You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/02/06 17:23:55 UTC

[01/15] lucene-solr git commit: current patch

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/lucene-5438-nrt-replication [created] bd6804bc6


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
new file mode 100644
index 0000000..5a073ff
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -0,0 +1,1175 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.charset.MalformedInputException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.ConcurrentMergeScheduler;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
+// nocommit sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
+
+// nocommit randomly p.destroy() one replica?
+
+/*
+  TODO
+    - why do we do the "rename temp to actual" all at the end...?  what really does that buy us?
+    - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
+    - test should not print scary exceptions and then succeed!
+    - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
+    - are the pre-copied-completed-merged files not being cleared in primary?
+      - hmm the logic isn't right today?  a replica may skip pulling a given copy state, that recorded the finished merged segments?
+    - beast & fix bugs
+    - graceful cluster restart
+    - better translog integration
+    - get "graceful primary shutdown" working
+    - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion
+    - clean up how version is persisted in commit data
+    - why am i not using hashes here?  how does ES use them?
+    - get all other "single shard" functions working too: this cluster should "act like" a single shard
+      - SLM
+      - controlled nrt reopen thread / returning long gen on write
+      - live field values
+      - add indexes
+    - make cluster level APIs to search, index, that deal w/ primary failover, etc.
+    - must prune xlog
+      - refuse to start primary unless we have quorum
+    - later
+      - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback?
+      - back pressure on indexing if replicas can't keep up?
+      - get xlog working on top?  needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit
+        quorum
+        - maybe fix IW to return "gen" or "seq id" or "segment name" or something?
+      - replica can copy files from other replicas too / use multicast / rsync / something
+      - each replica could also pre-open a SegmentReader after pre-copy when warming a merge
+      - we can pre-copy newly flushed files too, for cases where reopen rate is low vs IW's flushing because RAM buffer is full
+      - opto: pre-copy files as they are written; if they will become CFS, we can build CFS on the replica?
+      - what about multiple commit points?
+      - fix primary to init directly from an open replica, instead of having to commit/close the replica first
+*/
+
+// Tricky cases:
+//   - we are pre-copying a merge, then replica starts up part way through, so it misses that pre-copy and must do it on next nrt point
+//   - a down replica starts up, but it's "from the future" vs the current primary, and must copy over file names with different contents
+//     but referenced by its latest commit point, so it must fully remove that commit ... which is a hazardous window
+//   - replica comes up just as the primary is crashing / moving
+//   - electing a new primary when a replica is just finishing its nrt sync: we need to wait for it so we are sure to get the "most up to
+//     date" replica
+//   - replica comes up after merged segment finished so it doesn't copy over the merged segment "promptly" (i.e. only sees it on NRT refresh)
+
+/**
+ * Test case showing how to implement NRT replication.  This test spawns a sub-process per-node, running TestNRTReplicationChild.
+ *
+ * One node is primary, and segments are periodically flushed there, then concurrently the N replica nodes copy the new files over and open new readers, while
+ * primary also opens a new reader.
+ *
+ * Nodes randomly crash and are restarted.  If the primary crashes, a replica is promoted.
+ *
+ * Merges are currently first finished on the primary and then pre-copied out to replicas with a merged segment warmer so they don't block
+ * ongoing NRT reopens.  Probably replicas could do their own merging instead, but this is more complex and may not be better overall
+ * (merging takes a lot of IO resources).
+ *
+ * Slow network is simulated with a RateLimiter.
+ */
+
+// nocommit randomly delete all doc sometimes, 1) using IW.deleteAll and 2) doing it inefficiently (by query, by id)
+
+// MockRandom's .sd file has no index header/footer:
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class TestNRTReplication extends LuceneTestCase {
+
+  // Test evilness controls:
+
+  /** Randomly crash the current primary (losing data!) and promote the "next best" replica. */
+  static final boolean DO_CRASH_PRIMARY = true;
+
+  /** Randomly crash (JVM core dumps) a replica; it will later randomly be restarted and sync itself. */
+  static final boolean DO_CRASH_REPLICA = true;
+
+  /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
+  static final boolean DO_CLOSE_REPLICA = true;
+
+  /** If false, all child + parent output is interleaved into single stdout/err */
+  static final boolean SEPARATE_CHILD_OUTPUT = false;
+
+  // nocommit DO_CLOSE_PRIMARY?
+
+  /** Randomly crash whole cluster and then restart it */
+  static final boolean DO_FULL_CLUSTER_CRASH = true;
+
+  /** True if we randomly flip a bit while copying files out */
+  static final boolean DO_BIT_FLIPS_DURING_COPY = true;
+
+  /** Set to a non-null value to force exactly that many nodes; else, it's random. */
+  static final Integer NUM_NODES = null;
+
+  static final boolean DO_RANDOM_XLOG_REPLAY = false;
+
+  final AtomicBoolean failed = new AtomicBoolean();
+
+  final AtomicBoolean stop = new AtomicBoolean();
+
+  /** cwd where we start each child (server) node */
+  private Path childTempDir;
+
+  long primaryGen;
+
+  volatile long lastPrimaryVersion;
+
+  volatile NodeProcess primary;
+  volatile NodeProcess[] nodes;
+  volatile long[] nodeTimeStamps;
+  volatile boolean[] starting;
+  
+  Path[] indexPaths;
+
+  Path transLogPath;
+  SimpleTransLog transLog;
+  final AtomicInteger markerUpto = new AtomicInteger();
+
+  /** Maps searcher version to how many hits the query body:the matched. */
+  final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
+
+  /** Maps searcher version to how many marker documents matched.  This should only ever grow (we never delete marker documents). */
+  final Map<Long,Integer> versionToMarker = new ConcurrentHashMap<>();
+
+  /** Maps searcher version to xlog location when refresh of this version started. */
+  final Map<Long,Long> versionToTransLogLocation = new ConcurrentHashMap<>();
+
+  public void test() throws Exception {
+
+    Node.globalStartNS = System.nanoTime();
+
+    message("change thread name from " + Thread.currentThread().getName());
+    Thread.currentThread().setName("main");
+
+    childTempDir = createTempDir("child");
+
+    // We are parent process:
+
+    // Silly bootstrapping:
+    versionToTransLogLocation.put(0L, 0L);
+    versionToTransLogLocation.put(1L, 0L);
+
+    int numNodes;
+
+    if (NUM_NODES == null) {
+      numNodes = TestUtil.nextInt(random(), 2, 10);
+    } else {
+      numNodes = NUM_NODES.intValue();
+    }
+
+    System.out.println("TEST: using " + numNodes + " nodes");
+
+    transLogPath = createTempDir("NRTReplication").resolve("translog");
+    transLog = new SimpleTransLog(transLogPath);
+
+    //state.rateLimiters = new RateLimiter[numNodes];
+    indexPaths = new Path[numNodes];
+    nodes = new NodeProcess[numNodes];
+    nodeTimeStamps = new long[numNodes];
+    Arrays.fill(nodeTimeStamps, Node.globalStartNS);
+    starting = new boolean[numNodes];
+    
+    for(int i=0;i<numNodes;i++) {
+      indexPaths[i] = createTempDir("index" + i);
+    }
+
+    Thread[] indexers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+    System.out.println("TEST: launch " + indexers.length + " indexer threads");
+    for(int i=0;i<indexers.length;i++) {
+      indexers[i] = new IndexThread();
+      indexers[i].setName("indexer" + i);
+      indexers[i].setDaemon(true);
+      indexers[i].start();
+    }
+
+    Thread[] searchers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+    System.out.println("TEST: launch " + searchers.length + " searcher threads");
+    for(int i=0;i<searchers.length;i++) {
+      searchers[i] = new SearchThread();
+      searchers[i].setName("searcher" + i);
+      searchers[i].setDaemon(true);
+      searchers[i].start();
+    }
+
+    Thread restarter = new RestartThread();
+    restarter.setName("restarter");
+    restarter.setDaemon(true);
+    restarter.start();
+
+    int runTimeSec;
+    if (TEST_NIGHTLY) {
+      runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 120, 240);
+    } else {
+      runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 45, 120);
+    }
+
+    System.out.println("TEST: will run for " + runTimeSec + " sec");
+
+    long endTime = System.nanoTime() + runTimeSec*1000000000L;
+
+    sendReplicasToPrimary();
+
+    while (failed.get() == false && System.nanoTime() < endTime) {
+
+      // Wait a bit:
+      Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
+      if (primary != null && random().nextBoolean()) {
+        message("top: now flush primary");
+        NodeProcess curPrimary = primary;
+        if (curPrimary != null) {
+
+          // Save these before we start flush:
+          long nextTransLogLoc = transLog.getNextLocation();
+          int markerUptoSav = markerUpto.get();
+
+          long result;
+          try {
+            result = primary.flush();
+          } catch (Throwable t) {
+            message("top: flush failed; skipping: " + t.getMessage());
+            result = -1;
+          }
+          if (result > 0) {
+            // There were changes
+            lastPrimaryVersion = result;
+            addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
+            addVersionMarker(lastPrimaryVersion, markerUptoSav);
+          }
+        }
+      }
+
+      StringBuilder sb = new StringBuilder();
+      int liveCount = 0;
+      for(int i=0;i<nodes.length;i++) {
+        NodeProcess node = nodes[i];
+        if (node != null) {
+          if (sb.length() != 0) {
+            sb.append(" ");
+          }
+          liveCount++;
+          if (node.isPrimary) {
+            sb.append('P');
+          } else {
+            sb.append('R');
+          }
+          sb.append(i);
+        }
+      }
+
+      message("PG=" + (primary == null ? "X" : primaryGen) + " " + liveCount + " (of " + nodes.length + ") nodes running: " + sb);
+
+      // Commit a random node, primary or replica
+
+      {
+        NodeProcess node = nodes[random().nextInt(nodes.length)];
+        if (node != null) {
+          // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
+          // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
+          message("top: now commit node=" + node);
+          node.commitAsync();
+        }
+      }
+    }
+
+    message("TEST: top: test done, now close");
+    stop.set(true);
+    for(Thread thread : indexers) {
+      thread.join();
+    }
+    for(Thread thread : searchers) {
+      thread.join();
+    }
+    restarter.join();
+
+    // Close replicas before primary so we cancel any in-progres replications:
+    System.out.println("TEST: top: now close replicas");
+    List<Closeable> toClose = new ArrayList<>();
+    for(NodeProcess node : nodes) {
+      if (node != primary && node != null) {
+        toClose.add(node);
+      }
+    }
+    IOUtils.close(toClose);
+    IOUtils.close(primary);
+    IOUtils.close(transLog);
+
+    if (failed.get() == false) {
+      message("TEST: top: now checkIndex");    
+      for(Path path : indexPaths) {
+        message("TEST: check " + path);
+        MockDirectoryWrapper dir = newMockFSDirectory(path);
+        // Just too slow otherwise
+        dir.setCrossCheckTermVectorsOnClose(false);
+        dir.close();
+      }
+    } else {
+      message("TEST: failed; skip checkIndex");
+    }
+  }
+
+  private boolean anyNodesStarting() {
+    for(int id=0;id<nodes.length;id++) {
+      if (starting[id]) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /** Picks a replica and promotes it as new primary. */
+  private void promoteReplica() throws IOException {
+    message("top: primary crashed; now pick replica to promote");
+    long maxSearchingVersion = -1;
+    NodeProcess replicaToPromote = null;
+
+    // We must promote the most current replica, because otherwise file name reuse can cause a replication to fail when it needs to copy
+    // over a file currently held open for searching.  This also minimizes recovery work since the most current replica means less xlog
+    // replay to catch up:
+    for (NodeProcess node : nodes) {
+      if (node != null) {
+        message("ask " + node + " for its current searching version");
+        long searchingVersion = node.getSearchingVersion();
+        message(node + " has searchingVersion=" + searchingVersion);
+        if (searchingVersion > maxSearchingVersion) {
+          maxSearchingVersion = searchingVersion;
+          replicaToPromote = node;
+        }
+      }
+    }
+
+    if (replicaToPromote == null) {
+      message("top: no replicas running; skipping primary promotion");
+      return;
+    }
+
+    message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
+    if (replicaToPromote.commit() == false) {
+      message("top: commit failed; skipping primary promotion");
+      return;
+    }
+
+    message("top: now shutdown " + replicaToPromote);
+    if (replicaToPromote.shutdown() == false) {
+      message("top: shutdown failed for R" + replicaToPromote.id + "; skipping primary promotion");
+      return;
+    }
+
+    int id = replicaToPromote.id;
+    message("top: now startPrimary " + replicaToPromote);
+    startPrimary(replicaToPromote.id);
+  }
+
+  void startPrimary(int id) throws IOException {
+    message(id + ": top: startPrimary lastPrimaryVersion=" + lastPrimaryVersion);
+    assert nodes[id] == null;
+
+    // Force version of new primary to advance beyond where old primary was, so we never re-use versions.  It may have
+    // already advanced beyond newVersion, e.g. if it flushed new segments while during xlog replay:
+
+    // First start node as primary (it opens an IndexWriter) but do not publish it for searching until we replay xlog:
+    NodeProcess newPrimary = startNode(id, indexPaths[id], true, lastPrimaryVersion+1);
+    if (newPrimary == null) {
+      message("top: newPrimary failed to start; abort");
+      return;
+    }
+
+    // Get xlog location that this node was guaranteed to already have indexed through; this may replay some ops already indexed but it's OK
+    // because the ops are idempotent: we updateDocument (by docid) on replay even for original addDocument:
+    Long startTransLogLoc;
+    Integer markerCount;
+    if (newPrimary.initCommitVersion == 0) {
+      startTransLogLoc = 0L;
+      markerCount = 0;
+    } else {
+      startTransLogLoc = versionToTransLogLocation.get(newPrimary.initCommitVersion);
+      markerCount = versionToMarker.get(newPrimary.initCommitVersion);
+    }
+    assert startTransLogLoc != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToTransLogLocation: keys=" + versionToTransLogLocation.keySet();
+    assert markerCount != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToMarker: keys=" + versionToMarker.keySet();
+
+    // When the primary starts, the userData in its latest commit point tells us which version it had indexed up to, so we know where to
+    // replay from in the xlog.  However, we forcefuly advance the version, and then IW on init (or maybe getReader) also adds 1 to it.
+    // Since we publish the primary in this state (before xlog replay is done), a replica can start up at this point and pull this version,
+    // and possibly later be chosen as a primary, causing problems if the version is known recorded in the translog map.  So we record it
+    // here:
+
+    addTransLogLoc(newPrimary.initInfosVersion, startTransLogLoc);
+    addVersionMarker(newPrimary.initInfosVersion, markerCount);
+
+    assert newPrimary.initInfosVersion >= lastPrimaryVersion;
+    message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
+    lastPrimaryVersion = newPrimary.initInfosVersion;
+
+    // Publish new primary, before replaying xlog.  This means other indexing ops can come in at the same time as we catch up indexing
+    // previous ops.  Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
+    // translog) concurrently with new incoming ops.
+    nodes[id] = newPrimary;
+    primary = newPrimary;
+
+    sendReplicasToPrimary();
+
+    long nextTransLogLoc = transLog.getNextLocation();
+    int nextMarkerUpto = markerUpto.get();
+    message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
+    try {
+      transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
+    } catch (IOException ioe) {
+      message("top: replay xlog failed; abort");
+      return;
+    }
+    message("top: done replay trans log");
+  }
+
+  final AtomicLong nodeStartCounter = new AtomicLong();
+
+  final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
+
+  /** Launches a child "server" (separate JVM), which is either primary or replica node */
+  NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
+    nodeTimeStamps[id] = System.nanoTime();
+    List<String> cmd = new ArrayList<>();
+
+    NodeProcess curPrimary = primary;
+
+    cmd.add(System.getProperty("java.home") 
+        + System.getProperty("file.separator")
+        + "bin"
+        + System.getProperty("file.separator")
+        + "java");
+    cmd.add("-Xmx512m");
+
+    if (curPrimary != null) {
+      cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + curPrimary.tcpPort);
+    } else if (isPrimary == false) {
+      // We cannot start a replica when there is no primary:
+      return null;
+    }
+
+    cmd.add("-Dtests.nrtreplication.node=true");
+    cmd.add("-Dtests.nrtreplication.nodeid=" + id);
+    cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
+    cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
+    if (isPrimary) {
+      cmd.add("-Dtests.nrtreplication.isPrimary=true");
+      cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
+    }
+
+    long myPrimaryGen = primaryGen;
+    cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
+
+    // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
+    // new node:
+    long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
+
+    cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(seed));
+    cmd.add("-ea");
+    cmd.add("-cp");
+    cmd.add(System.getProperty("java.class.path"));
+    cmd.add("org.junit.runner.JUnitCore");
+    cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
+
+    Writer childLog;
+
+    if (SEPARATE_CHILD_OUTPUT) {
+      Path childOut = childTempDir.resolve(id + ".log");
+      message("logging to " + childOut);
+      childLog = Files.newBufferedWriter(childOut, StandardCharsets.UTF_8, StandardOpenOption.APPEND, StandardOpenOption.CREATE);
+      childLog.write("\n\nSTART NEW CHILD:\n");
+    } else {
+      childLog = null;
+    }
+
+    message("child process command: " + cmd);
+    ProcessBuilder pb = new ProcessBuilder(cmd);
+    pb.redirectErrorStream(true);
+
+    // Important, so that the scary looking hs_err_<pid>.log appear under our test temp dir:
+    pb.directory(childTempDir.toFile());
+
+    Process p = pb.start();
+
+    BufferedReader r;
+    try {
+      r = new BufferedReader(new InputStreamReader(p.getInputStream(), IOUtils.UTF_8));
+    } catch (UnsupportedEncodingException uee) {
+      throw new RuntimeException(uee);
+    }
+
+    int tcpPort = -1;
+    long initCommitVersion = -1;
+    long initInfosVersion = -1;
+    Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+    boolean willCrash = false;
+    boolean sawExistingSegmentsFile = false;
+
+    while (true) {
+      String l = r.readLine();
+      if (l == null) {
+        message("top: node=" + id + " failed to start");
+        try {
+          p.waitFor();
+        } catch (InterruptedException ie) {
+          throw new RuntimeException(ie);
+        }
+        message("exit value=" + p.exitValue());
+
+        // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
+        if (isPrimary == false) {
+          if (sawExistingSegmentsFile) {
+            // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
+            // and retry again later:
+            message("failed to remove segments_N; skipping");
+            return null;
+          }
+          for(int i=0;i<10;i++) {
+            if (primaryGen != myPrimaryGen || primary == null) {
+              // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
+              message("primary crashed/closed while replica R" + id + " tried to start; skipping");
+              return null;
+            } else {
+              try {
+                Thread.sleep(10);
+              } catch (InterruptedException ie) {
+                throw new ThreadInterruptedException(ie);
+              }
+            }
+          }
+        }
+
+        // Should fail the test:
+        message("top: now fail test replica R" + id + " failed to start");
+        failed.set(true);
+        throw new RuntimeException("replica R" + id + " failed to start");
+      }
+
+      if (childLog != null) {
+        childLog.write(l);
+        childLog.write("\n");
+        childLog.flush();
+      } else if (logTimeStart.matcher(l).matches()) {
+        // Already a well-formed log output:
+        System.out.println(l);
+      } else {
+        message(l);
+      }
+
+      if (l.startsWith("PORT: ")) {
+        tcpPort = Integer.parseInt(l.substring(6).trim());
+      } else if (l.startsWith("COMMIT VERSION: ")) {
+        initCommitVersion = Integer.parseInt(l.substring(16).trim());
+      } else if (l.startsWith("INFOS VERSION: ")) {
+        initInfosVersion = Integer.parseInt(l.substring(15).trim());
+      } else if (l.contains("will crash after")) {
+        willCrash = true;
+      } else if (l.startsWith("NODE STARTED")) {
+        break;
+      } else if (l.contains("replica cannot start: existing segments file=")) {
+        sawExistingSegmentsFile = true;
+      }
+    }
+
+    final boolean finalWillCrash = willCrash;
+
+    // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
+    Thread pumper = ThreadPumper.start(
+                                       new Runnable() {
+                                         @Override
+                                         public void run() {
+                                           message("now wait for process " + p);
+                                           try {
+                                             p.waitFor();
+                                           } catch (Throwable t) {
+                                             throw new RuntimeException(t);
+                                           }
+
+                                           message("done wait for process " + p);
+                                           int exitValue = p.exitValue();
+                                           message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
+                                           if (childLog != null) {
+                                             try {
+                                               childLog.write("process done; exitValue=" + exitValue + "\n");
+                                               childLog.close();
+                                             } catch (IOException ioe) {
+                                               throw new RuntimeException(ioe);
+                                             }
+                                           }
+                                           if (exitValue != 0 && finalWillCrash == false && crashingNodes.remove(id) == false) {
+                                             // should fail test
+                                             failed.set(true);
+                                             if (childLog != null) {
+                                               throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue + "; see " + childLog + " for details");
+                                             } else {
+                                               throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
+                                             }
+                                           }
+                                           nodeClosed(id);
+                                         }
+                                       }, r, System.out, childLog);
+    pumper.setName("pump" + id);
+
+    message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
+    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+  }
+
+  private void nodeClosed(int id) {
+    NodeProcess oldNode = nodes[id];
+    if (primary != null && oldNode == primary) {
+      message("top: " + primary + ": primary process finished");
+      primary = null;
+      primaryGen++;
+    } else {
+      message("top: " + oldNode + ": replica process finished");
+    }
+    if (oldNode != null) {
+      oldNode.isOpen = false;
+    }
+    nodes[id] = null;
+    nodeTimeStamps[id] = System.nanoTime();
+
+    sendReplicasToPrimary();
+  }
+
+  /** Sends currently alive replicas to primary, which uses this to know who to notify when it does a refresh */
+  private void sendReplicasToPrimary() {
+    NodeProcess curPrimary = primary;
+    if (curPrimary != null) {
+      List<NodeProcess> replicas = new ArrayList<>();
+      for (NodeProcess node : nodes) {
+        if (node != null && node.isPrimary == false) {
+          replicas.add(node);
+        }
+      }
+
+      message("top: send " + replicas.size() + " replicas to primary");
+
+      try (Connection c = new Connection(curPrimary.tcpPort)) {
+        c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+        c.out.writeVInt(replicas.size());        
+        for(NodeProcess replica : replicas) {
+          c.out.writeVInt(replica.id);
+          c.out.writeVInt(replica.tcpPort);
+        }
+        c.flush();
+        c.in.readByte();
+      } catch (Throwable t) {
+        message("top: ignore exc sending replicas to primary: " + t);
+      }
+    }
+  }
+
+  void addVersionMarker(long version, int count) {
+    //System.out.println("ADD VERSION MARKER version=" + version + " count=" + count);
+    if (versionToMarker.containsKey(version)) {
+      int curCount = versionToMarker.get(version);
+      if (curCount != count) {
+        message("top: wrong marker count version=" + version + " count=" + count + " curCount=" + curCount);
+        throw new IllegalStateException("version=" + version + " count=" + count + " curCount=" + curCount);
+      }
+    } else {
+      message("top: record marker count: version=" + version + " count=" + count);
+      versionToMarker.put(version, count);
+    }
+  }
+
+  void addTransLogLoc(long version, long loc) {
+    message("top: record transLogLoc: version=" + version + " loc=" + loc);
+    versionToTransLogLocation.put(version, loc);
+  }
+
+  // Periodically wakes up and starts up any down nodes:
+  private class RestartThread extends Thread {
+    @Override
+    public void run() {
+
+      List<Thread> startupThreads = Collections.synchronizedList(new ArrayList<>());
+
+      try {
+        while (stop.get() == false) {
+          Thread.sleep(TestUtil.nextInt(random(), 50, 500));
+          message("top: restarter cycle");
+
+          // Randomly crash full cluster:
+          if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
+            message("top: full cluster crash");
+            for(int i=0;i<nodes.length;i++) {
+              if (starting[i]) {
+                message("N" + i + ": top: wait for startup so we can crash...");
+                while (starting[i]) {
+                  Thread.sleep(10);
+                }
+                message("N" + i + ": top: done wait for startup");
+              }
+              NodeProcess node = nodes[i];
+              if (node != null) {
+                crashingNodes.add(i);
+                message("top: N" + node.id + ": top: now crash node");
+                node.crash();
+                message("top: N" + node.id + ": top: done crash node");
+              }
+            }
+          }
+
+          List<Integer> downNodes = new ArrayList<>();
+          StringBuilder b = new StringBuilder();
+          long nowNS = System.nanoTime();
+          for(int i=0;i<nodes.length;i++) {
+            b.append(' ');
+            double sec = (nowNS - nodeTimeStamps[i])/1000000000.0;
+            String prefix;
+            if (nodes[i] == null) {
+              downNodes.add(i);
+              if (starting[i]) {
+                prefix = "s";
+              } else {
+                prefix = "x";
+              }
+            } else {
+              prefix = "";
+            }
+            if (primary != null && nodes[i] == primary) {
+              prefix += "p";
+            }
+            b.append(String.format(Locale.ROOT, "%s%d(%.1fs)", prefix, i, sec));
+          }
+          message("node status" + b.toString());
+          message("downNodes=" + downNodes);
+
+          // If primary is down, promote a replica:
+          if (primary == null) {
+            if (anyNodesStarting()) {
+              message("top: skip promote replica: nodes are still starting");
+              continue;
+            }
+            promoteReplica();
+          }
+
+          // Randomly start up a down a replica:
+
+          // Stop or start a replica
+          if (downNodes.isEmpty() == false) {
+            int idx = downNodes.get(random().nextInt(downNodes.size()));
+            if (starting[idx] == false) {
+              if (primary == null) {
+                if (downNodes.size() == nodes.length) {
+                  // Cold start: entire cluster is down, start this node up as the new primary
+                  message("N" + idx + ": top: cold start as primary");
+                  startPrimary(idx);
+                }
+              } else if (random().nextDouble() < ((double) downNodes.size())/nodes.length) {
+                // Start up replica:
+                starting[idx] = true;
+                message("N" + idx + ": top: start up: launch thread");
+                Thread t = new Thread() {
+                    @Override
+                    public void run() {
+                      try {
+                        message("N" + idx + ": top: start up thread");
+                        nodes[idx] = startNode(idx, indexPaths[idx], false, -1);
+                        sendReplicasToPrimary();
+                      } catch (Throwable t) {
+                        failed.set(true);
+                        stop.set(true);
+                        throw new RuntimeException(t);
+                      } finally {
+                        starting[idx] = false;
+                        startupThreads.remove(Thread.currentThread());
+                      }
+                    }
+                  };
+                t.setName("start R" + idx);
+                t.start();
+                startupThreads.add(t);
+              }
+            } else {
+              message("node " + idx + " still starting");
+            }
+          }
+        }
+
+        System.out.println("Restarter: now stop: join " + startupThreads.size() + " startup threads");
+
+        while (startupThreads.size() > 0) {
+          Thread.sleep(10);
+        }
+
+      } catch (Throwable t) {
+        failed.set(true);
+        stop.set(true);
+        throw new RuntimeException(t);
+      }
+    }
+  }
+
+  /** Randomly picks a node and runs a search against it */
+  private class SearchThread extends Thread {
+
+    @Override
+    public void run() {
+      // Maps version to number of hits for silly 'the' TermQuery:
+      Query theQuery = new TermQuery(new Term("body", "the"));
+
+      // Persists connections
+      Map<Integer,Connection> connections = new HashMap<>();
+
+      while (stop.get() == false) {
+        NodeProcess node = nodes[random().nextInt(nodes.length)];
+        if (node == null || node.isOpen == false) {
+          continue;
+        }
+
+        if (node.lock.tryLock() == false) {
+          // Node is in the process of closing or crashing or something
+          continue;
+        }
+
+        try {
+
+          Thread.currentThread().setName("Searcher node=" + node);
+
+          //System.out.println("S: cycle; conns=" + connections);
+
+          Connection c = connections.get(node.id);
+
+          long version;
+          try {
+            if (c == null) {
+              //System.out.println("S: new connection " + node.id + " " + Thread.currentThread().getName());
+              c = new Connection(node.tcpPort);
+              connections.put(node.id, c);
+            } else {
+              //System.out.println("S: reuse connection " + node.id + " " + Thread.currentThread().getName());
+            }
+
+            c.out.writeByte(SimplePrimaryNode.CMD_SEARCH);
+            c.flush();
+
+            while (c.sockIn.available() == 0) {
+              if (stop.get()) {
+                break;
+              }
+              if (node.isOpen == false) {
+                throw new IOException("node closed");
+              }
+              Thread.sleep(1);
+            }
+            version = c.in.readVLong();
+
+            while (c.sockIn.available() == 0) {
+              if (stop.get()) {
+                break;
+              }
+              if (node.isOpen == false) {
+                throw new IOException("node closed");
+              }
+              Thread.sleep(1);
+            }
+            int hitCount = c.in.readVInt();
+
+            Integer oldHitCount = hitCounts.get(version);
+
+            // TODO: we never prune this map...
+            if (oldHitCount == null) {
+              hitCounts.put(version, hitCount);
+              message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
+            } else {
+              // Just ensure that all nodes show the same hit count for
+              // the same version, i.e. they really are replicas of one another:
+              if (oldHitCount.intValue() != hitCount) {
+                failed.set(true);
+                stop.set(true);
+                message("top: searcher: wrong version hitCount: version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+                fail("version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+              }
+            }
+          } catch (IOException ioe) {
+            //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+            //ioe.printStackTrace(System.out);
+            IOUtils.closeWhileHandlingException(c);
+            connections.remove(node.id);
+            continue;
+          }
+
+          // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
+          Integer expectedAtLeastHitCount = versionToMarker.get(version);
+
+          if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
+            try {
+              c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
+              c.flush();
+              while (c.sockIn.available() == 0) {
+                if (stop.get()) {
+                  break;
+                }
+                if (node.isOpen == false) {
+                  throw new IOException("node died");
+                }
+                Thread.sleep(1);
+              }
+
+              version = c.in.readVLong();
+
+              while (c.sockIn.available() == 0) {
+                if (stop.get()) {
+                  break;
+                }
+                if (node.isOpen == false) {
+                  throw new IOException("node died");
+                }
+                Thread.sleep(1);
+              }
+
+              int hitCount = c.in.readVInt();
+
+              // Look for data loss: make sure all marker docs are visible:
+            
+              if (hitCount < expectedAtLeastHitCount) {
+
+                String failMessage = "node=" + node + ": documents were lost version=" + version + " hitCount=" + hitCount + " vs expectedAtLeastHitCount=" + expectedAtLeastHitCount;
+                message(failMessage);
+                failed.set(true);
+                stop.set(true);
+                fail(failMessage);
+              }
+            } catch (IOException ioe) {
+              //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+              //throw new RuntimeException(ioe);
+              //ioe.printStackTrace(System.out);
+              IOUtils.closeWhileHandlingException(c);
+              connections.remove(node.id);
+              continue;
+            }
+          }
+
+          Thread.sleep(10);
+
+        } catch (Throwable t) {
+          failed.set(true);
+          stop.set(true);
+          throw new RuntimeException(t);
+        } finally {
+          node.lock.unlock();
+        }
+      }
+      System.out.println("Searcher: now stop");
+      IOUtils.closeWhileHandlingException(connections.values());
+    }
+  }
+
+  private class IndexThread extends Thread {
+
+    @Override
+    public void run() {
+
+      try {
+        LineFileDocs docs = new LineFileDocs(random());
+        int docCount = 0;
+
+        // How often we do an update/delete vs add:
+        double updatePct = random().nextDouble();
+
+        // Varies how many docs/sec we index:
+        int sleepChance = TestUtil.nextInt(random(), 4, 100);
+
+        message("top: indexer: updatePct=" + updatePct + " sleepChance=" + sleepChance);
+
+        long lastTransLogLoc = transLog.getNextLocation();
+        
+        NodeProcess curPrimary = null;
+        Connection c = null;
+
+        while (stop.get() == false) {
+
+          try {
+            while (stop.get() == false && curPrimary == null) {
+              Thread.sleep(10);
+              curPrimary = primary;
+              if (curPrimary != null) {
+                c = new Connection(curPrimary.tcpPort);
+                c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+                break;
+              }
+            }
+
+            if (stop.get()) {
+              break;
+            }
+
+            Thread.currentThread().setName("indexer p" + curPrimary.id);
+
+            if (random().nextInt(10) == 7) {
+              // We use the marker docs to check for data loss in search thread:
+              Document doc = new Document();
+              int id = markerUpto.getAndIncrement();
+              String idString = "m"+id;
+              doc.add(newStringField("docid", idString, Field.Store.YES));
+              doc.add(newStringField("marker", "marker", Field.Store.YES));
+              curPrimary.addOrUpdateDocument(c, doc, false);
+              transLog.addDocument(idString, doc);
+              message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
+            }
+
+            if (docCount > 0 && random().nextDouble() < updatePct) {
+              int randomID = random().nextInt(docCount);
+              String randomIDString = Integer.toString(randomID);
+              if (random().nextBoolean()) {
+                // Replace previous doc
+                Document doc = docs.nextDoc();
+                ((Field) doc.getField("docid")).setStringValue(randomIDString);
+                curPrimary.addOrUpdateDocument(c, doc, true);
+                transLog.updateDocument(randomIDString, doc);
+              } else {
+                // Delete previous doc
+                curPrimary.deleteDocument(c, randomIDString);
+                transLog.deleteDocuments(randomIDString);
+              }
+            } else {
+              // Add new doc:
+              Document doc = docs.nextDoc();
+              String idString = Integer.toString(docCount++);
+              ((Field) doc.getField("docid")).setStringValue(idString);
+              curPrimary.addOrUpdateDocument(c, doc, false);
+              transLog.addDocument(idString, doc);
+
+              if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
+                long curLoc = transLog.getNextLocation();
+                // randomly replay chunks of translog just to test replay:
+                message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
+                transLog.replay(curPrimary, lastTransLogLoc, curLoc);
+                lastTransLogLoc = curLoc;
+              }
+            }
+          } catch (IOException se) {
+            // Assume primary crashed
+            message("top: indexer lost connection to primary");
+            try {
+              c.close();
+            } catch (Throwable t) {
+            }
+            curPrimary = null;
+            c = null;
+          }
+
+          if (random().nextInt(sleepChance) == 0) {
+            Thread.sleep(1);
+          }
+
+          if (random().nextInt(100) == 17) {
+            System.out.println("Indexer: now pause for a bit...");
+            Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
+            System.out.println("Indexer: done pause for a bit...");
+          }
+        }
+        if (curPrimary != null) {
+          try {
+            c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
+            c.flush();
+            c.in.readByte();
+          } catch (IOException se) {
+            // Assume primary crashed
+            message("top: indexer lost connection to primary");
+            try {
+              c.close();
+            } catch (Throwable t) {
+            }
+            curPrimary = null;
+            c = null;
+          }
+        }
+        System.out.println("Indexer: now stop");
+      } catch (Throwable t) {
+        failed.set(true);
+        stop.set(true);
+        throw new RuntimeException(t);
+      }
+    }
+  }
+
+  static void message(String message) {
+    long now = System.nanoTime();
+    System.out.println(String.format(Locale.ROOT,
+                                     "%5.3fs       :     parent [%11s] %s",
+                                     (now-Node.globalStartNS)/1000000000.,
+                                     Thread.currentThread().getName(),
+                                     message));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
new file mode 100644
index 0000000..6ddb777
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
@@ -0,0 +1,59 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.util.regex.Pattern;
+
+/** A pipe thread. It'd be nice to reuse guava's implementation for this... */
+class ThreadPumper {
+  public static Thread start(final Runnable onExit, final BufferedReader from, final PrintStream to, final Writer toFile) {
+    Thread t = new Thread() {
+        @Override
+        public void run() {
+          try {
+            Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+            String line;
+            while ((line = from.readLine()) != null) {
+              if (toFile != null) {
+                toFile.write(line);
+                toFile.write("\n");
+                toFile.flush();
+              } else if (logTimeStart.matcher(line).matches()) {
+                // Already a well-formed log output:
+                System.out.println(line);
+              } else {
+                TestNRTReplication.message(line);
+              }
+            }
+            // Sub-process finished
+          } catch (IOException e) {
+            System.err.println("ignore IOExc reading from forked process pipe: " + e);
+          } finally {
+            onExit.run();
+          }
+        }
+      };
+    t.start();
+    return t;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/test.cmd
----------------------------------------------------------------------
diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd
new file mode 100644
index 0000000..14e3bd2
--- /dev/null
+++ b/lucene/replicator/test.cmd
@@ -0,0 +1 @@
+python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs  TestNRTReplication -jvms 1 -mult 4 -nightly

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
index 4236e88..9f876ef 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
@@ -637,7 +637,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
             for (int iter=0;iter<iters && failed.get() == false;iter++) {
 
               if (VERBOSE) {
-                System.out.println("\nTEST: iter=" + iter + " s=" + s);
+                System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " s=" + s);
               }
               Query query;
               VerifyHits verifyHits;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
index b4b6f7d..68eed39 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
@@ -457,7 +457,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
             if (random().nextBoolean()) {
               DirectoryReader ir = null;
               try {
-                ir = DirectoryReader.open(iw, random().nextBoolean());
+                ir = DirectoryReader.open(iw, random().nextBoolean(), false);
                 dir.setRandomIOExceptionRateOnOpen(0.0); // disable exceptions on openInput until next iteration
                 TestUtil.checkReader(ir);
               } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
index 047ef4b..22fee48 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
@@ -303,7 +303,7 @@ public class RandomIndexWriter implements Closeable {
 
   public DirectoryReader getReader() throws IOException {
     LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
-    return getReader(true);
+    return getReader(true, false);
   }
 
   private boolean doRandomForceMerge = true;
@@ -353,7 +353,7 @@ public class RandomIndexWriter implements Closeable {
     }
   }
 
-  public DirectoryReader getReader(boolean applyDeletions) throws IOException {
+  public DirectoryReader getReader(boolean applyDeletions, boolean writeAllDeletes) throws IOException {
     LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
     getReaderCalled = true;
     if (r.nextInt(20) == 2) {
@@ -366,7 +366,7 @@ public class RandomIndexWriter implements Closeable {
       if (r.nextInt(5) == 1) {
         w.commit();
       }
-      return w.getReader(applyDeletions);
+      return w.getReader(applyDeletions, writeAllDeletes);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("RIW.getReader: open new reader");
@@ -375,7 +375,7 @@ public class RandomIndexWriter implements Closeable {
       if (r.nextBoolean()) {
         return DirectoryReader.open(w.getDirectory());
       } else {
-        return w.getReader(applyDeletions);
+        return w.getReader(applyDeletions, writeAllDeletes);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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 ec99c7e..b19045b 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
@@ -38,8 +38,10 @@ import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
 
 import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.NoDeletionPolicy;
@@ -239,12 +241,24 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     
     if (openFiles.containsKey(source)) {
       if (assertNoDeleteOpenFile) {
-        throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
+        throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
       } else if (noDeleteOpenFile) {
-        throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
+        throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
       }
     }
 
+    if (openFiles.containsKey(dest)) {
+      if (assertNoDeleteOpenFile) {
+        throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
+      } else if (noDeleteOpenFile) {
+        throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
+      }
+    }
+
+    if (createdFiles.contains(dest)) {
+      throw new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" already exists: cannot rename");
+    }
+
     boolean success = false;
     try {
       in.renameFile(source, dest);
@@ -257,6 +271,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
           unSyncedFiles.add(dest);
         }
         openFilesDeleted.remove(source);
+        triedToDelete.remove(dest);
+        createdFiles.add(dest);
       }
     }
   }
@@ -278,89 +294,215 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     }
   }
 
-  /** Simulates a crash of OS or machine by overwriting
-   *  unsynced files. */
-  public synchronized void crash() throws IOException {
-    crashed = true;
-    openFiles = new HashMap<>();
-    openFilesForWrite = new HashSet<>();
-    openFilesDeleted = new HashSet<>();
-    Iterator<String> it = unSyncedFiles.iterator();
-    unSyncedFiles = new HashSet<>();
-    // first force-close all files, so we can corrupt on windows etc.
-    // clone the file map, as these guys want to remove themselves on close.
-    Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
-    for (Closeable f : m.keySet()) {
-      try {
-        f.close();
-      } catch (Exception ignored) {}
+  public synchronized void corruptUnknownFiles() throws IOException {
+
+    System.out.println("MDW: corrupt unknown files");
+    Set<String> knownFiles = new HashSet<>();
+    for(String fileName : listAll()) {
+      if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+        System.out.println("MDW: read " + fileName + " to gather files it references");
+        knownFiles.addAll(SegmentInfos.readCommit(this, fileName).files(true));
+      }
     }
-    
-    while(it.hasNext()) {
-      String name = it.next();
-      int damage = randomState.nextInt(5);
+
+    Set<String> toCorrupt = new HashSet<>();
+    Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
+    for(String fileName : listAll()) {
+      m.reset(fileName);
+      if (knownFiles.contains(fileName) == false &&
+          fileName.endsWith("write.lock") == false &&
+          (m.matches() || fileName.startsWith(IndexFileNames.PENDING_SEGMENTS))) {
+        toCorrupt.add(fileName);
+      }
+    }
+
+    corruptFiles(toCorrupt);
+  }
+
+  public synchronized void corruptFiles(Collection<String> files) {
+    // Must make a copy because we change the incoming unsyncedFiles
+    // when we create temp files, delete, etc., below:
+    for(String name : new ArrayList<>(files)) {
+      int damage = randomState.nextInt(6);
       String action = null;
 
-      if (damage == 0) {
+      switch(damage) {
+
+      case 0:
         action = "deleted";
-        deleteFile(name, true);
-      } else if (damage == 1) {
+        try {
+          deleteFile(name, true);
+        } catch (IOException ioe) {
+          // ignore
+        }
+        break;
+
+      case 1:
         action = "zeroed";
         // Zero out file entirely
-        long length = fileLength(name);
+        long length;
+        try {
+          length = fileLength(name);
+        } catch (IOException ioe) {
+          // Ignore
+          continue;
+        }
         byte[] zeroes = new byte[256];
         long upto = 0;
-        IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
-        while(upto < length) {
-          final int limit = (int) Math.min(length-upto, zeroes.length);
-          out.writeBytes(zeroes, 0, limit);
-          upto += limit;
+        try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
+          while(upto < length) {
+            final int limit = (int) Math.min(length-upto, zeroes.length);
+            out.writeBytes(zeroes, 0, limit);
+            upto += limit;
+          }
+        } catch (IOException ioe) {
+          // ignore
         }
-        out.close();
-      } else if (damage == 2) {
-        action = "partially truncated";
-        // Partially Truncate the file:
-
-        // First, make temp file and copy only half this
-        // file over:
-        String tempFileName;
-        while (true) {
-          tempFileName = ""+randomState.nextInt();
-          if (!LuceneTestCase.slowFileExists(in, tempFileName)) {
-            break;
+        break;
+
+      case 2:
+        {
+          action = "partially truncated";
+          // Partially Truncate the file:
+
+          // First, make temp file and copy only half this
+          // file over:
+          String tempFileName = null;
+          try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
+               IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
+              tempFileName = tempOut.getName();
+              tempOut.copyBytes(ii, ii.length()/2);
+            } catch (IOException ioe) {
+            // ignore
+          }
+
+          try {
+            // Delete original and copy bytes back:
+            deleteFile(name, true);
+          } catch (IOException ioe) {
+            // ignore
+          }
+
+          try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+               IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
+              out.copyBytes(ii, ii.length());
+            } catch (IOException ioe) {
+            // ignore
+          }
+          try {
+            deleteFile(tempFileName, true);
+          } catch (IOException ioe) {
+            // ignore
           }
         }
-        final IndexOutput tempOut = in.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
-        IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState));
-        tempOut.copyBytes(ii, ii.length()/2);
-        tempOut.close();
-        ii.close();
-
-        // Delete original and copy bytes back:
-        deleteFile(name, true);
-        
-        final IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
-        ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
-        out.copyBytes(ii, ii.length());
-        out.close();
-        ii.close();
-        deleteFile(tempFileName, true);
-      } else if (damage == 3) {
+        break;
+      
+      case 3:
         // The file survived intact:
         action = "didn't change";
-      } else {
+        break;
+
+      case 4:
+        // Corrupt one bit randomly in the file:
+
+        {
+
+          String tempFileName = null;
+          try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
+               IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
+              tempFileName = tempOut.getName();
+              if (ii.length() > 0) {
+                // Copy first part unchanged:
+                long byteToCorrupt = (long) (randomState.nextDouble() * ii.length());
+                if (byteToCorrupt > 0) {
+                  tempOut.copyBytes(ii, byteToCorrupt);
+                }
+
+                // Randomly flip one bit from this byte:
+                byte b = ii.readByte();
+                int bitToFlip = randomState.nextInt(8);
+                b = (byte) (b ^ (1 << bitToFlip));
+                tempOut.writeByte(b);
+
+                action = "flip bit " + bitToFlip + " of byte " + byteToCorrupt + " out of " + ii.length() + " bytes";
+
+                // Copy last part unchanged:
+                long bytesLeft = ii.length() - byteToCorrupt - 1;
+                if (bytesLeft > 0) {
+                  tempOut.copyBytes(ii, bytesLeft);
+                }
+              } else {
+                action = "didn't change";
+              }
+            } catch (IOException ioe) {
+            // ignore
+          }
+
+          try {
+            // Delete original and copy bytes back:
+            deleteFile(name, true);
+          } catch (IOException ioe) {
+            // ignore
+          }
+
+          try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+               IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
+              out.copyBytes(ii, ii.length());
+            } catch (IOException ioe) {
+            // ignore
+          }
+          try {
+            deleteFile(tempFileName, true);
+          } catch (IOException ioe) {
+            // ignore
+          }
+        }
+        break;
+        
+      case 5:
         action = "fully truncated";
         // Totally truncate the file to zero bytes
-        deleteFile(name, true);
-        IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
-        out.close();
+        try {
+          deleteFile(name, true);
+        } catch (IOException ioe) {
+          // ignore
+        }
+
+        try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
+        } catch (IOException ioe) {
+          // ignore
+        }
+        break;
+
+      default:
+        throw new AssertionError();
       }
-      if (LuceneTestCase.VERBOSE) {
+
+      if (true || LuceneTestCase.VERBOSE) {
         System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name);
       }
     }
   }
 
+  /** Simulates a crash of OS or machine by overwriting
+   *  unsynced files. */
+  public synchronized void crash() {
+    crashed = true;
+    openFiles = new HashMap<>();
+    openFilesForWrite = new HashSet<>();
+    openFilesDeleted = new HashSet<>();
+    // first force-close all files, so we can corrupt on windows etc.
+    // clone the file map, as these guys want to remove themselves on close.
+    Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
+    for (Closeable f : m.keySet()) {
+      try {
+        f.close();
+      } catch (Exception ignored) {}
+    }
+    corruptFiles(unSyncedFiles);
+    unSyncedFiles = new HashSet<>();
+  }
+
   public synchronized void clearCrash() {
     crashed = false;
     openLocks.clear();
@@ -520,9 +662,9 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     if (!forced && enableVirusScanner && (randomState.nextInt(4) == 0)) {
       triedToDelete.add(name);
       if (LuceneTestCase.VERBOSE) {
-        System.out.println("MDW: now refuse to delete file: " + name);
+        System.out.println(Thread.currentThread().getName() + ": MDW: now refuse to delete file: " + name + " this=" + this);
       }
-      throw new IOException("cannot delete file: " + name + ", a virus scanner has it open");
+      throw new IOException("cannot delete file: " + name + ", a virus scanner has it open (exists?=" + LuceneTestCase.slowFileExists(in, name));
     }
     triedToDelete.remove(name);
     in.deleteFile(name);
@@ -571,6 +713,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     
     unSyncedFiles.add(name);
     createdFiles.add(name);
+    triedToDelete.remove(name);
     
     if (in instanceof RAMDirectory) {
       RAMDirectory ramdir = (RAMDirectory) in;
@@ -801,7 +944,11 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
             IndexWriterConfig iwc = new IndexWriterConfig(null);
             iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
             new IndexWriter(in, iwc).rollback();
-            String[] endFiles = in.listAll();
+
+            Set<String> files = new HashSet<>(Arrays.asList(listAll()));
+            // Disregard what happens with the pendingDeletions files:
+            files.removeAll(pendingDeletions);
+            String[] endFiles = files.toArray(new String[0]);
             
             Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
             Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
@@ -839,7 +986,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
                       assert pendingDeletions.contains(s);
                       if (LuceneTestCase.VERBOSE) {
                         System.out.println("MDW: Unreferenced check: Ignoring referenced file: " + s + " " +
-                            "from " + file + " that we could not delete.");
+                                           "from " + file + " that we could not delete.");
                       }
                       startSet.add(s);
                     }
@@ -884,7 +1031,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
                 extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
               }
               
-              throw new RuntimeException("unreferenced files: before delete:\n    " + Arrays.toString(startFiles) + "\n  after delete:\n    " + Arrays.toString(endFiles) + extras);
+              throw new RuntimeException(this + ": unreferenced files: before delete:\n    " + Arrays.toString(startFiles) + "\n  after delete:\n    " + Arrays.toString(endFiles) + extras);
             }
             
             DirectoryReader ir1 = DirectoryReader.open(this);
@@ -1036,7 +1183,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     }
   }
 
-
   // don't override optional methods like copyFrom: we need the default impl for things like disk 
   // full checks. we randomly exercise "raw" directories anyway. We ensure default impls are used:
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index aaab030..e6536f3 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1155,11 +1155,14 @@ public abstract class LuceneTestCase extends Assert {
     }
     
     if (rarely(r)) {
-      // change warmer parameters
-      if (r.nextBoolean()) {
-        c.setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(c.getInfoStream()));
-      } else {
-        c.setMergedSegmentWarmer(null);
+      IndexWriter.IndexReaderWarmer curWarmer = c.getMergedSegmentWarmer();
+      if (curWarmer == null || curWarmer instanceof SimpleMergedSegmentWarmer) {
+        // change warmer parameters
+        if (r.nextBoolean()) {
+          c.setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(c.getInfoStream()));
+        } else {
+          c.setMergedSegmentWarmer(null);
+        }
       }
       didChange = true;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 99d4be3..de2cf57 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -976,15 +976,14 @@ public final class TestUtil {
   public static void reduceOpenFiles(IndexWriter w) {
     // keep number of open files lowish
     MergePolicy mp = w.getConfig().getMergePolicy();
+    mp.setNoCFSRatio(1.0);
     if (mp instanceof LogMergePolicy) {
       LogMergePolicy lmp = (LogMergePolicy) mp;
       lmp.setMergeFactor(Math.min(5, lmp.getMergeFactor()));
-      lmp.setNoCFSRatio(1.0);
     } else if (mp instanceof TieredMergePolicy) {
       TieredMergePolicy tmp = (TieredMergePolicy) mp;
       tmp.setMaxMergeAtOnce(Math.min(5, tmp.getMaxMergeAtOnce()));
       tmp.setSegmentsPerTier(Math.min(5, tmp.getSegmentsPerTier()));
-      tmp.setNoCFSRatio(1.0);
     }
     MergeScheduler ms = w.getConfig().getMergeScheduler();
     if (ms instanceof ConcurrentMergeScheduler) {


[08/15] lucene-solr git commit: reduce some verbosity; fix a silly vInt/int disagreement in the test protocol; reduce indexing rate so translog replay can keep up vs primary crashing itself

Posted by mi...@apache.org.
reduce some verbosity; fix a silly vInt/int disagreement in the test protocol; reduce indexing rate so translog replay can keep up vs primary crashing itself


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 8889469b8bb1445133c39069adfdb5db028dbad1
Parents: 022540e
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 2 06:09:57 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 2 06:09:57 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/index/SegmentInfos.java   |   2 +-
 .../lucene/replicator/nrt/PrimaryNode.java      |   7 +-
 .../lucene/replicator/nrt/ReplicaNode.java      |  26 +++--
 .../lucene/replicator/nrt/NodeProcess.java      |  22 +++-
 .../replicator/nrt/SimplePrimaryNode.java       |  55 ++++++++--
 .../replicator/nrt/SimpleReplicaNode.java       |  27 ++++-
 .../lucene/replicator/nrt/SimpleServer.java     |  20 +++-
 .../lucene/replicator/nrt/SimpleTransLog.java   |   2 +-
 .../replicator/nrt/TestNRTReplication.java      |  86 ++++++++++++++-
 .../nrt/TestStressNRTReplication.java           | 107 ++++++++++++-------
 .../lucene/replicator/nrt/ThreadPumper.java     |   6 +-
 lucene/replicator/test.cmd                      |   2 +-
 12 files changed, 285 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index 2f8d914..046f3ea 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -879,7 +879,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
    *  segments. */
   public void changed() {
     version++;
-    System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
+    //System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
     //new Throwable().printStackTrace(System.out);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index ccd8848..3cff95e 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -104,7 +104,8 @@ public abstract class PrimaryNode extends Node {
 
       IndexSearcher s = mgr.acquire();
       try {
-        message("init: marker hit count: " + s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits);
+        // TODO: this is test code specific!!
+        message("init: marker count: " + s.count(new TermQuery(new Term("marker", "marker"))));
       } finally {
         mgr.release(s);
       }
@@ -135,7 +136,7 @@ public abstract class PrimaryNode extends Node {
     if (result) {
       message("top: opened NRT reader version=" + curInfos.getVersion());
       finishedMergedFiles.removeAll(completedMergeFiles);
-      message("flushAndRefresh:  version=" + curInfos.getVersion() + " completedMergeFiles=" + completedMergeFiles + " finishedMergedFiles=" + finishedMergedFiles);
+      message("flushAndRefresh: version=" + curInfos.getVersion() + " completedMergeFiles=" + completedMergeFiles + " finishedMergedFiles=" + finishedMergedFiles);
     } else {
       message("top: no changes in flushAndRefresh; still version=" + curInfos.getVersion());
     }
@@ -208,6 +209,8 @@ public abstract class PrimaryNode extends Node {
     try {
       searcher = mgr.acquire();
       infos = ((StandardDirectoryReader) searcher.getIndexReader()).getSegmentInfos();
+      // TODO: this is test code specific!!
+      message("setCurrentInfos: marker count: " + searcher.count(new TermQuery(new Term("marker", "marker"))) + " version=" + infos.getVersion() + " searcher=" + searcher);
     } finally {
       if (searcher != null) {
         mgr.release(searcher);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/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 713c6f1..c7af429 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
@@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.SegmentInfos;
@@ -152,8 +153,6 @@ abstract class ReplicaNode extends Node {
       deleter.deleteUnknownFiles(segmentsFileName);
       message("top: done delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
 
-      // nocommit make test where index has all docs deleted (all segments dropped, so 0 segments) and is then replicated
-
       String s = infos.getUserData().get(PRIMARY_GEN_KEY);
       long myPrimaryGen;
       if (s == null) {
@@ -277,6 +276,15 @@ abstract class ReplicaNode extends Node {
       // Finally, we are open for business, since our index now "agrees" with the primary:
       mgr = new SegmentInfosSearcherManager(dir, this, infos, searcherFactory);
 
+      IndexSearcher searcher = mgr.acquire();
+      try {
+        // TODO: this is test specific:
+        int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+        message("top: marker count=" + hitCount + " version=" + ((DirectoryReader) searcher.getIndexReader()).getVersion());
+      } finally {
+        mgr.release(searcher);
+      }
+
       // Must commit after init mgr:
       if (doCommit) {
         // Very important to commit what we just sync'd over, because we removed the pre-existing commit point above if we had to
@@ -287,9 +295,13 @@ abstract class ReplicaNode extends Node {
       message("top: done start");
       state = "idle";
     } catch (Throwable t) {
-      message("exc on start:");
-      t.printStackTrace(System.out);
-      throw new RuntimeException(t);
+      if (t.getMessage().startsWith("replica cannot start") == false) {
+        message("exc on start:");
+        t.printStackTrace(System.out);
+      } else {
+        dir.close();
+      }
+      IOUtils.reThrow(t);
     }
   }
   
@@ -418,7 +430,7 @@ abstract class ReplicaNode extends Node {
     int markerCount;
     IndexSearcher s = mgr.acquire();
     try {
-      markerCount = s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+      markerCount = s.count(new TermQuery(new Term("marker", "marker")));
     } finally {
       mgr.release(s);
     }
@@ -496,7 +508,7 @@ abstract class ReplicaNode extends Node {
     } catch (NodeCommunicationException nce) {
       // E.g. primary could crash/close when we are asking it for the copy state:
       message("top: ignoring communication exception creating CopyJob: " + nce);
-      nce.printStackTrace(System.out);
+      //nce.printStackTrace(System.out);
       if (state.equals("syncing")) {
         state = "idle";
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index 9de2c04..dcbef87 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -19,6 +19,7 @@ package org.apache.lucene.replicator.nrt;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.document.Document;
@@ -47,7 +48,9 @@ class NodeProcess implements Closeable {
 
   volatile boolean isOpen = true;
 
-  public NodeProcess(Process p, int id, int tcpPort, Thread pumper, boolean isPrimary, long initCommitVersion, long initInfosVersion) {
+  final AtomicBoolean nodeIsClosing;
+
+  public NodeProcess(Process p, int id, int tcpPort, Thread pumper, boolean isPrimary, long initCommitVersion, long initInfosVersion, AtomicBoolean nodeIsClosing) {
     this.p = p;
     this.id = id;
     this.tcpPort = tcpPort;
@@ -55,6 +58,7 @@ class NodeProcess implements Closeable {
     this.isPrimary = isPrimary;
     this.initCommitVersion = initCommitVersion;
     this.initInfosVersion = initInfosVersion;
+    this.nodeIsClosing = nodeIsClosing;
     assert initInfosVersion >= initCommitVersion: "initInfosVersion=" + initInfosVersion + " initCommitVersion=" + initCommitVersion;
     lock = new ReentrantLock();
   }
@@ -122,10 +126,11 @@ class NodeProcess implements Closeable {
 
   /** Ask the primary node process to flush.  We send it all currently up replicas so it can notify them about the new NRT point.  Returns the newly
    *  flushed version, or a negative (current) version if there were no changes. */
-  public synchronized long flush() throws IOException {
+  public synchronized long flush(int atLeastMarkerCount) throws IOException {
     assert isPrimary;
     try (Connection c = new Connection(tcpPort)) {
       c.out.writeByte(SimplePrimaryNode.CMD_FLUSH);
+      c.out.writeVInt(atLeastMarkerCount);
       c.flush();
       c.s.shutdownOutput();
       return c.in.readLong();
@@ -218,7 +223,7 @@ class NodeProcess implements Closeable {
 
   public void deleteDocument(Connection c, String docid) throws IOException {
     if (isPrimary == false) {
-      throw new IllegalStateException("only primary can index");
+      throw new IllegalStateException("only primary can delete documents");
     }
     c.out.writeByte(SimplePrimaryNode.CMD_DELETE_DOC);
     c.out.writeString(docid);
@@ -228,11 +233,20 @@ class NodeProcess implements Closeable {
 
   public void deleteAllDocuments(Connection c) throws IOException {
     if (isPrimary == false) {
-      throw new IllegalStateException("only primary can index");
+      throw new IllegalStateException("only primary can delete documents");
     }
     c.out.writeByte(SimplePrimaryNode.CMD_DELETE_ALL_DOCS);
     c.flush();
     c.in.readByte();
   }
+
+  public void forceMerge(Connection c) throws IOException {
+    if (isPrimary == false) {
+      throw new IllegalStateException("only primary can force merge");
+    }
+    c.out.writeByte(SimplePrimaryNode.CMD_FORCE_MERGE);
+    c.flush();
+    c.in.readByte();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
index 0afd1b4..b9ecced 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -53,8 +53,10 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
@@ -272,6 +274,8 @@ class SimplePrimaryNode extends PrimaryNode {
   private void handleFlush(DataInput topIn, DataOutput topOut, BufferedOutputStream bos) throws IOException {
     Thread.currentThread().setName("flush");
 
+    int atLeastMarkerCount = topIn.readVInt();
+
     int[] replicaTCPPorts;
     int[] replicaIDs;
     synchronized (this) {
@@ -284,6 +288,8 @@ class SimplePrimaryNode extends PrimaryNode {
     if (flushAndRefresh()) {
       // Something did get flushed (there were indexing ops since the last flush):
 
+      verifyAtLeastMarkerCount(atLeastMarkerCount, null);
+
       // Tell caller the version before pushing to replicas, so that even if we crash after this, caller will know what version we
       // (possibly) pushed to some replicas.  Alternatively we could make this 2 separate ops?
       long version = getCopyStateVersion();
@@ -295,6 +301,7 @@ class SimplePrimaryNode extends PrimaryNode {
       for(int i=0;i<replicaIDs.length;i++) {
         int replicaID = replicaIDs[i];
         try (Connection c = new Connection(replicaTCPPorts[i])) {
+          message("send NEW_NRT_POINT to R" + replicaID + " at tcpPort=" + replicaTCPPorts[i]);
           c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
           c.out.writeVLong(version);
           c.out.writeInt(tcpPort);
@@ -452,6 +459,10 @@ class SimplePrimaryNode extends PrimaryNode {
         writer.deleteAll();
         out.writeByte((byte) 1);
         bos.flush();
+      } else if (cmd == CMD_FORCE_MERGE) {
+        writer.forceMerge(1);
+        out.writeByte((byte) 1);
+        bos.flush();
       } else if (cmd == CMD_INDEXING_DONE) {
         out.writeByte((byte) 1);
         bos.flush();
@@ -480,7 +491,6 @@ class SimplePrimaryNode extends PrimaryNode {
         throw new IllegalArgumentException("unhandled field name " + name);
       }
     }
-
     writer.addDocument(doc);
   }
 
@@ -537,6 +547,7 @@ class SimplePrimaryNode extends PrimaryNode {
   static final byte CMD_DELETE_DOC = 8;
   static final byte CMD_INDEXING_DONE = 19;
   static final byte CMD_DELETE_ALL_DOCS = 22;
+  static final byte CMD_FORCE_MERGE = 23;
 
   // Sent by replica to primary when replica first starts up, so primary can add it to any warming merges:
   static final byte CMD_NEW_REPLICA = 20;
@@ -617,15 +628,8 @@ class SimplePrimaryNode extends PrimaryNode {
       case CMD_MARKER_SEARCH:
         {
           Thread.currentThread().setName("msearch");
-          IndexSearcher searcher = mgr.acquire();
-          try {
-            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
-            int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
-            out.writeVLong(version);
-            out.writeVInt(hitCount);
-          } finally {
-            mgr.release(searcher);
-          }
+          int expectedAtLeastCount = in.readVInt();
+          verifyAtLeastMarkerCount(expectedAtLeastCount, out);
         }
         continue outer;
 
@@ -706,4 +710,35 @@ class SimplePrimaryNode extends PrimaryNode {
       break;
     }
   }
+
+  private void verifyAtLeastMarkerCount(int expectedAtLeastCount, DataOutput out) throws IOException {
+    IndexSearcher searcher = mgr.acquire();
+    try {
+      long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+      int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+
+      if (hitCount < expectedAtLeastCount) {
+        message("marker search: expectedAtLeastCount=" + expectedAtLeastCount + " but hitCount=" + hitCount);
+        TopDocs hits = searcher.search(new TermQuery(new Term("marker", "marker")), expectedAtLeastCount);
+        List<Integer> seen = new ArrayList<>();
+        for(ScoreDoc hit : hits.scoreDocs) {
+          Document doc = searcher.doc(hit.doc);
+          seen.add(Integer.parseInt(doc.get("docid").substring(1)));
+        }
+        Collections.sort(seen);
+        message("saw markers:");
+        for(int marker : seen) {
+          message("saw m" + marker);
+        }
+        throw new IllegalStateException("at flush: marker count " + hitCount + " but expected at least " + expectedAtLeastCount + " version=" + version);
+      }
+
+      if (out != null) {
+        out.writeVLong(version);
+        out.writeVInt(hitCount);
+      }
+    } finally {
+      mgr.release(searcher);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
index 27a5547..bc8bb03 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -24,18 +24,24 @@ import java.io.InputStream;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
@@ -148,6 +154,7 @@ class SimpleReplicaNode extends ReplicaNode {
 
   /** Handles incoming request to the naive TCP server wrapping this node */
   void handleOneConnection(ServerSocket ss, AtomicBoolean stop, InputStream is, Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
+    //message("one connection: " + socket);
     outer:
     while (true) {
       byte cmd;
@@ -173,6 +180,7 @@ class SimpleReplicaNode extends ReplicaNode {
           long version = in.readVLong();
           Thread.currentThread().setName("recv-" + version);
           curPrimaryTCPPort = in.readInt();
+          message("newNRTPoint primaryTCPPort=" + curPrimaryTCPPort);
           newNRTPoint(version);
         }
         break;
@@ -235,10 +243,26 @@ class SimpleReplicaNode extends ReplicaNode {
       case SimplePrimaryNode.CMD_MARKER_SEARCH:
         {
           Thread.currentThread().setName("msearch");
+          int expectedAtLeastCount = in.readVInt();
           IndexSearcher searcher = mgr.acquire();
           try {
             long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
-            int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+            int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+            if (hitCount < expectedAtLeastCount) {
+              message("marker search: expectedAtLeastCount=" + expectedAtLeastCount + " but hitCount=" + hitCount);
+              TopDocs hits = searcher.search(new TermQuery(new Term("marker", "marker")), expectedAtLeastCount);
+              List<Integer> seen = new ArrayList<>();
+              for(ScoreDoc hit : hits.scoreDocs) {
+                Document doc = searcher.doc(hit.doc);
+                seen.add(Integer.parseInt(doc.get("docid").substring(1)));
+              }
+              Collections.sort(seen);
+              message("saw markers:");
+              for(int marker : seen) {
+                message("saw m" + marker);
+              }
+            }
+
             out.writeVLong(version);
             out.writeVInt(hitCount);
             bos.flush();
@@ -305,7 +329,6 @@ class SimpleReplicaNode extends ReplicaNode {
       default:
         throw new IllegalArgumentException("unrecognized cmd=" + cmd);
       }
-      System.out.println("NOW FLUSH");
       bos.flush();
 
       break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index 5b04721..3fdc45f 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -22,6 +22,7 @@ import java.io.BufferedOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.lang.reflect.Method;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
@@ -42,9 +43,9 @@ import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
 /** Child process with silly naive TCP socket server to handle
@@ -99,8 +100,9 @@ public class SimpleServer extends LuceneTestCase {
         success = true;
       } catch (Throwable t) {
         if (t instanceof SocketException == false && t instanceof NodeCommunicationException == false) {
-          node.message("unexpected exception handling client connection:");
+          node.message("unexpected exception handling client connection; now failing test:");
           t.printStackTrace(System.out);
+          IOUtils.closeWhileHandlingException(ss);
           // Test should fail with this:
           throw new RuntimeException(t);
         } else {
@@ -218,7 +220,7 @@ public class SimpleServer extends LuceneTestCase {
   public void test() throws Exception {
 
     int id = Integer.parseInt(System.getProperty("tests.nrtreplication.nodeid"));
-    Thread.currentThread().setName("init child " + id);
+    Thread.currentThread().setName("main child " + id);
     Path indexPath = Paths.get(System.getProperty("tests.nrtreplication.indexpath"));
     boolean isPrimary = System.getProperty("tests.nrtreplication.isPrimary") != null;
     int primaryTCPPort;
@@ -238,7 +240,7 @@ public class SimpleServer extends LuceneTestCase {
     boolean doFlipBitsDuringCopy = "true".equals(System.getProperty("tests.nrtreplication.doFlipBitsDuringCopy"));
 
     // Create server socket that we listen for incoming requests on:
-    try (final ServerSocket ss = new ServerSocket(0)) {
+    try (final ServerSocket ss = new ServerSocket(0, 0, InetAddress.getLoopbackAddress())) {
 
       int tcpPort = ((InetSocketAddress) ss.getLocalSocketAddress()).getPort();
       System.out.println("\nPORT: " + tcpPort);
@@ -247,7 +249,15 @@ public class SimpleServer extends LuceneTestCase {
         node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy);
         System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
       } else {
-        node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+        try {
+          node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+        } catch (RuntimeException re) {
+          if (re.getMessage().startsWith("replica cannot start")) {
+            // this is "OK": it means MDW's refusal to delete a segments_N commit point means we cannot start:
+            assumeTrue(re.getMessage(), false);
+          }
+          throw re;
+        }
       }
       System.out.println("\nINFOS VERSION: " + node.getCurrentSearchingVersion());
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
index d409ffc..4c11e52 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
@@ -197,7 +197,7 @@ class SimpleTransLog implements Closeable {
     }
     String marker = readNullableString(in);
     if (marker != null) {
-      //System.out.println("xlog: replay marker=" + id);
+      //TestStressNRTReplication.message("xlog: replay marker=" + id);
       doc.add(new StringField("marker", marker, Field.Store.YES));
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 755b234..b2240eb 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -231,8 +231,8 @@ public class TestNRTReplication extends LuceneTestCase {
       replicaC.flush();
       version2 = replicaC.in.readVLong();
       hitCount = replicaC.in.readVInt();
-      if (hitCount == 10) {
-        assertTrue(version2 > version1);
+      if (version2 == primaryVersion1) {
+        assertEquals(10, hitCount);
         // good!
         break;
       }
@@ -269,8 +269,9 @@ public class TestNRTReplication extends LuceneTestCase {
       replicaC.flush();
       version4 = replicaC.in.readVLong();
       hitCount = replicaC.in.readVInt();
-      if (hitCount == 0) {
+      if (version4 == primaryVersion2) {
         assertTrue(version4 > version3);
+        assertEquals(0, hitCount);
         // good!
         break;
       }
@@ -293,7 +294,8 @@ public class TestNRTReplication extends LuceneTestCase {
       replicaC.flush();
       long version5 = replicaC.in.readVLong();
       hitCount = replicaC.in.readVInt();
-      if (hitCount == 10) {
+      if (version5 == primaryVersion3) {
+        assertEquals(10, hitCount);
         assertTrue(version5 > version4);
         // good!
         break;
@@ -308,6 +310,82 @@ public class TestNRTReplication extends LuceneTestCase {
     primary.close();
   }
 
+  public void testReplicateForceMerge() throws Exception {
+
+    Node.globalStartNS = System.nanoTime();
+    childTempDir = createTempDir("child");
+
+    message("change thread name from " + Thread.currentThread().getName());
+    Thread.currentThread().setName("main");
+    
+    Path primaryPath = createTempDir("primary");
+    NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+
+    Path replicaPath = createTempDir("replica");
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+
+    // Tell primary current replicas:
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+      c.out.writeVInt(1);
+      c.out.writeVInt(replica.id);
+      c.out.writeVInt(replica.tcpPort);
+      c.flush();
+      c.in.readByte();
+    }
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    Connection primaryC = new Connection(primary.tcpPort);
+    primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+    for(int i=0;i<10;i++) {
+      Document doc = docs.nextDoc();
+      primary.addOrUpdateDocument(primaryC, doc, false);
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush();
+    assertTrue(primaryVersion1 > 0);
+
+    // Index 10 more docs into primary:
+    for(int i=0;i<10;i++) {
+      Document doc = docs.nextDoc();
+      primary.addOrUpdateDocument(primaryC, doc, false);
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion2 = primary.flush();
+    assertTrue(primaryVersion2 > primaryVersion1);
+
+    primary.forceMerge(primaryC);
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion3 = primary.flush();
+    assertTrue(primaryVersion3 > primaryVersion2);
+
+    Connection replicaC = new Connection(replica.tcpPort);
+
+    // Wait for replica to show the change
+    while (true) {
+      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+      replicaC.flush();
+      long version = replicaC.in.readVLong();
+      int hitCount = replicaC.in.readVInt();
+      if (version == primaryVersion3) {
+        assertEquals(20, hitCount);
+        // good!
+        break;
+      }
+      Thread.sleep(10);
+    }
+
+    replicaC.close();
+    primaryC.close();
+
+    replica.close();
+    primary.close();
+  }
+
   static void message(String message) {
     long now = System.nanoTime();
     System.out.println(String.format(Locale.ROOT,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
index 271c5d2..63ff12a 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
@@ -58,10 +58,21 @@ import org.apache.lucene.util.ThreadInterruptedException;
 
 import com.carrotsearch.randomizedtesting.SeedUtils;
 
+// nocommit why so many "hit SocketException during commit with R0"?
+
+// nocommit why so much time when so many nodes are down
+
+// nocommit indexing is too fast?  (xlog replay fails to finish before primary crashes itself)
+
+// nocommit why all these NodeCommunicationExcs?
+
+// nocommit the sockets are a pita on jvm crashing ...
+
 /*
   TODO
     - fangs
       - sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
+      - graceful primary close
     - why do we do the "rename temp to actual" all at the end...?  what really does that buy us?
     - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
     - test should not print scary exceptions and then succeed!
@@ -137,11 +148,12 @@ public class TestStressNRTReplication extends LuceneTestCase {
   /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
   static final boolean DO_CLOSE_REPLICA = true;
 
+  /** Randomly gracefully close the primary; it will later be restarted and sync itself. */
+  static final boolean DO_CLOSE_PRIMARY = true;
+
   /** If false, all child + parent output is interleaved into single stdout/err */
   static final boolean SEPARATE_CHILD_OUTPUT = false;
 
-  // nocommit DO_CLOSE_PRIMARY?
-
   /** Randomly crash whole cluster and then restart it */
   static final boolean DO_FULL_CLUSTER_CRASH = true;
 
@@ -151,8 +163,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
   /** Set to a non-null value to force exactly that many nodes; else, it's random. */
   static final Integer NUM_NODES = null;
 
-  static final boolean DO_RANDOM_XLOG_REPLAY = false;
-
   final AtomicBoolean failed = new AtomicBoolean();
 
   final AtomicBoolean stop = new AtomicBoolean();
@@ -174,6 +184,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
   Path transLogPath;
   SimpleTransLog transLog;
   final AtomicInteger markerUpto = new AtomicInteger();
+  final AtomicInteger markerID = new AtomicInteger();
 
   /** Maps searcher version to how many hits the query body:the matched. */
   final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
@@ -201,7 +212,11 @@ public class TestStressNRTReplication extends LuceneTestCase {
 
     // Silly bootstrapping:
     versionToTransLogLocation.put(0L, 0L);
-    versionToTransLogLocation.put(1L, 0L);
+
+    // nocommit why also 1?
+    //versionToTransLogLocation.put(1L, 0L);
+
+    versionToMarker.put(0L, 0);
 
     int numNodes;
 
@@ -268,23 +283,24 @@ public class TestStressNRTReplication extends LuceneTestCase {
       // Wait a bit:
       Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
       if (primary != null && random().nextBoolean()) {
-        message("top: now flush primary");
         NodeProcess curPrimary = primary;
         if (curPrimary != null) {
 
           // Save these before we start flush:
           long nextTransLogLoc = transLog.getNextLocation();
           int markerUptoSav = markerUpto.get();
+          message("top: now flush primary; at least marker count=" + markerUptoSav);
 
           long result;
           try {
-            result = primary.flush();
+            result = primary.flush(markerUptoSav);
           } catch (Throwable t) {
             message("top: flush failed; skipping: " + t.getMessage());
             result = -1;
           }
           if (result > 0) {
             // There were changes
+            message("top: flush finished with changed; new primary version=" + result);
             lastPrimaryVersion = result;
             addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
             addVersionMarker(lastPrimaryVersion, markerUptoSav);
@@ -316,7 +332,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
 
       {
         NodeProcess node = nodes[random().nextInt(nodes.length)];
-        if (node != null) {
+        if (node != null && node.nodeIsClosing.get() == false) {
           // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
           // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
           message("top: now commit node=" + node);
@@ -452,27 +468,33 @@ public class TestStressNRTReplication extends LuceneTestCase {
     addVersionMarker(newPrimary.initInfosVersion, markerCount);
 
     assert newPrimary.initInfosVersion >= lastPrimaryVersion;
-    message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
+    message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion + "; startup marker count " + markerCount);
     lastPrimaryVersion = newPrimary.initInfosVersion;
 
-    // Publish new primary, before replaying xlog.  This means other indexing ops can come in at the same time as we catch up indexing
-    // previous ops.  Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
-    // translog) concurrently with new incoming ops.
-    nodes[id] = newPrimary;
-    primary = newPrimary;
-
-    sendReplicasToPrimary();
-
     long nextTransLogLoc = transLog.getNextLocation();
-    int nextMarkerUpto = markerUpto.get();
-    message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
+    long t0 = System.nanoTime();
+    message("top: start translog replay " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
     try {
       transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
     } catch (IOException ioe) {
+      // nocommit what if primary node is still running here, and we failed for some other reason?
       message("top: replay xlog failed; abort");
       return;
     }
-    message("top: done replay trans log");
+
+    long t1 = System.nanoTime();
+    message("top: done translog replay; took " + ((t1 - t0)/1000000.0) + " msec; now publish primary");
+
+    // Publish new primary only after translog has succeeded in replaying; this is important, for this test anyway, so we keep a "linear"
+    // history so enforcing marker counts is correct.  E.g., if we publish first and replay translog concurrently with incoming ops, then
+    // a primary commit that happens while translog is still replaying will incorrectly record the translog loc into the commit user data
+    // when in fact that commit did NOT reflect all prior ops.  So if we crash and start up again from that commit point, we are missing
+    // ops.
+    nodes[id] = newPrimary;
+    primary = newPrimary;
+
+    sendReplicasToPrimary();
+
   }
 
   /** Launches a child "server" (separate JVM), which is either primary or replica node */
@@ -506,6 +528,9 @@ public class TestStressNRTReplication extends LuceneTestCase {
       if (DO_CRASH_PRIMARY) {
         cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
       }
+      if (DO_CLOSE_PRIMARY) {
+        cmd.add("-Dtests.nrtreplication.doRandomClose=true");
+      }
     } else {
       if (DO_CRASH_REPLICA) {
         cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
@@ -544,7 +569,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
       childLog = null;
     }
 
-    message("child process command: " + cmd);
+    //message("child process command: " + cmd);
     ProcessBuilder pb = new ProcessBuilder(cmd);
     pb.redirectErrorStream(true);
 
@@ -577,6 +602,10 @@ public class TestStressNRTReplication extends LuceneTestCase {
           throw new RuntimeException(ie);
         }
         message("exit value=" + p.exitValue());
+        if (p.exitValue() == 0) {
+          message("zero exit status; assuming failed to remove segments_N; skipping");
+          return null;
+        }
 
         // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
         if (isPrimary == false) {
@@ -586,8 +615,9 @@ public class TestStressNRTReplication extends LuceneTestCase {
             message("failed to remove segments_N; skipping");
             return null;
           }
-          for(int i=0;i<10;i++) {
-            if (primaryGen != myPrimaryGen || primary == null) {
+          for(int i=0;i<100;i++) {
+            NodeProcess primary2 = primary;
+            if (primaryGen != myPrimaryGen || primary2 == null || primary2.nodeIsClosing.get()) {
               // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
               message("primary crashed/closed while replica R" + id + " tried to start; skipping");
               return null;
@@ -634,6 +664,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
     }
 
     final boolean finalWillCrash = willCrash;
+    final AtomicBoolean nodeIsClosing = new AtomicBoolean();
 
     // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
     Thread pumper = ThreadPumper.start(
@@ -669,11 +700,11 @@ public class TestStressNRTReplication extends LuceneTestCase {
                                            }
                                            nodeClosed(id);
                                          }
-                                       }, r, System.out, childLog);
+                                       }, r, System.out, childLog, nodeIsClosing);
     pumper.setName("pump" + id);
 
     message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
-    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion, nodeIsClosing);
   }
 
   private void nodeClosed(int id) {
@@ -754,7 +785,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
           message("top: restarter cycle");
 
           // Randomly crash full cluster:
-          if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
+          if (DO_FULL_CLUSTER_CRASH && random().nextInt(500) == 17) {
             message("top: full cluster crash");
             for(int i=0;i<nodes.length;i++) {
               if (starting[i]) {
@@ -954,12 +985,15 @@ public class TestStressNRTReplication extends LuceneTestCase {
             continue;
           }
 
+          // nocommit not anymore?
           // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
           Integer expectedAtLeastHitCount = versionToMarker.get(version);
+          assertNotNull("version=" + version, expectedAtLeastHitCount);
 
           if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
             try {
               c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
+              c.out.writeVInt(expectedAtLeastHitCount);
               c.flush();
               while (c.sockIn.available() == 0) {
                 if (stop.get()) {
@@ -1064,13 +1098,15 @@ public class TestStressNRTReplication extends LuceneTestCase {
             if (random().nextInt(10) == 7) {
               // We use the marker docs to check for data loss in search thread:
               Document doc = new Document();
-              int id = markerUpto.getAndIncrement();
+              int id = markerID.getAndIncrement();
               String idString = "m"+id;
               doc.add(newStringField("docid", idString, Field.Store.YES));
               doc.add(newStringField("marker", "marker", Field.Store.YES));
               curPrimary.addOrUpdateDocument(c, doc, false);
               transLog.addDocument(idString, doc);
-              message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
+              // Only increment after primary replies:
+              markerUpto.getAndIncrement();
+              //message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
             }
 
             if (docCount > 0 && random().nextDouble() < updatePct) {
@@ -1094,14 +1130,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
               ((Field) doc.getField("docid")).setStringValue(idString);
               curPrimary.addOrUpdateDocument(c, doc, false);
               transLog.addDocument(idString, doc);
-
-              if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
-                long curLoc = transLog.getNextLocation();
-                // randomly replay chunks of translog just to test replay:
-                message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
-                transLog.replay(curPrimary, lastTransLogLoc, curLoc);
-                lastTransLogLoc = curLoc;
-              }
             }
           } catch (IOException se) {
             // Assume primary crashed
@@ -1115,12 +1143,13 @@ public class TestStressNRTReplication extends LuceneTestCase {
           }
 
           if (random().nextInt(sleepChance) == 0) {
-            Thread.sleep(1);
+            Thread.sleep(10);
           }
 
           if (random().nextInt(100) == 17) {
-            System.out.println("Indexer: now pause for a bit...");
-            Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
+            int pauseMS = TestUtil.nextInt(random(), 500, 2000);
+            System.out.println("Indexer: now pause for " + pauseMS + " msec...");
+            Thread.sleep(pauseMS);
             System.out.println("Indexer: done pause for a bit...");
           }
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
index 6ddb777..d74e170 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
@@ -21,11 +21,12 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.Writer;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.regex.Pattern;
 
 /** A pipe thread. It'd be nice to reuse guava's implementation for this... */
 class ThreadPumper {
-  public static Thread start(final Runnable onExit, final BufferedReader from, final PrintStream to, final Writer toFile) {
+  public static Thread start(final Runnable onExit, final BufferedReader from, final PrintStream to, final Writer toFile, final AtomicBoolean nodeClosing) {
     Thread t = new Thread() {
         @Override
         public void run() {
@@ -43,6 +44,9 @@ class ThreadPumper {
               } else {
                 TestNRTReplication.message(line);
               }
+              if (line.contains("now force close server socket after")) {
+                nodeClosing.set(true);
+              }
             }
             // Sub-process finished
           } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/test.cmd
----------------------------------------------------------------------
diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd
index 14e3bd2..c38fc96 100644
--- a/lucene/replicator/test.cmd
+++ b/lucene/replicator/test.cmd
@@ -1 +1 @@
-python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs  TestNRTReplication -jvms 1 -mult 4 -nightly
+python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 1 -mult 4 -nightly


[03/15] lucene-solr git commit: current patch

Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
new file mode 100644
index 0000000..742b19f
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
@@ -0,0 +1,213 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ReferenceManager;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.StringHelper;
+
+/** Common base class for {@link PrimaryNode} and {@link ReplicaNode}. */
+
+abstract class Node implements Closeable {
+
+  static boolean VERBOSE_FILES = true;
+  static boolean VERBOSE_CONNECTIONS = false;
+
+  // Keys we store into IndexWriter's commit user data:
+
+  /** Key to store the primary gen in the commit data, which increments every time we promote a new primary, so replicas can detect when the
+   *  primary they were talking to is changed */
+  public static String PRIMARY_GEN_KEY = "__primaryGen";
+
+  /** Key to store the version in the commit data, which increments every time we open a new NRT reader */
+  public static String VERSION_KEY = "__version";
+
+  /** Compact ordinal for this node */
+  protected final int id;
+
+  protected final Directory dir;
+
+  protected final SearcherFactory searcherFactory;
+  
+  // Tracks NRT readers, opened from IW (primary) or opened from replicated SegmentInfos pulled across the wire (replica):
+  protected ReferenceManager<IndexSearcher> mgr;
+
+  /** Startup time of original test, carefully propogated to all nodes to produce consistent "seconds since start time" in messages */
+  public static long globalStartNS;
+
+  /** When this node was started */
+  public static final long localStartNS = System.nanoTime();
+
+  // public static final long globalStartNS;
+
+  // For debugging:
+  volatile String state = "idle";
+
+  /** File metadata for last sync that succeeded; we use this as a cache */
+  protected volatile Map<String,FileMetaData> lastFileMetaData;
+
+  public Node(int id, Directory dir, SearcherFactory searcherFactory) {
+    this.id = id;
+    this.dir = dir;
+    this.searcherFactory = searcherFactory;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(id=" + id + ")";
+  }
+
+  public abstract void commit() throws IOException;
+
+  public static void nodeMessage(String message) {
+    long now = System.nanoTime();
+    System.out.println(String.format(Locale.ROOT,
+                                     "%5.3fs %5.1fs:           [%11s] %s",
+                                     (now-globalStartNS)/1000000000.,
+                                     (now-localStartNS)/1000000000.,
+                                     Thread.currentThread().getName(),
+                                     message));
+
+  }
+
+  public static void nodeMessage(int id, String message) {
+    long now = System.nanoTime();
+    System.out.println(String.format(Locale.ROOT,
+                                     "%5.3fs %5.1fs:         N%d [%11s] %s",
+                                     (now-globalStartNS)/1000000000.,
+                                     (now-localStartNS)/1000000000.,
+                                     id,
+                                     Thread.currentThread().getName(),
+                                     message));
+
+  }
+
+  protected void message(String message) {
+    long now = System.nanoTime();
+    System.out.println(String.format(Locale.ROOT,
+                                     "%5.3fs %5.1fs: %7s %2s [%11s] %s",
+                                     (now-globalStartNS)/1000000000.,
+                                     (now-localStartNS)/1000000000.,
+                                     state, name(),
+                                     Thread.currentThread().getName(), message));
+  }
+
+  public String name() {
+    char mode = this instanceof PrimaryNode ? 'P' : 'R';
+    return mode + Integer.toString(id);
+  }
+
+  public abstract boolean isClosed();
+
+  public long getCurrentSearchingVersion() throws IOException {
+    IndexSearcher searcher = mgr.acquire();
+    try {
+      return ((DirectoryReader) searcher.getIndexReader()).getVersion();
+    } finally {
+      mgr.release(searcher);
+    }
+  }
+
+  public static String bytesToString(long bytes) {
+    if (bytes < 1024) {
+      return bytes + " b";
+    } else if (bytes < 1024 * 1024) {
+      return String.format(Locale.ROOT, "%.1f KB", bytes/1024.);
+    } else if (bytes < 1024 * 1024 * 1024) {
+      return String.format(Locale.ROOT, "%.1f MB", bytes/1024./1024.);
+    } else {
+      return String.format(Locale.ROOT, "%.1f GB", bytes/1024./1024./1024.);
+    }
+  }
+
+  /** Opens the specified file, reads its identifying information, including file length, full index header (includes the unique segment
+   *  ID) and the full footer (includes checksum), and returns the resulting {@link FileMetaData}.
+   *
+   *  <p>This returns null, logging a message, if there are any problems (the file does not exist, is corrupt, truncated, etc.).</p> */
+  public FileMetaData readLocalFileMetaData(String fileName) throws IOException {
+
+    Map<String,FileMetaData> cache = lastFileMetaData;
+    FileMetaData result;
+    if (cache != null) {
+      // We may already have this file cached from the last NRT point:
+      result = cache.get(fileName);
+    } else {
+      result = null;
+    }
+
+    if (result == null) {
+      // Pull from the filesystem
+      long checksum;
+      long length;
+      byte[] header;
+      byte[] footer;
+      try (IndexInput in = dir.openInput(fileName, IOContext.DEFAULT)) {
+          try {
+            length = in.length();
+            header = CodecUtil.readIndexHeader(in);
+            footer = CodecUtil.readFooter(in);
+            checksum = CodecUtil.retrieveChecksum(in);
+          } catch (EOFException | CorruptIndexException cie) {
+            // File exists but is busted: we must copy it.  This happens when node had crashed, corrupting an un-fsync'd file.  On init we try
+            // to delete such unreferenced files, but virus checker can block that, leaving this bad file.
+            if (VERBOSE_FILES) {
+              message("file " + fileName + ": will copy [existing file is corrupt]");
+            }
+            return null;
+          }
+          if (VERBOSE_FILES) {
+            message("file " + fileName + " has length=" + bytesToString(length));
+          }
+        } catch (FileNotFoundException | NoSuchFileException e) {
+        if (VERBOSE_FILES) {
+          message("file " + fileName + ": will copy [file does not exist]");
+        }
+        return null;
+      }
+
+      // NOTE: checksum is redundant w/ footer, but we break it out separately because when the bits cross the wire we need direct access to
+      // checksum when copying to catch bit flips:
+      result = new FileMetaData(header, footer, length, checksum);
+    }
+
+    return result;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
new file mode 100644
index 0000000..67a9d0a
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
@@ -0,0 +1,26 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class NodeCommunicationException extends RuntimeException {
+  public NodeCommunicationException(String when, Throwable cause) {
+    super(when);
+    assert cause != null;
+    initCause(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
new file mode 100644
index 0000000..1918ede
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
@@ -0,0 +1,80 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** A merged segment warmer that pre-copies the merged segment out to
+ *  replicas before primary cuts over to the merged segment.  This
+ *  ensures that NRT reopen time on replicas is only in proportion to
+ *  flushed segment sizes, not merged segments. */
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.replicator.nrt.CopyJob.OnceDone;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+// TODO: or ... replica node can do merging locally?  tricky to keep things in sync, when one node merges more slowly than others...
+
+class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
+
+  private final PrimaryNode primary;
+
+  public PreCopyMergedSegmentWarmer(PrimaryNode primary) {
+    this.primary = primary;
+  }
+
+  @Override
+  public void warm(LeafReader reader) throws IOException {
+    long startNS = System.nanoTime();
+    final SegmentCommitInfo info = ((SegmentReader) reader).getSegmentInfo();
+    //System.out.println("TEST: warm merged segment files " + info);
+    Map<String,FileMetaData> filesMetaData = new HashMap<>();
+    for(String fileName : info.files()) {
+      FileMetaData metaData = primary.readLocalFileMetaData(fileName);
+      assert metaData != null;
+      assert filesMetaData.containsKey(fileName) == false;
+      filesMetaData.put(fileName, metaData);
+    }
+
+    // nocommit if one replica is very slow then it dos's all other replicas?
+
+    primary.preCopyMergedSegmentFiles(info, filesMetaData);
+    primary.message(String.format(Locale.ROOT, "top: done warm merge " + info + ": took %.3f sec, %.1f MB", (System.nanoTime()-startNS)/1000000000., info.sizeInBytes()/1024/1024.));
+    primary.finishedMergedFiles.addAll(filesMetaData.keySet());
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
new file mode 100644
index 0000000..183f16f
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -0,0 +1,316 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.StandardDirectoryReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RAMFile;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.PrintStreamInfoStream;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/*
+ * This just asks IndexWriter to open new NRT reader, in order to publish a new NRT point.  This could be improved, if we separated out 1)
+ * nrt flush (and incRef the SIS) from 2) opening a new reader, but this is tricky with IW's concurrency, and it would also be hard-ish to share
+ * IW's reader pool with our searcher manager.  So we do the simpler solution now, but that adds some unecessary latency to NRT refresh on
+ * replicas since step 2) could otherwise be done concurrently with replicas copying files over.
+ */
+
+/** Node that holds an IndexWriter, indexing documents into its local index. */
+public abstract class PrimaryNode extends Node {
+
+  // Current NRT segment infos, incRef'd with IndexWriter.deleter:
+  private SegmentInfos curInfos;
+
+  final IndexWriter writer;
+
+  // IncRef'd state of the last published NRT point; when a replica comes asking, we give it this as the current NRT point:
+  private CopyState copyState;
+
+  protected final long primaryGen;
+
+  /** Contains merged segments that have been copied to all running replicas (as of when that merge started warming). */
+  final Set<String> finishedMergedFiles = Collections.synchronizedSet(new HashSet<String>());
+
+  private final AtomicInteger copyingCount = new AtomicInteger();
+
+  public PrimaryNode(IndexWriter writer, int id, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException {
+    super(id, writer.getDirectory(), searcherFactory);
+    message("top: now init primary");
+    this.writer = writer;
+    this.primaryGen = primaryGen;
+
+    try {
+      // So that when primary node's IndexWriter finishes a merge, but before it cuts over to the merged segment,
+      // it copies it out to the replicas.  This ensures the whole system's NRT latency remains low even when a
+      // large merge completes:
+      writer.getConfig().setMergedSegmentWarmer(new PreCopyMergedSegmentWarmer(this));
+
+      message("IWC:\n" + writer.getConfig());
+      message("dir:\n" + writer.getDirectory());
+      message("commitData: " + writer.getCommitData());
+
+      // Record our primaryGen in the userData, and set initial version to 0:
+      Map<String,String> commitData = new HashMap<>(writer.getCommitData());
+      commitData.put(PRIMARY_GEN_KEY, Long.toString(primaryGen));
+      if (commitData.get(VERSION_KEY) == null) {
+        commitData.put(VERSION_KEY, "0");
+        message("add initial commitData version=0");
+      } else {
+        message("keep current commitData version=" + commitData.get(VERSION_KEY));
+      }
+      writer.setCommitData(commitData, false);
+
+      // We forcefully advance the SIS version to an unused future version.  This is necessary if the previous primary crashed and we are
+      // starting up on an "older" index, else versions can be illegally reused but show different results:
+      if (forcePrimaryVersion != -1) {
+        message("now forcePrimaryVersion to version=" + forcePrimaryVersion);
+        writer.advanceSegmentInfosVersion(forcePrimaryVersion);
+      }
+
+      mgr = new SearcherManager(writer, true, true, searcherFactory);
+      setCurrentInfos(Collections.<String>emptySet());
+      message("init: infos version=" + curInfos.getVersion());
+
+      IndexSearcher s = mgr.acquire();
+      try {
+        message("init: marker hit count: " + s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits);
+      } finally {
+        mgr.release(s);
+      }
+
+    } catch (Throwable t) {
+      message("init: exception");
+      t.printStackTrace(System.out);
+      throw new RuntimeException(t);
+    }
+  }
+
+  // TODO: in the future, we should separate "flush" (returns an incRef'd SegmentInfos) from "refresh" (open new NRT reader from
+  // IndexWriter) so that the latter can be done concurrently while copying files out to replicas, minimizing the refresh time from the
+  // replicas.  But fixing this is tricky because e.g. IndexWriter may complete a big merge just after returning the incRef'd SegmentInfos
+  // and before we can open a new reader causing us to close the just-merged readers only to then open them again from the (now stale)
+  // SegmentInfos.  To fix this "properly" I think IW.inc/decRefDeleter must also incread the ReaderPool entry
+
+  /** Flush all index operations to disk and opens a new near-real-time reader.
+   *  new NRT point, to make the changes visible to searching.  Returns true if there were changes. */
+  public boolean flushAndRefresh() throws IOException {
+    message("top: now flushAndRefresh");
+    Set<String> completedMergeFiles;
+    synchronized(finishedMergedFiles) {
+      completedMergeFiles = Collections.unmodifiableSet(new HashSet<>(finishedMergedFiles));
+    }
+    mgr.maybeRefreshBlocking();
+    boolean result = setCurrentInfos(completedMergeFiles);
+    if (result) {
+      message("top: opened NRT reader version=" + curInfos.getVersion());
+      finishedMergedFiles.removeAll(completedMergeFiles);
+      message("flushAndRefresh:  version=" + curInfos.getVersion() + " completedMergeFiles=" + completedMergeFiles + " finishedMergedFiles=" + finishedMergedFiles);
+    } else {
+      message("top: no changes in flushAndRefresh; still version=" + curInfos.getVersion());
+    }
+    return result;
+  }
+
+  public long getCopyStateVersion() {
+    return copyState.version;
+  }
+
+  public synchronized long getLastCommitVersion() {
+    String s = curInfos.getUserData().get(VERSION_KEY);
+    // In ctor we always install an initial version:
+    assert s != null;
+    return Long.parseLong(s);
+  }
+
+  @Override
+  public void commit() throws IOException {
+    Map<String,String> commitData = new HashMap<>();
+    commitData.put(PRIMARY_GEN_KEY, Long.toString(primaryGen));
+    // TODO (opto): it's a bit wasteful that we put "last refresh" version here, not the actual version we are committing, because it means
+    // on xlog replay we are replaying more ops than necessary.
+    commitData.put(VERSION_KEY, Long.toString(copyState.version));
+    message("top: commit commitData=" + commitData);
+    // nocommit this is now an NRT-visible change!  make test where nothing is indexing and confirm we don't do silly commit + refresh loop forever!
+    writer.setCommitData(commitData, false);
+    writer.commit();
+  }
+
+  /** IncRef the current CopyState and return it */
+  public synchronized CopyState getCopyState() throws IOException {
+    ensureOpen(false);
+    //message("top: getCopyState replicaID=" + replicaID + " replicaNodeID=" + replicaNodeID + " version=" + curInfos.getVersion() + " infos=" + curInfos.toString());
+    assert curInfos == copyState.infos;
+    writer.incRefDeleter(copyState.infos);
+    int count = copyingCount.incrementAndGet();
+    assert count > 0;
+    return copyState;
+  }
+
+  /** Called once replica is done (or failed) copying an NRT point */
+  public void releaseCopyState(CopyState copyState) throws IOException {
+    //message("top: releaseCopyState version=" + copyState.version);
+    assert copyState.infos != null;
+    writer.decRefDeleter(copyState.infos);
+    int count = copyingCount.decrementAndGet();
+    assert count >= 0;
+  }
+
+  @Override
+  public boolean isClosed() {
+    return isClosed(false);
+  }
+
+  boolean isClosed(boolean allowClosing) {
+    return "closed".equals(state) || (allowClosing == false && "closing".equals(state));
+  }
+
+  private void ensureOpen(boolean allowClosing) {
+    if (isClosed(allowClosing)) {
+      throw new AlreadyClosedException(state);
+    }
+  }
+
+  /** Steals incoming infos refCount; returns true if there were changes. */
+  private synchronized boolean setCurrentInfos(Set<String> completedMergeFiles) throws IOException {
+
+    IndexSearcher searcher = null;
+    SegmentInfos infos;
+    try {
+      searcher = mgr.acquire();
+      infos = ((StandardDirectoryReader) searcher.getIndexReader()).getSegmentInfos();
+    } finally {
+      if (searcher != null) {
+        mgr.release(searcher);
+      }
+    }
+    if (curInfos != null && infos.getVersion() == curInfos.getVersion()) {
+      // no change
+      message("top: skip switch to infos: version=" + infos.getVersion() + " is unchanged: " + infos.toString());
+      return false;
+    }
+
+    SegmentInfos oldInfos = curInfos;
+    writer.incRefDeleter(infos);
+    curInfos = infos;
+    if (oldInfos != null) {
+      writer.decRefDeleter(oldInfos);
+    }
+
+    message("top: switch to infos=" + infos.toString() + " version=" + infos.getVersion());
+
+    // Serialize the SegmentInfos:
+    RAMOutputStream out = new RAMOutputStream(new RAMFile(), true);
+    infos.write(dir, out);
+    byte[] infosBytes = new byte[(int) out.getFilePointer()];
+    out.writeTo(infosBytes, 0);
+
+    Map<String,FileMetaData> filesMetaData = new HashMap<String,FileMetaData>();
+    for(SegmentCommitInfo info : infos) {
+      for(String fileName : info.files()) {
+        FileMetaData metaData = readLocalFileMetaData(fileName);
+        // NOTE: we hold a refCount on this infos, so this file better exist:
+        assert metaData != null;
+        assert filesMetaData.containsKey(fileName) == false;
+        filesMetaData.put(fileName, metaData);
+      }
+    }
+
+    lastFileMetaData = Collections.unmodifiableMap(filesMetaData);
+
+    message("top: set copyState primaryGen=" + primaryGen + " version=" + infos.getVersion() + " files=" + filesMetaData.keySet());
+    copyState = new CopyState(lastFileMetaData,
+                              infos.getVersion(), infos.getGeneration(), infosBytes, completedMergeFiles,
+                              primaryGen, curInfos);
+    return true;
+  }
+
+  private synchronized void waitForAllRemotesToClose() throws IOException {
+
+    // Wait for replicas to finish or crash:
+    while (true) {
+      int count = copyingCount.get();
+      if (count == 0) {
+        return;
+      }
+      message("pendingCopies: " + count);
+
+      try {
+        wait(10);
+      } catch (InterruptedException ie) {
+        throw new ThreadInterruptedException(ie);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    state = "closing";
+    message("top: close primary");
+
+    synchronized (this) {
+      waitForAllRemotesToClose();
+      if (curInfos != null) {
+        writer.decRefDeleter(curInfos);
+        curInfos = null;
+      }
+    }
+
+    mgr.close();
+
+    writer.rollback();
+    dir.close();
+
+    state = "closed";
+  }
+
+  /** Called when a merge has finished, but before IW switches to the merged segment */
+  protected abstract void preCopyMergedSegmentFiles(SegmentCommitInfo info, Map<String,FileMetaData> files) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
new file mode 100644
index 0000000..005f938
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
@@ -0,0 +1,218 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+// TODO: can we factor/share with IFD: this is doing exactly the same thing, but on the replica side
+
+// TODO: once LUCENE-6835 is in, this class becomes a lot simpler?
+
+class ReplicaFileDeleter {
+  private final Map<String,Integer> refCounts = new HashMap<String,Integer>();
+  private final Set<String> pending = new HashSet<String>();
+  private final Directory dir;
+  private final Node node;
+
+  public ReplicaFileDeleter(Node node, Directory dir) throws IOException {
+    this.dir = dir;
+    this.node = node;
+  }
+
+  /** Used only by asserts: returns true if the file exists
+   *  (can be opened), false if it cannot be opened, and
+   *  (unlike Java's File.exists) throws IOException if
+   *  there's some unexpected error. */
+  static boolean slowFileExists(Directory dir, String fileName) throws IOException {
+    try {
+      dir.openInput(fileName, IOContext.DEFAULT).close();
+      return true;
+    } catch (NoSuchFileException | FileNotFoundException e) {
+      return false;
+    }
+  }
+
+  public synchronized void incRef(Collection<String> fileNames) throws IOException {
+    for(String fileName : fileNames) {
+
+      if (pending.contains(fileName)) {
+        throw new IllegalStateException("cannot incRef file \"" + fileName + "\": it is pending delete");
+      }
+
+      assert slowFileExists(dir, fileName): "file " + fileName + " does not exist!";
+
+      Integer curCount = refCounts.get(fileName);
+      if (curCount == null) {
+        refCounts.put(fileName, 1);
+      } else {
+        refCounts.put(fileName, curCount.intValue() + 1);
+      }
+    }
+  }
+
+  public synchronized void decRef(Collection<String> fileNames) {
+    // We don't delete the files immediately when their RC drops to 0; instead, we add to the pending set, and then call deletePending in
+    // the end:
+    for(String fileName : fileNames) {
+      Integer curCount = refCounts.get(fileName);
+      assert curCount != null: "fileName=" + fileName;
+      assert curCount.intValue() > 0;
+      if (curCount.intValue() == 1) {
+        refCounts.remove(fileName);
+        pending.add(fileName);
+      } else {
+        refCounts.put(fileName, curCount.intValue() - 1);
+      }
+    }
+
+    deletePending();
+
+    // TODO: this local IR could incRef files here, like we do now with IW ... then we can assert this again:
+
+    // we can't assert this, e.g a search can be running when we switch to a new NRT point, holding a previous IndexReader still open for
+    // a bit:
+    /*
+    // We should never attempt deletion of a still-open file:
+    Set<String> delOpen = ((MockDirectoryWrapper) dir).getOpenDeletedFiles();
+    if (delOpen.isEmpty() == false) {
+      node.message("fail: we tried to delete these still-open files: " + delOpen);
+      throw new AssertionError("we tried to delete these still-open files: " + delOpen);
+    }
+    */
+  }
+
+  private synchronized boolean delete(String fileName) {
+    try {
+      if (Node.VERBOSE_FILES) {
+        node.message("file " + fileName + ": now delete");
+      }
+      dir.deleteFile(fileName);
+      pending.remove(fileName);
+      return true;
+    } catch (FileNotFoundException|NoSuchFileException missing) {
+      // This should never happen: we should only be asked to track files that do exist
+      node.message("file " + fileName + ": delete failed: " + missing);
+      throw new IllegalStateException("file " + fileName + ": we attempted delete but the file does not exist?", missing);
+    } catch (IOException ioe) {
+      if (Node.VERBOSE_FILES) {
+        node.message("file " + fileName + ": delete failed: " + ioe + "; will retry later");
+      }
+      pending.add(fileName);
+      return false;
+    }
+  }
+
+  public synchronized Integer getRefCount(String fileName) {
+    return refCounts.get(fileName);
+  }
+
+  public synchronized boolean isPending(String fileName) {
+    return pending.contains(fileName);
+  }
+
+  public synchronized void deletePending() {
+    if (Node.VERBOSE_FILES) {
+      node.message("now deletePending: " + pending.size() + " files to try: " + pending);
+    }
+
+    // Clone the set because it will change as we iterate:
+    List<String> toDelete = new ArrayList<>(pending);
+
+    // First pass: delete any segments_N files.  We do these first to be certain stale commit points are removed
+    // before we remove any files they reference.  If any delete of segments_N fails, we leave all other files
+    // undeleted so index is never in a corrupt state:
+    for (String fileName : toDelete) {
+      Integer rc = refCounts.get(fileName);
+      if (rc != null && rc > 0) {
+        // Should never happen!  This means we are about to pending-delete a referenced index file
+        throw new IllegalStateException("file \"" + fileName + "\" is in pending delete set but has non-zero refCount=" + rc);
+      } else if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+        if (delete(fileName) == false) {
+          if (Node.VERBOSE_FILES) {
+            node.message("failed to remove commit point \"" + fileName + "\"; skipping deletion of all other pending files");
+          }
+          return;
+        }
+      }
+    }
+
+    // Only delete other files if we were able to remove the segments_N files; this way we never
+    // leave a corrupt commit in the index even in the presense of virus checkers:
+    for(String fileName : toDelete) {
+      if (fileName.startsWith(IndexFileNames.SEGMENTS) == false) {
+        delete(fileName);
+      }
+    }
+
+    Set<String> copy = new HashSet<String>(pending);
+    pending.clear();
+    for(String fileName : copy) {
+      delete(fileName);
+    }
+  }
+
+  /** Necessary in case we had tried to delete this fileName before, it failed, but then it was later overwritten (because primary changed
+   *  and new primary didn't know this segment name had been previously attempted) and now has > 0 refCount */
+  public synchronized void clearPending(Collection<String> fileNames) {
+    for(String fileName : fileNames) {
+      if (pending.remove(fileName)) {
+        node.message("file " + fileName + ": deleter.clearPending now clear from pending");
+      }
+    }
+  }
+
+  public synchronized void deleteIfNoRef(String fileName) {
+    if (refCounts.containsKey(fileName) == false) {
+      deleteNewFile(fileName);
+    }
+  }
+
+  public synchronized void deleteNewFile(String fileName) {
+    delete(fileName);
+  }
+
+  public synchronized Set<String> getPending() {
+    return new HashSet<String>(pending);
+  }
+
+  public synchronized void deleteUnknownFiles(String segmentsFileName) throws IOException {
+    for(String fileName : dir.listAll()) {
+      if (refCounts.containsKey(fileName) == false &&
+          fileName.equals("write.lock") == false &&
+          fileName.equals(segmentsFileName) == false) {
+        node.message("will delete unknown file \"" + fileName + "\"");
+        pending.add(fileName);
+      }
+    }
+
+    deletePending();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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
new file mode 100644
index 0000000..af142d5
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -0,0 +1,772 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteArrayIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Lock;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */
+
+abstract class ReplicaNode extends Node {
+
+  ReplicaFileDeleter deleter;
+
+  /** IncRef'd files in the current commit point: */
+  private final Collection<String> lastCommitFiles = new HashSet<>();
+
+  /** IncRef'd files in the current NRT point: */
+  protected final Collection<String> lastNRTFiles = new HashSet<>();
+
+  /** Currently running merge pre-copy jobs */
+  protected final Set<CopyJob> mergeCopyJobs = Collections.synchronizedSet(new HashSet<>());
+
+  /** Non-null when we are currently copying files from a new NRT point: */
+  protected CopyJob curNRTCopy;
+
+  /** We hold this to ensure an external IndexWriter cannot also open on our directory: */
+  private final Lock writeFileLock;
+
+  /** Merged segment files that we pre-copied, but have not yet made visible in a new NRT point. */
+  final Set<String> pendingMergeFiles = Collections.synchronizedSet(new HashSet<String>());
+
+  /** Primary gen last time we successfully replicated: */
+  protected long lastPrimaryGen;
+
+  public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory) throws IOException {
+    super(id, dir, searcherFactory);
+
+    boolean success = false;
+
+    try {
+      message("top: init replica dir=" + dir);
+
+      // Obtain a write lock on this index since we "act like" an IndexWriter, to prevent any other IndexWriter or ReplicaNode from using it:
+      writeFileLock = dir.obtainLock(IndexWriter.WRITE_LOCK_NAME);
+
+      state = "init";
+      deleter = new ReplicaFileDeleter(this, dir);
+    } catch (Throwable t) {
+      message("exc on init:");
+      t.printStackTrace(System.out);
+      throw t;
+    } finally {
+      if (success == false) {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  /** Start up this replica, which possibly requires heavy copying of files from the primary node, if we were down for a long time */
+  protected synchronized void start(long curPrimaryGen) throws IOException {
+
+    if (state.equals("init") == false) {
+      throw new IllegalStateException("already started");
+    }
+
+    message("top: now start");
+    try {
+
+      // Figure out what state our local index is in now:
+      String segmentsFileName = SegmentInfos.getLastCommitSegmentsFileName(dir);
+
+      // Also look for any pending_segments_N, in case we crashed mid-commit.  We must "inflate" our infos gen to at least this, since
+      // otherwise we may wind up re-using the pending_segments_N file name on commit, and then our deleter can get angry because it still
+      // wants to delete this file:
+      long maxPendingGen = -1;
+      for(String fileName : dir.listAll()) {
+        if (fileName.startsWith(IndexFileNames.PENDING_SEGMENTS)) {
+          long gen = Long.parseLong(fileName.substring(IndexFileNames.PENDING_SEGMENTS.length()+1), Character.MAX_RADIX);
+          if (gen > maxPendingGen) {
+            maxPendingGen = gen;
+          }
+        }
+      }
+
+      SegmentInfos infos;
+      if (segmentsFileName == null) {
+        // No index here yet:
+        infos = new SegmentInfos();
+        message("top: init: no segments in index");
+      } else {
+        message("top: init: read existing segments commit " + segmentsFileName);
+        infos = SegmentInfos.readCommit(dir, segmentsFileName);
+        message("top: init: segments: " + infos.toString() + " version=" + infos.getVersion());
+        Collection<String> indexFiles = infos.files(false);
+
+        lastCommitFiles.add(segmentsFileName);
+        lastCommitFiles.addAll(indexFiles);
+
+        // Always protect the last commit:
+        deleter.incRef(lastCommitFiles);
+
+        lastNRTFiles.addAll(indexFiles);
+        deleter.incRef(lastNRTFiles);
+        message("top: commitFiles=" + lastCommitFiles);
+        message("top: nrtFiles=" + lastNRTFiles);
+      }
+
+      message("top: delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
+      deleter.deleteUnknownFiles(segmentsFileName);
+      message("top: done delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
+
+      // nocommit make test where index has all docs deleted (all segments dropped, so 0 segments) and is then replicated
+
+      String s = infos.getUserData().get(PRIMARY_GEN_KEY);
+      long myPrimaryGen;
+      if (s == null) {
+        assert infos.size() == 0;
+        myPrimaryGen = -1;
+      } else {
+        myPrimaryGen = Long.parseLong(s);
+      }
+      message("top: myPrimaryGen=" + myPrimaryGen);
+
+      boolean doCommit;
+
+      if (infos.size() > 0 && myPrimaryGen != -1 && myPrimaryGen != curPrimaryGen) {
+
+        assert myPrimaryGen < curPrimaryGen;
+
+        // Primary changed while we were down.  In this case, we must sync from primary before opening a reader, because it's possible current
+        // files we have will need to be overwritten with different ones (if index rolled back and "forked"), and we can't overwrite open
+        // files on Windows:
+
+        final long initSyncStartNS = System.nanoTime();
+
+        message("top: init: primary changed while we were down myPrimaryGen=" + myPrimaryGen +
+                " vs curPrimaryGen=" + curPrimaryGen +
+                "; sync now before mgr init");
+
+        // Try until we succeed in copying over the latest NRT point:
+        CopyJob job = null;
+
+        // We may need to overwrite files referenced by our latest commit, either right now on initial sync, or on a later sync.  To make
+        // sure the index is never even in an "apparently" corrupt state (where an old segments_N references invalid files) we forcefully
+        // remove the commit now, and refuse to start the replica if this delete fails:
+        message("top: now delete starting commit point " + segmentsFileName);
+
+        // If this throws exc (e.g. due to virus checker), we cannot start this replica:
+        assert deleter.getRefCount(segmentsFileName) == 1;
+        deleter.decRef(Collections.singleton(segmentsFileName));
+        if (deleter.isPending(segmentsFileName)) {
+          // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else we can cause corruption:
+          throw new RuntimeException("replica cannot start: existing segments file=" + segmentsFileName + " must be removed in order to start, but the file delete failed");
+        }
+        // So we don't later try to decRef it (illegally) again:
+        boolean didRemove = lastCommitFiles.remove(segmentsFileName);
+        assert didRemove;
+
+        while (true) {
+          job = newCopyJob("sync on startup replica=" + name() + " myVersion=" + infos.getVersion(),
+                           null,
+                           null,
+                           true,
+                           null);
+          job.start();
+
+          message("top: init: sync sis.version=" + job.getCopyState().version);
+
+          Collection<String> fileNamesToCopy = job.getFileNamesToCopy();
+
+          // Force this copy job to finish while we wait, now.  Note that this can be very time consuming!
+          // NOTE: newNRTPoint detects we are still in init (mgr is null) and does not cancel our copy if a flush happens
+          try {
+            job.runBlocking();
+            job.finish();
+
+            // Success!
+            break;
+          } catch (IOException ioe) {
+            job.cancel("startup failed", ioe);
+            if (ioe.getMessage().contains("checksum mismatch after file copy")) {
+              // OK-ish
+              message("top: failed to copy: " + ioe + "; retrying");
+            } else {
+              throw ioe;
+            }
+          }
+        }
+
+        lastPrimaryGen = job.getCopyState().primaryGen;
+        byte[] infosBytes = job.getCopyState().infosBytes;
+
+        SegmentInfos syncInfos = SegmentInfos.readCommit(dir,
+                                                         new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", job.getCopyState().infosBytes)),
+                                                         job.getCopyState().gen);
+
+        // Must always commit to a larger generation than what's currently in the index:
+        syncInfos.updateGeneration(infos);
+        infos = syncInfos;
+
+        assert infos.getVersion() == job.getCopyState().version;
+        message("  version=" + infos.getVersion() + " segments=" + infos.toString());
+        message("top: init: incRef nrtFiles=" + job.getFileNames());
+        deleter.incRef(job.getFileNames());
+        message("top: init: decRef lastNRTFiles=" + lastNRTFiles);
+        deleter.decRef(lastNRTFiles);
+
+        lastNRTFiles.clear();
+        lastNRTFiles.addAll(job.getFileNames());
+
+        message("top: init: set lastNRTFiles=" + lastNRTFiles);
+        lastFileMetaData = job.getCopyState().files;
+        message(String.format(Locale.ROOT, "top: %d: start: done sync: took %.3fs for %s, opened NRT reader version=%d",
+                              id,
+                              (System.nanoTime()-initSyncStartNS)/1000000000.0,
+                              bytesToString(job.getTotalBytesCopied()),
+                              job.getCopyState().version));
+
+        doCommit = true;
+      } else {
+        doCommit = false;
+        lastPrimaryGen = curPrimaryGen;
+        message("top: same primary as before");
+      }
+
+      if (infos.getGeneration() < maxPendingGen) {
+        message("top: move infos generation from " + infos.getGeneration() + " to " + maxPendingGen);
+        infos.setNextWriteGeneration(maxPendingGen);
+      }
+
+      // Notify primary we started, to give it a chance to send any warming merges our way to reduce NRT latency of first sync:
+      sendNewReplica();
+
+      // Finally, we are open for business, since our index now "agrees" with the primary:
+      mgr = new SegmentInfosSearcherManager(dir, this, infos, searcherFactory);
+
+      // Must commit after init mgr:
+      if (doCommit) {
+        // Very important to commit what we just sync'd over, because we removed the pre-existing commit point above if we had to
+        // overwrite any files it referenced:
+        commit();
+      }
+
+      message("top: done start");
+      state = "idle";
+    } catch (Throwable t) {
+      message("exc on start:");
+      t.printStackTrace(System.out);
+      throw new RuntimeException(t);
+    }
+  }
+  
+  final Object commitLock = new Object();
+
+  @Override
+  public void commit() throws IOException {
+
+    synchronized(commitLock) {
+
+      SegmentInfos infos;
+      Collection<String> indexFiles;
+
+      synchronized (this) {
+        infos = ((SegmentInfosSearcherManager) mgr).getCurrentInfos();
+        indexFiles = infos.files(false);
+        deleter.incRef(indexFiles);
+      }
+
+      message("top: commit primaryGen=" + lastPrimaryGen + " infos=" + infos.toString() + " files=" + indexFiles);
+
+      // fsync all index files we are now referencing
+      dir.sync(indexFiles);
+
+      Map<String,String> commitData = new HashMap<>();
+      commitData.put(PRIMARY_GEN_KEY, Long.toString(lastPrimaryGen));
+      commitData.put(VERSION_KEY, Long.toString(getCurrentSearchingVersion()));
+      infos.setUserData(commitData, false);
+
+      // write and fsync a new segments_N
+      infos.commit(dir);
+
+      // Notify current infos (which may have changed while we were doing dir.sync above) what generation we are up to; this way future
+      // commits are guaranteed to go to the next (unwritten) generations:
+      if (mgr != null) {
+        ((SegmentInfosSearcherManager) mgr).getCurrentInfos().updateGeneration(infos);
+      }
+      String segmentsFileName = infos.getSegmentsFileName();
+      message("top: commit wrote segments file " + segmentsFileName + " version=" + infos.getVersion() + " sis=" + infos.toString() + " commitData=" + commitData);
+      deleter.incRef(Collections.singletonList(segmentsFileName));
+      message("top: commit decRef lastCommitFiles=" + lastCommitFiles);
+      deleter.decRef(lastCommitFiles);
+      lastCommitFiles.clear();
+      lastCommitFiles.addAll(indexFiles);
+      lastCommitFiles.add(segmentsFileName);
+      message("top: commit version=" + infos.getVersion() + " files now " + lastCommitFiles);
+    }
+  }
+
+  void finishNRTCopy(CopyJob job, long startNS) throws IOException {
+    CopyState copyState = job.getCopyState();
+    message("top: finishNRTCopy: version=" + copyState.version + (job.getFailed() ? " FAILED" : "") + " job=" + job);
+
+    // NOTE: if primary crashed while we were still copying then the job will hit an exc trying to read bytes for the files from the primary node,
+    // and the job will be marked as failed here:
+
+    synchronized (this) {
+
+      if ("syncing".equals(state)) {
+        state = "idle";
+      }
+
+      if (curNRTCopy == job) {
+        message("top: now clear curNRTCopy; job=" + job);
+        curNRTCopy = null;
+      } else {
+        assert job.getFailed();
+        message("top: skip clear curNRTCopy: we were cancelled; job=" + job);
+      }
+
+      if (job.getFailed()) {
+        return;
+      }
+
+      // Does final file renames:
+      job.finish();
+
+      // Turn byte[] back to SegmentInfos:
+      byte[] infosBytes = copyState.infosBytes;
+      SegmentInfos infos = SegmentInfos.readCommit(dir,
+                                                   new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", copyState.infosBytes)),
+                                                   copyState.gen);
+      assert infos.getVersion() == copyState.version;
+
+      message("  version=" + infos.getVersion() + " segments=" + infos.toString());
+
+      // Cutover to new searcher:
+      if (mgr != null) {
+        ((SegmentInfosSearcherManager) mgr).setCurrentInfos(infos);
+      }
+
+      // Must first incRef new NRT files, then decRef old ones, to make sure we don't remove an NRT file that's in common to both:
+      Collection<String> newFiles = copyState.files.keySet();
+      message("top: incRef newNRTFiles=" + newFiles);
+      deleter.incRef(newFiles);
+
+      // If any of our new files were previously copied merges, we clear them now, so we don't try to later delete a non-existent file:
+      pendingMergeFiles.removeAll(newFiles);
+      message("top: after remove from pending merges pendingMergeFiles=" + pendingMergeFiles);
+
+      message("top: decRef lastNRTFiles=" + lastNRTFiles);
+      deleter.decRef(lastNRTFiles);
+      lastNRTFiles.clear();
+      lastNRTFiles.addAll(newFiles);
+      message("top: set lastNRTFiles=" + lastNRTFiles);
+
+      // At this point we can remove any completed merge segment files that we still do not reference.  This can happen when a merge
+      // finishes, copies its files out to us, but is then merged away (or dropped due to 100% deletions) before we ever cutover to it
+      // in an NRT point:
+      if (copyState.completedMergeFiles.isEmpty() == false) {
+        message("now remove-if-not-ref'd completed merge files: " + copyState.completedMergeFiles);
+        for(String fileName : copyState.completedMergeFiles) {
+          if (pendingMergeFiles.contains(fileName)) {
+            pendingMergeFiles.remove(fileName);
+            deleter.deleteIfNoRef(fileName);
+          }
+        }
+      }
+
+      lastFileMetaData = copyState.files;
+
+      // It's a good time to delete pending files, since we just refreshed and some previously open files are now closed:
+      deleter.deletePending();
+    }
+
+    int markerCount;
+    IndexSearcher s = mgr.acquire();
+    try {
+      markerCount = s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+    } finally {
+      mgr.release(s);
+    }
+
+    message(String.format(Locale.ROOT, "top: done sync: took %.3fs for %s, opened NRT reader version=%d markerCount=%d",
+                          (System.nanoTime()-startNS)/1000000000.0,
+                          bytesToString(job.getTotalBytesCopied()),
+                          copyState.version,
+                          markerCount));
+  }
+
+  /** Start a background copying job, to copy the specified files from the current primary node.  If files is null then the latest copy
+   *  state should be copied.  If prevJob is not null, then the new copy job is replacing it and should 1) cancel the previous one, and
+   *  2) optionally salvage e.g. partially copied and, shared with the new copy job, files. */
+  protected abstract CopyJob newCopyJob(String reason, Map<String,FileMetaData> files, Map<String,FileMetaData> prevFiles,
+                                        boolean highPriority, CopyJob.OnceDone onceDone) throws IOException;
+
+  /** Runs this job async'd */
+  protected abstract void launch(CopyJob job);
+
+  /** Tell primary we (replica) just started, so primary can tell us to warm any already warming merges.  This lets us keep low nrt refresh
+   *  time for the first nrt sync after we started. */
+  protected abstract void sendNewReplica() throws IOException;
+
+  /** Call this to notify this replica node that a new NRT infos is available on the primary.
+   *  We kick off a job (runs in the background) to copy files across, and open a new reader once that's done. */
+  public synchronized CopyJob newNRTPoint(long version) throws IOException {
+
+    if (isClosed()) {
+      throw new AlreadyClosedException("this replica is closed: state=" + state);
+    }
+
+    // Caller should not "publish" us until we have finished .start():
+    assert mgr != null;
+
+    if ("idle".equals(state)) {
+      state = "syncing";
+    }
+
+    long curVersion = getCurrentSearchingVersion();
+
+    message("top: start sync sis.version=" + version);
+
+    if (version == curVersion) {
+      // Caller releases the CopyState:
+      message("top: new NRT point has same version as current; skipping");
+      return null;
+    }
+
+    if (version < curVersion) {
+      // This can happen, if two syncs happen close together, and due to thread scheduling, the incoming older version runs after the newer version
+      message("top: new NRT point (version=" + version + ") is older than current (version=" + version + "); skipping");
+      return null;
+    }
+
+    final long startNS = System.nanoTime();
+
+    message("top: newNRTPoint");
+    CopyJob job = null;
+    try {
+      job = newCopyJob("NRT point sync version=" + version,
+                       null,
+                       lastFileMetaData,
+                       true,
+                       new CopyJob.OnceDone() {
+                         @Override
+                         public void run(CopyJob job) {
+                           try {
+                             finishNRTCopy(job, startNS);
+                           } catch (IOException ioe) {
+                             throw new RuntimeException(ioe);
+                           }
+                         }
+                       });
+    } catch (NodeCommunicationException nce) {
+      // E.g. primary could crash/close when we are asking it for the copy state:
+      message("top: ignoring communication exception creating CopyJob: " + nce);
+      nce.printStackTrace(System.out);
+      if (state.equals("syncing")) {
+        state = "idle";
+      }
+      return null;
+    }
+
+    Collection<String> newNRTFiles = job.getFileNames();
+    long newPrimaryGen = job.getCopyState().primaryGen;
+    maybeNewPrimary(newPrimaryGen);
+
+    message("top: newNRTPoint: job files=" + newNRTFiles);
+
+    if (curNRTCopy != null) {
+      job.transferAndCancel(curNRTCopy);
+      assert curNRTCopy.getFailed();
+    }
+
+    curNRTCopy = job;
+
+    for(String fileName : curNRTCopy.getFileNamesToCopy()) {
+      assert lastCommitFiles.contains(fileName) == false: "fileName=" + fileName + " is in lastCommitFiles and is being copied?";
+      synchronized (mergeCopyJobs) {
+        for (CopyJob mergeJob : mergeCopyJobs) {
+          if (mergeJob.getFileNames().contains(fileName)) {
+            // nocommit can't we .transferAndCancel?
+            message("top: now cancel merge copy job=" + mergeJob + ": file " + fileName + " is now being copied via NRT point");
+            mergeJob.cancel("newNRTPoint is copying over the same file", null);
+          }
+        }
+      }
+    }
+
+    try {
+      job.start();
+    } catch (NodeCommunicationException nce) {
+      // E.g. primary could crash/close when we are asking it for the copy state:
+      message("top: ignoring exception starting CopyJob: " + nce);
+      nce.printStackTrace(System.out);
+      if (state.equals("syncing")) {
+        state = "idle";
+      }
+      return null;
+    }
+
+    // Runs in the background jobs thread, maybe slowly/throttled, and calls finishSync once it's done:
+    launch(curNRTCopy);
+    return curNRTCopy;
+  }
+
+  public synchronized boolean isCopying() {
+    return curNRTCopy != null;
+  }
+
+  @Override
+  public boolean isClosed() {
+    return "closed".equals(state) || "closing".equals(state) || "crashing".equals(state) || "crashed".equals(state);
+  }
+
+  @Override
+  public void close() throws IOException {
+    message("top: now close");
+
+    synchronized (this) {
+      state = "closing";
+      if (curNRTCopy != null) {
+        curNRTCopy.cancel("closing", null);
+      }
+    }
+
+    synchronized (this) {
+      message("top: close mgr");
+      mgr.close();
+
+      message("top: decRef lastNRTFiles=" + lastNRTFiles);
+      deleter.decRef(lastNRTFiles);
+      lastNRTFiles.clear();
+
+      // NOTE: do not decRef these!
+      lastCommitFiles.clear();
+
+      message("top: delete if no ref pendingMergeFiles=" + pendingMergeFiles);
+      for(String fileName : pendingMergeFiles) {
+        deleter.deleteIfNoRef(fileName);
+      }
+      pendingMergeFiles.clear();
+    
+      message("top: close dir");
+      IOUtils.close(writeFileLock, dir);
+    }
+    message("top: done close");
+    state = "closed";
+  }
+
+  /** Called when the primary changed */
+  protected synchronized void maybeNewPrimary(long newPrimaryGen) {
+    if (newPrimaryGen != lastPrimaryGen) {
+      message("top: now change lastPrimaryGen from " + lastPrimaryGen + " to " + newPrimaryGen + " pendingMergeFiles=" + pendingMergeFiles);
+      assert newPrimaryGen > lastPrimaryGen: "newPrimaryGen=" + newPrimaryGen + " vs lastPrimaryGen=" + lastPrimaryGen;
+      lastPrimaryGen = newPrimaryGen;
+      pendingMergeFiles.clear();
+    } else {
+      message("top: keep current lastPrimaryGen=" + lastPrimaryGen);
+    }
+  }
+
+  protected synchronized CopyJob launchPreCopyMerge(AtomicBoolean finished, long newPrimaryGen, Map<String,FileMetaData> files) throws IOException {
+
+    CopyJob job;
+
+    maybeNewPrimary(newPrimaryGen);
+    final long primaryGenStart = lastPrimaryGen;
+    Set<String> fileNames = files.keySet();
+    message("now pre-copy warm merge files=" + fileNames + " primaryGen=" + newPrimaryGen);
+
+    for(String fileName : fileNames) {
+      assert pendingMergeFiles.contains(fileName) == false: "file \"" + fileName + "\" is already being warmed!";
+      assert lastNRTFiles.contains(fileName) == false: "file \"" + fileName + "\" is already NRT visible!";
+    }
+
+    job = newCopyJob("warm merge on " + name() + " filesNames=" + fileNames,
+                     files, null, false,
+                     new CopyJob.OnceDone() {
+
+                       @Override
+                       public void run(CopyJob job) throws IOException {
+                         // Signals that this replica has finished
+                         mergeCopyJobs.remove(job);
+                         message("done warming merge " + fileNames + " failed?=" + job.getFailed());
+                         synchronized(this) {
+                           if (job.getFailed() == false) {
+                             if (lastPrimaryGen != primaryGenStart) {
+                               message("merge pre copy finished but primary has changed; cancelling job files=" + fileNames);
+                               job.cancel("primary changed during merge copy", null);
+                             } else {
+                               boolean abort = false;
+                               for (String fileName : fileNames) {
+                                 if (lastNRTFiles.contains(fileName)) {
+                                   message("abort merge finish: file " + fileName + " is referenced by last NRT point");
+                                   abort = true;
+                                 }
+                                 if (lastCommitFiles.contains(fileName)) {
+                                   message("abort merge finish: file " + fileName + " is referenced by last commit point");
+                                   abort = true;
+                                 }
+                               }
+                               if (abort) {
+                                 // Even though in newNRTPoint we have similar logic, which cancels any merge copy jobs if an NRT point
+                                 // shows up referencing the files we are warming (because primary got impatient and gave up on us), we also
+                                 // need it here in case replica is way far behind and fails to even receive the merge pre-copy request
+                                 // until after the newNRTPoint referenced those files:
+                                 job.cancel("merged segment was separately copied via NRT point", null);
+                               } else {
+                                 job.finish();
+                                 message("merge pre copy finished files=" + fileNames);
+                                 for(String fileName : fileNames) {
+                                   assert pendingMergeFiles.contains(fileName) == false : "file \"" + fileName + "\" is already in pendingMergeFiles";
+                                   message("add file " + fileName + " to pendingMergeFiles");
+                                   pendingMergeFiles.add(fileName);
+                                 }
+                               }
+                             }
+                           } else {
+                             message("merge copy finished with failure");
+                           }
+                         }
+                         finished.set(true);
+                       }
+                     });
+
+    job.start();
+
+    // When warming a merge we better not already have any of these files copied!
+    assert job.getFileNamesToCopy().size() == files.size();
+
+    mergeCopyJobs.add(job);
+    launch(job);
+
+    return job;
+  }
+
+  public IndexOutput createTempOutput(String prefix, String suffix, IOContext ioContext) throws IOException {
+    return dir.createTempOutput(prefix, suffix, IOContext.DEFAULT);
+  }
+
+  /** Compares incoming per-file identity (id, checksum, header, footer) versus what we have locally and returns the subset of the incoming
+   *  files that need copying */
+  public List<Map.Entry<String,FileMetaData>> getFilesToCopy(Map<String,FileMetaData> files) throws IOException {
+
+    boolean doCopyCommitFiles = false;
+    List<Map.Entry<String,FileMetaData>> toCopy = new ArrayList<>();
+    for (Map.Entry<String,FileMetaData> ent : files.entrySet()) {
+      String fileName = ent.getKey();
+      FileMetaData fileMetaData = ent.getValue();
+      if (fileIsIdentical(fileName, fileMetaData) == false) {
+        toCopy.add(ent);
+      }
+    }
+
+    return toCopy;
+  }
+
+  /** Carefully determine if the file on the primary, identified by its {@code String fileName} along with the {@link FileMetaData}
+   * "summarizing" its contents, is precisely the same file that we have locally.  If the file does not exist locally, or if its its header
+   * (inclues the segment id), length, footer (including checksum) differ, then this returns false, else true. */
+  private boolean fileIsIdentical(String fileName, FileMetaData srcMetaData) throws IOException {
+
+    if (deleter.isPending(fileName)) {
+      // This was a file we had wanted to delete yet a virus checker prevented us, and now we need to overwrite it.
+      // Such files are in an unknown state, and even if their header and footer and length all
+      // match, since they may not have been fsync'd by the previous node instance on this directory,
+      // they could in theory have corruption internally.  So we always force ourselves to copy them here:
+      if (Node.VERBOSE_FILES) {
+        message("file " + fileName + ": will copy [we had wanted to delete this file on init, but failed]");
+      }
+      return false;
+    }
+
+    FileMetaData destMetaData = readLocalFileMetaData(fileName);
+    if (destMetaData == null) {
+      // Something went wrong in reading the file (it's corrupt, truncated, does not exist, etc.):
+      return false;
+    }
+
+    if (Arrays.equals(destMetaData.header, srcMetaData.header) == false ||
+        Arrays.equals(destMetaData.footer, srcMetaData.footer) == false) {
+      // Segment name was reused!  This is rare but possible and otherwise devastating:
+      if (Node.VERBOSE_FILES) {
+        message("file " + fileName + ": will copy [header/footer is different]");
+      }
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  private ConcurrentMap<String,Boolean> copying = new ConcurrentHashMap<>();
+
+  // Used only to catch bugs, ensuring a given file name is only ever being copied bye one job:
+  public void startCopyFile(String name) {
+    if (copying.putIfAbsent(name, Boolean.TRUE) != null) {
+      throw new IllegalStateException("file " + name + " is being copied in two places!");
+    }
+  }
+
+  public void finishCopyFile(String name) {
+    if (copying.remove(name) == null) {
+      throw new IllegalStateException("file " + name + " was not actually being copied?");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
new file mode 100644
index 0000000..72ed921
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
@@ -0,0 +1,129 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.StandardDirectoryReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ReferenceManager;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** A SearcherManager that refreshes via an externally provided (NRT) SegmentInfos, either from {@link IndexWriter} or via
+ *  nrt replication to another index. */
+class SegmentInfosSearcherManager extends ReferenceManager<IndexSearcher> {
+  private volatile SegmentInfos currentInfos;
+  private final Directory dir;
+  private final Node node;
+  private final AtomicInteger openReaderCount = new AtomicInteger();
+  private final SearcherFactory searcherFactory;
+
+  public SegmentInfosSearcherManager(Directory dir, Node node, SegmentInfos infosIn, SearcherFactory searcherFactory) throws IOException {
+    this.dir = dir;
+    this.node = node;
+    if (searcherFactory == null) {
+      searcherFactory = new SearcherFactory();
+    }
+    this.searcherFactory = searcherFactory;
+    currentInfos = infosIn;
+    node.message("SegmentInfosSearcherManager.init: use incoming infos=" + infosIn.toString());
+    current = SearcherManager.getSearcher(searcherFactory, StandardDirectoryReader.open(dir, currentInfos, null), null);
+    addReaderClosedListener(current.getIndexReader());
+  }
+
+  @Override
+  protected int getRefCount(IndexSearcher s) {
+    return s.getIndexReader().getRefCount();
+  }
+
+  @Override
+  protected boolean tryIncRef(IndexSearcher s) {
+    return s.getIndexReader().tryIncRef();
+  }
+
+  @Override
+  protected void decRef(IndexSearcher s) throws IOException {
+    s.getIndexReader().decRef();
+  }
+
+  public SegmentInfos getCurrentInfos() {
+    return currentInfos;
+  }
+
+  /** Switch to new segments, refreshing if necessary.  Note that it's the caller job to ensure there's a held refCount for the
+   *  incoming infos, so all files exist. */
+  public void setCurrentInfos(SegmentInfos infos) throws IOException {
+    if (currentInfos != null) {
+      // So that if we commit, we will go to the next
+      // (unwritten so far) generation:
+      infos.updateGeneration(currentInfos);
+      node.message("mgr.setCurrentInfos: carry over infos gen=" + infos.getSegmentsFileName());
+    }
+    currentInfos = infos;
+    maybeRefresh();
+  }
+
+  @Override
+  protected IndexSearcher refreshIfNeeded(IndexSearcher old) throws IOException {
+    List<LeafReader> subs;
+    if (old == null) {
+      subs = null;
+    } else {
+      subs = new ArrayList<>();
+      for(LeafReaderContext ctx : old.getIndexReader().leaves()) {
+        subs.add(ctx.reader());
+      }
+    }
+
+    // Open a new reader, sharing any common segment readers with the old one:
+    DirectoryReader r = StandardDirectoryReader.open(dir, currentInfos, subs);
+    addReaderClosedListener(r);
+    node.message("refreshed to version=" + currentInfos.getVersion() + " r=" + r);
+    return SearcherManager.getSearcher(searcherFactory, r, (DirectoryReader) old.getIndexReader());
+  }
+
+  private void addReaderClosedListener(IndexReader r) {
+    openReaderCount.incrementAndGet();
+    r.addReaderClosedListener(new IndexReader.ReaderClosedListener() {
+        @Override
+        public void onClose(IndexReader reader) {
+          onReaderClosed();
+        }
+      });
+  }
+
+  /** Tracks how many readers are still open, so that when we are closed,
+   *  we can additionally wait until all in-flight searchers are
+   *  closed. */
+  synchronized void onReaderClosed() {
+    if (openReaderCount.decrementAndGet() == 0) {
+      notifyAll();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java
new file mode 100644
index 0000000..7db7bc1
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java
@@ -0,0 +1,63 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+
+/** Simple point-to-point TCP connection */
+class Connection implements Closeable {
+  public final DataInput in;
+  public final DataOutput out;
+  public final InputStream sockIn;
+  public final BufferedOutputStream bos;
+  public final Socket s;
+  public final int destTCPPort;
+  public long lastKeepAliveNS = System.nanoTime();
+
+  public Connection(int tcpPort) throws IOException {
+    this.destTCPPort = tcpPort;
+    this.s = new Socket(InetAddress.getLoopbackAddress(), tcpPort);
+    this.sockIn = s.getInputStream();
+    this.in = new InputStreamDataInput(sockIn);
+    this.bos = new BufferedOutputStream(s.getOutputStream());
+    this.out = new OutputStreamDataOutput(bos);
+    if (Node.VERBOSE_CONNECTIONS) {
+      System.out.println("make new client Connection socket=" + this.s + " destPort=" + tcpPort);
+    }
+  }
+
+  public void flush() throws IOException {
+    bos.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    s.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
new file mode 100644
index 0000000..369414f
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
@@ -0,0 +1,152 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.lucene.store.AlreadyClosedException;
+
+/** Runs CopyJob(s) in background thread; each ReplicaNode has an instance of this
+ *  running.  At a given there could be one NRT copy job running, and multiple
+ *  pre-warm merged segments jobs. */
+class Jobs extends Thread implements Closeable {
+
+  private final PriorityQueue<CopyJob> queue = new PriorityQueue<>();
+
+  private final Node node;
+
+  public Jobs(Node node) {
+    this.node = node;
+  }
+
+  private boolean finish;
+
+  /** Returns null if we are closing, else, returns the top job or waits for one to arrive if the queue is empty. */
+  private synchronized SimpleCopyJob getNextJob() {
+    while (true) {
+      if (finish) {
+        return null;
+      } else if (queue.isEmpty()) {
+        try {
+          wait();
+        } catch (InterruptedException ie) {
+          throw new RuntimeException(ie);
+        }
+      } else {
+        return (SimpleCopyJob) queue.poll();
+      }
+    }
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      SimpleCopyJob topJob = getNextJob();
+      if (topJob == null) {
+        assert finish;
+        break;
+      }
+
+      this.setName("jobs o" + topJob.ord);
+
+      assert topJob != null;
+
+      boolean result;
+      try {
+        result = topJob.visit();
+      } catch (Throwable t) {
+        if ((t instanceof AlreadyClosedException) == false) {
+          node.message("exception during job.visit job=" + topJob + "; now cancel");
+          t.printStackTrace(System.out);
+        } else {
+          node.message("AlreadyClosedException during job.visit job=" + topJob + "; now cancel");
+        }
+        topJob.cancel("unexpected exception in visit", t);
+        try {
+          topJob.onceDone.run(topJob);
+        } catch (Throwable t2) {
+          node.message("ignore exception calling OnceDone: " + t2);
+        }
+        continue;
+      }
+
+      if (result == false) {
+        // Job isn't done yet; put it back:
+        synchronized (this) {
+          queue.offer(topJob);
+        }
+      } else {
+        // Job finished, now notify caller:
+        try {
+          topJob.onceDone.run(topJob);
+        } catch (Throwable t) {
+          node.message("ignore exception calling OnceDone: " + t);
+        }
+      }
+    }
+
+    node.message("top: jobs now exit run thread");
+
+    synchronized(this) {
+      // Gracefully cancel any jobs we didn't finish:
+      while (queue.isEmpty() == false) {
+        SimpleCopyJob job = (SimpleCopyJob) queue.poll();
+        node.message("top: Jobs: now cancel job=" + job);
+        job.cancel("jobs closing", null);
+        try {
+          job.onceDone.run(job);
+        } catch (Throwable t) {
+          node.message("ignore exception calling OnceDone: " + t);
+        }
+      }
+    }
+  }
+
+  public synchronized void launch(CopyJob job) {
+    if (finish == false) {
+      queue.offer(job);
+      notify();
+    } else {
+      throw new AlreadyClosedException("closed");
+    }
+  }
+
+  /** Cancels any existing jobs that are copying the same file names as this one */
+  public synchronized void cancelConflictingJobs(CopyJob newJob) {
+    for (CopyJob job : queue) {
+      if (job.conflicts(newJob)) {
+        node.message("top: now cancel existing conflicting job=" + job + " due to newJob=" + newJob);
+        job.cancel("conflicts with new job", null);
+      }
+    }
+  }
+
+  @Override
+  public synchronized void close() {
+    finish = true;
+    notify();
+    try {
+      join();
+    } catch (InterruptedException ie) {
+      throw new RuntimeException(ie);
+    }
+  }
+}


[05/15] lucene-solr git commit: exempt NodeCommunicationException as well

Posted by mi...@apache.org.
exempt NodeCommunicationException as well


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 33890681a0efd840874c5ec79c8862a7e94e59ab
Parents: 1ae7291
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 24 19:50:59 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 24 19:50:59 2016 -0500

----------------------------------------------------------------------
 .../src/test/org/apache/lucene/replicator/nrt/SimpleServer.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/33890681/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index f03a5c3..7a257de 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -127,13 +127,13 @@ public class SimpleServer extends LuceneTestCase {
 
         success = true;
       } catch (Throwable t) {
-        if (t instanceof SocketException == false) {
+        if (t instanceof SocketException == false && t instanceof NodeCommunicationException == false) {
           node.message("unexpected exception handling client connection:");
           t.printStackTrace(System.out);
           // Test should fail with this:
           throw new RuntimeException(t);
         } else {
-          node.message("SocketException " + t + " handling client connection; ignoring");
+          node.message("exception " + t + " handling client connection; ignoring");
         }
       } finally {
         if (success) {


[11/15] lucene-solr git commit: Merge branch 'nrt_replicas' of github.com:mikemccand/lucene-solr into nrt_replicas

Posted by mi...@apache.org.
Merge branch 'nrt_replicas' of github.com:mikemccand/lucene-solr into nrt_replicas

Conflicts:
	lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 82ecccf8e9efbafd13c8a66362e8bebf5dca1c29
Parents: 7af83c4 d9f20f5
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 3 16:46:27 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 3 16:46:27 2016 -0500

----------------------------------------------------------------------
 .../replicator/nrt/TestNRTReplication.java      | 36 +++++++++++---------
 1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82ecccf8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --cc lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 510a075,e2824af..7ba3bc2
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@@ -28,17 -38,6 +38,8 @@@ import java.util.concurrent.atomic.Atom
  import java.util.concurrent.atomic.AtomicLong;
  import java.util.regex.Pattern;
  
- import org.apache.lucene.document.Document;
- import org.apache.lucene.util.IOUtils;
- import org.apache.lucene.util.LineFileDocs;
- import org.apache.lucene.util.LuceneTestCase;
- import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
- import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
- 
- import com.carrotsearch.randomizedtesting.SeedUtils;
- 
 +// nocommit make some explicit failure tests
 +
  // MockRandom's .sd file has no index header/footer:
  @SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
  @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")


[04/15] lucene-solr git commit: current patch

Posted by mi...@apache.org.
current patch


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 1ae7291429bad742715344f86cfa5200229b3698
Parents: b62c671
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 24 18:17:20 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 24 18:17:20 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/codecs/CodecUtil.java     |   30 +
 .../apache/lucene/index/DirectoryReader.java    |    9 +-
 .../org/apache/lucene/index/IndexWriter.java    |   43 +-
 .../org/apache/lucene/index/ReaderManager.java  |    8 +-
 .../org/apache/lucene/index/SegmentInfos.java   |  369 +++---
 .../lucene/index/StandardDirectoryReader.java   |   31 +-
 .../apache/lucene/search/SearcherManager.java   |    8 +-
 .../lucene/store/ByteArrayIndexInput.java       |  163 +++
 .../lucene/store/NRTCachingDirectory.java       |    6 +-
 .../java/org/apache/lucene/util/IOUtils.java    |    3 +
 .../apache/lucene/index/TestIndexWriter.java    |    2 +-
 .../lucene/index/TestIndexWriterDelete.java     |    2 +-
 .../index/TestIndexWriterExceptions2.java       |    2 +-
 .../lucene/index/TestIndexWriterOnJRECrash.java |   42 +-
 .../lucene/index/TestIndexWriterOnVMError.java  |    2 +-
 .../apache/lucene/index/TestRollingUpdates.java |    2 +-
 .../apache/lucene/index/TestStressDeletes.java  |    2 +-
 .../TestControlledRealTimeReopenThread.java     |    6 +-
 .../apache/lucene/search/TestLRUQueryCache.java |    2 +-
 .../lucene/search/TestSearcherManager.java      |   10 +-
 .../facet/taxonomy/SearcherTaxonomyManager.java |    2 +-
 .../directory/DirectoryTaxonomyWriter.java      |    2 +-
 .../apache/lucene/replicator/nrt/CopyJob.java   |  237 ++++
 .../lucene/replicator/nrt/CopyOneFile.java      |  132 ++
 .../apache/lucene/replicator/nrt/CopyState.java |   56 +
 .../lucene/replicator/nrt/FileMetaData.java     |   40 +
 .../org/apache/lucene/replicator/nrt/Node.java  |  213 ++++
 .../nrt/NodeCommunicationException.java         |   26 +
 .../nrt/PreCopyMergedSegmentWarmer.java         |   80 ++
 .../lucene/replicator/nrt/PrimaryNode.java      |  316 +++++
 .../replicator/nrt/ReplicaFileDeleter.java      |  218 ++++
 .../lucene/replicator/nrt/ReplicaNode.java      |  772 ++++++++++++
 .../nrt/SegmentInfosSearcherManager.java        |  129 ++
 .../lucene/replicator/nrt/Connection.java       |   63 +
 .../org/apache/lucene/replicator/nrt/Jobs.java  |  152 +++
 .../lucene/replicator/nrt/NodeProcess.java      |  238 ++++
 .../lucene/replicator/nrt/SimpleCopyJob.java    |  294 +++++
 .../replicator/nrt/SimplePrimaryNode.java       |  674 ++++++++++
 .../replicator/nrt/SimpleReplicaNode.java       |  316 +++++
 .../lucene/replicator/nrt/SimpleServer.java     |  390 ++++++
 .../lucene/replicator/nrt/SimpleTransLog.java   |  250 ++++
 .../replicator/nrt/TestNRTReplication.java      | 1175 ++++++++++++++++++
 .../lucene/replicator/nrt/ThreadPumper.java     |   59 +
 lucene/replicator/test.cmd                      |    1 +
 .../lucene/util/BaseGeoPointTestCase.java       |    2 +-
 .../index/BaseIndexFileFormatTestCase.java      |    2 +-
 .../apache/lucene/index/RandomIndexWriter.java  |    8 +-
 .../lucene/store/MockDirectoryWrapper.java      |  286 +++--
 .../org/apache/lucene/util/LuceneTestCase.java  |   13 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    3 +-
 50 files changed, 6568 insertions(+), 323 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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 1bc2f40..4ddad22 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
@@ -258,6 +258,36 @@ public final class CodecUtil {
     checkIndexHeaderSuffix(in, expectedSuffix);
     return version;
   }
+
+  /** Retrieves the full index header from the provided {@link IndexInput}.
+   *  This throws {@link CorruptIndexException} if this file does
+   * not appear to be an index file. */
+  public static byte[] readIndexHeader(IndexInput in) throws IOException {
+    in.seek(0);
+    final int actualHeader = in.readInt();
+    if (actualHeader != CODEC_MAGIC) {
+      throw new CorruptIndexException("codec header mismatch: actual header=" + actualHeader + " vs expected header=" + CODEC_MAGIC, in);
+    }
+    String codec = in.readString();
+    in.readInt();
+    in.seek(in.getFilePointer() + StringHelper.ID_LENGTH);
+    int suffixLength = in.readByte() & 0xFF;
+    byte[] bytes = new byte[headerLength(codec) + StringHelper.ID_LENGTH + 1 + suffixLength];
+    in.seek(0);
+    in.readBytes(bytes, 0, bytes.length);
+    return bytes;
+  }
+
+  /** Retrieves the full footer from the provided {@link IndexInput}.  This throws
+   *  {@link CorruptIndexException} if this file does not have a valid footer. */
+  public static byte[] readFooter(IndexInput in) throws IOException {
+    in.seek(in.length() - footerLength());
+    validateFooter(in);
+    in.seek(in.length() - footerLength());
+    byte[] bytes = new byte[footerLength()];
+    in.readBytes(bytes, 0, bytes.length);
+    return bytes;
+  }
   
   /** Expert: just reads and verifies the object ID of an index header */
   public static byte[] checkIndexHeaderID(DataInput in, byte[] expectedID) throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
index 3df0b70..a926726 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
@@ -76,7 +76,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
    * @lucene.experimental
    */
   public static DirectoryReader open(final IndexWriter writer) throws IOException {
-    return open(writer, true);
+    return open(writer, true, false);
   }
 
   /**
@@ -91,13 +91,16 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
    * future.  Applying deletes can be costly, so if your app
    * can tolerate deleted documents being returned you might
    * gain some performance by passing false.
+   * @param writeAllDeletes If true, new deletes will be written
+   * down to index files instead of carried over from writer to
+   * reader in heap
    *
    * @see #open(IndexWriter)
    *
    * @lucene.experimental
    */
-  public static DirectoryReader open(final IndexWriter writer, boolean applyAllDeletes) throws IOException {
-    return writer.getReader(applyAllDeletes);
+  public static DirectoryReader open(final IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
+    return writer.getReader(applyAllDeletes, writeAllDeletes);
   }
 
   /** Expert: returns an IndexReader reading the index in the given

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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 b05e15a..96dfa0b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -332,7 +332,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
   final CloseableThreadLocal<MergeRateLimiter> rateLimiters = new CloseableThreadLocal<>();
 
   DirectoryReader getReader() throws IOException {
-    return getReader(true);
+    return getReader(true, false);
   }
 
   /**
@@ -393,9 +393,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    *
    * @throws IOException If there is a low-level I/O error
    */
-  DirectoryReader getReader(boolean applyAllDeletes) throws IOException {
+  DirectoryReader getReader(boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
     ensureOpen();
 
+    if (writeAllDeletes && applyAllDeletes == false) {
+      throw new IllegalArgumentException("applyAllDeletes must be true when writeAllDeletes=true");
+    }
+
     final long tStart = System.currentTimeMillis();
 
     if (infoStream.isEnabled("IW")) {
@@ -431,7 +435,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           // just like we do when loading segments_N
           synchronized(this) {
             anyChanges |= maybeApplyDeletes(applyAllDeletes);
-            r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes);
+            if (writeAllDeletes) {
+              // Must move the deletes to disk:
+              System.out.println("IW: now readerPool.commit");
+              readerPool.commit(segmentInfos);
+            }
+
+            r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes, writeAllDeletes);
             if (infoStream.isEnabled("IW")) {
               infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
             }
@@ -1159,6 +1169,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     return docWriter.getNumDocs() + segmentInfos.totalMaxDoc();
   }
 
+  /** If {@link SegmentInfos#getVersion} is below {@code newVersion} then update it to this value. */
+  public synchronized void advanceSegmentInfosVersion(long newVersion) {
+    ensureOpen();
+    if (segmentInfos.getVersion() < newVersion) {
+      segmentInfos.setVersion(newVersion);
+    }
+    changed();
+  }
+
   /** Returns total number of docs in this index, including
    *  docs not yet flushed (still in the RAM buffer), and
    *  including deletions.  <b>NOTE:</b> buffered deletions
@@ -2870,7 +2889,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
    * contents after calling this method has no effect.
    */
   public final synchronized void setCommitData(Map<String,String> commitUserData) {
-    segmentInfos.setUserData(new HashMap<>(commitUserData));
+    setCommitData(commitUserData, true);
+  }
+
+  public final synchronized void setCommitData(Map<String,String> commitUserData, boolean doIncrementVersion) {
+    segmentInfos.setUserData(new HashMap<>(commitUserData), doIncrementVersion);
     changeCount.incrementAndGet();
   }
   
@@ -4576,10 +4599,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
   synchronized boolean nrtIsCurrent(SegmentInfos infos) {
     //System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
     ensureOpen();
-    boolean isCurrent = infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedUpdatesStream.any();
+    boolean isCurrent = infos.getVersion() == segmentInfos.getVersion() && !docWriter.anyChanges() && !bufferedUpdatesStream.any();
     if (infoStream.isEnabled("IW")) {
       if (isCurrent == false) {
-        infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any());
+        infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.getVersion() == segmentInfos.getVersion()) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any());
       }
     }
     return isCurrent;
@@ -4708,15 +4731,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     }
   }
   
-  synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
+  /** @lucene.internal */
+  public synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
     ensureOpen();
     deleter.incRef(segmentInfos, false);
     if (infoStream.isEnabled("IW")) {
       infoStream.message("IW", "incRefDeleter for NRT reader version=" + segmentInfos.getVersion() + " segments=" + segString(segmentInfos));
     }
   }
-  
-  synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
+
+  /** @lucene.internal */
+  public synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
     ensureOpen();
     deleter.decRef(segmentInfos);
     if (infoStream.isEnabled("IW")) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java b/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
index 79d9a94..8044dc1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
@@ -45,7 +45,7 @@ public final class ReaderManager extends ReferenceManager<DirectoryReader> {
    * @throws IOException If there is a low-level I/O error
    */
   public ReaderManager(IndexWriter writer) throws IOException {
-    this(writer, true);
+    this(writer, true, false);
   }
 
   /**
@@ -63,11 +63,13 @@ public final class ReaderManager extends ReferenceManager<DirectoryReader> {
    *          tolerate deleted documents being returned you might gain some
    *          performance by passing <code>false</code>. See
    *          {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}.
+   * @param writeAllDeletes
+   *          If <code>true</code>, new deletes will be forcefully written to index files.
    * 
    * @throws IOException If there is a low-level I/O error
    */
-  public ReaderManager(IndexWriter writer, boolean applyAllDeletes) throws IOException {
-    current = DirectoryReader.open(writer, applyAllDeletes);
+  public ReaderManager(IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
+    current = DirectoryReader.open(writer, applyAllDeletes, writeAllDeletes);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index de5dbff..2f8d914 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -279,124 +279,131 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
   public static final SegmentInfos readCommit(Directory directory, String segmentFileName) throws IOException {
 
     long generation = generationFromSegmentsFileName(segmentFileName);
+    //System.out.println(Thread.currentThread() + ": SegmentInfos.readCommit " + segmentFileName);
     try (ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ)) {
-      // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
-      // to read the magic ourselves.
-      int magic = input.readInt();
-      if (magic != CodecUtil.CODEC_MAGIC) {
-        throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
-      }
-      int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
-      byte id[] = new byte[StringHelper.ID_LENGTH];
-      input.readBytes(id, 0, id.length);
-      CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
-
-      SegmentInfos infos = new SegmentInfos();
-      infos.id = id;
-      infos.generation = generation;
-      infos.lastGeneration = generation;
-      if (format >= VERSION_53) {
-        // TODO: in the future (7.0?  sigh) we can use this to throw IndexFormatTooOldException ... or just rely on the
-        // minSegmentLuceneVersion check instead:
-        infos.luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
-      } else {
-        // else compute the min version down below in the for loop
-      }
+      return readCommit(directory, input, generation);
+    }
+  }
 
-      infos.version = input.readLong();
-      infos.counter = input.readInt();
-      int numSegments = input.readInt();
-      if (numSegments < 0) {
-        throw new CorruptIndexException("invalid segment count: " + numSegments, input);
-      }
+  public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput input, long generation) throws IOException {
 
-      if (format >= VERSION_53) {
-        if (numSegments > 0) {
-          infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
-          if (infos.minSegmentLuceneVersion.onOrAfter(Version.LUCENE_5_0_0) == false) {
-            throw new IndexFormatTooOldException(input, "this index contains a too-old segment (version: " + infos.minSegmentLuceneVersion + ")");
-          }
-        } else {
-          // else leave as null: no segments
+    // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
+    // to read the magic ourselves.
+    int magic = input.readInt();
+    if (magic != CodecUtil.CODEC_MAGIC) {
+      throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
+    }
+    int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
+    byte id[] = new byte[StringHelper.ID_LENGTH];
+    input.readBytes(id, 0, id.length);
+    CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
+
+    SegmentInfos infos = new SegmentInfos();
+    infos.id = id;
+    infos.generation = generation;
+    infos.lastGeneration = generation;
+    if (format >= VERSION_53) {
+      // TODO: in the future (7.0?  sigh) we can use this to throw IndexFormatTooOldException ... or just rely on the
+      // minSegmentLuceneVersion check instead:
+      infos.luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+    } else {
+      // else compute the min version down below in the for loop
+    }
+
+    infos.version = input.readLong();
+    //System.out.println("READ sis version=" + infos.version);
+    infos.counter = input.readInt();
+    int numSegments = input.readInt();
+    if (numSegments < 0) {
+      throw new CorruptIndexException("invalid segment count: " + numSegments, input);
+    }
+
+    if (format >= VERSION_53) {
+      if (numSegments > 0) {
+        infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+        if (infos.minSegmentLuceneVersion.onOrAfter(Version.LUCENE_5_0_0) == false) {
+          throw new IndexFormatTooOldException(input, "this index contains a too-old segment (version: " + infos.minSegmentLuceneVersion + ")");
         }
       } else {
-        // else we recompute it below as we visit segments; it can't be used for throwing IndexFormatTooOldExc, but consumers of
-        // SegmentInfos can maybe still use it for other reasons
+        // else leave as null: no segments
       }
+    } else {
+      // else we recompute it below as we visit segments; it can't be used for throwing IndexFormatTooOldExc, but consumers of
+      // SegmentInfos can maybe still use it for other reasons
+    }
 
-      long totalDocs = 0;
-      for (int seg = 0; seg < numSegments; seg++) {
-        String segName = input.readString();
-        final byte segmentID[];
-        byte hasID = input.readByte();
-        if (hasID == 1) {
-          segmentID = new byte[StringHelper.ID_LENGTH];
-          input.readBytes(segmentID, 0, segmentID.length);
-        } else if (hasID == 0) {
-          throw new IndexFormatTooOldException(input, "Segment is from Lucene 4.x");
-        } else {
-          throw new CorruptIndexException("invalid hasID byte, got: " + hasID, input);
-        }
-        Codec codec = readCodec(input, format < VERSION_53);
-        SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
-        info.setCodec(codec);
-        totalDocs += info.maxDoc();
-        long delGen = input.readLong();
-        int delCount = input.readInt();
-        if (delCount < 0 || delCount > info.maxDoc()) {
-          throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
-        }
-        long fieldInfosGen = input.readLong();
-        long dvGen = input.readLong();
-        SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
-        if (format >= VERSION_51) {
-          siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
-        } else {
-          siPerCommit.setFieldInfosFiles(Collections.unmodifiableSet(input.readStringSet()));
-        }
-        final Map<Integer,Set<String>> dvUpdateFiles;
-        final int numDVFields = input.readInt();
-        if (numDVFields == 0) {
-          dvUpdateFiles = Collections.emptyMap();
-        } else {
-          Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
-          for (int i = 0; i < numDVFields; i++) {
-            if (format >= VERSION_51) {
-              map.put(input.readInt(), input.readSetOfStrings());
-            } else {
-              map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
-            }
+    long totalDocs = 0;
+    for (int seg = 0; seg < numSegments; seg++) {
+      String segName = input.readString();
+      final byte segmentID[];
+      byte hasID = input.readByte();
+      if (hasID == 1) {
+        segmentID = new byte[StringHelper.ID_LENGTH];
+        input.readBytes(segmentID, 0, segmentID.length);
+      } else if (hasID == 0) {
+        throw new IndexFormatTooOldException(input, "Segment is from Lucene 4.x");
+      } else {
+        throw new CorruptIndexException("invalid hasID byte, got: " + hasID, input);
+      }
+      Codec codec = readCodec(input, format < VERSION_53);
+      SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
+      info.setCodec(codec);
+      totalDocs += info.maxDoc();
+      long delGen = input.readLong();
+      int delCount = input.readInt();
+      if (delCount < 0 || delCount > info.maxDoc()) {
+        throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
+      }
+      long fieldInfosGen = input.readLong();
+      long dvGen = input.readLong();
+      SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
+      if (format >= VERSION_51) {
+        siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
+      } else {
+        siPerCommit.setFieldInfosFiles(Collections.unmodifiableSet(input.readStringSet()));
+      }
+      final Map<Integer,Set<String>> dvUpdateFiles;
+      final int numDVFields = input.readInt();
+      if (numDVFields == 0) {
+        dvUpdateFiles = Collections.emptyMap();
+      } else {
+        Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
+        for (int i = 0; i < numDVFields; i++) {
+          if (format >= VERSION_51) {
+            map.put(input.readInt(), input.readSetOfStrings());
+          } else {
+            map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
           }
-          dvUpdateFiles = Collections.unmodifiableMap(map);
         }
-        siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
-        infos.add(siPerCommit);
+        dvUpdateFiles = Collections.unmodifiableMap(map);
+      }
+      siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
+      infos.add(siPerCommit);
 
-        Version segmentVersion = info.getVersion();
-        if (format < VERSION_53) {
-          if (infos.minSegmentLuceneVersion == null || segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
-            infos.minSegmentLuceneVersion = segmentVersion;
-          }
-        } else if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
-          throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
+      Version segmentVersion = info.getVersion();
+      if (format < VERSION_53) {
+        if (infos.minSegmentLuceneVersion == null || segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
+          infos.minSegmentLuceneVersion = segmentVersion;
         }
+      } else if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
+        throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
       }
+    }
 
-      if (format >= VERSION_51) {
-        infos.userData = input.readMapOfStrings();
-      } else {
-        infos.userData = Collections.unmodifiableMap(input.readStringStringMap());
-      }
+    if (format >= VERSION_51) {
+      infos.userData = input.readMapOfStrings();
+    } else {
+      infos.userData = Collections.unmodifiableMap(input.readStringStringMap());
+    }
 
-      CodecUtil.checkFooter(input);
+    CodecUtil.checkFooter(input);
 
-      // LUCENE-6299: check we are in bounds
-      if (totalDocs > IndexWriter.getActualMaxDocs()) {
-        throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
-      }
-
-      return infos;
+    // LUCENE-6299: check we are in bounds
+    if (totalDocs > IndexWriter.getActualMaxDocs()) {
+      throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
     }
+
+    return infos;
   }
 
   private static final List<String> unsupportedCodecs = Arrays.asList(
@@ -454,68 +461,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
 
     try {
       segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
-      CodecUtil.writeIndexHeader(segnOutput, "segments", VERSION_CURRENT, 
-                                 StringHelper.randomId(), Long.toString(nextGeneration, Character.MAX_RADIX));
-      segnOutput.writeVInt(Version.LATEST.major);
-      segnOutput.writeVInt(Version.LATEST.minor);
-      segnOutput.writeVInt(Version.LATEST.bugfix);
-
-      segnOutput.writeLong(version); 
-      segnOutput.writeInt(counter); // write counter
-      segnOutput.writeInt(size());
-
-      if (size() > 0) {
-
-        Version minSegmentVersion = null;
-
-        // We do a separate loop up front so we can write the minSegmentVersion before
-        // any SegmentInfo; this makes it cleaner to throw IndexFormatTooOldExc at read time:
-        for (SegmentCommitInfo siPerCommit : this) {
-          Version segmentVersion = siPerCommit.info.getVersion();
-          if (minSegmentVersion == null || segmentVersion.onOrAfter(minSegmentVersion) == false) {
-            minSegmentVersion = segmentVersion;
-          }
-        }
-
-        segnOutput.writeVInt(minSegmentVersion.major);
-        segnOutput.writeVInt(minSegmentVersion.minor);
-        segnOutput.writeVInt(minSegmentVersion.bugfix);
-      }
-
-      // write infos
-      for (SegmentCommitInfo siPerCommit : this) {
-        SegmentInfo si = siPerCommit.info;
-        segnOutput.writeString(si.name);
-        byte segmentID[] = si.getId();
-        // TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
-        if (segmentID == null) {
-          segnOutput.writeByte((byte)0);
-        } else {
-          if (segmentID.length != StringHelper.ID_LENGTH) {
-            throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
-          }
-          segnOutput.writeByte((byte)1);
-          segnOutput.writeBytes(segmentID, segmentID.length);
-        }
-        segnOutput.writeString(si.getCodec().getName());
-        segnOutput.writeLong(siPerCommit.getDelGen());
-        int delCount = siPerCommit.getDelCount();
-        if (delCount < 0 || delCount > si.maxDoc()) {
-          throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
-        }
-        segnOutput.writeInt(delCount);
-        segnOutput.writeLong(siPerCommit.getFieldInfosGen());
-        segnOutput.writeLong(siPerCommit.getDocValuesGen());
-        segnOutput.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
-        final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
-        segnOutput.writeInt(dvUpdatesFiles.size());
-        for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
-          segnOutput.writeInt(e.getKey());
-          segnOutput.writeSetOfStrings(e.getValue());
-        }
-      }
-      segnOutput.writeMapOfStrings(userData);
-      CodecUtil.writeFooter(segnOutput);
+      write(directory, segnOutput);
       segnOutput.close();
       directory.sync(Collections.singleton(segmentFileName));
       success = true;
@@ -533,6 +479,72 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     }
   }
 
+  public void write(Directory directory, IndexOutput out) throws IOException {
+    CodecUtil.writeIndexHeader(out, "segments", VERSION_CURRENT, 
+                               StringHelper.randomId(), Long.toString(generation, Character.MAX_RADIX));
+    out.writeVInt(Version.LATEST.major);
+    out.writeVInt(Version.LATEST.minor);
+    out.writeVInt(Version.LATEST.bugfix);
+    System.out.println(Thread.currentThread().getName() + ": now write " + out.getName() + " with version=" + version);
+
+    out.writeLong(version); 
+    out.writeInt(counter); // write counter
+    out.writeInt(size());
+
+    if (size() > 0) {
+
+      Version minSegmentVersion = null;
+
+      // We do a separate loop up front so we can write the minSegmentVersion before
+      // any SegmentInfo; this makes it cleaner to throw IndexFormatTooOldExc at read time:
+      for (SegmentCommitInfo siPerCommit : this) {
+        Version segmentVersion = siPerCommit.info.getVersion();
+        if (minSegmentVersion == null || segmentVersion.onOrAfter(minSegmentVersion) == false) {
+          minSegmentVersion = segmentVersion;
+        }
+      }
+
+      out.writeVInt(minSegmentVersion.major);
+      out.writeVInt(minSegmentVersion.minor);
+      out.writeVInt(minSegmentVersion.bugfix);
+    }
+
+    // write infos
+    for (SegmentCommitInfo siPerCommit : this) {
+      SegmentInfo si = siPerCommit.info;
+      out.writeString(si.name);
+      byte segmentID[] = si.getId();
+      // TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
+      if (segmentID == null) {
+        out.writeByte((byte)0);
+      } else {
+        if (segmentID.length != StringHelper.ID_LENGTH) {
+          throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
+        }
+        out.writeByte((byte)1);
+        out.writeBytes(segmentID, segmentID.length);
+      }
+      out.writeString(si.getCodec().getName());
+      out.writeLong(siPerCommit.getDelGen());
+      int delCount = siPerCommit.getDelCount();
+      if (delCount < 0 || delCount > si.maxDoc()) {
+        throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
+      }
+      out.writeInt(delCount);
+      out.writeLong(siPerCommit.getFieldInfosGen());
+      out.writeLong(siPerCommit.getDocValuesGen());
+      out.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
+      final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
+      out.writeInt(dvUpdatesFiles.size());
+      for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
+        out.writeInt(e.getKey());
+        out.writeSetOfStrings(e.getValue());
+      }
+    }
+    out.writeMapOfStrings(userData);
+    CodecUtil.writeFooter(out);
+  }
+
   /**
    * Returns a copy of this instance, also copying each
    * SegmentInfo.
@@ -698,8 +710,10 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     protected abstract T doBody(String segmentFileName) throws IOException;
   }
 
-  // Carry over generation numbers from another SegmentInfos
-  void updateGeneration(SegmentInfos other) {
+  /** Carry over generation numbers from another SegmentInfos
+   *
+   * @lucene.internal */
+  public void updateGeneration(SegmentInfos other) {
     lastGeneration = other.lastGeneration;
     generation = other.generation;
   }
@@ -711,7 +725,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     this.counter = other.counter;
   }
 
-  void setNextWriteGeneration(long generation) {
+  public void setNextWriteGeneration(long generation) {
     assert generation >= this.generation;
     this.generation = generation;
   }
@@ -777,7 +791,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     boolean success = false;
     final String dest;
     try {
-      final String src =  IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
+      final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
       dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", generation);
       dir.renameFile(src, dest);
       success = true;
@@ -800,7 +814,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
    *  method if changes have been made to this {@link SegmentInfos} instance
    *  </p>  
    **/
-  final void commit(Directory dir) throws IOException {
+  public final void commit(Directory dir) throws IOException {
     prepareCommit(dir);
     finishCommit(dir);
   }
@@ -829,14 +843,15 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     return userData;
   }
 
-  void setUserData(Map<String,String> data) {
+  public void setUserData(Map<String,String> data, boolean doIncrementVersion) {
     if (data == null) {
       userData = Collections.<String,String>emptyMap();
     } else {
       userData = data;
     }
-
-    changed();
+    if (doIncrementVersion) {
+      changed();
+    }
   }
 
   /** Replaces all segments in this instance, but keeps
@@ -864,8 +879,18 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
    *  segments. */
   public void changed() {
     version++;
+    System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
+    //new Throwable().printStackTrace(System.out);
   }
   
+  void setVersion(long newVersion) {
+    if (newVersion < version) {
+      throw new IllegalArgumentException("newVersion (=" + newVersion + ") cannot be less than current version (=" + version + ")");
+    }
+    System.out.println(Thread.currentThread().getName() + ": SIS.setVersion change from " + version + " to " + newVersion);
+    version = newVersion;
+  }
+
   /** applies all changes caused by committing a merge to this SegmentInfos */
   void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) {
     final Set<SegmentCommitInfo> mergedAway = new HashSet<>(merge.segments);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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 8d51532..28dd55f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
@@ -30,19 +30,21 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
 
-final class StandardDirectoryReader extends DirectoryReader {
+public final class StandardDirectoryReader extends DirectoryReader {
 
   final IndexWriter writer;
   final SegmentInfos segmentInfos;
   private final boolean applyAllDeletes;
+  private final boolean writeAllDeletes;
   
   /** called only from static open() methods */
   StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
-    SegmentInfos sis, boolean applyAllDeletes) throws IOException {
+                          SegmentInfos sis, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
     super(directory, readers);
     this.writer = writer;
     this.segmentInfos = sis;
     this.applyAllDeletes = applyAllDeletes;
+    this.writeAllDeletes = writeAllDeletes;
   }
 
   /** called from DirectoryReader.open(...) methods */
@@ -60,7 +62,7 @@ final class StandardDirectoryReader extends DirectoryReader {
 
           // This may throw CorruptIndexException if there are too many docs, so
           // it must be inside try clause so we close readers in that case:
-          DirectoryReader reader = new StandardDirectoryReader(directory, readers, null, sis, false);
+          DirectoryReader reader = new StandardDirectoryReader(directory, readers, null, sis, false, false);
           success = true;
 
           return reader;
@@ -74,7 +76,7 @@ final class StandardDirectoryReader extends DirectoryReader {
   }
 
   /** Used by near real-time search */
-  static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
+  static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
     // IndexWriter synchronizes externally before calling
     // us, which ensures infos will not change; so there's
     // no need to process segments in reverse order
@@ -113,8 +115,8 @@ final class StandardDirectoryReader extends DirectoryReader {
       writer.incRefDeleter(segmentInfos);
       
       StandardDirectoryReader result = new StandardDirectoryReader(dir,
-          readers.toArray(new SegmentReader[readers.size()]), writer,
-          segmentInfos, applyAllDeletes);
+                                                                   readers.toArray(new SegmentReader[readers.size()]), writer,
+                                                                   segmentInfos, applyAllDeletes, writeAllDeletes);
       success = true;
       return result;
     } finally {
@@ -131,8 +133,10 @@ final class StandardDirectoryReader extends DirectoryReader {
     }
   }
 
-  /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
-  private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends LeafReader> oldReaders) throws IOException {
+  /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)}, as well as NRT replication.
+   *
+   *  @lucene.internal */
+  public static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends LeafReader> oldReaders) throws IOException {
 
     // we put the old SegmentReaders in a map, that allows us
     // to lookup a reader using its segment name
@@ -210,7 +214,7 @@ final class StandardDirectoryReader extends DirectoryReader {
         }
       }
     }    
-    return new StandardDirectoryReader(directory, newReaders, null, infos, false);
+    return new StandardDirectoryReader(directory, newReaders, null, infos, false, false);
   }
 
   // TODO: move somewhere shared if it's useful elsewhere
@@ -270,7 +274,7 @@ final class StandardDirectoryReader extends DirectoryReader {
     if (writer == this.writer && applyAllDeletes == this.applyAllDeletes) {
       return doOpenFromWriter(null);
     } else {
-      return writer.getReader(applyAllDeletes);
+      return writer.getReader(applyAllDeletes, writeAllDeletes);
     }
   }
 
@@ -283,7 +287,7 @@ final class StandardDirectoryReader extends DirectoryReader {
       return null;
     }
 
-    DirectoryReader reader = writer.getReader(applyAllDeletes);
+    DirectoryReader reader = writer.getReader(applyAllDeletes, writeAllDeletes);
 
     // If in fact no changes took place, return null:
     if (reader.getVersion() == segmentInfos.getVersion()) {
@@ -332,6 +336,11 @@ final class StandardDirectoryReader extends DirectoryReader {
     return segmentInfos.getVersion();
   }
 
+  /** @lucene.internal */
+  public SegmentInfos getSegmentInfos() {
+    return segmentInfos;
+  }
+
   @Override
   public boolean isCurrent() throws IOException {
     ensureOpen();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java b/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
index 3d3b064..a57f260 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
@@ -73,7 +73,7 @@ public final class SearcherManager extends ReferenceManager<IndexSearcher> {
    * @throws IOException if there is a low-level I/O error
    */
   public SearcherManager(IndexWriter writer, SearcherFactory searcherFactory) throws IOException {
-    this(writer, true, searcherFactory);
+    this(writer, true, false, searcherFactory);
   }
 
   /**
@@ -91,6 +91,8 @@ public final class SearcherManager extends ReferenceManager<IndexSearcher> {
    *          tolerate deleted documents being returned you might gain some
    *          performance by passing <code>false</code>. See
    *          {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}.
+   * @param writeAllDeletes
+   *          If <code>true</code>, new deletes will be forcefully written to index files.
    * @param searcherFactory
    *          An optional {@link SearcherFactory}. Pass <code>null</code> if you
    *          don't require the searcher to be warmed before going live or other
@@ -98,12 +100,12 @@ public final class SearcherManager extends ReferenceManager<IndexSearcher> {
    * 
    * @throws IOException if there is a low-level I/O error
    */
-  public SearcherManager(IndexWriter writer, boolean applyAllDeletes, SearcherFactory searcherFactory) throws IOException {
+  public SearcherManager(IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes, SearcherFactory searcherFactory) throws IOException {
     if (searcherFactory == null) {
       searcherFactory = new SearcherFactory();
     }
     this.searcherFactory = searcherFactory;
-    current = getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes), null);
+    current = getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes, writeAllDeletes), null);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
new file mode 100644
index 0000000..6363ed1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
@@ -0,0 +1,163 @@
+package org.apache.lucene.store;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+
+/** 
+ * DataInput backed by a byte array.
+ * <b>WARNING:</b> This class omits all low-level checks.
+ * @lucene.experimental 
+ */
+public final class ByteArrayIndexInput extends IndexInput {
+
+  private byte[] bytes;
+
+  private int pos;
+  private int limit;
+
+  public ByteArrayIndexInput(String description, byte[] bytes) {
+    super(description);
+    this.bytes = bytes;
+    this.limit = bytes.length;
+  }
+
+  public long getFilePointer() {
+    return pos;
+  }
+  
+  public void seek(long pos) {
+    this.pos = (int) pos;
+  }
+
+  public void reset(byte[] bytes, int offset, int len) {
+    this.bytes = bytes;
+    pos = offset;
+    limit = offset + len;
+  }
+
+  @Override
+  public long length() {
+    return limit;
+  }
+
+  public boolean eof() {
+    return pos == limit;
+  }
+
+  @Override
+  public void skipBytes(long count) {
+    pos += count;
+  }
+
+  @Override
+  public short readShort() {
+    return (short) (((bytes[pos++] & 0xFF) <<  8) |  (bytes[pos++] & 0xFF));
+  }
+ 
+  @Override
+  public int readInt() {
+    return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
+      | ((bytes[pos++] & 0xFF) <<  8) |  (bytes[pos++] & 0xFF);
+  }
+ 
+  @Override
+  public long readLong() {
+    final int i1 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
+      ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
+    final int i2 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
+      ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
+    return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
+  }
+
+  @Override
+  public int readVInt() {
+    byte b = bytes[pos++];
+    if (b >= 0) return b;
+    int i = b & 0x7F;
+    b = bytes[pos++];
+    i |= (b & 0x7F) << 7;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7F) << 14;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7F) << 21;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    // Warning: the next ands use 0x0F / 0xF0 - beware copy/paste errors:
+    i |= (b & 0x0F) << 28;
+    if ((b & 0xF0) == 0) return i;
+    throw new RuntimeException("Invalid vInt detected (too many bits)");
+  }
+ 
+  @Override
+  public long readVLong() {
+    byte b = bytes[pos++];
+    if (b >= 0) return b;
+    long i = b & 0x7FL;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 7;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 14;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 21;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 28;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 35;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 42;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 49;
+    if (b >= 0) return i;
+    b = bytes[pos++];
+    i |= (b & 0x7FL) << 56;
+    if (b >= 0) return i;
+    throw new RuntimeException("Invalid vLong detected (negative values disallowed)");
+  }
+
+  // NOTE: AIOOBE not EOF if you read too much
+  @Override
+  public byte readByte() {
+    return bytes[pos++];
+  }
+
+  // NOTE: AIOOBE not EOF if you read too much
+  @Override
+  public void readBytes(byte[] b, int offset, int len) {
+    System.arraycopy(bytes, pos, b, offset, len);
+    pos += len;
+  }
+
+  @Override
+  public void close() {
+  }
+
+  public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
index fd5e3d7..538e7c0 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
@@ -174,8 +174,12 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
 
   @Override
   public void renameFile(String source, String dest) throws IOException {
-    // NOTE: uncache is unnecessary for lucene's usage, as we always sync() before renaming.
     unCache(source);
+    try {
+      cache.deleteFile(dest);
+    } catch (FileNotFoundException fnfe) {
+      // OK -- it may not exist
+    }
     in.renameFile(source, dest);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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 510545f..1c5aabd 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -107,6 +107,9 @@ public final class IOUtils {
    *          objects to call <tt>close()</tt> on
    */
   public static void closeWhileHandlingException(Closeable... objects) {
+    if (objects.length == 0) {
+      throw new IllegalArgumentException("pass at least one Closeable");
+    }
     closeWhileHandlingException(Arrays.asList(objects));
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index baa2484..ab75fb8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -1036,7 +1036,7 @@ public class TestIndexWriter extends LuceneTestCase {
               if (random.nextInt(3) == 0) {
                 IndexReader r = null;
                 try {
-                  r = DirectoryReader.open(w, random.nextBoolean());
+                  r = DirectoryReader.open(w, random.nextBoolean(), false);
                   if (random.nextBoolean() && r.maxDoc() > 0) {
                     int docid = random.nextInt(r.maxDoc());
                     w.tryDeleteDocument(r, docid);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
index 9213c25..b970519 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
@@ -1249,7 +1249,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
     iwc = new IndexWriterConfig(new MockAnalyzer(random()));
     iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND);
     w = new IndexWriter(d, iwc);
-    IndexReader r = DirectoryReader.open(w, false);
+    IndexReader r = DirectoryReader.open(w, false, false);
     assertTrue(w.tryDeleteDocument(r, 1));
     assertTrue(w.tryDeleteDocument(r.leaves().get(0).reader(), 0));
     r.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
index 58e3ac7..2013ab3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
@@ -200,7 +200,7 @@ public class TestIndexWriterExceptions2 extends LuceneTestCase {
             if (random().nextBoolean()) {
               DirectoryReader ir = null;
               try {
-                ir = DirectoryReader.open(iw, random().nextBoolean());
+                ir = DirectoryReader.open(iw, random().nextBoolean(), false);
                 TestUtil.checkReader(ir);
               } finally {
                 IOUtils.closeWhileHandlingException(ir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
index 777ef46..3fecf71 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
@@ -196,29 +196,29 @@ public class TestIndexWriterOnJRECrash extends TestNRTThreads {
         vendor.startsWith("Sun") || 
         vendor.startsWith("Apple");
 
-      try {
-        if (supportsUnsafeNpeDereference) {
-          try {
-            Class<?> clazz = Class.forName("sun.misc.Unsafe");
-            Field field = clazz.getDeclaredField("theUnsafe");
-            field.setAccessible(true);
-            Object o = field.get(null);
-            Method m = clazz.getMethod("putAddress", long.class, long.class);
-            m.invoke(o, 0L, 0L);
-          } catch (Throwable e) {
-            System.out.println("Couldn't kill the JVM via Unsafe.");
-            e.printStackTrace(System.out); 
-          }
+    try {
+      if (supportsUnsafeNpeDereference) {
+        try {
+          Class<?> clazz = Class.forName("sun.misc.Unsafe");
+          Field field = clazz.getDeclaredField("theUnsafe");
+          field.setAccessible(true);
+          Object o = field.get(null);
+          Method m = clazz.getMethod("putAddress", long.class, long.class);
+          m.invoke(o, 0L, 0L);
+        } catch (Throwable e) {
+          System.out.println("Couldn't kill the JVM via Unsafe.");
+          e.printStackTrace(System.out); 
         }
-
-        // Fallback attempt to Runtime.halt();
-        Runtime.getRuntime().halt(-1);
-      } catch (Exception e) {
-        System.out.println("Couldn't kill the JVM.");
-        e.printStackTrace(System.out); 
       }
 
-      // We couldn't get the JVM to crash for some reason.
-      fail();
+      // Fallback attempt to Runtime.halt();
+      Runtime.getRuntime().halt(-1);
+    } catch (Exception e) {
+      System.out.println("Couldn't kill the JVM.");
+      e.printStackTrace(System.out); 
+    }
+
+    // We couldn't get the JVM to crash for some reason.
+    fail();
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
index 90371fe..1c8a43e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
@@ -169,7 +169,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
               if (random().nextBoolean()) {
                 DirectoryReader ir = null;
                 try {
-                  ir = DirectoryReader.open(iw, random().nextBoolean());
+                  ir = DirectoryReader.open(iw, random().nextBoolean(), false);
                   TestUtil.checkReader(ir);
                 } finally {
                   IOUtils.closeWhileHandlingException(ir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
index f367024..51c0eb5 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
@@ -124,7 +124,7 @@ public class TestRollingUpdates extends LuceneTestCase {
           System.out.println("TEST: reopen applyDeletions=" + applyDeletions);
         }
 
-        r = w.getReader(applyDeletions);
+        r = w.getReader(applyDeletions, false);
         if (applyDeletions) {
           s = newSearcher(r);
         } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
index 1df019c..cc4b80c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
@@ -83,7 +83,7 @@ public class TestStressDeletes extends LuceneTestCase {
                   }
                 }
                 if (random().nextInt(500) == 2) {
-                  DirectoryReader.open(w, random().nextBoolean()).close();
+                  DirectoryReader.open(w, random().nextBoolean(), false).close();
                 }
                 if (random().nextInt(500) == 2) {
                   w.commit();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
index 2e1f385..d90eaba 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
@@ -231,7 +231,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
         }
       };
 
-    nrtNoDeletes = new SearcherManager(writer, false, sf);
+    nrtNoDeletes = new SearcherManager(writer, false, false, sf);
     nrtDeletes = new SearcherManager(writer, sf);
                          
     nrtDeletesThread = new ControlledRealTimeReopenThread<>(genWriter, nrtDeletes, maxReopenSec, minReopenSec);
@@ -313,7 +313,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
 
     LatchedIndexWriter _writer = new LatchedIndexWriter(d, conf, latch, signal);
     final TrackingIndexWriter writer = new TrackingIndexWriter(_writer);
-    final SearcherManager manager = new SearcherManager(_writer, false, null);
+    final SearcherManager manager = new SearcherManager(_writer, false, false, null);
     Document doc = new Document();
     doc.add(newTextField("test", "test", Field.Store.YES));
     writer.addDocument(doc);
@@ -423,7 +423,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
       };
 
     try {
-      new SearcherManager(w.w, false, theEvilOne);
+      new SearcherManager(w.w, false, false, theEvilOne);
       fail("didn't hit expected exception");
     } catch (IllegalStateException ise) {
       // expected

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index db632de..8eb7b44 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -94,7 +94,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
       }
     };
     final boolean applyDeletes = random().nextBoolean();
-    final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, searcherFactory);
+    final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, false, searcherFactory);
     final AtomicBoolean indexing = new AtomicBoolean(true);
     final AtomicReference<Throwable> error = new AtomicReference<>();
     final int numDocs = atLeast(10000);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
index 76b56a5..3218569 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
@@ -231,7 +231,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
     };
     final SearcherManager searcherManager = random().nextBoolean() 
         ? new SearcherManager(dir, factory) 
-        : new SearcherManager(writer, random().nextBoolean(), factory);
+      : new SearcherManager(writer, random().nextBoolean(), false, factory);
     if (VERBOSE) {
       System.out.println("sm created");
     }
@@ -311,7 +311,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
     Directory dir = newDirectory();
     IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
         new MockAnalyzer(random())).setMergeScheduler(new ConcurrentMergeScheduler()));
-    SearcherManager sm = new SearcherManager(writer, false, new SearcherFactory());
+    SearcherManager sm = new SearcherManager(writer, false, false, new SearcherFactory());
     writer.addDocument(new Document());
     writer.commit();
     sm.maybeRefreshBlocking();
@@ -368,7 +368,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
     Directory dir = newDirectory();
     IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
     final AtomicBoolean afterRefreshCalled = new AtomicBoolean(false);
-    SearcherManager sm = new SearcherManager(iw, false, new SearcherFactory());
+    SearcherManager sm = new SearcherManager(iw, false, false, new SearcherFactory());
     sm.addListener(new ReferenceManager.RefreshListener() {
       @Override
       public void beforeRefresh() {
@@ -411,7 +411,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
       // expected
     }
     try {
-      new SearcherManager(w.w, random.nextBoolean(), theEvilOne);
+      new SearcherManager(w.w, random.nextBoolean(), false, theEvilOne);
     } catch (IllegalStateException ise) {
       // expected
     }
@@ -522,7 +522,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
     }
 
     MySearcherFactory factory = new MySearcherFactory();
-    final SearcherManager sm = new SearcherManager(w, random().nextBoolean(), factory);
+    final SearcherManager sm = new SearcherManager(w, random().nextBoolean(), false, factory);
     assertEquals(1, factory.called);
     assertNull(factory.lastPreviousReader);
     assertNotNull(factory.lastReader);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
index cdd0a15..5a3a551 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
@@ -76,7 +76,7 @@ public class SearcherTaxonomyManager extends ReferenceManager<SearcherTaxonomyMa
     this.searcherFactory = searcherFactory;
     this.taxoWriter = taxoWriter;
     DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
-    current = new SearcherAndTaxonomy(SearcherManager.getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes), null), taxoReader);
+    current = new SearcherAndTaxonomy(SearcherManager.getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes, false), null), taxoReader);
     this.taxoEpoch = taxoWriter.getTaxonomyEpoch();
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
index e241007..6396b16 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
@@ -271,7 +271,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
         // verify that the taxo-writer hasn't been closed on us.
         ensureOpen();
         if (!initializedReaderManager) {
-          readerManager = new ReaderManager(indexWriter, false);
+          readerManager = new ReaderManager(indexWriter, false, false);
           shouldRefreshReaderManager = false;
           initializedReaderManager = true;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/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
new file mode 100644
index 0000000..5baf1d3
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
@@ -0,0 +1,237 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.util.IOUtils;
+
+/** Handles copying one set of files, e.g. all files for a new NRT point, or files for pre-copying a merged segment.
+ *  This notifies the caller via OnceDone when the job finishes or failed. */
+
+public abstract class CopyJob implements Comparable<CopyJob> {
+  private final static AtomicLong counter = new AtomicLong();
+  protected final ReplicaNode dest;
+
+  protected final Map<String,FileMetaData> files;
+
+  public final long ord = counter.incrementAndGet();
+
+  /** True for an NRT sync, false for pre-copying a newly merged segment */
+  public final boolean highPriority;
+
+  public final OnceDone onceDone;
+
+  public final long startNS = System.nanoTime();
+
+  public final String reason;
+
+  protected final List<Map.Entry<String,FileMetaData>> toCopy;
+
+  protected long totBytes;
+
+  protected long totBytesCopied;
+
+  // The file we are currently copying:
+  protected CopyOneFile current;
+
+  // Set when we are cancelled
+  protected volatile Throwable exc;
+  protected volatile String cancelReason;
+
+  // toString may concurrently access this:
+  protected final Map<String,String> copiedFiles = new ConcurrentHashMap<>();
+
+  protected CopyJob(String reason, Map<String,FileMetaData> files, ReplicaNode dest, boolean highPriority, OnceDone onceDone) throws IOException {
+    this.reason = reason;
+    this.files = files;
+    this.dest = dest;
+    this.highPriority = highPriority;
+    this.onceDone = onceDone;
+
+    // Exceptions in here are bad:
+    try {
+      this.toCopy = dest.getFilesToCopy(this.files);
+    } catch (Throwable t) {
+      cancel("exc during init", t);
+      throw new CorruptIndexException("exception while checking local files", "n/a", t);
+    }
+  }
+
+  /** Callback invoked by CopyJob once all files have (finally) finished copying */
+  public interface OnceDone {
+    public void run(CopyJob job) throws IOException;
+  }
+
+  /** Transfers whatever tmp files were already copied in this previous job and cancels the previous job */
+  public synchronized void transferAndCancel(CopyJob prevJob) throws IOException {
+    synchronized(prevJob) {
+      dest.message("CopyJob: now transfer prevJob " + prevJob);
+      try {
+        _transferAndCancel(prevJob);
+      } catch (Throwable t) {
+        dest.message("xfer: exc during transferAndCancel");
+        cancel("exc during transferAndCancel", t);
+        IOUtils.reThrow(t);
+      }
+    }
+  }
+
+  private synchronized void _transferAndCancel(CopyJob prevJob) throws IOException {
+
+    // Caller must already be sync'd on prevJob:
+    assert Thread.holdsLock(prevJob);
+
+    if (prevJob.exc != null) {
+      // Already cancelled
+      dest.message("xfer: prevJob was already cancelled; skip transfer");
+      return;
+    }
+
+    // Cancel the previous job
+    prevJob.exc = new Throwable();
+
+    // Carry over already copied files that we also want to copy
+    Iterator<Map.Entry<String,FileMetaData>> it = toCopy.iterator();
+    long bytesAlreadyCopied = 0;
+
+    // Iterate over all files we think we need to copy:
+    while (it.hasNext()) {
+      Map.Entry<String,FileMetaData> ent = it.next();
+      String fileName = ent.getKey();
+      String prevTmpFileName = prevJob.copiedFiles.get(fileName);
+      if (prevTmpFileName != null) {
+        // This fileName is common to both jobs, and the old job already finished copying it (to a temp file), so we keep it:
+        long fileLength = ent.getValue().length;
+        bytesAlreadyCopied += fileLength;
+        dest.message("xfer: carry over already-copied file " + fileName + " (" + prevTmpFileName + ", " + fileLength + " bytes)");
+        copiedFiles.put(fileName, prevTmpFileName);
+
+        // So we don't try to delete it, below:
+        prevJob.copiedFiles.remove(fileName);
+
+        // So it's not in our copy list anymore:
+        it.remove();
+      } else if (prevJob.current != null && prevJob.current.name.equals(fileName)) {
+        // This fileName is common to both jobs, and it's the file that the previous job was in the process of copying.  In this case
+        // we continue copying it from the prevoius job.  This is important for cases where we are copying over a large file
+        // because otherwise we could keep failing the NRT copy and restarting this file from the beginning and never catch up:
+        dest.message("xfer: carry over in-progress file " + fileName + " (" + prevJob.current.tmpName + ") bytesCopied=" + prevJob.current.getBytesCopied() + " of " + prevJob.current.bytesToCopy);
+        bytesAlreadyCopied += prevJob.current.getBytesCopied();
+
+        assert current == null;
+
+        // must set current first, before writing/read to c.in/out in case that hits an exception, so that we then close the temp
+        // IndexOutput when cancelling ourselves:
+        current = newCopyOneFile(prevJob.current);
+
+        // Tell our new (primary) connection we'd like to copy this file first, but resuming from how many bytes we already copied last time:
+        // We do this even if bytesToCopy == bytesCopied, because we still need to readLong() the checksum from the primary connection:
+        assert prevJob.current.getBytesCopied() <= prevJob.current.bytesToCopy;
+
+        prevJob.current = null;
+
+        totBytes += current.metaData.length;
+
+        // So it's not in our copy list anymore:
+        it.remove();
+      } else {
+        dest.message("xfer: file " + fileName + " will be fully copied");
+      }
+    }
+    dest.message("xfer: " + bytesAlreadyCopied + " bytes already copied of " + totBytes);
+
+    // Delete all temp files the old job wrote but we don't need:
+    dest.message("xfer: now delete old temp files: " + prevJob.copiedFiles.values());
+    IOUtils.deleteFilesIgnoringExceptions(dest.dir, prevJob.copiedFiles.values());
+
+    if (prevJob.current != null) { 
+      IOUtils.closeWhileHandlingException(prevJob.current);
+      if (Node.VERBOSE_FILES) {
+        dest.message("remove partial file " + prevJob.current.tmpName);
+      }
+      dest.deleter.deleteNewFile(prevJob.current.tmpName);
+      prevJob.current = null;
+    }
+  }
+
+  protected abstract CopyOneFile newCopyOneFile(CopyOneFile current);
+
+  /** Begin copying files */
+  public abstract void start() throws IOException;
+
+  /** Use current thread (blocking) to do all copying and then return once done, or throw exception on failure */
+  public abstract void runBlocking() throws Exception;
+
+  public void cancel(String reason, Throwable exc) {
+    if (this.exc != null) {
+      // Already cancelled
+      return;
+    }
+
+    dest.message(String.format(Locale.ROOT, "top: cancel after copying %s; exc=%s:\n  files=%s\n  copiedFiles=%s",
+                               Node.bytesToString(totBytesCopied),
+                               exc,
+                               files == null ? "null" : files.keySet(), copiedFiles.keySet()));
+
+    if (exc == null) {
+      exc = new Throwable();
+    }
+
+    this.exc = exc;
+    this.cancelReason = reason;
+
+    // Delete all temp files we wrote:
+    IOUtils.deleteFilesIgnoringExceptions(dest.dir, copiedFiles.values());
+
+    if (current != null) { 
+      IOUtils.closeWhileHandlingException(current);
+      if (Node.VERBOSE_FILES) {
+        dest.message("remove partial file " + current.tmpName);
+      }
+      dest.deleter.deleteNewFile(current.tmpName);
+      current = null;
+    }
+  }
+
+  /** Return true if this job is trying to copy any of the same files as the other job */
+  public abstract boolean conflicts(CopyJob other);
+
+  /** Renames all copied (tmp) files to their true file names */
+  public abstract void finish() throws IOException;
+
+  public abstract boolean getFailed();
+
+  /** Returns only those file names (a subset of {@link #getFileNames}) that need to be copied */
+  public abstract Set<String> getFileNamesToCopy();
+
+  /** Returns all file names referenced in this copy job */
+  public abstract Set<String> getFileNames();
+
+  public abstract CopyState getCopyState();
+
+  public abstract long getTotalBytesCopied();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
new file mode 100644
index 0000000..e3f0f7d
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
@@ -0,0 +1,132 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Locale;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+/** Copies one file from an incoming DataInput to a dest filename in a local Directory */
+
+class CopyOneFile implements Closeable {
+  private final DataInput in;
+  private final IndexOutput out;
+  private final ReplicaNode dest;
+  public final String name;
+  public final String tmpName;
+  public final FileMetaData metaData;
+  public final long bytesToCopy;
+  private final long copyStartNS;
+  private final byte[] buffer;
+
+  private long bytesCopied;
+
+  public CopyOneFile(DataInput in, ReplicaNode dest, String name, FileMetaData metaData, byte[] buffer) throws IOException {
+    this.in = in;
+    this.name = name;
+    this.dest = dest;
+    this.buffer = buffer;
+    // TODO: pass correct IOCtx, e.g. seg total size
+    out = dest.createTempOutput(name, "copy", IOContext.DEFAULT);
+    tmpName = out.getName();
+
+    // last 8 bytes are checksum:
+    bytesToCopy = metaData.length - 8;
+
+    if (Node.VERBOSE_FILES) {
+      dest.message("file " + name + ": start copying to tmp file " + tmpName + " length=" + (8+bytesToCopy));
+    }
+
+    copyStartNS = System.nanoTime();
+    this.metaData = metaData;
+    dest.startCopyFile(name);
+  }
+
+  /** Transfers this file copy to another input, continuing where the first one left off */
+  public CopyOneFile(CopyOneFile other, DataInput in) {
+    this.in = in;
+    this.dest = other.dest;
+    this.name = other.name;
+    this.out = other.out;
+    this.tmpName = other.tmpName;
+    this.metaData = other.metaData;
+    this.bytesCopied = other.bytesCopied;
+    this.bytesToCopy = other.bytesToCopy;
+    this.copyStartNS = other.copyStartNS;
+    this.buffer = other.buffer;
+  }
+
+  public void close() throws IOException {
+    out.close();
+    dest.finishCopyFile(name);
+  }
+
+  /** Copy another chunk of bytes, returning true once the copy is done */
+  public boolean visit() throws IOException {
+    // Copy up to 640 KB per visit:
+    for(int i=0;i<10;i++) {
+      long bytesLeft = bytesToCopy - bytesCopied;
+      if (bytesLeft == 0) {
+        long checksum = out.getChecksum();
+        if (checksum != metaData.checksum) {
+          // Bits flipped during copy!
+          dest.message("file " + tmpName + ": checksum mismatch after copy (bits flipped during network copy?) after-copy checksum=" + checksum + " vs expected=" + metaData.checksum + "; cancel job");
+          throw new IOException("file " + name + ": checksum mismatch after file copy");
+        }
+
+        // Paranoia: make sure the primary node is not smoking crack, by somehow sending us an already corrupted file whose checksum (in its
+        // footer) disagrees with reality:
+        long actualChecksumIn = in.readLong();
+        if (actualChecksumIn != checksum) {
+          dest.message("file " + tmpName + ": checksum claimed by primary disagrees with the file's footer: claimed checksum=" + checksum + " vs actual=" + actualChecksumIn);
+          throw new IOException("file " + name + ": checksum mismatch after file copy");
+        }
+        out.writeLong(checksum);
+        close();
+
+        if (Node.VERBOSE_FILES) {
+          dest.message(String.format(Locale.ROOT, "file %s: done copying [%s, %.3fms]",
+                                     name,
+                                     Node.bytesToString(metaData.length),
+                                     (System.nanoTime() - copyStartNS)/1000000.0));
+        }
+
+        return true;
+      }
+
+      int toCopy = (int) Math.min(bytesLeft, buffer.length);
+      in.readBytes(buffer, 0, toCopy);
+      out.writeBytes(buffer, 0, toCopy);
+
+      // TODO: rsync will fsync a range of the file; maybe we should do that here for large files in case we crash/killed
+      bytesCopied += toCopy;
+    }
+
+    return false;
+  }
+
+  public long getBytesCopied() {
+    return bytesCopied;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
new file mode 100644
index 0000000..c19fabc
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
@@ -0,0 +1,56 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.store.Directory;
+
+/** Holds incRef'd file level details for one point-in-time segment infos on the primary node. */
+public class CopyState {
+
+  public final Map<String,FileMetaData> files;
+  public final long version;
+  public final long gen;
+  public final byte[] infosBytes;
+  public final Set<String> completedMergeFiles;
+  public final long primaryGen;
+  
+  // only non-null on the primary node
+  public final SegmentInfos infos;
+
+  public CopyState(Map<String,FileMetaData> files, long version, long gen, byte[] infosBytes,
+                   Set<String> completedMergeFiles, long primaryGen, SegmentInfos infos) {
+    assert completedMergeFiles != null;
+    this.files = Collections.unmodifiableMap(files);
+    this.version = version;
+    this.gen = gen;
+    this.infosBytes = infosBytes;
+    this.completedMergeFiles = Collections.unmodifiableSet(completedMergeFiles);
+    this.primaryGen = primaryGen;
+    this.infos = infos;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(version=" + version + ")";
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
new file mode 100644
index 0000000..aca408c
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
@@ -0,0 +1,40 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Holds metadata details about a single file that we use to confirm two files (one remote, one local) are in fact "identical". */
+
+class FileMetaData {
+
+  // Header and footer of the file must be identical between primary and replica to consider the files equal:
+  public final byte[] header;
+  public final byte[] footer;
+
+  public final long length;
+
+  // Used to ensure no bit flips when copying the file:
+  public final long checksum;
+
+  public FileMetaData(byte[] header, byte[] footer, long length, long checksum) {
+    this.header = header;
+    this.footer = footer;
+    this.length = length;
+    this.checksum = checksum;
+  }
+}
+


[07/15] lucene-solr git commit: fix more nocommits; add separate test that deleteAll can replicate

Posted by mi...@apache.org.
fix more nocommits; add separate test that deleteAll can replicate


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 022540e8c23e57046f1b54a9cd485dab5ff4b563
Parents: 3389068
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Jan 27 06:00:12 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Jan 27 06:00:12 2016 -0500

----------------------------------------------------------------------
 .../nrt/PreCopyMergedSegmentWarmer.java         |   18 -
 .../lucene/replicator/nrt/PrimaryNode.java      |   18 -
 .../lucene/replicator/nrt/ReplicaNode.java      |   25 +-
 .../lucene/replicator/nrt/NodeProcess.java      |   18 +-
 .../replicator/nrt/SimplePrimaryNode.java       |   53 +-
 .../replicator/nrt/SimpleReplicaNode.java       |   30 +-
 .../lucene/replicator/nrt/SimpleServer.java     |   38 +-
 .../replicator/nrt/TestNRTReplication.java      | 1090 ++--------------
 .../nrt/TestStressNRTReplication.java           | 1160 ++++++++++++++++++
 9 files changed, 1359 insertions(+), 1091 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
index 1918ede..77f23ab 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
@@ -23,30 +23,14 @@ package org.apache.lucene.replicator.nrt;
  *  flushed segment sizes, not merged segments. */
 
 import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
 
 import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.replicator.nrt.CopyJob.OnceDone;
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.ThreadInterruptedException;
 
 // TODO: or ... replica node can do merging locally?  tricky to keep things in sync, when one node merges more slowly than others...
 
@@ -71,8 +55,6 @@ class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
       filesMetaData.put(fileName, metaData);
     }
 
-    // nocommit if one replica is very slow then it dos's all other replicas?
-
     primary.preCopyMergedSegmentFiles(info, filesMetaData);
     primary.message(String.format(Locale.ROOT, "top: done warm merge " + info + ": took %.3f sec, %.1f MB", (System.nanoTime()-startNS)/1000000000., info.sizeInBytes()/1024/1024.));
     primary.finishedMergedFiles.addAll(filesMetaData.keySet());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index 183f16f..ccd8848 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -17,43 +17,26 @@ package org.apache.lucene.replicator.nrt;
  * limitations under the License.
  */
 
-import java.io.Closeable;
 import java.io.IOException;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogMergePolicy;
-import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.SegmentCommitInfo;
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.index.StandardDirectoryReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.SearcherManager;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.RAMFile;
 import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PrintStreamInfoStream;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /*
@@ -178,7 +161,6 @@ public abstract class PrimaryNode extends Node {
     // on xlog replay we are replaying more ops than necessary.
     commitData.put(VERSION_KEY, Long.toString(copyState.version));
     message("top: commit commitData=" + commitData);
-    // nocommit this is now an NRT-visible change!  make test where nothing is indexing and confirm we don't do silly commit + refresh loop forever!
     writer.setCommitData(commitData, false);
     writer.commit();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/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 af142d5..713c6f1 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
@@ -17,14 +17,7 @@ package org.apache.lucene.replicator.nrt;
  * limitations under the License.
  */
 
-import java.io.EOFException;
-import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -34,15 +27,11 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.SegmentInfos;
@@ -52,24 +41,12 @@ import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedChecksumIndexInput;
-import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.ByteArrayIndexInput;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.store.RateLimiter;
 import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */
 
@@ -544,7 +521,7 @@ abstract class ReplicaNode extends Node {
       synchronized (mergeCopyJobs) {
         for (CopyJob mergeJob : mergeCopyJobs) {
           if (mergeJob.getFileNames().contains(fileName)) {
-            // nocommit can't we .transferAndCancel?
+            // TODO: we could maybe transferAndCancel here?  except CopyJob can't transferAndCancel more than one currently
             message("top: now cancel merge copy job=" + mergeJob + ": file " + fileName + " is now being copied via NRT point");
             mergeJob.cancel("newNRTPoint is copying over the same file", null);
           }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index 4e29508..9de2c04 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -19,18 +19,9 @@ package org.apache.lucene.replicator.nrt;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
 
 /** Parent JVM hold this "wrapper" to refer to each child JVM.  This is roughly equivalent e.g. to a client-side "sugar" API. */
 class NodeProcess implements Closeable {
@@ -234,5 +225,14 @@ class NodeProcess implements Closeable {
     c.flush();
     c.in.readByte();
   }
+
+  public void deleteAllDocuments(Connection c) throws IOException {
+    if (isPrimary == false) {
+      throw new IllegalStateException("only primary can index");
+    }
+    c.out.writeByte(SimplePrimaryNode.CMD_DELETE_ALL_DOCS);
+    c.flush();
+    c.in.readByte();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
index 18e77ef..0afd1b4 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -43,17 +43,16 @@ import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.SegmentCommitInfo;
-import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TieredMergePolicy;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.DataInput;
@@ -61,8 +60,6 @@ import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -74,6 +71,8 @@ class SimplePrimaryNode extends PrimaryNode {
 
   final int tcpPort;
 
+  final Random random;
+
   // These are updated by parent test process whenever replicas change:
   int[] replicaTCPPorts;
   int[] replicaIDs;
@@ -81,6 +80,10 @@ class SimplePrimaryNode extends PrimaryNode {
   // So we only flip a bit once per file name:
   final Set<String> bitFlipped = Collections.synchronizedSet(new HashSet<>());
 
+  final List<MergePreCopy> warmingSegments = Collections.synchronizedList(new ArrayList<>());
+
+  final boolean doFlipBitsDuringCopy;
+
   static class MergePreCopy {
     final List<Connection> connections = Collections.synchronizedList(new ArrayList<>());
     final Map<String,FileMetaData> files;
@@ -109,11 +112,12 @@ class SimplePrimaryNode extends PrimaryNode {
     }
   }
 
-  final List<MergePreCopy> warmingSegments = Collections.synchronizedList(new ArrayList<>());
-
-  public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException {
+  public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory,
+                           boolean doFlipBitsDuringCopy) throws IOException {
     super(initWriter(id, random, indexPath), id, primaryGen, forcePrimaryVersion, searcherFactory);
     this.tcpPort = tcpPort;
+    this.random = new Random(random.nextLong());
+    this.doFlipBitsDuringCopy = doFlipBitsDuringCopy;
   }
 
   /** Records currently alive replicas. */
@@ -187,7 +191,7 @@ class SimplePrimaryNode extends PrimaryNode {
       long startNS = System.nanoTime();
       long lastWarnNS = startNS;
 
-      // TODO: maybe ... place some sort of time limit on how long we are willing to wait for slow replicas to finish copying?
+      // TODO: maybe ... place some sort of time limit on how long we are willing to wait for slow replica(s) to finish copying?
       while (preCopy.finished() == false) {
         try {
           Thread.sleep(10);
@@ -241,7 +245,16 @@ class SimplePrimaryNode extends PrimaryNode {
                 message("top: warning: replica socket=" + c.s + " for segment=" + info + " seems to be dead; closing files=" + files.keySet());
                 IOUtils.closeWhileHandlingException(c);
                 it.remove();
+                done = true;
               }
+
+              if (done == false && random.nextInt(1000) == 17) {
+                message("top: warning: now randomly dropping replica from merge warming; files=" + files.keySet());
+                IOUtils.closeWhileHandlingException(c);
+                it.remove();
+                done = true;
+              }
+
             } catch (Throwable t) {
               message("top: ignore exception trying to read byte during warm for segment=" + info + " to replica socket=" + c.s + ": " + t + " files=" + files.keySet());
               IOUtils.closeWhileHandlingException(c);
@@ -368,7 +381,7 @@ class SimplePrimaryNode extends PrimaryNode {
           while (upto < len) {
             int chunk = (int) Math.min(buffer.length, (len-upto));
             in.readBytes(buffer, 0, chunk);
-            if (TestNRTReplication.DO_BIT_FLIPS_DURING_COPY) {
+            if (doFlipBitsDuringCopy) {
               if (random.nextInt(3000) == 17 && bitFlipped.contains(fileName) == false) {
                 bitFlipped.add(fileName);
                 message("file " + fileName + " to R" + replicaID + ": now randomly flipping a bit at byte=" + upto);
@@ -435,6 +448,10 @@ class SimplePrimaryNode extends PrimaryNode {
         handleDeleteDocument(in, out);
         out.writeByte((byte) 1);
         bos.flush();
+      } else if (cmd == CMD_DELETE_ALL_DOCS) {
+        writer.deleteAll();
+        out.writeByte((byte) 1);
+        bos.flush();
       } else if (cmd == CMD_INDEXING_DONE) {
         out.writeByte((byte) 1);
         bos.flush();
@@ -508,6 +525,7 @@ class SimplePrimaryNode extends PrimaryNode {
   static final byte CMD_MARKER_SEARCH = 3;
   static final byte CMD_COMMIT = 4;
   static final byte CMD_CLOSE = 5;
+  static final byte CMD_SEARCH_ALL = 21;
 
   // Send (to primary) the list of currently running replicas:
   static final byte CMD_SET_REPLICAS = 16;
@@ -518,6 +536,7 @@ class SimplePrimaryNode extends PrimaryNode {
   static final byte CMD_UPDATE_DOC = 7;
   static final byte CMD_DELETE_DOC = 8;
   static final byte CMD_INDEXING_DONE = 19;
+  static final byte CMD_DELETE_ALL_DOCS = 22;
 
   // Sent by replica to primary when replica first starts up, so primary can add it to any warming merges:
   static final byte CMD_NEW_REPLICA = 20;
@@ -579,6 +598,22 @@ class SimplePrimaryNode extends PrimaryNode {
         }
         continue outer;
 
+      case CMD_SEARCH_ALL:
+        {
+          Thread.currentThread().setName("search all");
+          IndexSearcher searcher = mgr.acquire();
+          try {
+            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+            int hitCount = searcher.search(new MatchAllDocsQuery(), 1).totalHits;
+            //message("version=" + version + " searcher=" + searcher);
+            out.writeVLong(version);
+            out.writeVInt(hitCount);
+          } finally {
+            mgr.release(searcher);
+          }
+        }
+        continue outer;
+
       case CMD_MARKER_SEARCH:
         {
           Thread.currentThread().setName("msearch");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
index 8667df1..27a5547 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -24,19 +24,16 @@ import java.io.InputStream;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.nio.file.Path;
-import java.util.Collections;
-import java.util.HashSet;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.DataInput;
@@ -45,7 +42,6 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.store.RateLimitedIndexOutput;
 import org.apache.lucene.store.RateLimiter;
 import org.apache.lucene.util.LuceneTestCase;
@@ -182,7 +178,7 @@ class SimpleReplicaNode extends ReplicaNode {
         break;
 
       case SimplePrimaryNode.CMD_GET_SEARCHING_VERSION:
-        // nocommit this is hacky:
+        // This is called when primary has crashed and we need to elect a new primary from all the still running replicas:
 
         // Tricky: if a sync is just finishing up, i.e. managed to finish copying all files just before we crashed primary, and is now
         // in the process of opening a new reader, we need to wait for it, to be sure we really pick the most current replica:
@@ -190,7 +186,7 @@ class SimpleReplicaNode extends ReplicaNode {
           message("top: getSearchingVersion: now wait for finish sync");
           // TODO: use immediate concurrency instead of polling:
           while (isCopying() && stop.get() == false) {
-            Thread.sleep(50);
+            Thread.sleep(10);
             message("top: curNRTCopy=" + curNRTCopy);
           }
           message("top: getSearchingVersion: done wait for finish sync");
@@ -212,6 +208,24 @@ class SimpleReplicaNode extends ReplicaNode {
             //node.message("version=" + version + " searcher=" + searcher);
             out.writeVLong(version);
             out.writeVInt(hitCount);
+            bos.flush();
+          } finally {
+            mgr.release(searcher);
+          }
+        }
+        continue outer;
+
+      case SimplePrimaryNode.CMD_SEARCH_ALL:
+        {
+          Thread.currentThread().setName("search all");
+          IndexSearcher searcher = mgr.acquire();
+          try {
+            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+            int hitCount = searcher.search(new MatchAllDocsQuery(), 1).totalHits;
+            //node.message("version=" + version + " searcher=" + searcher);
+            out.writeVLong(version);
+            out.writeVInt(hitCount);
+            bos.flush();
           } finally {
             mgr.release(searcher);
           }
@@ -227,6 +241,7 @@ class SimpleReplicaNode extends ReplicaNode {
             int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
             out.writeVLong(version);
             out.writeVInt(hitCount);
+            bos.flush();
           } finally {
             mgr.release(searcher);
           }
@@ -290,6 +305,7 @@ class SimpleReplicaNode extends ReplicaNode {
       default:
         throw new IllegalArgumentException("unrecognized cmd=" + cmd);
       }
+      System.out.println("NOW FLUSH");
       bos.flush();
 
       break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index 7a257de..5b04721 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -19,61 +19,32 @@ package org.apache.lucene.replicator.nrt;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
-import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
 import java.lang.reflect.Method;
-import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketException;
 import java.nio.file.Path;
-import java.nio.file.Path;
-import java.nio.file.Paths;
 import java.nio.file.Paths;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
-import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
 import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
 /** Child process with silly naive TCP socket server to handle
@@ -262,8 +233,9 @@ public class SimpleServer extends LuceneTestCase {
     long primaryGen = Long.parseLong(System.getProperty("tests.nrtreplication.primaryGen"));
     Node.globalStartNS = Long.parseLong(System.getProperty("tests.nrtreplication.startNS"));
 
-    boolean doRandomCrash = isPrimary ? TestNRTReplication.DO_CRASH_PRIMARY : TestNRTReplication.DO_CRASH_REPLICA;
-    boolean doRandomClose = isPrimary ? false : TestNRTReplication.DO_CLOSE_REPLICA;
+    boolean doRandomCrash = "true".equals(System.getProperty("tests.nrtreplication.doRandomCrash"));
+    boolean doRandomClose = "true".equals(System.getProperty("tests.nrtreplication.doRandomClose"));
+    boolean doFlipBitsDuringCopy = "true".equals(System.getProperty("tests.nrtreplication.doFlipBitsDuringCopy"));
 
     // Create server socket that we listen for incoming requests on:
     try (final ServerSocket ss = new ServerSocket(0)) {
@@ -272,7 +244,7 @@ public class SimpleServer extends LuceneTestCase {
       System.out.println("\nPORT: " + tcpPort);
       final Node node;
       if (isPrimary) {
-        node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null);
+        node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy);
         System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
       } else {
         node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 5a073ff..755b234 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -18,499 +18,39 @@ package org.apache.lucene.replicator.nrt;
  */
 
 import java.io.BufferedReader;
-import java.io.Closeable;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.UnsupportedEncodingException;
-import java.io.Writer;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.charset.MalformedInputException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 
-import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.ConcurrentMergeScheduler;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.store.RateLimiter;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.ThreadInterruptedException;
 
 import com.carrotsearch.randomizedtesting.SeedUtils;
 
-// nocommit sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
-
-// nocommit randomly p.destroy() one replica?
-
-/*
-  TODO
-    - why do we do the "rename temp to actual" all at the end...?  what really does that buy us?
-    - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
-    - test should not print scary exceptions and then succeed!
-    - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
-    - are the pre-copied-completed-merged files not being cleared in primary?
-      - hmm the logic isn't right today?  a replica may skip pulling a given copy state, that recorded the finished merged segments?
-    - beast & fix bugs
-    - graceful cluster restart
-    - better translog integration
-    - get "graceful primary shutdown" working
-    - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion
-    - clean up how version is persisted in commit data
-    - why am i not using hashes here?  how does ES use them?
-    - get all other "single shard" functions working too: this cluster should "act like" a single shard
-      - SLM
-      - controlled nrt reopen thread / returning long gen on write
-      - live field values
-      - add indexes
-    - make cluster level APIs to search, index, that deal w/ primary failover, etc.
-    - must prune xlog
-      - refuse to start primary unless we have quorum
-    - later
-      - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback?
-      - back pressure on indexing if replicas can't keep up?
-      - get xlog working on top?  needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit
-        quorum
-        - maybe fix IW to return "gen" or "seq id" or "segment name" or something?
-      - replica can copy files from other replicas too / use multicast / rsync / something
-      - each replica could also pre-open a SegmentReader after pre-copy when warming a merge
-      - we can pre-copy newly flushed files too, for cases where reopen rate is low vs IW's flushing because RAM buffer is full
-      - opto: pre-copy files as they are written; if they will become CFS, we can build CFS on the replica?
-      - what about multiple commit points?
-      - fix primary to init directly from an open replica, instead of having to commit/close the replica first
-*/
-
-// Tricky cases:
-//   - we are pre-copying a merge, then replica starts up part way through, so it misses that pre-copy and must do it on next nrt point
-//   - a down replica starts up, but it's "from the future" vs the current primary, and must copy over file names with different contents
-//     but referenced by its latest commit point, so it must fully remove that commit ... which is a hazardous window
-//   - replica comes up just as the primary is crashing / moving
-//   - electing a new primary when a replica is just finishing its nrt sync: we need to wait for it so we are sure to get the "most up to
-//     date" replica
-//   - replica comes up after merged segment finished so it doesn't copy over the merged segment "promptly" (i.e. only sees it on NRT refresh)
-
-/**
- * Test case showing how to implement NRT replication.  This test spawns a sub-process per-node, running TestNRTReplicationChild.
- *
- * One node is primary, and segments are periodically flushed there, then concurrently the N replica nodes copy the new files over and open new readers, while
- * primary also opens a new reader.
- *
- * Nodes randomly crash and are restarted.  If the primary crashes, a replica is promoted.
- *
- * Merges are currently first finished on the primary and then pre-copied out to replicas with a merged segment warmer so they don't block
- * ongoing NRT reopens.  Probably replicas could do their own merging instead, but this is more complex and may not be better overall
- * (merging takes a lot of IO resources).
- *
- * Slow network is simulated with a RateLimiter.
- */
-
-// nocommit randomly delete all doc sometimes, 1) using IW.deleteAll and 2) doing it inefficiently (by query, by id)
-
 // MockRandom's .sd file has no index header/footer:
 @SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
 @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
 public class TestNRTReplication extends LuceneTestCase {
 
-  // Test evilness controls:
-
-  /** Randomly crash the current primary (losing data!) and promote the "next best" replica. */
-  static final boolean DO_CRASH_PRIMARY = true;
-
-  /** Randomly crash (JVM core dumps) a replica; it will later randomly be restarted and sync itself. */
-  static final boolean DO_CRASH_REPLICA = true;
-
-  /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
-  static final boolean DO_CLOSE_REPLICA = true;
-
-  /** If false, all child + parent output is interleaved into single stdout/err */
-  static final boolean SEPARATE_CHILD_OUTPUT = false;
-
-  // nocommit DO_CLOSE_PRIMARY?
-
-  /** Randomly crash whole cluster and then restart it */
-  static final boolean DO_FULL_CLUSTER_CRASH = true;
-
-  /** True if we randomly flip a bit while copying files out */
-  static final boolean DO_BIT_FLIPS_DURING_COPY = true;
-
-  /** Set to a non-null value to force exactly that many nodes; else, it's random. */
-  static final Integer NUM_NODES = null;
-
-  static final boolean DO_RANDOM_XLOG_REPLAY = false;
-
-  final AtomicBoolean failed = new AtomicBoolean();
-
-  final AtomicBoolean stop = new AtomicBoolean();
-
   /** cwd where we start each child (server) node */
   private Path childTempDir;
 
-  long primaryGen;
-
-  volatile long lastPrimaryVersion;
-
-  volatile NodeProcess primary;
-  volatile NodeProcess[] nodes;
-  volatile long[] nodeTimeStamps;
-  volatile boolean[] starting;
-  
-  Path[] indexPaths;
-
-  Path transLogPath;
-  SimpleTransLog transLog;
-  final AtomicInteger markerUpto = new AtomicInteger();
-
-  /** Maps searcher version to how many hits the query body:the matched. */
-  final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
-
-  /** Maps searcher version to how many marker documents matched.  This should only ever grow (we never delete marker documents). */
-  final Map<Long,Integer> versionToMarker = new ConcurrentHashMap<>();
-
-  /** Maps searcher version to xlog location when refresh of this version started. */
-  final Map<Long,Long> versionToTransLogLocation = new ConcurrentHashMap<>();
-
-  public void test() throws Exception {
-
-    Node.globalStartNS = System.nanoTime();
-
-    message("change thread name from " + Thread.currentThread().getName());
-    Thread.currentThread().setName("main");
-
-    childTempDir = createTempDir("child");
-
-    // We are parent process:
-
-    // Silly bootstrapping:
-    versionToTransLogLocation.put(0L, 0L);
-    versionToTransLogLocation.put(1L, 0L);
-
-    int numNodes;
-
-    if (NUM_NODES == null) {
-      numNodes = TestUtil.nextInt(random(), 2, 10);
-    } else {
-      numNodes = NUM_NODES.intValue();
-    }
-
-    System.out.println("TEST: using " + numNodes + " nodes");
-
-    transLogPath = createTempDir("NRTReplication").resolve("translog");
-    transLog = new SimpleTransLog(transLogPath);
-
-    //state.rateLimiters = new RateLimiter[numNodes];
-    indexPaths = new Path[numNodes];
-    nodes = new NodeProcess[numNodes];
-    nodeTimeStamps = new long[numNodes];
-    Arrays.fill(nodeTimeStamps, Node.globalStartNS);
-    starting = new boolean[numNodes];
-    
-    for(int i=0;i<numNodes;i++) {
-      indexPaths[i] = createTempDir("index" + i);
-    }
-
-    Thread[] indexers = new Thread[TestUtil.nextInt(random(), 1, 3)];
-    System.out.println("TEST: launch " + indexers.length + " indexer threads");
-    for(int i=0;i<indexers.length;i++) {
-      indexers[i] = new IndexThread();
-      indexers[i].setName("indexer" + i);
-      indexers[i].setDaemon(true);
-      indexers[i].start();
-    }
-
-    Thread[] searchers = new Thread[TestUtil.nextInt(random(), 1, 3)];
-    System.out.println("TEST: launch " + searchers.length + " searcher threads");
-    for(int i=0;i<searchers.length;i++) {
-      searchers[i] = new SearchThread();
-      searchers[i].setName("searcher" + i);
-      searchers[i].setDaemon(true);
-      searchers[i].start();
-    }
-
-    Thread restarter = new RestartThread();
-    restarter.setName("restarter");
-    restarter.setDaemon(true);
-    restarter.start();
-
-    int runTimeSec;
-    if (TEST_NIGHTLY) {
-      runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 120, 240);
-    } else {
-      runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 45, 120);
-    }
-
-    System.out.println("TEST: will run for " + runTimeSec + " sec");
-
-    long endTime = System.nanoTime() + runTimeSec*1000000000L;
-
-    sendReplicasToPrimary();
-
-    while (failed.get() == false && System.nanoTime() < endTime) {
-
-      // Wait a bit:
-      Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
-      if (primary != null && random().nextBoolean()) {
-        message("top: now flush primary");
-        NodeProcess curPrimary = primary;
-        if (curPrimary != null) {
-
-          // Save these before we start flush:
-          long nextTransLogLoc = transLog.getNextLocation();
-          int markerUptoSav = markerUpto.get();
-
-          long result;
-          try {
-            result = primary.flush();
-          } catch (Throwable t) {
-            message("top: flush failed; skipping: " + t.getMessage());
-            result = -1;
-          }
-          if (result > 0) {
-            // There were changes
-            lastPrimaryVersion = result;
-            addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
-            addVersionMarker(lastPrimaryVersion, markerUptoSav);
-          }
-        }
-      }
-
-      StringBuilder sb = new StringBuilder();
-      int liveCount = 0;
-      for(int i=0;i<nodes.length;i++) {
-        NodeProcess node = nodes[i];
-        if (node != null) {
-          if (sb.length() != 0) {
-            sb.append(" ");
-          }
-          liveCount++;
-          if (node.isPrimary) {
-            sb.append('P');
-          } else {
-            sb.append('R');
-          }
-          sb.append(i);
-        }
-      }
-
-      message("PG=" + (primary == null ? "X" : primaryGen) + " " + liveCount + " (of " + nodes.length + ") nodes running: " + sb);
-
-      // Commit a random node, primary or replica
-
-      {
-        NodeProcess node = nodes[random().nextInt(nodes.length)];
-        if (node != null) {
-          // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
-          // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
-          message("top: now commit node=" + node);
-          node.commitAsync();
-        }
-      }
-    }
-
-    message("TEST: top: test done, now close");
-    stop.set(true);
-    for(Thread thread : indexers) {
-      thread.join();
-    }
-    for(Thread thread : searchers) {
-      thread.join();
-    }
-    restarter.join();
-
-    // Close replicas before primary so we cancel any in-progres replications:
-    System.out.println("TEST: top: now close replicas");
-    List<Closeable> toClose = new ArrayList<>();
-    for(NodeProcess node : nodes) {
-      if (node != primary && node != null) {
-        toClose.add(node);
-      }
-    }
-    IOUtils.close(toClose);
-    IOUtils.close(primary);
-    IOUtils.close(transLog);
-
-    if (failed.get() == false) {
-      message("TEST: top: now checkIndex");    
-      for(Path path : indexPaths) {
-        message("TEST: check " + path);
-        MockDirectoryWrapper dir = newMockFSDirectory(path);
-        // Just too slow otherwise
-        dir.setCrossCheckTermVectorsOnClose(false);
-        dir.close();
-      }
-    } else {
-      message("TEST: failed; skip checkIndex");
-    }
-  }
-
-  private boolean anyNodesStarting() {
-    for(int id=0;id<nodes.length;id++) {
-      if (starting[id]) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  /** Picks a replica and promotes it as new primary. */
-  private void promoteReplica() throws IOException {
-    message("top: primary crashed; now pick replica to promote");
-    long maxSearchingVersion = -1;
-    NodeProcess replicaToPromote = null;
-
-    // We must promote the most current replica, because otherwise file name reuse can cause a replication to fail when it needs to copy
-    // over a file currently held open for searching.  This also minimizes recovery work since the most current replica means less xlog
-    // replay to catch up:
-    for (NodeProcess node : nodes) {
-      if (node != null) {
-        message("ask " + node + " for its current searching version");
-        long searchingVersion = node.getSearchingVersion();
-        message(node + " has searchingVersion=" + searchingVersion);
-        if (searchingVersion > maxSearchingVersion) {
-          maxSearchingVersion = searchingVersion;
-          replicaToPromote = node;
-        }
-      }
-    }
-
-    if (replicaToPromote == null) {
-      message("top: no replicas running; skipping primary promotion");
-      return;
-    }
-
-    message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
-    if (replicaToPromote.commit() == false) {
-      message("top: commit failed; skipping primary promotion");
-      return;
-    }
-
-    message("top: now shutdown " + replicaToPromote);
-    if (replicaToPromote.shutdown() == false) {
-      message("top: shutdown failed for R" + replicaToPromote.id + "; skipping primary promotion");
-      return;
-    }
-
-    int id = replicaToPromote.id;
-    message("top: now startPrimary " + replicaToPromote);
-    startPrimary(replicaToPromote.id);
-  }
-
-  void startPrimary(int id) throws IOException {
-    message(id + ": top: startPrimary lastPrimaryVersion=" + lastPrimaryVersion);
-    assert nodes[id] == null;
-
-    // Force version of new primary to advance beyond where old primary was, so we never re-use versions.  It may have
-    // already advanced beyond newVersion, e.g. if it flushed new segments while during xlog replay:
-
-    // First start node as primary (it opens an IndexWriter) but do not publish it for searching until we replay xlog:
-    NodeProcess newPrimary = startNode(id, indexPaths[id], true, lastPrimaryVersion+1);
-    if (newPrimary == null) {
-      message("top: newPrimary failed to start; abort");
-      return;
-    }
-
-    // Get xlog location that this node was guaranteed to already have indexed through; this may replay some ops already indexed but it's OK
-    // because the ops are idempotent: we updateDocument (by docid) on replay even for original addDocument:
-    Long startTransLogLoc;
-    Integer markerCount;
-    if (newPrimary.initCommitVersion == 0) {
-      startTransLogLoc = 0L;
-      markerCount = 0;
-    } else {
-      startTransLogLoc = versionToTransLogLocation.get(newPrimary.initCommitVersion);
-      markerCount = versionToMarker.get(newPrimary.initCommitVersion);
-    }
-    assert startTransLogLoc != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToTransLogLocation: keys=" + versionToTransLogLocation.keySet();
-    assert markerCount != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToMarker: keys=" + versionToMarker.keySet();
-
-    // When the primary starts, the userData in its latest commit point tells us which version it had indexed up to, so we know where to
-    // replay from in the xlog.  However, we forcefuly advance the version, and then IW on init (or maybe getReader) also adds 1 to it.
-    // Since we publish the primary in this state (before xlog replay is done), a replica can start up at this point and pull this version,
-    // and possibly later be chosen as a primary, causing problems if the version is known recorded in the translog map.  So we record it
-    // here:
-
-    addTransLogLoc(newPrimary.initInfosVersion, startTransLogLoc);
-    addVersionMarker(newPrimary.initInfosVersion, markerCount);
-
-    assert newPrimary.initInfosVersion >= lastPrimaryVersion;
-    message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
-    lastPrimaryVersion = newPrimary.initInfosVersion;
-
-    // Publish new primary, before replaying xlog.  This means other indexing ops can come in at the same time as we catch up indexing
-    // previous ops.  Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
-    // translog) concurrently with new incoming ops.
-    nodes[id] = newPrimary;
-    primary = newPrimary;
-
-    sendReplicasToPrimary();
-
-    long nextTransLogLoc = transLog.getNextLocation();
-    int nextMarkerUpto = markerUpto.get();
-    message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
-    try {
-      transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
-    } catch (IOException ioe) {
-      message("top: replay xlog failed; abort");
-      return;
-    }
-    message("top: done replay trans log");
-  }
-
   final AtomicLong nodeStartCounter = new AtomicLong();
 
-  final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
-
   /** Launches a child "server" (separate JVM), which is either primary or replica node */
-  NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
-    nodeTimeStamps[id] = System.nanoTime();
+  NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
     List<String> cmd = new ArrayList<>();
 
-    NodeProcess curPrimary = primary;
-
     cmd.add(System.getProperty("java.home") 
         + System.getProperty("file.separator")
         + "bin"
@@ -518,12 +58,13 @@ public class TestNRTReplication extends LuceneTestCase {
         + "java");
     cmd.add("-Xmx512m");
 
-    if (curPrimary != null) {
-      cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + curPrimary.tcpPort);
+    if (primaryTCPPort != -1) {
+      cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + primaryTCPPort);
     } else if (isPrimary == false) {
       // We cannot start a replica when there is no primary:
       return null;
     }
+    cmd.add("-Dtests.nrtreplication.closeorcrash=false");
 
     cmd.add("-Dtests.nrtreplication.node=true");
     cmd.add("-Dtests.nrtreplication.nodeid=" + id);
@@ -534,7 +75,7 @@ public class TestNRTReplication extends LuceneTestCase {
       cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
     }
 
-    long myPrimaryGen = primaryGen;
+    long myPrimaryGen = 0;
     cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
 
     // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
@@ -548,17 +89,6 @@ public class TestNRTReplication extends LuceneTestCase {
     cmd.add("org.junit.runner.JUnitCore");
     cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
 
-    Writer childLog;
-
-    if (SEPARATE_CHILD_OUTPUT) {
-      Path childOut = childTempDir.resolve(id + ".log");
-      message("logging to " + childOut);
-      childLog = Files.newBufferedWriter(childOut, StandardCharsets.UTF_8, StandardOpenOption.APPEND, StandardOpenOption.CREATE);
-      childLog.write("\n\nSTART NEW CHILD:\n");
-    } else {
-      childLog = null;
-    }
-
     message("child process command: " + cmd);
     ProcessBuilder pb = new ProcessBuilder(cmd);
     pb.redirectErrorStream(true);
@@ -592,41 +122,11 @@ public class TestNRTReplication extends LuceneTestCase {
           throw new RuntimeException(ie);
         }
         message("exit value=" + p.exitValue());
-
-        // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
-        if (isPrimary == false) {
-          if (sawExistingSegmentsFile) {
-            // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
-            // and retry again later:
-            message("failed to remove segments_N; skipping");
-            return null;
-          }
-          for(int i=0;i<10;i++) {
-            if (primaryGen != myPrimaryGen || primary == null) {
-              // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
-              message("primary crashed/closed while replica R" + id + " tried to start; skipping");
-              return null;
-            } else {
-              try {
-                Thread.sleep(10);
-              } catch (InterruptedException ie) {
-                throw new ThreadInterruptedException(ie);
-              }
-            }
-          }
-        }
-
-        // Should fail the test:
         message("top: now fail test replica R" + id + " failed to start");
-        failed.set(true);
         throw new RuntimeException("replica R" + id + " failed to start");
       }
 
-      if (childLog != null) {
-        childLog.write(l);
-        childLog.write("\n");
-        childLog.flush();
-      } else if (logTimeStart.matcher(l).matches()) {
+      if (logTimeStart.matcher(l).matches()) {
         // Already a well-formed log output:
         System.out.println(l);
       } else {
@@ -650,7 +150,7 @@ public class TestNRTReplication extends LuceneTestCase {
 
     final boolean finalWillCrash = willCrash;
 
-    // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
+    // Baby sits the child process, pulling its stdout and printing to our stdout:
     Thread pumper = ThreadPumper.start(
                                        new Runnable() {
                                          @Override
@@ -665,503 +165,147 @@ public class TestNRTReplication extends LuceneTestCase {
                                            message("done wait for process " + p);
                                            int exitValue = p.exitValue();
                                            message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
-                                           if (childLog != null) {
-                                             try {
-                                               childLog.write("process done; exitValue=" + exitValue + "\n");
-                                               childLog.close();
-                                             } catch (IOException ioe) {
-                                               throw new RuntimeException(ioe);
-                                             }
-                                           }
-                                           if (exitValue != 0 && finalWillCrash == false && crashingNodes.remove(id) == false) {
+                                           if (exitValue != 0) {
                                              // should fail test
-                                             failed.set(true);
-                                             if (childLog != null) {
-                                               throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue + "; see " + childLog + " for details");
-                                             } else {
-                                               throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
-                                             }
+                                             throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
                                            }
-                                           nodeClosed(id);
                                          }
-                                       }, r, System.out, childLog);
+                                       }, r, System.out, null);
     pumper.setName("pump" + id);
 
     message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
     return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
   }
 
-  private void nodeClosed(int id) {
-    NodeProcess oldNode = nodes[id];
-    if (primary != null && oldNode == primary) {
-      message("top: " + primary + ": primary process finished");
-      primary = null;
-      primaryGen++;
-    } else {
-      message("top: " + oldNode + ": replica process finished");
+  public void testReplicateDeleteAllDocuments() throws Exception {
+
+    Node.globalStartNS = System.nanoTime();
+    childTempDir = createTempDir("child");
+
+    message("change thread name from " + Thread.currentThread().getName());
+    Thread.currentThread().setName("main");
+    
+    Path primaryPath = createTempDir("primary");
+    NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+
+    Path replicaPath = createTempDir("replica");
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+
+    // Tell primary current replicas:
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+      c.out.writeVInt(1);
+      c.out.writeVInt(replica.id);
+      c.out.writeVInt(replica.tcpPort);
+      c.flush();
+      c.in.readByte();
     }
-    if (oldNode != null) {
-      oldNode.isOpen = false;
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    Connection primaryC = new Connection(primary.tcpPort);
+    primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+    for(int i=0;i<10;i++) {
+      Document doc = docs.nextDoc();
+      primary.addOrUpdateDocument(primaryC, doc, false);
     }
-    nodes[id] = null;
-    nodeTimeStamps[id] = System.nanoTime();
 
-    sendReplicasToPrimary();
-  }
+    // Nothing in replica index yet
+    Connection replicaC = new Connection(replica.tcpPort);
+    replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+    replicaC.flush();
+    long version1 = replicaC.in.readVLong();
+    assertEquals(0L, version1);
+    int hitCount = replicaC.in.readVInt();
+    assertEquals(0, hitCount);
 
-  /** Sends currently alive replicas to primary, which uses this to know who to notify when it does a refresh */
-  private void sendReplicasToPrimary() {
-    NodeProcess curPrimary = primary;
-    if (curPrimary != null) {
-      List<NodeProcess> replicas = new ArrayList<>();
-      for (NodeProcess node : nodes) {
-        if (node != null && node.isPrimary == false) {
-          replicas.add(node);
-        }
-      }
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush();
+    assertTrue(primaryVersion1 > 0);
 
-      message("top: send " + replicas.size() + " replicas to primary");
+    long version2;
 
-      try (Connection c = new Connection(curPrimary.tcpPort)) {
-        c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
-        c.out.writeVInt(replicas.size());        
-        for(NodeProcess replica : replicas) {
-          c.out.writeVInt(replica.id);
-          c.out.writeVInt(replica.tcpPort);
-        }
-        c.flush();
-        c.in.readByte();
-      } catch (Throwable t) {
-        message("top: ignore exc sending replicas to primary: " + t);
+    // Wait for replica to show the change
+    while (true) {
+      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+      replicaC.flush();
+      version2 = replicaC.in.readVLong();
+      hitCount = replicaC.in.readVInt();
+      if (hitCount == 10) {
+        assertTrue(version2 > version1);
+        // good!
+        break;
       }
+      Thread.sleep(10);
     }
-  }
 
-  void addVersionMarker(long version, int count) {
-    //System.out.println("ADD VERSION MARKER version=" + version + " count=" + count);
-    if (versionToMarker.containsKey(version)) {
-      int curCount = versionToMarker.get(version);
-      if (curCount != count) {
-        message("top: wrong marker count version=" + version + " count=" + count + " curCount=" + curCount);
-        throw new IllegalStateException("version=" + version + " count=" + count + " curCount=" + curCount);
+    // Delete all docs from primary
+    if (random().nextBoolean()) {
+      // Inefficiently:
+      for(int id=0;id<10;id++) {
+        primary.deleteDocument(primaryC, Integer.toString(id));
       }
     } else {
-      message("top: record marker count: version=" + version + " count=" + count);
-      versionToMarker.put(version, count);
+      // Efficiently:
+      primary.deleteAllDocuments(primaryC);
     }
-  }
-
-  void addTransLogLoc(long version, long loc) {
-    message("top: record transLogLoc: version=" + version + " loc=" + loc);
-    versionToTransLogLocation.put(version, loc);
-  }
-
-  // Periodically wakes up and starts up any down nodes:
-  private class RestartThread extends Thread {
-    @Override
-    public void run() {
-
-      List<Thread> startupThreads = Collections.synchronizedList(new ArrayList<>());
-
-      try {
-        while (stop.get() == false) {
-          Thread.sleep(TestUtil.nextInt(random(), 50, 500));
-          message("top: restarter cycle");
-
-          // Randomly crash full cluster:
-          if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
-            message("top: full cluster crash");
-            for(int i=0;i<nodes.length;i++) {
-              if (starting[i]) {
-                message("N" + i + ": top: wait for startup so we can crash...");
-                while (starting[i]) {
-                  Thread.sleep(10);
-                }
-                message("N" + i + ": top: done wait for startup");
-              }
-              NodeProcess node = nodes[i];
-              if (node != null) {
-                crashingNodes.add(i);
-                message("top: N" + node.id + ": top: now crash node");
-                node.crash();
-                message("top: N" + node.id + ": top: done crash node");
-              }
-            }
-          }
-
-          List<Integer> downNodes = new ArrayList<>();
-          StringBuilder b = new StringBuilder();
-          long nowNS = System.nanoTime();
-          for(int i=0;i<nodes.length;i++) {
-            b.append(' ');
-            double sec = (nowNS - nodeTimeStamps[i])/1000000000.0;
-            String prefix;
-            if (nodes[i] == null) {
-              downNodes.add(i);
-              if (starting[i]) {
-                prefix = "s";
-              } else {
-                prefix = "x";
-              }
-            } else {
-              prefix = "";
-            }
-            if (primary != null && nodes[i] == primary) {
-              prefix += "p";
-            }
-            b.append(String.format(Locale.ROOT, "%s%d(%.1fs)", prefix, i, sec));
-          }
-          message("node status" + b.toString());
-          message("downNodes=" + downNodes);
-
-          // If primary is down, promote a replica:
-          if (primary == null) {
-            if (anyNodesStarting()) {
-              message("top: skip promote replica: nodes are still starting");
-              continue;
-            }
-            promoteReplica();
-          }
-
-          // Randomly start up a down a replica:
-
-          // Stop or start a replica
-          if (downNodes.isEmpty() == false) {
-            int idx = downNodes.get(random().nextInt(downNodes.size()));
-            if (starting[idx] == false) {
-              if (primary == null) {
-                if (downNodes.size() == nodes.length) {
-                  // Cold start: entire cluster is down, start this node up as the new primary
-                  message("N" + idx + ": top: cold start as primary");
-                  startPrimary(idx);
-                }
-              } else if (random().nextDouble() < ((double) downNodes.size())/nodes.length) {
-                // Start up replica:
-                starting[idx] = true;
-                message("N" + idx + ": top: start up: launch thread");
-                Thread t = new Thread() {
-                    @Override
-                    public void run() {
-                      try {
-                        message("N" + idx + ": top: start up thread");
-                        nodes[idx] = startNode(idx, indexPaths[idx], false, -1);
-                        sendReplicasToPrimary();
-                      } catch (Throwable t) {
-                        failed.set(true);
-                        stop.set(true);
-                        throw new RuntimeException(t);
-                      } finally {
-                        starting[idx] = false;
-                        startupThreads.remove(Thread.currentThread());
-                      }
-                    }
-                  };
-                t.setName("start R" + idx);
-                t.start();
-                startupThreads.add(t);
-              }
-            } else {
-              message("node " + idx + " still starting");
-            }
-          }
-        }
 
-        System.out.println("Restarter: now stop: join " + startupThreads.size() + " startup threads");
-
-        while (startupThreads.size() > 0) {
-          Thread.sleep(10);
-        }
+    // Replica still shows 10 docs:
+    replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+    replicaC.flush();
+    long version3 = replicaC.in.readVLong();
+    assertEquals(version2, version3);
+    hitCount = replicaC.in.readVInt();
+    assertEquals(10, hitCount);
+    
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion2 = primary.flush();
+    assertTrue(primaryVersion2 > primaryVersion1);
 
-      } catch (Throwable t) {
-        failed.set(true);
-        stop.set(true);
-        throw new RuntimeException(t);
+    // Wait for replica to show the change
+    long version4;
+    while (true) {
+      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+      replicaC.flush();
+      version4 = replicaC.in.readVLong();
+      hitCount = replicaC.in.readVInt();
+      if (hitCount == 0) {
+        assertTrue(version4 > version3);
+        // good!
+        break;
       }
+      Thread.sleep(10);
     }
-  }
-
-  /** Randomly picks a node and runs a search against it */
-  private class SearchThread extends Thread {
-
-    @Override
-    public void run() {
-      // Maps version to number of hits for silly 'the' TermQuery:
-      Query theQuery = new TermQuery(new Term("body", "the"));
-
-      // Persists connections
-      Map<Integer,Connection> connections = new HashMap<>();
-
-      while (stop.get() == false) {
-        NodeProcess node = nodes[random().nextInt(nodes.length)];
-        if (node == null || node.isOpen == false) {
-          continue;
-        }
-
-        if (node.lock.tryLock() == false) {
-          // Node is in the process of closing or crashing or something
-          continue;
-        }
-
-        try {
-
-          Thread.currentThread().setName("Searcher node=" + node);
-
-          //System.out.println("S: cycle; conns=" + connections);
-
-          Connection c = connections.get(node.id);
-
-          long version;
-          try {
-            if (c == null) {
-              //System.out.println("S: new connection " + node.id + " " + Thread.currentThread().getName());
-              c = new Connection(node.tcpPort);
-              connections.put(node.id, c);
-            } else {
-              //System.out.println("S: reuse connection " + node.id + " " + Thread.currentThread().getName());
-            }
-
-            c.out.writeByte(SimplePrimaryNode.CMD_SEARCH);
-            c.flush();
-
-            while (c.sockIn.available() == 0) {
-              if (stop.get()) {
-                break;
-              }
-              if (node.isOpen == false) {
-                throw new IOException("node closed");
-              }
-              Thread.sleep(1);
-            }
-            version = c.in.readVLong();
-
-            while (c.sockIn.available() == 0) {
-              if (stop.get()) {
-                break;
-              }
-              if (node.isOpen == false) {
-                throw new IOException("node closed");
-              }
-              Thread.sleep(1);
-            }
-            int hitCount = c.in.readVInt();
-
-            Integer oldHitCount = hitCounts.get(version);
-
-            // TODO: we never prune this map...
-            if (oldHitCount == null) {
-              hitCounts.put(version, hitCount);
-              message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
-            } else {
-              // Just ensure that all nodes show the same hit count for
-              // the same version, i.e. they really are replicas of one another:
-              if (oldHitCount.intValue() != hitCount) {
-                failed.set(true);
-                stop.set(true);
-                message("top: searcher: wrong version hitCount: version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
-                fail("version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
-              }
-            }
-          } catch (IOException ioe) {
-            //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
-            //ioe.printStackTrace(System.out);
-            IOUtils.closeWhileHandlingException(c);
-            connections.remove(node.id);
-            continue;
-          }
-
-          // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
-          Integer expectedAtLeastHitCount = versionToMarker.get(version);
-
-          if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
-            try {
-              c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
-              c.flush();
-              while (c.sockIn.available() == 0) {
-                if (stop.get()) {
-                  break;
-                }
-                if (node.isOpen == false) {
-                  throw new IOException("node died");
-                }
-                Thread.sleep(1);
-              }
-
-              version = c.in.readVLong();
 
-              while (c.sockIn.available() == 0) {
-                if (stop.get()) {
-                  break;
-                }
-                if (node.isOpen == false) {
-                  throw new IOException("node died");
-                }
-                Thread.sleep(1);
-              }
-
-              int hitCount = c.in.readVInt();
-
-              // Look for data loss: make sure all marker docs are visible:
-            
-              if (hitCount < expectedAtLeastHitCount) {
-
-                String failMessage = "node=" + node + ": documents were lost version=" + version + " hitCount=" + hitCount + " vs expectedAtLeastHitCount=" + expectedAtLeastHitCount;
-                message(failMessage);
-                failed.set(true);
-                stop.set(true);
-                fail(failMessage);
-              }
-            } catch (IOException ioe) {
-              //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
-              //throw new RuntimeException(ioe);
-              //ioe.printStackTrace(System.out);
-              IOUtils.closeWhileHandlingException(c);
-              connections.remove(node.id);
-              continue;
-            }
-          }
+    // Index 10 docs again:
+    for(int i=0;i<10;i++) {
+      Document doc = docs.nextDoc();
+      primary.addOrUpdateDocument(primaryC, doc, false);
+    }
 
-          Thread.sleep(10);
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion3 = primary.flush();
+    assertTrue(primaryVersion3 > primaryVersion2);
 
-        } catch (Throwable t) {
-          failed.set(true);
-          stop.set(true);
-          throw new RuntimeException(t);
-        } finally {
-          node.lock.unlock();
-        }
+    // Wait for replica to show the change
+    while (true) {
+      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+      replicaC.flush();
+      long version5 = replicaC.in.readVLong();
+      hitCount = replicaC.in.readVInt();
+      if (hitCount == 10) {
+        assertTrue(version5 > version4);
+        // good!
+        break;
       }
-      System.out.println("Searcher: now stop");
-      IOUtils.closeWhileHandlingException(connections.values());
+      Thread.sleep(10);
     }
-  }
-
-  private class IndexThread extends Thread {
-
-    @Override
-    public void run() {
-
-      try {
-        LineFileDocs docs = new LineFileDocs(random());
-        int docCount = 0;
-
-        // How often we do an update/delete vs add:
-        double updatePct = random().nextDouble();
-
-        // Varies how many docs/sec we index:
-        int sleepChance = TestUtil.nextInt(random(), 4, 100);
 
-        message("top: indexer: updatePct=" + updatePct + " sleepChance=" + sleepChance);
+    replicaC.close();
+    primaryC.close();
 
-        long lastTransLogLoc = transLog.getNextLocation();
-        
-        NodeProcess curPrimary = null;
-        Connection c = null;
-
-        while (stop.get() == false) {
-
-          try {
-            while (stop.get() == false && curPrimary == null) {
-              Thread.sleep(10);
-              curPrimary = primary;
-              if (curPrimary != null) {
-                c = new Connection(curPrimary.tcpPort);
-                c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
-                break;
-              }
-            }
-
-            if (stop.get()) {
-              break;
-            }
-
-            Thread.currentThread().setName("indexer p" + curPrimary.id);
-
-            if (random().nextInt(10) == 7) {
-              // We use the marker docs to check for data loss in search thread:
-              Document doc = new Document();
-              int id = markerUpto.getAndIncrement();
-              String idString = "m"+id;
-              doc.add(newStringField("docid", idString, Field.Store.YES));
-              doc.add(newStringField("marker", "marker", Field.Store.YES));
-              curPrimary.addOrUpdateDocument(c, doc, false);
-              transLog.addDocument(idString, doc);
-              message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
-            }
-
-            if (docCount > 0 && random().nextDouble() < updatePct) {
-              int randomID = random().nextInt(docCount);
-              String randomIDString = Integer.toString(randomID);
-              if (random().nextBoolean()) {
-                // Replace previous doc
-                Document doc = docs.nextDoc();
-                ((Field) doc.getField("docid")).setStringValue(randomIDString);
-                curPrimary.addOrUpdateDocument(c, doc, true);
-                transLog.updateDocument(randomIDString, doc);
-              } else {
-                // Delete previous doc
-                curPrimary.deleteDocument(c, randomIDString);
-                transLog.deleteDocuments(randomIDString);
-              }
-            } else {
-              // Add new doc:
-              Document doc = docs.nextDoc();
-              String idString = Integer.toString(docCount++);
-              ((Field) doc.getField("docid")).setStringValue(idString);
-              curPrimary.addOrUpdateDocument(c, doc, false);
-              transLog.addDocument(idString, doc);
-
-              if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
-                long curLoc = transLog.getNextLocation();
-                // randomly replay chunks of translog just to test replay:
-                message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
-                transLog.replay(curPrimary, lastTransLogLoc, curLoc);
-                lastTransLogLoc = curLoc;
-              }
-            }
-          } catch (IOException se) {
-            // Assume primary crashed
-            message("top: indexer lost connection to primary");
-            try {
-              c.close();
-            } catch (Throwable t) {
-            }
-            curPrimary = null;
-            c = null;
-          }
-
-          if (random().nextInt(sleepChance) == 0) {
-            Thread.sleep(1);
-          }
-
-          if (random().nextInt(100) == 17) {
-            System.out.println("Indexer: now pause for a bit...");
-            Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
-            System.out.println("Indexer: done pause for a bit...");
-          }
-        }
-        if (curPrimary != null) {
-          try {
-            c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
-            c.flush();
-            c.in.readByte();
-          } catch (IOException se) {
-            // Assume primary crashed
-            message("top: indexer lost connection to primary");
-            try {
-              c.close();
-            } catch (Throwable t) {
-            }
-            curPrimary = null;
-            c = null;
-          }
-        }
-        System.out.println("Indexer: now stop");
-      } catch (Throwable t) {
-        failed.set(true);
-        stop.set(true);
-        throw new RuntimeException(t);
-      }
-    }
+    replica.close();
+    primary.close();
   }
 
   static void message(String message) {


[10/15] lucene-solr git commit: add nocommit; try to fix false test failure due to concurrent check index running while primary is still closing

Posted by mi...@apache.org.
add nocommit; try to fix false test failure due to concurrent check index running while primary is still closing


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 7af83c486b50ceb13d58e79dd65c158bc96130dc
Parents: 8889469
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 3 16:45:46 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 3 16:45:46 2016 -0500

----------------------------------------------------------------------
 .../src/test/org/apache/lucene/replicator/nrt/NodeProcess.java   | 4 +---
 .../org/apache/lucene/replicator/nrt/TestNRTReplication.java     | 2 ++
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af83c48/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index dcbef87..be0b3df3 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -157,9 +157,7 @@ class NodeProcess implements Closeable {
             throw new RuntimeException("shutdown failed");
           }
         } catch (Throwable t) {
-          System.out.println("top: shutdown failed; skipping");
-          t.printStackTrace(System.out);
-          return false;
+          System.out.println("top: shutdown failed; ignoring");
         }
         try {
           p.waitFor();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af83c48/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index b2240eb..510a075 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -37,6 +37,8 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 
 import com.carrotsearch.randomizedtesting.SeedUtils;
 
+// nocommit make some explicit failure tests
+
 // MockRandom's .sd file has no index header/footer:
 @SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
 @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")


[13/15] lucene-solr git commit: Merge branch 'master' into nrt_replicas

Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --cc lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
index 0000000,aa1da81..f2f8de7
mode 000000,100644..100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
@@@ -1,0 -1,760 +1,760 @@@
+ /*
+  * Licensed to the Apache Software Foundation (ASF) under one or more
+  * contributor license agreements.  See the NOTICE file distributed with
+  * this work for additional information regarding copyright ownership.
+  * The ASF licenses this file to You under the Apache License, Version 2.0
+  * (the "License"); you may not use this file except in compliance with
+  * the License.  You may obtain a copy of the License at
+  *
+  *     http://www.apache.org/licenses/LICENSE-2.0
+  *
+  * Unless required by applicable law or agreed to in writing, software
+  * distributed under the License is distributed on an "AS IS" BASIS,
+  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  * See the License for the specific language governing permissions and
+  * limitations under the License.
+  */
+ package org.apache.lucene.spatial.util;
+ 
+ import java.io.IOException;
+ import java.text.DecimalFormat;
+ import java.text.DecimalFormatSymbols;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Locale;
+ import java.util.Set;
+ import java.util.concurrent.CountDownLatch;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ 
+ import org.apache.lucene.document.Document;
+ import org.apache.lucene.document.Field;
+ import org.apache.lucene.document.NumericDocValuesField;
+ import org.apache.lucene.index.DirectoryReader;
+ import org.apache.lucene.index.IndexReader;
+ import org.apache.lucene.index.IndexWriter;
+ import org.apache.lucene.index.IndexWriterConfig;
+ import org.apache.lucene.index.LeafReaderContext;
+ import org.apache.lucene.index.MultiDocValues;
+ import org.apache.lucene.index.NumericDocValues;
+ import org.apache.lucene.index.RandomIndexWriter;
+ import org.apache.lucene.index.Term;
+ import org.apache.lucene.search.IndexSearcher;
+ import org.apache.lucene.search.Query;
+ import org.apache.lucene.search.SimpleCollector;
+ import org.apache.lucene.store.Directory;
+ import org.apache.lucene.store.MockDirectoryWrapper;
+ import org.apache.lucene.util.FixedBitSet;
+ import org.apache.lucene.util.IOUtils;
+ import org.apache.lucene.util.LuceneTestCase;
+ import org.apache.lucene.util.SloppyMath;
+ import org.apache.lucene.util.TestUtil;
+ import org.junit.BeforeClass;
+ 
+ // TODO: cutover TestGeoUtils too?
+ 
+ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
+ 
+   protected static final String FIELD_NAME = "point";
+ 
+   private static final double LON_SCALE = (0x1L<< GeoEncodingUtils.BITS)/360.0D;
+   private static final double LAT_SCALE = (0x1L<< GeoEncodingUtils.BITS)/180.0D;
+ 
+   private static double originLat;
+   private static double originLon;
+   private static double lonRange;
+   private static double latRange;
+ 
+   @BeforeClass
+   public static void beforeClassBase() throws Exception {
+     // Between 1.0 and 3.0:
+     lonRange = 2 * (random().nextDouble() + 0.5);
+     latRange = 2 * (random().nextDouble() + 0.5);
+ 
+     originLon = GeoUtils.normalizeLon(GeoUtils.MIN_LON_INCL + lonRange + (GeoUtils.MAX_LON_INCL - GeoUtils.MIN_LON_INCL - 2 * lonRange) * random().nextDouble());
+     originLat = GeoUtils.normalizeLat(GeoUtils.MIN_LAT_INCL + latRange + (GeoUtils.MAX_LAT_INCL - GeoUtils.MIN_LAT_INCL - 2 * latRange) * random().nextDouble());
+   }
+ 
+   /** Return true when testing on a non-small region may be too slow (GeoPoint*Query) */
+   protected boolean forceSmall() {
+     return false;
+   }
+ 
+   // A particularly tricky adversary for BKD tree:
+   public void testSamePointManyTimes() throws Exception {
+ 
+     // For GeoPointQuery, only run this test nightly:
+     assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+ 
+     int numPoints = atLeast(1000);
+     boolean small = random().nextBoolean();
+ 
+     // Every doc has 2 points:
+     double theLat = randomLat(small);
+     double theLon = randomLon(small);
+ 
+     double[] lats = new double[numPoints];
+     Arrays.fill(lats, theLat);
+ 
+     double[] lons = new double[numPoints];
+     Arrays.fill(lons, theLon);
+ 
+     verify(small, lats, lons);
+   }
+ 
+   public void testAllLatEqual() throws Exception {
+ 
+     // For GeoPointQuery, only run this test nightly:
+     assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+ 
+     int numPoints = atLeast(10000);
+     boolean small = forceSmall() || random().nextBoolean();
+     double lat = randomLat(small);
+     double[] lats = new double[numPoints];
+     double[] lons = new double[numPoints];
+ 
+     boolean haveRealDoc = false;
+ 
+     for(int docID=0;docID<numPoints;docID++) {
+       int x = random().nextInt(20);
+       if (x == 17) {
+         // Some docs don't have a point:
+         lats[docID] = Double.NaN;
+         if (VERBOSE) {
+           System.out.println("  doc=" + docID + " is missing");
+         }
+         continue;
+       }
+ 
+       if (docID > 0 && x == 14 && haveRealDoc) {
+         int oldDocID;
+         while (true) {
+           oldDocID = random().nextInt(docID);
+           if (Double.isNaN(lats[oldDocID]) == false) {
+             break;
+           }
+         }
+             
+         // Fully identical point:
+         lons[docID] = lons[oldDocID];
+         if (VERBOSE) {
+           System.out.println("  doc=" + docID + " lat=" + lat + " lon=" + lons[docID] + " (same lat/lon as doc=" + oldDocID + ")");
+         }
+       } else {
+         lons[docID] = randomLon(small);
+         haveRealDoc = true;
+         if (VERBOSE) {
+           System.out.println("  doc=" + docID + " lat=" + lat + " lon=" + lons[docID]);
+         }
+       }
+       lats[docID] = lat;
+     }
+ 
+     verify(small, lats, lons);
+   }
+ 
+   public void testAllLonEqual() throws Exception {
+ 
+     // For GeoPointQuery, only run this test nightly:
+     assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+ 
+     int numPoints = atLeast(10000);
+     boolean small = forceSmall() || random().nextBoolean();
+     double theLon = randomLon(small);
+     double[] lats = new double[numPoints];
+     double[] lons = new double[numPoints];
+ 
+     boolean haveRealDoc = false;
+ 
+     //System.out.println("theLon=" + theLon);
+ 
+     for(int docID=0;docID<numPoints;docID++) {
+       int x = random().nextInt(20);
+       if (x == 17) {
+         // Some docs don't have a point:
+         lats[docID] = Double.NaN;
+         if (VERBOSE) {
+           System.out.println("  doc=" + docID + " is missing");
+         }
+         continue;
+       }
+ 
+       if (docID > 0 && x == 14 && haveRealDoc) {
+         int oldDocID;
+         while (true) {
+           oldDocID = random().nextInt(docID);
+           if (Double.isNaN(lats[oldDocID]) == false) {
+             break;
+           }
+         }
+             
+         // Fully identical point:
+         lats[docID] = lats[oldDocID];
+         if (VERBOSE) {
+           System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + theLon + " (same lat/lon as doc=" + oldDocID + ")");
+         }
+       } else {
+         lats[docID] = randomLat(small);
+         haveRealDoc = true;
+         if (VERBOSE) {
+           System.out.println("  doc=" + docID + " lat=" + lats[docID] + " lon=" + theLon);
+         }
+       }
+       lons[docID] = theLon;
+     }
+ 
+     verify(small, lats, lons);
+   }
+ 
+   public void testMultiValued() throws Exception {
+ 
+     // For GeoPointQuery, only run this test nightly:
+     assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+ 
+     int numPoints = atLeast(10000);
+     // Every doc has 2 points:
+     double[] lats = new double[2*numPoints];
+     double[] lons = new double[2*numPoints];
+     Directory dir = newDirectory();
+     IndexWriterConfig iwc = newIndexWriterConfig();
+     initIndexWriterConfig(FIELD_NAME, iwc);
+ 
+     // We rely on docID order:
+     iwc.setMergePolicy(newLogMergePolicy());
+     RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+ 
+     boolean small = random().nextBoolean();
+ 
+     for (int id=0;id<numPoints;id++) {
+       Document doc = new Document();
+       lats[2*id] = randomLat(small);
+       lons[2*id] = randomLon(small);
+       doc.add(newStringField("id", ""+id, Field.Store.YES));
+       addPointToDoc(FIELD_NAME, doc, lats[2*id], lons[2*id]);
+       lats[2*id+1] = randomLat(small);
+       lons[2*id+1] = randomLon(small);
+       addPointToDoc(FIELD_NAME, doc, lats[2*id+1], lons[2*id+1]);
+ 
+       if (VERBOSE) {
+         System.out.println("id=" + id);
+         System.out.println("  lat=" + lats[2*id] + " lon=" + lons[2*id]);
+         System.out.println("  lat=" + lats[2*id+1] + " lon=" + lons[2*id+1]);
+       }
+       w.addDocument(doc);
+     }
+ 
+     // TODO: share w/ verify; just need parallel array of the expected ids
+     if (random().nextBoolean()) {
+       w.forceMerge(1);
+     }
+     IndexReader r = w.getReader();
+     w.close();
+ 
+     // We can't wrap with "exotic" readers because the BKD query must see the BKDDVFormat:
+     IndexSearcher s = newSearcher(r, false);
+ 
+     int iters = atLeast(75);
+     for (int iter=0;iter<iters;iter++) {
+       GeoRect rect = randomRect(small, small == false);
+ 
+       if (VERBOSE) {
+         System.out.println("\nTEST: iter=" + iter + " rect=" + rect);
+       }
+ 
+       Query query = newRectQuery(FIELD_NAME, rect);
+ 
+       final FixedBitSet hits = new FixedBitSet(r.maxDoc());
+       s.search(query, new SimpleCollector() {
+ 
+           private int docBase;
+ 
+           @Override
+           public boolean needsScores() {
+             return false;
+           }
+ 
+           @Override
+           protected void doSetNextReader(LeafReaderContext context) throws IOException {
+             docBase = context.docBase;
+           }
+ 
+           @Override
+           public void collect(int doc) {
+             hits.set(docBase+doc);
+           }
+         });
+ 
+       boolean fail = false;
+ 
+       for(int docID=0;docID<lats.length/2;docID++) {
+         double latDoc1 = lats[2*docID];
+         double lonDoc1 = lons[2*docID];
+         double latDoc2 = lats[2*docID+1];
+         double lonDoc2 = lons[2*docID+1];
+         
+         Boolean result1 = rectContainsPoint(rect, latDoc1, lonDoc1);
+         if (result1 == null) {
+           // borderline case: cannot test
+           continue;
+         }
+ 
+         Boolean result2 = rectContainsPoint(rect, latDoc2, lonDoc2);
+         if (result2 == null) {
+           // borderline case: cannot test
+           continue;
+         }
+ 
+         boolean expected = result1 == Boolean.TRUE || result2 == Boolean.TRUE;
+ 
+         if (hits.get(docID) != expected) {
+           String id = s.doc(docID).get("id");
+           if (expected) {
+             System.out.println(Thread.currentThread().getName() + ": id=" + id + " docID=" + docID + " should match but did not");
+           } else {
+             System.out.println(Thread.currentThread().getName() + ": id=" + id + " docID=" + docID + " should not match but did");
+           }
+           System.out.println("  rect=" + rect);
+           System.out.println("  lat=" + latDoc1 + " lon=" + lonDoc1 + "\n  lat=" + latDoc2 + " lon=" + lonDoc2);
+           System.out.println("  result1=" + result1 + " result2=" + result2);
+           fail = true;
+         }
+       }
+ 
+       if (fail) {
+         fail("some hits were wrong");
+       }
+     }
+     r.close();
+     dir.close();
+   }
+ 
+   public void testRandomTiny() throws Exception {
+     // Make sure single-leaf-node case is OK:
+     doTestRandom(10);
+   }
+ 
+   public void testRandomMedium() throws Exception {
+     doTestRandom(10000);
+   }
+ 
+   @Nightly
+   public void testRandomBig() throws Exception {
+     assumeFalse("Direct codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Direct"));
+     assumeFalse("Memory codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Memory"));
+     doTestRandom(200000);
+   }
+ 
+   private void doTestRandom(int count) throws Exception {
+ 
+     int numPoints = atLeast(count);
+ 
+     if (VERBOSE) {
+       System.out.println("TEST: numPoints=" + numPoints);
+     }
+ 
+     double[] lats = new double[numPoints];
+     double[] lons = new double[numPoints];
+ 
+     boolean small = random().nextBoolean();
+ 
+     boolean haveRealDoc = false;
+ 
+     for (int id=0;id<numPoints;id++) {
+       int x = random().nextInt(20);
+       if (x == 17) {
+         // Some docs don't have a point:
+         lats[id] = Double.NaN;
+         if (VERBOSE) {
+           System.out.println("  id=" + id + " is missing");
+         }
+         continue;
+       }
+ 
+       if (id > 0 && x < 3 && haveRealDoc) {
+         int oldID;
+         while (true) {
+           oldID = random().nextInt(id);
+           if (Double.isNaN(lats[oldID]) == false) {
+             break;
+           }
+         }
+             
+         if (x == 0) {
+           // Identical lat to old point
+           lats[id] = lats[oldID];
+           lons[id] = randomLon(small);
+           if (VERBOSE) {
+             System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lat as doc=" + oldID + ")");
+           }
+         } else if (x == 1) {
+           // Identical lon to old point
+           lats[id] = randomLat(small);
+           lons[id] = lons[oldID];
+           if (VERBOSE) {
+             System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lon as doc=" + oldID + ")");
+           }
+         } else {
+           assert x == 2;
+           // Fully identical point:
+           lats[id] = lats[oldID];
+           lons[id] = lons[oldID];
+           if (VERBOSE) {
+             System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lat/lon as doc=" + oldID + ")");
+           }
+         }
+       } else {
+         lats[id] = randomLat(small);
+         lons[id] = randomLon(small);
+         haveRealDoc = true;
+         if (VERBOSE) {
+           System.out.println("  id=" + id + " lat=" + lats[id] + " lon=" + lons[id]);
+         }
+       }
+     }
+ 
+     verify(small, lats, lons);
+   }
+ 
+   public double randomLat(boolean small) {
+     double result;
+     if (small) {
+       result = GeoUtils.normalizeLat(originLat + latRange * (random().nextDouble() - 0.5));
+     } else {
+       result = -90 + 180.0 * random().nextDouble();
+     }
+     return result;
+   }
+ 
+   public double randomLon(boolean small) {
+     double result;
+     if (small) {
+       result = GeoUtils.normalizeLon(originLon + lonRange * (random().nextDouble() - 0.5));
+     } else {
+       result = -180 + 360.0 * random().nextDouble();
+     }
+     return result;
+   }
+ 
+   protected GeoRect randomRect(boolean small, boolean canCrossDateLine) {
+     double lat0 = randomLat(small);
+     double lat1 = randomLat(small);
+     double lon0 = randomLon(small);
+     double lon1 = randomLon(small);
+ 
+     if (lat1 < lat0) {
+       double x = lat0;
+       lat0 = lat1;
+       lat1 = x;
+     }
+ 
+     if (canCrossDateLine == false && lon1 < lon0) {
+       double x = lon0;
+       lon0 = lon1;
+       lon1 = x;
+     }
+ 
+     return new GeoRect(lon0, lon1, lat0, lat1);
+   }
+ 
+   protected void initIndexWriterConfig(String field, IndexWriterConfig iwc) {
+   }
+ 
+   protected abstract void addPointToDoc(String field, Document doc, double lat, double lon);
+ 
+   protected abstract Query newRectQuery(String field, GeoRect bbox);
+ 
+   protected abstract Query newDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters);
+ 
+   protected abstract Query newDistanceRangeQuery(String field, double centerLat, double centerLon, double minRadiusMeters, double radiusMeters);
+ 
+   protected abstract Query newPolygonQuery(String field, double[] lats, double[] lons);
+ 
+   /** Returns null if it's borderline case */
+   protected abstract Boolean rectContainsPoint(GeoRect rect, double pointLat, double pointLon);
+ 
+   /** Returns null if it's borderline case */
+   protected abstract Boolean polyRectContainsPoint(GeoRect rect, double pointLat, double pointLon);
+ 
+   /** Returns null if it's borderline case */
+   protected abstract Boolean circleContainsPoint(double centerLat, double centerLon, double radiusMeters, double pointLat, double pointLon);
+ 
+   protected abstract Boolean distanceRangeContainsPoint(double centerLat, double centerLon, double minRadiusMeters, double radiusMeters, double pointLat, double pointLon);
+ 
+   private static abstract class VerifyHits {
+ 
+     public void test(AtomicBoolean failed, boolean small, IndexSearcher s, NumericDocValues docIDToID, Set<Integer> deleted, Query query, double[] lats, double[] lons) throws Exception {
+       int maxDoc = s.getIndexReader().maxDoc();
+       final FixedBitSet hits = new FixedBitSet(maxDoc);
+       s.search(query, new SimpleCollector() {
+ 
+           private int docBase;
+ 
+           @Override
+           public boolean needsScores() {
+             return false;
+           }
+ 
+           @Override
+           protected void doSetNextReader(LeafReaderContext context) throws IOException {
+             docBase = context.docBase;
+           }
+ 
+           @Override
+           public void collect(int doc) {
+             hits.set(docBase+doc);
+           }
+         });
+ 
+       boolean fail = false;
+ 
+       for(int docID=0;docID<maxDoc;docID++) {
+         int id = (int) docIDToID.get(docID);
+         Boolean expected;
+         if (deleted.contains(id)) {
+           expected = false;
+         } else if (Double.isNaN(lats[id])) {
+           expected = false;
+         } else {
+           expected = shouldMatch(lats[id], lons[id]);
+         }
+ 
+         // null means it's a borderline case which is allowed to be wrong:
+         if (expected != null && hits.get(docID) != expected) {
+           if (expected) {
+             System.out.println(Thread.currentThread().getName() + ": id=" + id + " should match but did not");
+           } else {
+             System.out.println(Thread.currentThread().getName() + ": id=" + id + " should not match but did");
+           }
+           System.out.println("  small=" + small + " query=" + query +
+                              " docID=" + docID + "\n  lat=" + lats[id] + " lon=" + lons[id] +
+                              "\n  deleted?=" + deleted.contains(id));
+           if (Double.isNaN(lats[id]) == false) {
+             describe(docID, lats[id], lons[id]);
+           }
+           fail = true;
+         }
+       }
+ 
+       if (fail) {
+         failed.set(true);
+         fail("some hits were wrong");
+       }
+     }
+ 
+     /** Return true if we definitely should match, false if we definitely
+      *  should not match, and null if it's a borderline case which might
+      *  go either way. */
+     protected abstract Boolean shouldMatch(double lat, double lon);
+ 
+     protected abstract void describe(int docID, double lat, double lon);
+   }
+ 
+   protected void verify(boolean small, double[] lats, double[] lons) throws Exception {
+     IndexWriterConfig iwc = newIndexWriterConfig();
+     // Else we can get O(N^2) merging:
+     int mbd = iwc.getMaxBufferedDocs();
+     if (mbd != -1 && mbd < lats.length/100) {
+       iwc.setMaxBufferedDocs(lats.length/100);
+     }
+     Directory dir;
+     if (lats.length > 100000) {
+       dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+     } else {
+       dir = newDirectory();
+     }
+ 
+     Set<Integer> deleted = new HashSet<>();
+     // RandomIndexWriter is too slow here:
+     IndexWriter w = new IndexWriter(dir, iwc);
+     for(int id=0;id<lats.length;id++) {
+       Document doc = new Document();
+       doc.add(newStringField("id", ""+id, Field.Store.NO));
+       doc.add(new NumericDocValuesField("id", id));
+       if (Double.isNaN(lats[id]) == false) {
+         addPointToDoc(FIELD_NAME, doc, lats[id], lons[id]);
+       }
+       w.addDocument(doc);
+       if (id > 0 && random().nextInt(100) == 42) {
+         int idToDelete = random().nextInt(id);
+         w.deleteDocuments(new Term("id", ""+idToDelete));
+         deleted.add(idToDelete);
+         if (VERBOSE) {
+           System.out.println("  delete id=" + idToDelete);
+         }
+       }
+     }
+ 
+     if (random().nextBoolean()) {
+       w.forceMerge(1);
+     }
+     final IndexReader r = DirectoryReader.open(w);
+     w.close();
+ 
+     // We can't wrap with "exotic" readers because the BKD query must see the BKDDVFormat:
+     IndexSearcher s = newSearcher(r, false);
+ 
+     // Make sure queries are thread safe:
+     int numThreads = TestUtil.nextInt(random(), 2, 5);
+ 
+     List<Thread> threads = new ArrayList<>();
+     final int iters = atLeast(75);
+ 
+     final CountDownLatch startingGun = new CountDownLatch(1);
+     final AtomicBoolean failed = new AtomicBoolean();
+ 
+     for(int i=0;i<numThreads;i++) {
+       Thread thread = new Thread() {
+           @Override
+           public void run() {
+             try {
+               _run();
+             } catch (Exception e) {
+               failed.set(true);
+               throw new RuntimeException(e);
+             }
+           }
+ 
+           private void _run() throws Exception {
+             startingGun.await();
+ 
+             NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+ 
+             for (int iter=0;iter<iters && failed.get() == false;iter++) {
+ 
+               if (VERBOSE) {
 -                System.out.println("\nTEST: iter=" + iter + " s=" + s);
++                System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " s=" + s);
+               }
+               Query query;
+               VerifyHits verifyHits;
+ 
+               if (random().nextBoolean()) {
+                 // Rect: don't allow dateline crossing when testing small:
+                 final GeoRect rect = randomRect(small, small == false);
+ 
+                 query = newRectQuery(FIELD_NAME, rect);
+ 
+                 verifyHits = new VerifyHits() {
+                     @Override
+                     protected Boolean shouldMatch(double pointLat, double pointLon) {
+                       return rectContainsPoint(rect, pointLat, pointLon);
+                     }
+                     @Override
+                     protected void describe(int docID, double lat, double lon) {
+                     }
+                   };
+ 
+               } else if (random().nextBoolean()) {
+                 // Distance
+                 final boolean rangeQuery = random().nextBoolean();
+                 final double centerLat = randomLat(small);
+                 final double centerLon = randomLon(small);
+ 
+                 double radiusMeters;
+                 double minRadiusMeters;
+ 
+                 if (small) {
+                   // Approx 3 degrees lon at the equator:
+                   radiusMeters = random().nextDouble() * 333000 + 1.0;
+                 } else {
+                   // So the query can cover at most 50% of the earth's surface:
+                   radiusMeters = random().nextDouble() * GeoProjectionUtils.SEMIMAJOR_AXIS * Math.PI / 2.0 + 1.0;
+                 }
+ 
+                 // generate a random minimum radius between 1% and 95% the max radius
+                 minRadiusMeters = (0.01 + 0.94 * random().nextDouble()) * radiusMeters;
+ 
+                 if (VERBOSE) {
+                   final DecimalFormat df = new DecimalFormat("#,###.00", DecimalFormatSymbols.getInstance(Locale.ENGLISH));
+                   System.out.println("  radiusMeters = " + df.format(radiusMeters)
+                       + ((rangeQuery == true) ? " minRadiusMeters = " + df.format(minRadiusMeters) : ""));
+                 }
+ 
+                 try {
+                   if (rangeQuery == true) {
+                     query = newDistanceRangeQuery(FIELD_NAME, centerLat, centerLon, minRadiusMeters, radiusMeters);
+                   } else {
+                     query = newDistanceQuery(FIELD_NAME, centerLat, centerLon, radiusMeters);
+                   }
+                 } catch (IllegalArgumentException e) {
+                   if (e.getMessage().contains("exceeds maxRadius")) {
+                     continue;
+                   }
+                   throw e;
+                 }
+ 
+                 verifyHits = new VerifyHits() {
+                     @Override
+                     protected Boolean shouldMatch(double pointLat, double pointLon) {
+                       if (rangeQuery == false) {
+                         return circleContainsPoint(centerLat, centerLon, radiusMeters, pointLat, pointLon);
+                       } else {
+                         return distanceRangeContainsPoint(centerLat, centerLon, minRadiusMeters, radiusMeters, pointLat, pointLon);
+                       }
+                     }
+ 
+                     @Override
+                     protected void describe(int docID, double pointLat, double pointLon) {
+                       double distanceKM = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon);
+                       System.out.println("  docID=" + docID + " centerLon=" + centerLon + " centerLat=" + centerLat
+                           + " pointLon=" + pointLon + " pointLat=" + pointLat + " distanceMeters=" + (distanceKM * 1000)
+                           + " vs" + ((rangeQuery == true) ? " minRadiusMeters=" + minRadiusMeters : "") + " radiusMeters=" + radiusMeters);
+                     }
+                    };
+ 
+               // TODO: get poly query working with dateline crossing too (how?)!
+               } else {
+ 
+                 // TODO: poly query can't handle dateline crossing yet:
+                 final GeoRect bbox = randomRect(small, false);
+ 
+                 // Polygon
+                 double[] lats = new double[5];
+                 double[] lons = new double[5];
+                 lats[0] = bbox.minLat;
+                 lons[0] = bbox.minLon;
+                 lats[1] = bbox.maxLat;
+                 lons[1] = bbox.minLon;
+                 lats[2] = bbox.maxLat;
+                 lons[2] = bbox.maxLon;
+                 lats[3] = bbox.minLat;
+                 lons[3] = bbox.maxLon;
+                 lats[4] = bbox.minLat;
+                 lons[4] = bbox.minLon;
+                 query = newPolygonQuery(FIELD_NAME, lats, lons);
+ 
+                 verifyHits = new VerifyHits() {
+                     @Override
+                     protected Boolean shouldMatch(double pointLat, double pointLon) {
+                       return polyRectContainsPoint(bbox, pointLat, pointLon);
+                     }
+ 
+                     @Override
+                     protected void describe(int docID, double lat, double lon) {
+                     }
+                   };
+               }
+ 
+               if (query != null) {
+ 
+                 if (VERBOSE) {
+                   System.out.println("  query=" + query);
+                 }
+ 
+                 verifyHits.test(failed, small, s, docIDToID, deleted, query, lats, lons);
+               }
+             }
+           }
+       };
+       thread.setName("T" + i);
+       thread.start();
+       threads.add(thread);
+     }
+     startingGun.countDown();
+     for(Thread thread : threads) {
+       thread.join();
+     }
+     IOUtils.close(r, dir);
+     assertFalse(failed.get());
+   }
+ }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
----------------------------------------------------------------------
diff --cc lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index b19045b,51dbffe..a5fc397
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@@ -45,7 -43,8 +45,9 @@@ import org.apache.lucene.index.IndexFil
  import org.apache.lucene.index.IndexWriter;
  import org.apache.lucene.index.IndexWriterConfig;
  import org.apache.lucene.index.NoDeletionPolicy;
 +import org.apache.lucene.index.SegmentInfos;
+ import org.apache.lucene.mockfile.FilterFileSystem;
+ import org.apache.lucene.mockfile.VirusCheckingFS;
  import org.apache.lucene.util.IOUtils;
  import org.apache.lucene.util.LuceneTestCase;
  import org.apache.lucene.util.TestUtil;
@@@ -239,24 -219,8 +222,16 @@@ public class MockDirectoryWrapper exten
        throw new IOException("cannot rename after crash");
      }
      
-     if (openFiles.containsKey(source)) {
-       if (assertNoDeleteOpenFile) {
-         throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
-       } else if (noDeleteOpenFile) {
-         throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
-       }
 -    if (assertNoDeleteOpenFile && openFiles.containsKey(source)) {
 -      throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
++    if (openFiles.containsKey(source) && assertNoDeleteOpenFile) {
++      throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
 +    }
 +
-     if (openFiles.containsKey(dest)) {
-       if (assertNoDeleteOpenFile) {
-         throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
-       } else if (noDeleteOpenFile) {
-         throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
-       }
++    if (openFiles.containsKey(dest) && assertNoDeleteOpenFile) {
++      throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
 +    }
 +
 +    if (createdFiles.contains(dest)) {
 +      throw new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" already exists: cannot rename");
      }
  
      boolean success = false;
@@@ -271,8 -235,6 +246,7 @@@
            unSyncedFiles.add(dest);
          }
          openFilesDeleted.remove(source);
-         triedToDelete.remove(dest);
 +        createdFiles.add(dest);
        }
      }
    }
@@@ -294,213 -256,94 +268,190 @@@
      }
    }
  
 -  /** Simulates a crash of OS or machine by overwriting
 -   *  unsynced files. */
 -  public synchronized void crash() throws IOException {
 -    openFiles = new HashMap<>();
 -    openFilesForWrite = new HashSet<>();
 -    openFilesDeleted = new HashSet<>();
 -    Iterator<String> it = unSyncedFiles.iterator();
 -    unSyncedFiles = new HashSet<>();
 -    // first force-close all files, so we can corrupt on windows etc.
 -    // clone the file map, as these guys want to remove themselves on close.
 -    Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
 -    for (Closeable f : m.keySet()) {
 -      try {
 -        f.close();
 -      } catch (Exception ignored) {}
 +  public synchronized void corruptUnknownFiles() throws IOException {
 +
 +    System.out.println("MDW: corrupt unknown files");
 +    Set<String> knownFiles = new HashSet<>();
 +    for(String fileName : listAll()) {
 +      if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
 +        System.out.println("MDW: read " + fileName + " to gather files it references");
 +        knownFiles.addAll(SegmentInfos.readCommit(this, fileName).files(true));
 +      }
      }
  
 -    // Maybe disable virus checker so it doesn't interfere with our efforts to corrupt files below:
 -    boolean virusCheckerWasEnabled = TestUtil.disableVirusChecker(in);
 +    Set<String> toCorrupt = new HashSet<>();
 +    Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
 +    for(String fileName : listAll()) {
 +      m.reset(fileName);
 +      if (knownFiles.contains(fileName) == false &&
 +          fileName.endsWith("write.lock") == false &&
 +          (m.matches() || fileName.startsWith(IndexFileNames.PENDING_SEGMENTS))) {
 +        toCorrupt.add(fileName);
 +      }
 +    }
  
 -    while(it.hasNext()) {
 -      String name = it.next();
 -      int damage = randomState.nextInt(5);
 +    corruptFiles(toCorrupt);
 +  }
 +
-   public synchronized void corruptFiles(Collection<String> files) {
++  public synchronized void corruptFiles(Collection<String> files) throws IOException {
 +    // Must make a copy because we change the incoming unsyncedFiles
 +    // when we create temp files, delete, etc., below:
 +    for(String name : new ArrayList<>(files)) {
 +      int damage = randomState.nextInt(6);
        String action = null;
  
 -      if (damage == 0) {
 +      switch(damage) {
 +
 +      case 0:
          action = "deleted";
-         try {
-           deleteFile(name, true);
-         } catch (IOException ioe) {
-           // ignore
-         }
+         deleteFile(name);
 -      } else if (damage == 1) {
 +        break;
 +
 +      case 1:
          action = "zeroed";
          // Zero out file entirely
 -        long length = fileLength(name);
 +        long length;
 +        try {
 +          length = fileLength(name);
 +        } catch (IOException ioe) {
 +          // Ignore
 +          continue;
 +        }
          byte[] zeroes = new byte[256];
          long upto = 0;
 -        IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
 -        while(upto < length) {
 -          final int limit = (int) Math.min(length-upto, zeroes.length);
 -          out.writeBytes(zeroes, 0, limit);
 -          upto += limit;
 -        }
 -        out.close();
 -      } else if (damage == 2) {
 -        action = "partially truncated";
 -        // Partially Truncate the file:
 -
 -        // First, make temp file and copy only half this
 -        // file over:
 -        String tempFileName;
 -        while (true) {
 -          tempFileName = ""+randomState.nextInt();
 -          if (!LuceneTestCase.slowFileExists(in, tempFileName)) {
 -            break;
 +        try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
 +          while(upto < length) {
 +            final int limit = (int) Math.min(length-upto, zeroes.length);
 +            out.writeBytes(zeroes, 0, limit);
 +            upto += limit;
            }
 +        } catch (IOException ioe) {
 +          // ignore
          }
 -        final IndexOutput tempOut = in.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
 -        IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState));
 -        tempOut.copyBytes(ii, ii.length()/2);
 -        tempOut.close();
 -        ii.close();
 +        break;
  
 -        // Delete original and copy bytes back:
 -        deleteFile(name);
 -        
 -        try(IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
 -          ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
 -          out.copyBytes(ii, ii.length());
 -          ii.close();
 +      case 2:
 +        {
 +          action = "partially truncated";
 +          // Partially Truncate the file:
 +
 +          // First, make temp file and copy only half this
 +          // file over:
 +          String tempFileName = null;
 +          try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
 +               IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
 +              tempFileName = tempOut.getName();
 +              tempOut.copyBytes(ii, ii.length()/2);
 +            } catch (IOException ioe) {
 +            // ignore
 +          }
 +
-           try {
-             // Delete original and copy bytes back:
-             deleteFile(name, true);
-           } catch (IOException ioe) {
-             // ignore
-           }
++          // Delete original and copy bytes back:
++          deleteFile(name);
 +
 +          try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
 +               IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
 +              out.copyBytes(ii, ii.length());
 +            } catch (IOException ioe) {
 +            // ignore
 +          }
-           try {
-             deleteFile(tempFileName, true);
-           } catch (IOException ioe) {
-             // ignore
-           }
++          deleteFile(tempFileName);
          }
 -        deleteFile(tempFileName);
 -      } else if (damage == 3) {
 +        break;
 +      
 +      case 3:
          // The file survived intact:
          action = "didn't change";
 -      } else {
 +        break;
 +
 +      case 4:
 +        // Corrupt one bit randomly in the file:
 +
 +        {
 +
 +          String tempFileName = null;
 +          try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
 +               IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
 +              tempFileName = tempOut.getName();
 +              if (ii.length() > 0) {
 +                // Copy first part unchanged:
 +                long byteToCorrupt = (long) (randomState.nextDouble() * ii.length());
 +                if (byteToCorrupt > 0) {
 +                  tempOut.copyBytes(ii, byteToCorrupt);
 +                }
 +
 +                // Randomly flip one bit from this byte:
 +                byte b = ii.readByte();
 +                int bitToFlip = randomState.nextInt(8);
 +                b = (byte) (b ^ (1 << bitToFlip));
 +                tempOut.writeByte(b);
 +
 +                action = "flip bit " + bitToFlip + " of byte " + byteToCorrupt + " out of " + ii.length() + " bytes";
 +
 +                // Copy last part unchanged:
 +                long bytesLeft = ii.length() - byteToCorrupt - 1;
 +                if (bytesLeft > 0) {
 +                  tempOut.copyBytes(ii, bytesLeft);
 +                }
 +              } else {
 +                action = "didn't change";
 +              }
 +            } catch (IOException ioe) {
 +            // ignore
 +          }
 +
-           try {
-             // Delete original and copy bytes back:
-             deleteFile(name, true);
-           } catch (IOException ioe) {
-             // ignore
-           }
++          // Delete original and copy bytes back:
++          deleteFile(name);
 +
 +          try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
 +               IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
 +              out.copyBytes(ii, ii.length());
 +            } catch (IOException ioe) {
 +            // ignore
 +          }
-           try {
-             deleteFile(tempFileName, true);
-           } catch (IOException ioe) {
-             // ignore
-           }
++
++          deleteFile(tempFileName);
 +        }
 +        break;
 +        
 +      case 5:
          action = "fully truncated";
          // Totally truncate the file to zero bytes
-         try {
-           deleteFile(name, true);
-         } catch (IOException ioe) {
-           // ignore
-         }
+         deleteFile(name);
 +
          try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
 +        } catch (IOException ioe) {
 +          // ignore
          }
 +        break;
 +
 +      default:
 +        throw new AssertionError();
        }
 -      // Re-enable
 -      if (virusCheckerWasEnabled) {
 -        TestUtil.enableVirusChecker(in);
 -      }
 -      if (LuceneTestCase.VERBOSE) {
 +
 +      if (true || LuceneTestCase.VERBOSE) {
          System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name);
        }
      }
 +  }
 +
 +  /** Simulates a crash of OS or machine by overwriting
 +   *  unsynced files. */
-   public synchronized void crash() {
++  public synchronized void crash() throws IOException {
      crashed = true;
 +    openFiles = new HashMap<>();
 +    openFilesForWrite = new HashSet<>();
 +    openFilesDeleted = new HashSet<>();
 +    // first force-close all files, so we can corrupt on windows etc.
 +    // clone the file map, as these guys want to remove themselves on close.
 +    Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
 +    for (Closeable f : m.keySet()) {
 +      try {
 +        f.close();
 +      } catch (Exception ignored) {}
 +    }
 +    corruptFiles(unSyncedFiles);
 +    unSyncedFiles = new HashSet<>();
    }
  
    public synchronized void clearCrash() {
@@@ -942,13 -745,12 +853,14 @@@
              String[] startFiles = allFiles.toArray(new String[0]);
              IndexWriterConfig iwc = new IndexWriterConfig(null);
              iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
+ 
+             // We must do this before opening writer otherwise writer will be angry if there are pending deletions:
+             TestUtil.disableVirusChecker(in);
+ 
              new IndexWriter(in, iwc).rollback();
 -            String[] endFiles = in.listAll();
 +
 +            Set<String> files = new HashSet<>(Arrays.asList(listAll()));
-             // Disregard what happens with the pendingDeletions files:
-             files.removeAll(pendingDeletions);
 +            String[] endFiles = files.toArray(new String[0]);
              
              Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
              Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
@@@ -1027,11 -784,7 +894,7 @@@
                  extras += "\n\nThese files were added (waaaaaaaaaat!): " + added;
                }
                
-               if (pendingDeletions.size() != 0) {
-                 extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
-               }
-               
 -              throw new RuntimeException("unreferenced files: before delete:\n    " + Arrays.toString(startFiles) + "\n  after delete:\n    " + Arrays.toString(endFiles) + extras);
 +              throw new RuntimeException(this + ": unreferenced files: before delete:\n    " + Arrays.toString(startFiles) + "\n  after delete:\n    " + Arrays.toString(endFiles) + extras);
              }
              
              DirectoryReader ir1 = DirectoryReader.open(this);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------


[15/15] lucene-solr git commit: Merge branch 'master' into nrt_replicas

Posted by mi...@apache.org.
Merge branch 'master' into nrt_replicas

Conflicts:
	lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: bd6804bc6312c1a1d18294b5a670cc5eba7c9f48
Parents: b1b2c79 8e78469
Author: Mike McCandless <mi...@apache.org>
Authored: Sat Feb 6 11:14:15 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Feb 6 11:14:15 2016 -0500

----------------------------------------------------------------------
 .../dot.settings/org.eclipse.jdt.ui.prefs       |   3 +-
 dev-tools/git/HELP.txt                          |  14 +
 .../idea/.idea/copyright/profiles_settings.xml  |   2 +-
 dev-tools/idea/lucene/sandbox/sandbox.iml       |   1 +
 lucene/CHANGES.txt                              |  39 +
 .../lucene/analysis/ar/ArabicAnalyzer.java      |   4 +-
 .../analysis/ar/ArabicNormalizationFilter.java  |   4 +-
 .../ar/ArabicNormalizationFilterFactory.java    |   4 +-
 .../lucene/analysis/ar/ArabicNormalizer.java    |   4 +-
 .../lucene/analysis/ar/ArabicStemFilter.java    |   4 +-
 .../analysis/ar/ArabicStemFilterFactory.java    |   4 +-
 .../lucene/analysis/ar/ArabicStemmer.java       |   6 +-
 .../lucene/analysis/bg/BulgarianAnalyzer.java   |   4 +-
 .../lucene/analysis/bg/BulgarianStemFilter.java |   4 +-
 .../analysis/bg/BulgarianStemFilterFactory.java |   4 +-
 .../lucene/analysis/bg/BulgarianStemmer.java    |   4 +-
 .../lucene/analysis/br/BrazilianAnalyzer.java   |   4 +-
 .../lucene/analysis/br/BrazilianStemFilter.java |   4 +-
 .../analysis/br/BrazilianStemFilterFactory.java |   4 +-
 .../lucene/analysis/br/BrazilianStemmer.java    |   8 +-
 .../lucene/analysis/ca/CatalanAnalyzer.java     |   4 +-
 .../analysis/charfilter/BaseCharFilter.java     |   1 -
 .../charfilter/HTMLStripCharFilter.java         |   3 +-
 .../charfilter/HTMLStripCharFilter.jflex        |   3 +-
 .../charfilter/HTMLStripCharFilterFactory.java  |   4 +-
 .../analysis/charfilter/MappingCharFilter.java  |   1 -
 .../charfilter/MappingCharFilterFactory.java    |   4 +-
 .../analysis/charfilter/NormalizeCharMap.java   |   1 -
 .../apache/lucene/analysis/cjk/CJKAnalyzer.java |   4 +-
 .../lucene/analysis/cjk/CJKBigramFilter.java    |   4 +-
 .../analysis/cjk/CJKBigramFilterFactory.java    |   4 +-
 .../lucene/analysis/cjk/CJKWidthFilter.java     |   4 +-
 .../analysis/cjk/CJKWidthFilterFactory.java     |   4 +-
 .../lucene/analysis/ckb/SoraniAnalyzer.java     |   4 +-
 .../analysis/ckb/SoraniNormalizationFilter.java |   4 +-
 .../ckb/SoraniNormalizationFilterFactory.java   |   4 +-
 .../lucene/analysis/ckb/SoraniNormalizer.java   |   4 +-
 .../lucene/analysis/ckb/SoraniStemFilter.java   |   4 +-
 .../analysis/ckb/SoraniStemFilterFactory.java   |   4 +-
 .../lucene/analysis/ckb/SoraniStemmer.java      |   4 +-
 .../analysis/commongrams/CommonGramsFilter.java |   1 -
 .../commongrams/CommonGramsFilterFactory.java   |   4 +-
 .../CommonGramsQueryFilterFactory.java          |   4 +-
 .../compound/CompoundWordTokenFilterBase.java   |   4 +-
 .../DictionaryCompoundWordTokenFilter.java      |   4 +-
 ...ictionaryCompoundWordTokenFilterFactory.java |   4 +-
 .../HyphenationCompoundWordTokenFilter.java     |   4 +-
 ...phenationCompoundWordTokenFilterFactory.java |   4 +-
 .../compound/hyphenation/ByteVector.java        |   7 +-
 .../compound/hyphenation/CharVector.java        |   7 +-
 .../analysis/compound/hyphenation/Hyphen.java   |   7 +-
 .../compound/hyphenation/Hyphenation.java       |   7 +-
 .../compound/hyphenation/HyphenationTree.java   |   7 +-
 .../compound/hyphenation/PatternConsumer.java   |   7 +-
 .../compound/hyphenation/PatternParser.java     |   7 +-
 .../compound/hyphenation/TernaryTree.java       |   7 +-
 .../analysis/core/DecimalDigitFilter.java       |   4 +-
 .../core/DecimalDigitFilterFactory.java         |   4 +-
 .../lucene/analysis/core/KeywordAnalyzer.java   |   4 +-
 .../lucene/analysis/core/KeywordTokenizer.java  |   4 +-
 .../analysis/core/KeywordTokenizerFactory.java  |   4 +-
 .../lucene/analysis/core/LetterTokenizer.java   |   4 +-
 .../analysis/core/LetterTokenizerFactory.java   |   4 +-
 .../lucene/analysis/core/LowerCaseFilter.java   |   4 +-
 .../analysis/core/LowerCaseFilterFactory.java   |   4 +-
 .../analysis/core/LowerCaseTokenizer.java       |   4 +-
 .../core/LowerCaseTokenizerFactory.java         |   4 +-
 .../lucene/analysis/core/SimpleAnalyzer.java    |   4 +-
 .../lucene/analysis/core/StopAnalyzer.java      |   4 +-
 .../apache/lucene/analysis/core/StopFilter.java |   4 +-
 .../lucene/analysis/core/StopFilterFactory.java |   4 +-
 .../lucene/analysis/core/TypeTokenFilter.java   |   4 +-
 .../analysis/core/TypeTokenFilterFactory.java   |   4 +-
 .../core/UnicodeWhitespaceAnalyzer.java         |   4 +-
 .../core/UnicodeWhitespaceTokenizer.java        |   4 +-
 .../lucene/analysis/core/UpperCaseFilter.java   |   4 +-
 .../analysis/core/UpperCaseFilterFactory.java   |   4 +-
 .../analysis/core/WhitespaceAnalyzer.java       |   4 +-
 .../analysis/core/WhitespaceTokenizer.java      |   4 +-
 .../core/WhitespaceTokenizerFactory.java        |   4 +-
 .../lucene/analysis/custom/CustomAnalyzer.java  |   4 +-
 .../lucene/analysis/cz/CzechAnalyzer.java       |   4 +-
 .../lucene/analysis/cz/CzechStemFilter.java     |  20 +-
 .../analysis/cz/CzechStemFilterFactory.java     |   4 +-
 .../apache/lucene/analysis/cz/CzechStemmer.java |   4 +-
 .../lucene/analysis/da/DanishAnalyzer.java      |   4 +-
 .../lucene/analysis/de/GermanAnalyzer.java      |   6 +-
 .../analysis/de/GermanLightStemFilter.java      |   4 +-
 .../de/GermanLightStemFilterFactory.java        |   4 +-
 .../lucene/analysis/de/GermanLightStemmer.java  |   4 +-
 .../analysis/de/GermanMinimalStemFilter.java    |   4 +-
 .../de/GermanMinimalStemFilterFactory.java      |   4 +-
 .../analysis/de/GermanMinimalStemmer.java       |   4 +-
 .../analysis/de/GermanNormalizationFilter.java  |   4 +-
 .../de/GermanNormalizationFilterFactory.java    |   4 +-
 .../lucene/analysis/de/GermanStemFilter.java    |   4 +-
 .../analysis/de/GermanStemFilterFactory.java    |   4 +-
 .../lucene/analysis/de/GermanStemmer.java       |  12 +-
 .../lucene/analysis/el/GreekAnalyzer.java       |  14 +-
 .../analysis/el/GreekLowerCaseFilter.java       |  14 +-
 .../el/GreekLowerCaseFilterFactory.java         |   4 +-
 .../lucene/analysis/el/GreekStemFilter.java     |   4 +-
 .../analysis/el/GreekStemFilterFactory.java     |   4 +-
 .../apache/lucene/analysis/el/GreekStemmer.java |  12 +-
 .../lucene/analysis/en/EnglishAnalyzer.java     |   4 +-
 .../analysis/en/EnglishMinimalStemFilter.java   |   4 +-
 .../en/EnglishMinimalStemFilterFactory.java     |   4 +-
 .../analysis/en/EnglishMinimalStemmer.java      |   4 +-
 .../analysis/en/EnglishPossessiveFilter.java    |   4 +-
 .../en/EnglishPossessiveFilterFactory.java      |   4 +-
 .../apache/lucene/analysis/en/KStemFilter.java  |   4 +-
 .../lucene/analysis/en/KStemFilterFactory.java  |   4 +-
 .../org/apache/lucene/analysis/en/KStemmer.java |   1 -
 .../lucene/analysis/en/PorterStemFilter.java    |   4 +-
 .../analysis/en/PorterStemFilterFactory.java    |   4 +-
 .../lucene/analysis/en/PorterStemmer.java       |   4 +-
 .../lucene/analysis/es/SpanishAnalyzer.java     |   4 +-
 .../analysis/es/SpanishLightStemFilter.java     |   4 +-
 .../es/SpanishLightStemFilterFactory.java       |   4 +-
 .../lucene/analysis/es/SpanishLightStemmer.java |   4 +-
 .../lucene/analysis/eu/BasqueAnalyzer.java      |   4 +-
 .../lucene/analysis/fa/PersianAnalyzer.java     |   4 +-
 .../lucene/analysis/fa/PersianCharFilter.java   |   4 +-
 .../analysis/fa/PersianCharFilterFactory.java   |   4 +-
 .../analysis/fa/PersianNormalizationFilter.java |   4 +-
 .../fa/PersianNormalizationFilterFactory.java   |   4 +-
 .../lucene/analysis/fa/PersianNormalizer.java   |   4 +-
 .../lucene/analysis/fi/FinnishAnalyzer.java     |   4 +-
 .../analysis/fi/FinnishLightStemFilter.java     |   4 +-
 .../fi/FinnishLightStemFilterFactory.java       |   4 +-
 .../lucene/analysis/fi/FinnishLightStemmer.java |   4 +-
 .../lucene/analysis/fr/FrenchAnalyzer.java      |   4 +-
 .../analysis/fr/FrenchLightStemFilter.java      |   4 +-
 .../fr/FrenchLightStemFilterFactory.java        |   4 +-
 .../lucene/analysis/fr/FrenchLightStemmer.java  |   4 +-
 .../analysis/fr/FrenchMinimalStemFilter.java    |   4 +-
 .../fr/FrenchMinimalStemFilterFactory.java      |   4 +-
 .../analysis/fr/FrenchMinimalStemmer.java       |   4 +-
 .../lucene/analysis/ga/IrishAnalyzer.java       |   4 +-
 .../analysis/ga/IrishLowerCaseFilter.java       |   4 +-
 .../ga/IrishLowerCaseFilterFactory.java         |   4 +-
 .../lucene/analysis/gl/GalicianAnalyzer.java    |   4 +-
 .../analysis/gl/GalicianMinimalStemFilter.java  |   4 +-
 .../gl/GalicianMinimalStemFilterFactory.java    |   4 +-
 .../analysis/gl/GalicianMinimalStemmer.java     |   4 +-
 .../lucene/analysis/gl/GalicianStemFilter.java  |   4 +-
 .../analysis/gl/GalicianStemFilterFactory.java  |   4 +-
 .../lucene/analysis/gl/GalicianStemmer.java     |   4 +-
 .../lucene/analysis/hi/HindiAnalyzer.java       |   4 +-
 .../analysis/hi/HindiNormalizationFilter.java   |   4 +-
 .../hi/HindiNormalizationFilterFactory.java     |   4 +-
 .../lucene/analysis/hi/HindiNormalizer.java     |   4 +-
 .../lucene/analysis/hi/HindiStemFilter.java     |   4 +-
 .../analysis/hi/HindiStemFilterFactory.java     |   4 +-
 .../apache/lucene/analysis/hi/HindiStemmer.java |   4 +-
 .../lucene/analysis/hu/HungarianAnalyzer.java   |   4 +-
 .../analysis/hu/HungarianLightStemFilter.java   |   4 +-
 .../hu/HungarianLightStemFilterFactory.java     |   4 +-
 .../analysis/hu/HungarianLightStemmer.java      |   4 +-
 .../lucene/analysis/hunspell/Dictionary.java    |   4 +-
 .../analysis/hunspell/HunspellStemFilter.java   |   4 +-
 .../hunspell/HunspellStemFilterFactory.java     |   4 +-
 .../analysis/hunspell/ISO8859_14Decoder.java    |   4 +-
 .../lucene/analysis/hunspell/Stemmer.java       |   4 +-
 .../lucene/analysis/hy/ArmenianAnalyzer.java    |   4 +-
 .../lucene/analysis/id/IndonesianAnalyzer.java  |   4 +-
 .../analysis/id/IndonesianStemFilter.java       |   4 +-
 .../id/IndonesianStemFilterFactory.java         |   4 +-
 .../lucene/analysis/id/IndonesianStemmer.java   |   4 +-
 .../analysis/in/IndicNormalizationFilter.java   |   4 +-
 .../in/IndicNormalizationFilterFactory.java     |   4 +-
 .../lucene/analysis/in/IndicNormalizer.java     |   4 +-
 .../lucene/analysis/it/ItalianAnalyzer.java     |   4 +-
 .../analysis/it/ItalianLightStemFilter.java     |   4 +-
 .../it/ItalianLightStemFilterFactory.java       |   4 +-
 .../lucene/analysis/it/ItalianLightStemmer.java |   4 +-
 .../lucene/analysis/lt/LithuanianAnalyzer.java  |   4 +-
 .../lucene/analysis/lv/LatvianAnalyzer.java     |   4 +-
 .../lucene/analysis/lv/LatvianStemFilter.java   |   4 +-
 .../analysis/lv/LatvianStemFilterFactory.java   |   4 +-
 .../lucene/analysis/lv/LatvianStemmer.java      |   8 +-
 .../miscellaneous/ASCIIFoldingFilter.java       |   4 +-
 .../ASCIIFoldingFilterFactory.java              |   4 +-
 .../miscellaneous/CapitalizationFilter.java     |   4 +-
 .../CapitalizationFilterFactory.java            |   4 +-
 .../miscellaneous/CodepointCountFilter.java     |   4 +-
 .../CodepointCountFilterFactory.java            |   4 +-
 .../miscellaneous/DateRecognizerFilter.java     |   4 +-
 .../DateRecognizerFilterFactory.java            |   4 +-
 .../miscellaneous/EmptyTokenStream.java         |   4 +-
 .../miscellaneous/FingerprintFilter.java        |   4 +-
 .../miscellaneous/FingerprintFilterFactory.java |   4 +-
 .../miscellaneous/HyphenatedWordsFilter.java    |   4 +-
 .../HyphenatedWordsFilterFactory.java           |   4 +-
 .../analysis/miscellaneous/KeepWordFilter.java  |   1 -
 .../miscellaneous/KeepWordFilterFactory.java    |   4 +-
 .../miscellaneous/KeywordMarkerFilter.java      |   4 +-
 .../KeywordMarkerFilterFactory.java             |   4 +-
 .../miscellaneous/KeywordRepeatFilter.java      |   4 +-
 .../KeywordRepeatFilterFactory.java             |   4 +-
 .../analysis/miscellaneous/LengthFilter.java    |   4 +-
 .../miscellaneous/LengthFilterFactory.java      |   4 +-
 .../miscellaneous/LimitTokenCountAnalyzer.java  |   4 +-
 .../miscellaneous/LimitTokenCountFilter.java    |   4 +-
 .../LimitTokenCountFilterFactory.java           |   4 +-
 .../miscellaneous/LimitTokenOffsetFilter.java   |   4 +-
 .../LimitTokenOffsetFilterFactory.java          |   4 +-
 .../miscellaneous/LimitTokenPositionFilter.java |   2 +-
 .../LimitTokenPositionFilterFactory.java        |   2 +-
 .../PatternKeywordMarkerFilter.java             |   4 +-
 .../miscellaneous/PerFieldAnalyzerWrapper.java  |   4 +-
 .../PrefixAndSuffixAwareTokenFilter.java        |   4 +-
 .../miscellaneous/PrefixAwareTokenFilter.java   |   4 +-
 .../RemoveDuplicatesTokenFilter.java            |   1 -
 .../RemoveDuplicatesTokenFilterFactory.java     |   4 +-
 .../ScandinavianFoldingFilter.java              |   4 +-
 .../ScandinavianFoldingFilterFactory.java       |   4 +-
 .../ScandinavianNormalizationFilter.java        |   4 +-
 .../ScandinavianNormalizationFilterFactory.java |   4 +-
 .../miscellaneous/SetKeywordMarkerFilter.java   |   2 +-
 .../miscellaneous/StemmerOverrideFilter.java    |   4 +-
 .../StemmerOverrideFilterFactory.java           |   4 +-
 .../analysis/miscellaneous/TrimFilter.java      |   1 -
 .../miscellaneous/TrimFilterFactory.java        |   4 +-
 .../miscellaneous/TruncateTokenFilter.java      |   4 +-
 .../TruncateTokenFilterFactory.java             |   4 +-
 .../miscellaneous/WordDelimiterFilter.java      |   3 +-
 .../WordDelimiterFilterFactory.java             |   4 +-
 .../miscellaneous/WordDelimiterIterator.java    |   4 +-
 .../analysis/ngram/EdgeNGramFilterFactory.java  |   4 +-
 .../analysis/ngram/EdgeNGramTokenFilter.java    |   4 +-
 .../analysis/ngram/EdgeNGramTokenizer.java      |   4 +-
 .../ngram/EdgeNGramTokenizerFactory.java        |   4 +-
 .../analysis/ngram/NGramFilterFactory.java      |   4 +-
 .../lucene/analysis/ngram/NGramTokenFilter.java |   4 +-
 .../lucene/analysis/ngram/NGramTokenizer.java   |   4 +-
 .../analysis/ngram/NGramTokenizerFactory.java   |   4 +-
 .../lucene/analysis/nl/DutchAnalyzer.java       |   4 +-
 .../lucene/analysis/no/NorwegianAnalyzer.java   |   4 +-
 .../analysis/no/NorwegianLightStemFilter.java   |   4 +-
 .../no/NorwegianLightStemFilterFactory.java     |   4 +-
 .../analysis/no/NorwegianLightStemmer.java      |   4 +-
 .../analysis/no/NorwegianMinimalStemFilter.java |   4 +-
 .../no/NorwegianMinimalStemFilterFactory.java   |   4 +-
 .../analysis/no/NorwegianMinimalStemmer.java    |   4 +-
 .../analysis/path/PathHierarchyTokenizer.java   |   2 +-
 .../path/PathHierarchyTokenizerFactory.java     |   4 +-
 .../path/ReversePathHierarchyTokenizer.java     |   2 +-
 .../PatternCaptureGroupFilterFactory.java       |   4 +-
 .../pattern/PatternCaptureGroupTokenFilter.java |   4 +-
 .../pattern/PatternReplaceCharFilter.java       |   1 -
 .../PatternReplaceCharFilterFactory.java        |   4 +-
 .../analysis/pattern/PatternReplaceFilter.java  |   1 -
 .../pattern/PatternReplaceFilterFactory.java    |   4 +-
 .../analysis/pattern/PatternTokenizer.java      |   1 -
 .../pattern/PatternTokenizerFactory.java        |   4 +-
 .../analysis/payloads/AbstractEncoder.java      |   8 +-
 .../payloads/DelimitedPayloadTokenFilter.java   |   2 +-
 .../DelimitedPayloadTokenFilterFactory.java     |   4 +-
 .../lucene/analysis/payloads/FloatEncoder.java  |   8 +-
 .../analysis/payloads/IdentityEncoder.java      |   2 +-
 .../analysis/payloads/IntegerEncoder.java       |   2 +-
 .../payloads/NumericPayloadTokenFilter.java     |   2 +-
 .../NumericPayloadTokenFilterFactory.java       |   4 +-
 .../analysis/payloads/PayloadEncoder.java       |   8 +-
 .../lucene/analysis/payloads/PayloadHelper.java |   2 +-
 .../payloads/TokenOffsetPayloadTokenFilter.java |   2 +-
 .../TokenOffsetPayloadTokenFilterFactory.java   |   4 +-
 .../payloads/TypeAsPayloadTokenFilter.java      |   2 +-
 .../TypeAsPayloadTokenFilterFactory.java        |   4 +-
 .../lucene/analysis/pt/PortugueseAnalyzer.java  |   4 +-
 .../analysis/pt/PortugueseLightStemFilter.java  |   4 +-
 .../pt/PortugueseLightStemFilterFactory.java    |   4 +-
 .../analysis/pt/PortugueseLightStemmer.java     |   4 +-
 .../pt/PortugueseMinimalStemFilter.java         |   4 +-
 .../pt/PortugueseMinimalStemFilterFactory.java  |   4 +-
 .../analysis/pt/PortugueseMinimalStemmer.java   |   4 +-
 .../analysis/pt/PortugueseStemFilter.java       |   4 +-
 .../pt/PortugueseStemFilterFactory.java         |   4 +-
 .../lucene/analysis/pt/PortugueseStemmer.java   |   4 +-
 .../lucene/analysis/pt/RSLPStemmerBase.java     |   4 +-
 .../query/QueryAutoStopWordAnalyzer.java        |   2 +-
 .../analysis/reverse/ReverseStringFilter.java   |   1 -
 .../reverse/ReverseStringFilterFactory.java     |   4 +-
 .../lucene/analysis/ro/RomanianAnalyzer.java    |   4 +-
 .../lucene/analysis/ru/RussianAnalyzer.java     |   4 +-
 .../analysis/ru/RussianLightStemFilter.java     |   4 +-
 .../ru/RussianLightStemFilterFactory.java       |   4 +-
 .../lucene/analysis/ru/RussianLightStemmer.java |   4 +-
 .../shingle/ShingleAnalyzerWrapper.java         |   4 +-
 .../lucene/analysis/shingle/ShingleFilter.java  |   4 +-
 .../analysis/shingle/ShingleFilterFactory.java  |   4 +-
 .../analysis/sinks/TeeSinkTokenFilter.java      |   4 +-
 .../analysis/snowball/SnowballFilter.java       |   4 +-
 .../snowball/SnowballPorterFilterFactory.java   |   4 +-
 .../analysis/sr/SerbianNormalizationFilter.java |   4 +-
 .../sr/SerbianNormalizationFilterFactory.java   |   4 +-
 .../sr/SerbianNormalizationRegularFilter.java   |   4 +-
 .../analysis/standard/ClassicAnalyzer.java      |   4 +-
 .../lucene/analysis/standard/ClassicFilter.java |   4 +-
 .../analysis/standard/ClassicFilterFactory.java |   4 +-
 .../analysis/standard/ClassicTokenizer.java     |   1 -
 .../standard/ClassicTokenizerFactory.java       |   4 +-
 .../analysis/standard/ClassicTokenizerImpl.java |   3 +-
 .../standard/ClassicTokenizerImpl.jflex         |   3 +-
 .../analysis/standard/StandardAnalyzer.java     |   4 +-
 .../analysis/standard/StandardFilter.java       |   4 +-
 .../standard/StandardFilterFactory.java         |   4 +-
 .../analysis/standard/StandardTokenizer.java    |   1 -
 .../standard/StandardTokenizerFactory.java      |   4 +-
 .../standard/StandardTokenizerImpl.java         |   3 +-
 .../standard/StandardTokenizerImpl.jflex        |   3 +-
 .../standard/UAX29URLEmailAnalyzer.java         |   4 +-
 .../standard/UAX29URLEmailTokenizer.java        |   4 +-
 .../standard/UAX29URLEmailTokenizerFactory.java |   4 +-
 .../standard/UAX29URLEmailTokenizerImpl.java    |   3 +-
 .../standard/UAX29URLEmailTokenizerImpl.jflex   |   3 +-
 .../lucene/analysis/sv/SwedishAnalyzer.java     |   4 +-
 .../analysis/sv/SwedishLightStemFilter.java     |   4 +-
 .../sv/SwedishLightStemFilterFactory.java       |   4 +-
 .../lucene/analysis/sv/SwedishLightStemmer.java |   4 +-
 .../analysis/synonym/SolrSynonymParser.java     |   4 +-
 .../lucene/analysis/synonym/SynonymFilter.java  |   4 +-
 .../analysis/synonym/SynonymFilterFactory.java  |   4 +-
 .../lucene/analysis/synonym/SynonymMap.java     |   4 +-
 .../analysis/synonym/WordnetSynonymParser.java  |   4 +-
 .../apache/lucene/analysis/th/ThaiAnalyzer.java |  14 +-
 .../lucene/analysis/th/ThaiTokenizer.java       |   4 +-
 .../analysis/th/ThaiTokenizerFactory.java       |   4 +-
 .../lucene/analysis/tr/ApostropheFilter.java    |   4 +-
 .../analysis/tr/ApostropheFilterFactory.java    |   4 +-
 .../lucene/analysis/tr/TurkishAnalyzer.java     |   4 +-
 .../analysis/tr/TurkishLowerCaseFilter.java     |   4 +-
 .../tr/TurkishLowerCaseFilterFactory.java       |   4 +-
 .../analysis/util/AbstractAnalysisFactory.java  |   4 +-
 .../lucene/analysis/util/AnalysisSPILoader.java |   4 +-
 .../lucene/analysis/util/CharArrayIterator.java |   4 +-
 .../lucene/analysis/util/CharArrayMap.java      |   4 +-
 .../lucene/analysis/util/CharArraySet.java      |   4 +-
 .../lucene/analysis/util/CharFilterFactory.java |   4 +-
 .../lucene/analysis/util/CharTokenizer.java     |   4 +-
 .../lucene/analysis/util/CharacterUtils.java    |   4 +-
 .../analysis/util/ClasspathResourceLoader.java  |   4 +-
 .../lucene/analysis/util/ElisionFilter.java     |   4 +-
 .../analysis/util/ElisionFilterFactory.java     |   4 +-
 .../analysis/util/FilesystemResourceLoader.java |   4 +-
 .../analysis/util/FilteringTokenFilter.java     |   4 +-
 .../analysis/util/MultiTermAwareComponent.java  |   4 +-
 .../lucene/analysis/util/OpenStringBuilder.java |   4 +-
 .../lucene/analysis/util/ResourceLoader.java    |   4 +-
 .../analysis/util/ResourceLoaderAware.java      |   1 -
 .../lucene/analysis/util/RollingCharBuffer.java |   4 +-
 .../analysis/util/SegmentingTokenizerBase.java  |   4 +-
 .../lucene/analysis/util/StemmerUtil.java       |   4 +-
 .../analysis/util/StopwordAnalyzerBase.java     |   1 -
 .../analysis/util/TokenFilterFactory.java       |   4 +-
 .../lucene/analysis/util/TokenizerFactory.java  |   4 +-
 .../lucene/analysis/util/WordlistLoader.java    |   4 +-
 .../analysis/wikipedia/WikipediaTokenizer.java  |   1 -
 .../wikipedia/WikipediaTokenizerFactory.java    |   4 +-
 .../wikipedia/WikipediaTokenizerImpl.java       |   3 +-
 .../wikipedia/WikipediaTokenizerImpl.jflex      |   3 +-
 .../collation/CollationAttributeFactory.java    |   4 +-
 .../collation/CollationDocValuesField.java      |   4 +-
 .../lucene/collation/CollationKeyAnalyzer.java  |   4 +-
 .../CollatedTermAttributeImpl.java              |   4 +-
 .../lucene/analysis/ar/TestArabicAnalyzer.java  |   4 +-
 .../lucene/analysis/ar/TestArabicFilters.java   |   4 +-
 .../ar/TestArabicNormalizationFilter.java       |   4 +-
 .../analysis/ar/TestArabicStemFilter.java       |   4 +-
 .../analysis/bg/TestBulgarianAnalyzer.java      |   4 +-
 .../bg/TestBulgarianStemFilterFactory.java      |   4 +-
 .../analysis/bg/TestBulgarianStemmer.java       |   4 +-
 .../analysis/br/TestBrazilianAnalyzer.java      |   4 +-
 .../br/TestBrazilianStemFilterFactory.java      |   4 +-
 .../lucene/analysis/ca/TestCatalanAnalyzer.java |   4 +-
 .../charfilter/HTMLStripCharFilterTest.java     |   4 +-
 .../TestHTMLStripCharFilterFactory.java         |   4 +-
 .../charfilter/TestMappingCharFilter.java       |   1 -
 .../TestMappingCharFilterFactory.java           |   4 +-
 .../lucene/analysis/cjk/TestCJKAnalyzer.java    |   4 +-
 .../analysis/cjk/TestCJKBigramFilter.java       |   4 +-
 .../cjk/TestCJKBigramFilterFactory.java         |   4 +-
 .../lucene/analysis/cjk/TestCJKWidthFilter.java |   4 +-
 .../analysis/cjk/TestCJKWidthFilterFactory.java |   4 +-
 .../lucene/analysis/ckb/TestSoraniAnalyzer.java |   4 +-
 .../ckb/TestSoraniNormalizationFilter.java      |   4 +-
 .../TestSoraniNormalizationFilterFactory.java   |   4 +-
 .../analysis/ckb/TestSoraniStemFilter.java      |   4 +-
 .../ckb/TestSoraniStemFilterFactory.java        |   4 +-
 .../TestCommonGramsFilterFactory.java           |   4 +-
 .../TestCommonGramsQueryFilterFactory.java      |   4 +-
 .../compound/TestCompoundWordTokenFilter.java   |   4 +-
 ...ictionaryCompoundWordTokenFilterFactory.java |   4 +-
 ...phenationCompoundWordTokenFilterFactory.java |   4 +-
 .../core/TestAllAnalyzersHaveFactories.java     |   4 +-
 .../lucene/analysis/core/TestAnalyzers.java     |   4 +-
 .../analysis/core/TestBugInSomething.java       |  32 +-
 .../analysis/core/TestDecimalDigitFilter.java   |   4 +-
 .../core/TestDecimalDigitFilterFactory.java     |   4 +-
 .../analysis/core/TestDuelingAnalyzers.java     |   4 +-
 .../lucene/analysis/core/TestFactories.java     |   4 +-
 .../analysis/core/TestKeywordAnalyzer.java      |   4 +-
 .../lucene/analysis/core/TestRandomChains.java  |   4 +-
 .../lucene/analysis/core/TestStopAnalyzer.java  |   4 +-
 .../lucene/analysis/core/TestStopFilter.java    |  15 +-
 .../analysis/core/TestStopFilterFactory.java    |   4 +-
 .../analysis/core/TestTypeTokenFilter.java      |   4 +-
 .../core/TestTypeTokenFilterFactory.java        |   4 +-
 .../core/TestUnicodeWhitespaceTokenizer.java    |   4 +-
 .../analysis/custom/TestCustomAnalyzer.java     |   4 +-
 .../lucene/analysis/cz/TestCzechAnalyzer.java   |   4 +-
 .../analysis/cz/TestCzechStemFilterFactory.java |   4 +-
 .../lucene/analysis/cz/TestCzechStemmer.java    |   4 +-
 .../lucene/analysis/da/TestDanishAnalyzer.java  |   4 +-
 .../lucene/analysis/de/TestGermanAnalyzer.java  |   4 +-
 .../analysis/de/TestGermanLightStemFilter.java  |   4 +-
 .../de/TestGermanLightStemFilterFactory.java    |   4 +-
 .../de/TestGermanMinimalStemFilter.java         |   4 +-
 .../de/TestGermanMinimalStemFilterFactory.java  |   4 +-
 .../de/TestGermanNormalizationFilter.java       |   4 +-
 .../TestGermanNormalizationFilterFactory.java   |   4 +-
 .../analysis/de/TestGermanStemFilter.java       |   4 +-
 .../de/TestGermanStemFilterFactory.java         |   4 +-
 .../lucene/analysis/el/GreekAnalyzerTest.java   |  14 +-
 .../el/TestGreekLowerCaseFilterFactory.java     |   4 +-
 .../analysis/el/TestGreekStemFilterFactory.java |   4 +-
 .../lucene/analysis/el/TestGreekStemmer.java    |   4 +-
 .../lucene/analysis/en/TestEnglishAnalyzer.java |   4 +-
 .../en/TestEnglishMinimalStemFilter.java        |   4 +-
 .../en/TestEnglishMinimalStemFilterFactory.java |   4 +-
 .../analysis/en/TestKStemFilterFactory.java     |   4 +-
 .../apache/lucene/analysis/en/TestKStemmer.java |   4 +-
 .../analysis/en/TestPorterStemFilter.java       |   4 +-
 .../en/TestPorterStemFilterFactory.java         |   4 +-
 .../lucene/analysis/es/TestSpanishAnalyzer.java |   4 +-
 .../analysis/es/TestSpanishLightStemFilter.java |   4 +-
 .../es/TestSpanishLightStemFilterFactory.java   |   4 +-
 .../lucene/analysis/eu/TestBasqueAnalyzer.java  |   4 +-
 .../lucene/analysis/fa/TestPersianAnalyzer.java |   4 +-
 .../analysis/fa/TestPersianCharFilter.java      |   4 +-
 .../fa/TestPersianNormalizationFilter.java      |   4 +-
 .../TestPersianNormalizationFilterFactory.java  |   4 +-
 .../lucene/analysis/fi/TestFinnishAnalyzer.java |   4 +-
 .../analysis/fi/TestFinnishLightStemFilter.java |   4 +-
 .../fi/TestFinnishLightStemFilterFactory.java   |   4 +-
 .../lucene/analysis/fr/TestFrenchAnalyzer.java  |   4 +-
 .../analysis/fr/TestFrenchLightStemFilter.java  |   4 +-
 .../fr/TestFrenchLightStemFilterFactory.java    |   4 +-
 .../fr/TestFrenchMinimalStemFilter.java         |   4 +-
 .../fr/TestFrenchMinimalStemFilterFactory.java  |   4 +-
 .../lucene/analysis/ga/TestIrishAnalyzer.java   |   4 +-
 .../analysis/ga/TestIrishLowerCaseFilter.java   |   4 +-
 .../ga/TestIrishLowerCaseFilterFactory.java     |   4 +-
 .../analysis/gl/TestGalicianAnalyzer.java       |   4 +-
 .../gl/TestGalicianMinimalStemFilter.java       |   4 +-
 .../TestGalicianMinimalStemFilterFactory.java   |   4 +-
 .../analysis/gl/TestGalicianStemFilter.java     |   4 +-
 .../gl/TestGalicianStemFilterFactory.java       |   4 +-
 .../lucene/analysis/hi/TestHindiAnalyzer.java   |  14 +-
 .../lucene/analysis/hi/TestHindiFilters.java    |   4 +-
 .../lucene/analysis/hi/TestHindiNormalizer.java |   4 +-
 .../lucene/analysis/hi/TestHindiStemmer.java    |   4 +-
 .../analysis/hu/TestHungarianAnalyzer.java      |   4 +-
 .../hu/TestHungarianLightStemFilter.java        |   4 +-
 .../hu/TestHungarianLightStemFilterFactory.java |   4 +-
 .../analysis/hunspell/StemmerTestBase.java      |   4 +-
 .../analysis/hunspell/Test64kAffixes.java       |   8 +-
 .../analysis/hunspell/TestAllDictionaries.java  |  10 +-
 .../analysis/hunspell/TestAllDictionaries2.java |  10 +-
 .../analysis/hunspell/TestAlternateCasing.java  |   4 +-
 .../analysis/hunspell/TestCaseInsensitive.java  |   4 +-
 .../analysis/hunspell/TestCaseSensitive.java    |   4 +-
 .../lucene/analysis/hunspell/TestCircumfix.java |   4 +-
 .../analysis/hunspell/TestComplexPrefix.java    |   4 +-
 .../lucene/analysis/hunspell/TestCondition.java |   4 +-
 .../analysis/hunspell/TestCondition2.java       |   8 +-
 .../lucene/analysis/hunspell/TestConv.java      |   8 +-
 .../analysis/hunspell/TestDependencies.java     |   4 +-
 .../analysis/hunspell/TestDictionary.java       |  10 +-
 .../analysis/hunspell/TestDoubleEscape.java     |   4 +-
 .../lucene/analysis/hunspell/TestEscaped.java   |   4 +-
 .../lucene/analysis/hunspell/TestFlagLong.java  |   4 +-
 .../lucene/analysis/hunspell/TestFlagNum.java   |   4 +-
 .../lucene/analysis/hunspell/TestFullStrip.java |   4 +-
 .../lucene/analysis/hunspell/TestHomonyms.java  |   4 +-
 .../hunspell/TestHunspellStemFilter.java        |  10 +-
 .../hunspell/TestHunspellStemFilterFactory.java |   4 +-
 .../lucene/analysis/hunspell/TestIgnore.java    |   4 +-
 .../lucene/analysis/hunspell/TestKeepCase.java  |   4 +-
 .../lucene/analysis/hunspell/TestMorph.java     |   4 +-
 .../analysis/hunspell/TestMorphAlias.java       |   4 +-
 .../lucene/analysis/hunspell/TestMorphData.java |   4 +-
 .../lucene/analysis/hunspell/TestNeedAffix.java |   4 +-
 .../analysis/hunspell/TestOnlyInCompound.java   |   4 +-
 .../hunspell/TestOptionalCondition.java         |   4 +-
 .../lucene/analysis/hunspell/TestSpaces.java    |   4 +-
 .../lucene/analysis/hunspell/TestStemmer.java   |   4 +-
 .../hunspell/TestStrangeOvergeneration.java     |   4 +-
 .../lucene/analysis/hunspell/TestTwoFold.java   |   4 +-
 .../analysis/hunspell/TestTwoSuffixes.java      |   4 +-
 .../lucene/analysis/hunspell/TestZeroAffix.java |   4 +-
 .../analysis/hunspell/TestZeroAffix2.java       |   4 +-
 .../analysis/hy/TestArmenianAnalyzer.java       |   4 +-
 .../analysis/id/TestIndonesianAnalyzer.java     |   4 +-
 .../id/TestIndonesianStemFilterFactory.java     |   4 +-
 .../analysis/id/TestIndonesianStemmer.java      |   4 +-
 .../lucene/analysis/in/TestIndicNormalizer.java |   4 +-
 .../lucene/analysis/it/TestItalianAnalyzer.java |   4 +-
 .../analysis/it/TestItalianLightStemFilter.java |   4 +-
 .../it/TestItalianLightStemFilterFactory.java   |   4 +-
 .../analysis/lt/TestLithuanianAnalyzer.java     |   4 +-
 .../analysis/lt/TestLithuanianStemming.java     |   4 +-
 .../lucene/analysis/lv/TestLatvianAnalyzer.java |   4 +-
 .../lv/TestLatvianStemFilterFactory.java        |   4 +-
 .../lucene/analysis/lv/TestLatvianStemmer.java  |   4 +-
 .../DateRecognizerFilterFactoryTest.java        |  14 +-
 .../miscellaneous/DateRecognizerFilterTest.java |  14 +-
 .../miscellaneous/TestASCIIFoldingFilter.java   |   4 +-
 .../miscellaneous/TestCapitalizationFilter.java |   1 -
 .../TestCapitalizationFilterFactory.java        |   4 +-
 .../miscellaneous/TestCodepointCountFilter.java |   4 +-
 .../TestCodepointCountFilterFactory.java        |   4 +-
 .../miscellaneous/TestEmptyTokenStream.java     |   4 +-
 .../miscellaneous/TestFingerprintFilter.java    |   4 +-
 .../TestFingerprintFilterFactory.java           |   4 +-
 .../TestHyphenatedWordsFilter.java              |   1 -
 .../miscellaneous/TestKeepFilterFactory.java    |   4 +-
 .../miscellaneous/TestKeepWordFilter.java       |   1 -
 .../miscellaneous/TestKeywordMarkerFilter.java  |  32 +-
 .../TestKeywordMarkerFilterFactory.java         |   4 +-
 .../miscellaneous/TestKeywordRepeatFilter.java  |   4 +-
 .../miscellaneous/TestLengthFilter.java         |   4 +-
 .../miscellaneous/TestLengthFilterFactory.java  |   4 +-
 .../TestLimitTokenCountAnalyzer.java            |   4 +-
 .../TestLimitTokenCountFilter.java              |   4 +-
 .../TestLimitTokenCountFilterFactory.java       |   4 +-
 .../TestLimitTokenOffsetFilter.java             |   4 +-
 .../TestLimitTokenOffsetFilterFactory.java      |   4 +-
 .../TestLimitTokenPositionFilter.java           |   2 +-
 .../TestLimitTokenPositionFilterFactory.java    |   2 +-
 .../TestPerFieldAnalyzerWrapper.java            |  32 +-
 .../TestPrefixAndSuffixAwareTokenFilter.java    |   4 +-
 .../TestPrefixAwareTokenFilter.java             |   4 +-
 .../TestRemoveDuplicatesTokenFilter.java        |   1 -
 .../TestRemoveDuplicatesTokenFilterFactory.java |   4 +-
 .../TestScandinavianFoldingFilter.java          |   4 +-
 .../TestScandinavianFoldingFilterFactory.java   |  15 +-
 .../TestScandinavianNormalizationFilter.java    |   4 +-
 ...tScandinavianNormalizationFilterFactory.java |  18 +-
 .../TestStemmerOverrideFilter.java              |   2 +-
 .../TestStemmerOverrideFilterFactory.java       |   4 +-
 .../analysis/miscellaneous/TestTrimFilter.java  |   1 -
 .../miscellaneous/TestTrimFilterFactory.java    |   4 +-
 .../miscellaneous/TestTruncateTokenFilter.java  |   4 +-
 .../TestTruncateTokenFilterFactory.java         |   4 +-
 .../miscellaneous/TestWordDelimiterFilter.java  |   1 -
 .../ngram/EdgeNGramTokenFilterTest.java         |   4 +-
 .../analysis/ngram/EdgeNGramTokenizerTest.java  |   4 +-
 .../analysis/ngram/NGramTokenFilterTest.java    |   4 +-
 .../analysis/ngram/NGramTokenizerTest.java      |   4 +-
 .../lucene/analysis/ngram/TestNGramFilters.java |   4 +-
 .../lucene/analysis/nl/TestDutchAnalyzer.java   |   4 +-
 .../analysis/no/TestNorwegianAnalyzer.java      |   4 +-
 .../no/TestNorwegianLightStemFilter.java        |   4 +-
 .../no/TestNorwegianLightStemFilterFactory.java |   4 +-
 .../no/TestNorwegianMinimalStemFilter.java      |   4 +-
 .../TestNorwegianMinimalStemFilterFactory.java  |   4 +-
 .../path/TestPathHierarchyTokenizer.java        |   4 +-
 .../path/TestReversePathHierarchyTokenizer.java |   4 +-
 .../TestPatternCaptureGroupTokenFilter.java     |   4 +-
 .../pattern/TestPatternReplaceCharFilter.java   |   1 -
 .../TestPatternReplaceCharFilterFactory.java    |   4 +-
 .../pattern/TestPatternReplaceFilter.java       |   1 -
 .../TestPatternReplaceFilterFactory.java        |   4 +-
 .../analysis/pattern/TestPatternTokenizer.java  |   1 -
 .../pattern/TestPatternTokenizerFactory.java    |   4 +-
 .../DelimitedPayloadTokenFilterTest.java        |   2 +-
 .../payloads/NumericPayloadTokenFilterTest.java |  14 +-
 .../TestDelimitedPayloadTokenFilterFactory.java |   4 +-
 .../TokenOffsetPayloadTokenFilterTest.java      |  15 +-
 .../payloads/TypeAsPayloadTokenFilterTest.java  |  17 +-
 .../analysis/pt/TestPortugueseAnalyzer.java     |   4 +-
 .../pt/TestPortugueseLightStemFilter.java       |   4 +-
 .../TestPortugueseLightStemFilterFactory.java   |   4 +-
 .../pt/TestPortugueseMinimalStemFilter.java     |   4 +-
 .../TestPortugueseMinimalStemFilterFactory.java |   4 +-
 .../analysis/pt/TestPortugueseStemFilter.java   |   4 +-
 .../pt/TestPortugueseStemFilterFactory.java     |   4 +-
 .../query/QueryAutoStopWordAnalyzerTest.java    |   2 +-
 .../reverse/TestReverseStringFilter.java        |   1 -
 .../reverse/TestReverseStringFilterFactory.java |   4 +-
 .../analysis/ro/TestRomanianAnalyzer.java       |   4 +-
 .../lucene/analysis/ru/TestRussianAnalyzer.java |   4 +-
 .../analysis/ru/TestRussianLightStemFilter.java |   4 +-
 .../ru/TestRussianLightStemFilterFactory.java   |   4 +-
 .../shingle/ShingleAnalyzerWrapperTest.java     |   4 +-
 .../analysis/shingle/ShingleFilterTest.java     |   4 +-
 .../shingle/TestShingleFilterFactory.java       |   4 +-
 .../analysis/sinks/TestTeeSinkTokenFilter.java  |  14 +-
 .../lucene/analysis/snowball/TestSnowball.java  |   4 +-
 .../TestSnowballPorterFilterFactory.java        |  15 +-
 .../analysis/snowball/TestSnowballVocab.java    |   4 +-
 .../sr/TestSerbianNormalizationFilter.java      |   4 +-
 .../TestSerbianNormalizationFilterFactory.java  |   4 +-
 .../TestSerbianNormalizationRegularFilter.java  |   4 +-
 .../analysis/standard/TestClassicAnalyzer.java  |   4 +-
 .../analysis/standard/TestStandardAnalyzer.java |   4 +-
 .../standard/TestStandardFactories.java         |   4 +-
 .../standard/TestUAX29URLEmailAnalyzer.java     |   4 +-
 .../standard/TestUAX29URLEmailTokenizer.java    |  32 +-
 .../TestUAX29URLEmailTokenizerFactory.java      |   4 +-
 .../standard/WordBreakTestUnicode_6_3_0.java    |   4 +-
 .../lucene/analysis/sv/TestSwedishAnalyzer.java |   4 +-
 .../analysis/sv/TestSwedishLightStemFilter.java |   4 +-
 .../sv/TestSwedishLightStemFilterFactory.java   |   4 +-
 .../synonym/BaseSynonymParserTestCase.java      |   4 +-
 .../analysis/synonym/TestMultiWordSynonyms.java |   4 +-
 .../analysis/synonym/TestSolrSynonymParser.java |   4 +-
 .../synonym/TestSynonymFilterFactory.java       |   4 +-
 .../analysis/synonym/TestSynonymMapFilter.java  |   1 -
 .../synonym/TestWordnetSynonymParser.java       |   1 -
 .../lucene/analysis/th/TestThaiAnalyzer.java    |   4 +-
 .../analysis/th/TestThaiTokenizerFactory.java   |   4 +-
 .../analysis/tr/TestApostropheFilter.java       |   4 +-
 .../tr/TestApostropheFilterFactory.java         |   4 +-
 .../lucene/analysis/tr/TestTurkishAnalyzer.java |   4 +-
 .../analysis/tr/TestTurkishLowerCaseFilter.java |   4 +-
 .../tr/TestTurkishLowerCaseFilterFactory.java   |   4 +-
 .../util/BaseTokenStreamFactoryTestCase.java    |   4 +-
 .../analysis/util/StringMockResourceLoader.java |   4 +-
 .../analysis/util/TestAnalysisSPILoader.java    |   4 +-
 .../analysis/util/TestCharArrayIterator.java    |   4 +-
 .../lucene/analysis/util/TestCharArrayMap.java  |   1 -
 .../lucene/analysis/util/TestCharArraySet.java  |   4 +-
 .../analysis/util/TestCharTokenizers.java       |   4 +-
 .../analysis/util/TestCharacterUtils.java       |   4 +-
 .../lucene/analysis/util/TestElision.java       |   4 +-
 .../analysis/util/TestElisionFilterFactory.java |   4 +-
 .../util/TestFilesystemResourceLoader.java      |  32 +-
 .../analysis/util/TestRollingCharBuffer.java    |   4 +-
 .../util/TestSegmentingTokenizerBase.java       |   4 +-
 .../analysis/util/TestWordlistLoader.java       |   4 +-
 .../TestWikipediaTokenizerFactory.java          |   4 +-
 .../wikipedia/WikipediaTokenizerTest.java       |   1 -
 .../collation/TestCollationDocValuesField.java  |   4 +-
 .../collation/TestCollationKeyAnalyzer.java     |   4 +-
 .../standard/GenerateJflexTLDMacros.java        |  16 +-
 .../lucene/analysis/icu/ICUFoldingFilter.java   |   4 +-
 .../analysis/icu/ICUFoldingFilterFactory.java   |   4 +-
 .../analysis/icu/ICUNormalizer2CharFilter.java  |   4 +-
 .../icu/ICUNormalizer2CharFilterFactory.java    |   4 +-
 .../analysis/icu/ICUNormalizer2Filter.java      |   4 +-
 .../icu/ICUNormalizer2FilterFactory.java        |   4 +-
 .../lucene/analysis/icu/ICUTransformFilter.java |   4 +-
 .../analysis/icu/ICUTransformFilterFactory.java |   4 +-
 .../icu/segmentation/BreakIteratorWrapper.java  |   4 +-
 .../icu/segmentation/CharArrayIterator.java     |   4 +-
 .../segmentation/CompositeBreakIterator.java    |   4 +-
 .../segmentation/DefaultICUTokenizerConfig.java |   4 +-
 .../analysis/icu/segmentation/ICUTokenizer.java |   4 +-
 .../icu/segmentation/ICUTokenizerConfig.java    |   4 +-
 .../icu/segmentation/ICUTokenizerFactory.java   |   4 +-
 .../icu/segmentation/ScriptIterator.java        |   3 +-
 .../icu/tokenattributes/ScriptAttribute.java    |   4 +-
 .../tokenattributes/ScriptAttributeImpl.java    |   4 +-
 .../collation/ICUCollationAttributeFactory.java |   4 +-
 .../collation/ICUCollationDocValuesField.java   |   4 +-
 .../collation/ICUCollationKeyAnalyzer.java      |   4 +-
 .../ICUCollatedTermAttributeImpl.java           |   4 +-
 .../analysis/icu/TestICUFoldingFilter.java      |   4 +-
 .../icu/TestICUFoldingFilterFactory.java        |   4 +-
 .../icu/TestICUNormalizer2CharFilter.java       |   4 +-
 .../TestICUNormalizer2CharFilterFactory.java    |   4 +-
 .../analysis/icu/TestICUNormalizer2Filter.java  |   4 +-
 .../icu/TestICUNormalizer2FilterFactory.java    |   4 +-
 .../analysis/icu/TestICUTransformFilter.java    |   4 +-
 .../icu/TestICUTransformFilterFactory.java      |   4 +-
 .../icu/segmentation/TestCharArrayIterator.java |   4 +-
 .../icu/segmentation/TestICUTokenizer.java      |   4 +-
 .../icu/segmentation/TestICUTokenizerCJK.java   |   4 +-
 .../segmentation/TestICUTokenizerFactory.java   |   4 +-
 .../segmentation/TestWithCJKBigramFilter.java   |   4 +-
 .../TestICUCollationDocValuesField.java         |   4 +-
 .../collation/TestICUCollationKeyAnalyzer.java  |   4 +-
 .../analysis/icu/GenerateUTR30DataFiles.java    |   4 +-
 .../lucene/analysis/icu/RBBIRuleCompiler.java   |   4 +-
 .../lucene/analysis/ja/GraphvizFormatter.java   |   4 +-
 .../lucene/analysis/ja/JapaneseAnalyzer.java    |   4 +-
 .../analysis/ja/JapaneseBaseFormFilter.java     |   4 +-
 .../ja/JapaneseBaseFormFilterFactory.java       |   4 +-
 .../ja/JapaneseIterationMarkCharFilter.java     |   4 +-
 .../JapaneseIterationMarkCharFilterFactory.java |   4 +-
 .../analysis/ja/JapaneseKatakanaStemFilter.java |   4 +-
 .../ja/JapaneseKatakanaStemFilterFactory.java   |   4 +-
 .../analysis/ja/JapaneseNumberFilter.java       |   4 +-
 .../ja/JapaneseNumberFilterFactory.java         |   4 +-
 .../ja/JapanesePartOfSpeechStopFilter.java      |   4 +-
 .../JapanesePartOfSpeechStopFilterFactory.java  |   4 +-
 .../analysis/ja/JapaneseReadingFormFilter.java  |   4 +-
 .../ja/JapaneseReadingFormFilterFactory.java    |   4 +-
 .../lucene/analysis/ja/JapaneseTokenizer.java   |   4 +-
 .../analysis/ja/JapaneseTokenizerFactory.java   |   4 +-
 .../org/apache/lucene/analysis/ja/Token.java    |   4 +-
 .../analysis/ja/dict/BinaryDictionary.java      |   4 +-
 .../analysis/ja/dict/CharacterDefinition.java   |   4 +-
 .../analysis/ja/dict/ConnectionCosts.java       |   4 +-
 .../lucene/analysis/ja/dict/Dictionary.java     |   4 +-
 .../analysis/ja/dict/TokenInfoDictionary.java   |   4 +-
 .../lucene/analysis/ja/dict/TokenInfoFST.java   |   4 +-
 .../analysis/ja/dict/UnknownDictionary.java     |   4 +-
 .../lucene/analysis/ja/dict/UserDictionary.java |   4 +-
 .../ja/tokenattributes/BaseFormAttribute.java   |   4 +-
 .../tokenattributes/BaseFormAttributeImpl.java  |   4 +-
 .../ja/tokenattributes/InflectionAttribute.java |   4 +-
 .../InflectionAttributeImpl.java                |   4 +-
 .../tokenattributes/PartOfSpeechAttribute.java  |   4 +-
 .../PartOfSpeechAttributeImpl.java              |   4 +-
 .../ja/tokenattributes/ReadingAttribute.java    |   4 +-
 .../tokenattributes/ReadingAttributeImpl.java   |   4 +-
 .../apache/lucene/analysis/ja/util/CSVUtil.java |   4 +-
 .../lucene/analysis/ja/util/ToStringUtil.java   |   4 +-
 .../analysis/ja/StringMockResourceLoader.java   |   4 +-
 .../lucene/analysis/ja/TestExtendedMode.java    |   4 +-
 .../analysis/ja/TestJapaneseAnalyzer.java       |   4 +-
 .../analysis/ja/TestJapaneseBaseFormFilter.java |   4 +-
 .../ja/TestJapaneseBaseFormFilterFactory.java   |   4 +-
 .../ja/TestJapaneseIterationMarkCharFilter.java |   4 +-
 ...tJapaneseIterationMarkCharFilterFactory.java |   4 +-
 .../ja/TestJapaneseKatakanaStemFilter.java      |   4 +-
 .../TestJapaneseKatakanaStemFilterFactory.java  |   4 +-
 .../analysis/ja/TestJapaneseNumberFilter.java   |   4 +-
 .../ja/TestJapaneseNumberFilterFactory.java     |   4 +-
 ...stJapanesePartOfSpeechStopFilterFactory.java |   4 +-
 .../ja/TestJapaneseReadingFormFilter.java       |   4 +-
 .../TestJapaneseReadingFormFilterFactory.java   |   4 +-
 .../analysis/ja/TestJapaneseTokenizer.java      |   4 +-
 .../ja/TestJapaneseTokenizerFactory.java        |   4 +-
 .../lucene/analysis/ja/TestSearchMode.java      |   4 +-
 .../ja/dict/TestTokenInfoDictionary.java        |   4 +-
 .../analysis/ja/dict/UserDictionaryTest.java    |   4 +-
 .../analysis/ja/util/TestToStringUtil.java      |   4 +-
 .../ja/util/BinaryDictionaryWriter.java         |   4 +-
 .../ja/util/CharacterDefinitionWriter.java      |   4 +-
 .../ja/util/ConnectionCostsBuilder.java         |   4 +-
 .../analysis/ja/util/ConnectionCostsWriter.java |   4 +-
 .../analysis/ja/util/DictionaryBuilder.java     |   4 +-
 .../ja/util/TokenInfoDictionaryBuilder.java     |   4 +-
 .../ja/util/TokenInfoDictionaryWriter.java      |   4 +-
 .../ja/util/UnknownDictionaryBuilder.java       |   4 +-
 .../ja/util/UnknownDictionaryWriter.java        |   4 +-
 .../analysis/ja/dict/UnknownDictionaryTest.java |   4 +-
 .../analysis/morfologik/MorfologikAnalyzer.java |   3 +-
 .../analysis/morfologik/MorfologikFilter.java   |   4 +-
 .../morfologik/MorfologikFilterFactory.java     |   4 +-
 .../MorphosyntacticTagsAttribute.java           |   3 +-
 .../MorphosyntacticTagsAttributeImpl.java       |   3 +-
 .../morfologik/TestMorfologikAnalyzer.java      |   4 +-
 .../morfologik/TestMorfologikFilterFactory.java |   4 +-
 .../analysis/phonetic/BeiderMorseFilter.java    |   4 +-
 .../phonetic/BeiderMorseFilterFactory.java      |   4 +-
 .../phonetic/DaitchMokotoffSoundexFilter.java   |   2 +-
 .../DaitchMokotoffSoundexFilterFactory.java     |   2 +-
 .../phonetic/DoubleMetaphoneFilterFactory.java  |   4 +-
 .../analysis/phonetic/PhoneticFilter.java       |   1 -
 .../phonetic/PhoneticFilterFactory.java         |   4 +-
 .../phonetic/TestBeiderMorseFilter.java         |   4 +-
 .../phonetic/TestBeiderMorseFilterFactory.java  |   4 +-
 .../TestDaitchMokotoffSoundexFilter.java        |   2 +-
 .../TestDaitchMokotoffSoundexFilterFactory.java |   4 +-
 .../TestDoubleMetaphoneFilterFactory.java       |   4 +-
 .../analysis/phonetic/TestPhoneticFilter.java   |   1 -
 .../phonetic/TestPhoneticFilterFactory.java     |   4 +-
 .../analysis/cn/smart/AnalyzerProfile.java      |   1 -
 .../lucene/analysis/cn/smart/CharType.java      |   1 -
 .../analysis/cn/smart/HMMChineseTokenizer.java  |   4 +-
 .../cn/smart/HMMChineseTokenizerFactory.java    |   1 -
 .../analysis/cn/smart/SmartChineseAnalyzer.java |   1 -
 .../lucene/analysis/cn/smart/Utility.java       |   1 -
 .../lucene/analysis/cn/smart/WordSegmenter.java |   1 -
 .../lucene/analysis/cn/smart/WordType.java      |   1 -
 .../cn/smart/hhmm/AbstractDictionary.java       |   1 -
 .../analysis/cn/smart/hhmm/BiSegGraph.java      |   1 -
 .../cn/smart/hhmm/BigramDictionary.java         |   1 -
 .../analysis/cn/smart/hhmm/HHMMSegmenter.java   |   1 -
 .../lucene/analysis/cn/smart/hhmm/PathNode.java |   1 -
 .../lucene/analysis/cn/smart/hhmm/SegGraph.java |   1 -
 .../lucene/analysis/cn/smart/hhmm/SegToken.java |   1 -
 .../analysis/cn/smart/hhmm/SegTokenFilter.java  |   1 -
 .../analysis/cn/smart/hhmm/SegTokenPair.java    |   1 -
 .../analysis/cn/smart/hhmm/WordDictionary.java  |   1 -
 .../smart/TestHMMChineseTokenizerFactory.java   |   4 +-
 .../cn/smart/TestSmartChineseAnalyzer.java      |   1 -
 .../lucene/analysis/pl/PolishAnalyzer.java      |   4 +-
 .../lucene/analysis/stempel/StempelFilter.java  |  26 +-
 .../stempel/StempelPolishStemFilterFactory.java |   4 +-
 .../lucene/analysis/stempel/StempelStemmer.java |  26 +-
 .../lucene/analysis/pl/TestPolishAnalyzer.java  |   4 +-
 .../TestStempelPolishStemFilterFactory.java     |   4 +-
 .../lucene/analysis/uima/BaseUIMATokenizer.java |   4 +-
 .../analysis/uima/UIMAAnnotationsTokenizer.java |   4 +-
 .../uima/UIMAAnnotationsTokenizerFactory.java   |   4 +-
 .../lucene/analysis/uima/UIMABaseAnalyzer.java  |   4 +-
 .../analysis/uima/UIMATypeAwareAnalyzer.java    |   4 +-
 .../uima/UIMATypeAwareAnnotationsTokenizer.java |   4 +-
 ...IMATypeAwareAnnotationsTokenizerFactory.java |   4 +-
 .../lucene/analysis/uima/ae/AEProvider.java     |   4 +-
 .../analysis/uima/ae/AEProviderFactory.java     |   4 +-
 .../analysis/uima/ae/BasicAEProvider.java       |   4 +-
 .../uima/ae/OverridingParamsAEProvider.java     |   4 +-
 .../analysis/uima/UIMABaseAnalyzerTest.java     |   4 +-
 .../uima/UIMATypeAwareAnalyzerTest.java         |   4 +-
 .../analysis/uima/ae/AEProviderFactoryTest.java |   4 +-
 .../analysis/uima/ae/BasicAEProviderTest.java   |   4 +-
 .../uima/ae/OverridingParamsAEProviderTest.java |   4 +-
 .../analysis/uima/an/SampleEntityAnnotator.java |   4 +-
 .../analysis/uima/an/SamplePoSTagger.java       |   4 +-
 .../uima/an/SampleWSTokenizerAnnotator.java     |   4 +-
 .../org/apache/lucene/codecs/Placeholder.java   |   4 +-
 .../lucene/codecs/lucene50/Lucene50Codec.java   |   4 +-
 .../lucene50/Lucene50DocValuesConsumer.java     |   4 +-
 .../lucene50/Lucene50DocValuesFormat.java       |   4 +-
 .../lucene50/Lucene50DocValuesProducer.java     |   4 +-
 .../codecs/lucene50/Lucene50NormsFormat.java    |   4 +-
 .../codecs/lucene50/Lucene50NormsProducer.java  |   4 +-
 .../lucene/codecs/lucene53/Lucene53Codec.java   |   4 +-
 .../lucene/codecs/lucene54/Lucene54Codec.java   |   4 +-
 .../codecs/lucene50/Lucene50NormsConsumer.java  |   4 +-
 .../lucene/codecs/lucene50/Lucene50RWCodec.java |   4 +-
 .../codecs/lucene50/Lucene50RWNormsFormat.java  |   4 +-
 .../lucene50/TestLucene50DocValuesFormat.java   |   4 +-
 .../lucene50/TestLucene50NormsFormat.java       |   4 +-
 .../index/TestBackwardsCompatibility.java       |  12 +-
 .../lucene/index/TestMaxPositionInOldIndex.java |   4 +-
 lucene/benchmark/.gitignore                     |   2 +
 .../org/apache/lucene/benchmark/Constants.java  |  19 +-
 .../lucene/benchmark/byTask/Benchmark.java      |   4 +-
 .../lucene/benchmark/byTask/PerfRunData.java    |   4 +-
 .../byTask/feeds/AbstractQueryMaker.java        |  13 +-
 .../byTask/feeds/ContentItemsSource.java        |   4 +-
 .../benchmark/byTask/feeds/ContentSource.java   |   4 +-
 .../benchmark/byTask/feeds/DemoHTMLParser.java  |   4 +-
 .../byTask/feeds/DirContentSource.java          |   4 +-
 .../lucene/benchmark/byTask/feeds/DocData.java  |   4 +-
 .../lucene/benchmark/byTask/feeds/DocMaker.java |   4 +-
 .../byTask/feeds/EnwikiContentSource.java       |   4 +-
 .../byTask/feeds/EnwikiQueryMaker.java          |   4 +-
 .../benchmark/byTask/feeds/FacetSource.java     |   4 +-
 .../byTask/feeds/FileBasedQueryMaker.java       |  18 +-
 .../byTask/feeds/GeonamesLineParser.java        |   4 +-
 .../benchmark/byTask/feeds/HTMLParser.java      |   4 +-
 .../benchmark/byTask/feeds/LineDocSource.java   |   4 +-
 .../feeds/LongToEnglishContentSource.java       |   4 +-
 .../byTask/feeds/LongToEnglishQueryMaker.java   |   4 +-
 .../byTask/feeds/NoMoreDataException.java       |   4 +-
 .../benchmark/byTask/feeds/QueryMaker.java      |   4 +-
 .../byTask/feeds/RandomFacetSource.java         |   4 +-
 .../byTask/feeds/ReutersContentSource.java      |   4 +-
 .../byTask/feeds/ReutersQueryMaker.java         |   4 +-
 .../byTask/feeds/SimpleQueryMaker.java          |   4 +-
 .../feeds/SimpleSloppyPhraseQueryMaker.java     |   4 +-
 .../benchmark/byTask/feeds/SingleDocSource.java |   8 +-
 .../byTask/feeds/SortableSingleDocSource.java   |   4 +-
 .../benchmark/byTask/feeds/SpatialDocMaker.java |   4 +-
 .../byTask/feeds/SpatialFileQueryMaker.java     |   4 +-
 .../byTask/feeds/TrecContentSource.java         |   4 +-
 .../benchmark/byTask/feeds/TrecDocParser.java   |   4 +-
 .../benchmark/byTask/feeds/TrecFBISParser.java  |   4 +-
 .../benchmark/byTask/feeds/TrecFR94Parser.java  |   4 +-
 .../benchmark/byTask/feeds/TrecFTParser.java    |   4 +-
 .../benchmark/byTask/feeds/TrecGov2Parser.java  |   4 +-
 .../byTask/feeds/TrecLATimesParser.java         |   4 +-
 .../byTask/feeds/TrecParserByPath.java          |   4 +-
 .../benchmark/byTask/programmatic/Sample.java   |  15 +-
 .../lucene/benchmark/byTask/stats/Points.java   |   4 +-
 .../lucene/benchmark/byTask/stats/Report.java   |   4 +-
 .../benchmark/byTask/stats/TaskStats.java       |   4 +-
 .../benchmark/byTask/tasks/AddDocTask.java      |   4 +-
 .../byTask/tasks/AddFacetedDocTask.java         |   4 +-
 .../benchmark/byTask/tasks/AddIndexesTask.java  |   4 +-
 .../byTask/tasks/AnalyzerFactoryTask.java       |   4 +-
 .../byTask/tasks/BenchmarkHighlighter.java      |   4 +-
 .../benchmark/byTask/tasks/ClearStatsTask.java  |   8 +-
 .../benchmark/byTask/tasks/CloseIndexTask.java  |   4 +-
 .../benchmark/byTask/tasks/CloseReaderTask.java |   4 +-
 .../byTask/tasks/CloseTaxonomyIndexTask.java    |   4 +-
 .../byTask/tasks/CloseTaxonomyReaderTask.java   |   4 +-
 .../benchmark/byTask/tasks/CommitIndexTask.java |   4 +-
 .../byTask/tasks/CommitTaxonomyIndexTask.java   |   2 +-
 .../byTask/tasks/ConsumeContentSourceTask.java  |   4 +-
 .../benchmark/byTask/tasks/CreateIndexTask.java |   4 +-
 .../byTask/tasks/CreateTaxonomyIndexTask.java   |   4 +-
 .../benchmark/byTask/tasks/ForceMergeTask.java  |   4 +-
 .../byTask/tasks/NearRealtimeReaderTask.java    |   4 +-
 .../benchmark/byTask/tasks/NewAnalyzerTask.java |  13 +-
 .../byTask/tasks/NewCollationAnalyzerTask.java  |   4 +-
 .../benchmark/byTask/tasks/NewLocaleTask.java   |   4 +-
 .../benchmark/byTask/tasks/NewRoundTask.java    |   8 +-
 .../benchmark/byTask/tasks/OpenIndexTask.java   |   4 +-
 .../benchmark/byTask/tasks/OpenReaderTask.java  |   4 +-
 .../byTask/tasks/OpenTaxonomyIndexTask.java     |   4 +-
 .../byTask/tasks/OpenTaxonomyReaderTask.java    |   4 +-
 .../lucene/benchmark/byTask/tasks/PerfTask.java |   4 +-
 .../benchmark/byTask/tasks/PrintReaderTask.java |   4 +-
 .../lucene/benchmark/byTask/tasks/ReadTask.java |   4 +-
 .../benchmark/byTask/tasks/ReadTokensTask.java  |   4 +-
 .../byTask/tasks/ReopenReaderTask.java          |   2 +-
 .../benchmark/byTask/tasks/RepAllTask.java      |   4 +-
 .../byTask/tasks/RepSelectByPrefTask.java       |   4 +-
 .../byTask/tasks/RepSumByNameRoundTask.java     |   4 +-
 .../byTask/tasks/RepSumByNameTask.java          |   4 +-
 .../byTask/tasks/RepSumByPrefRoundTask.java     |   4 +-
 .../byTask/tasks/RepSumByPrefTask.java          |   4 +-
 .../benchmark/byTask/tasks/ReportTask.java      |  18 +-
 .../benchmark/byTask/tasks/ResetInputsTask.java |   8 +-
 .../byTask/tasks/ResetSystemEraseTask.java      |   8 +-
 .../byTask/tasks/ResetSystemSoftTask.java       |   8 +-
 .../byTask/tasks/RollbackIndexTask.java         |   4 +-
 .../benchmark/byTask/tasks/SearchTask.java      |   4 +-
 .../tasks/SearchTravRetHighlightTask.java       |   4 +-
 .../SearchTravRetLoadFieldSelectorTask.java     |  14 +-
 .../byTask/tasks/SearchTravRetTask.java         |   4 +-
 .../tasks/SearchTravRetVectorHighlightTask.java |   4 +-
 .../benchmark/byTask/tasks/SearchTravTask.java  |   4 +-
 .../byTask/tasks/SearchWithCollectorTask.java   |   2 +-
 .../byTask/tasks/SearchWithSortTask.java        |   2 +-
 .../benchmark/byTask/tasks/SetPropTask.java     |   8 +-
 .../benchmark/byTask/tasks/TaskSequence.java    |   4 +-
 .../benchmark/byTask/tasks/UpdateDocTask.java   |   4 +-
 .../lucene/benchmark/byTask/tasks/WaitTask.java |   4 +-
 .../lucene/benchmark/byTask/tasks/WarmTask.java |   4 +-
 .../byTask/tasks/WriteEnwikiLineDocTask.java    |  32 +-
 .../byTask/tasks/WriteLineDocTask.java          |   4 +-
 .../benchmark/byTask/utils/Algorithm.java       |   4 +-
 .../benchmark/byTask/utils/AnalyzerFactory.java |   4 +-
 .../lucene/benchmark/byTask/utils/Config.java   |   4 +-
 .../lucene/benchmark/byTask/utils/Format.java   |   4 +-
 .../benchmark/byTask/utils/StreamUtils.java     |   4 +-
 .../benchmark/quality/trec/QueryDriver.java     |   4 +-
 .../lucene/benchmark/utils/ExtractReuters.java  |  15 +-
 .../benchmark/utils/ExtractWikipedia.java       |   4 +-
 lucene/benchmark/src/test/conf/ConfLoader.java  |   3 +-
 .../lucene/benchmark/BenchmarkTestCase.java     |   4 +-
 .../benchmark/byTask/TestPerfTasksLogic.java    |   3 -
 .../benchmark/byTask/TestPerfTasksParse.java    |   1 -
 .../benchmark/byTask/feeds/DocMakerTest.java    |   4 +-
 .../byTask/feeds/EnwikiContentSourceTest.java   |   4 +-
 .../byTask/feeds/LineDocSourceTest.java         |   4 +-
 .../benchmark/byTask/feeds/TestHtmlParser.java  |   4 +-
 .../byTask/feeds/TrecContentSourceTest.java     |   4 +-
 .../byTask/tasks/AddIndexesTaskTest.java        |   4 +-
 .../byTask/tasks/CommitIndexTaskTest.java       |   4 +-
 .../tasks/CountingHighlighterTestTask.java      |   1 -
 .../byTask/tasks/CountingSearchTestTask.java    |   1 -
 .../byTask/tasks/CreateIndexTaskTest.java       |   4 +-
 .../benchmark/byTask/tasks/PerfTaskTest.java    |   4 +-
 .../byTask/tasks/SearchWithSortTaskTest.java    |   4 +-
 .../tasks/WriteEnwikiLineDocTaskTest.java       |   4 +-
 .../byTask/tasks/WriteLineDocTaskTest.java      |   4 +-
 .../byTask/tasks/alt/AltPackageTaskTest.java    |   4 +-
 .../benchmark/byTask/tasks/alt/AltTestTask.java |   4 +-
 .../benchmark/byTask/utils/StreamUtilsTest.java |  18 +-
 .../benchmark/byTask/utils/TestConfig.java      |   4 +-
 .../benchmark/quality/TestQualityRun.java       |   4 +-
 .../CachingNaiveBayesClassifier.java            |  32 +-
 .../document/DocumentClassifier.java            |   4 +-
 .../KNearestNeighborDocumentClassifier.java     |   4 +-
 .../SimpleNaiveBayesDocumentClassifier.java     |   4 +-
 .../utils/ConfusionMatrixGenerator.java         |   4 +-
 .../classification/utils/DatasetSplitter.java   |   4 +-
 .../CachingNaiveBayesClassifierTest.java        |   4 +-
 .../DocumentClassificationTestBase.java         |   4 +-
 .../KNearestNeighborDocumentClassifierTest.java |   4 +-
 .../SimpleNaiveBayesDocumentClassifierTest.java |   4 +-
 .../utils/ConfusionMatrixGeneratorTest.java     |   4 +-
 .../classification/utils/DataSplitterTest.java  |   4 +-
 .../autoprefix/AutoPrefixPostingsFormat.java    |   4 +-
 .../codecs/blockterms/BlockTermsReader.java     |   4 +-
 .../codecs/blockterms/BlockTermsWriter.java     |   4 +-
 .../blockterms/FixedGapTermsIndexReader.java    |   4 +-
 .../blockterms/FixedGapTermsIndexWriter.java    |   4 +-
 .../codecs/blockterms/TermsIndexReaderBase.java |   4 +-
 .../codecs/blockterms/TermsIndexWriterBase.java |   4 +-
 .../blockterms/VariableGapTermsIndexReader.java |   4 +-
 .../blockterms/VariableGapTermsIndexWriter.java |   4 +-
 .../BlockTreeOrdsPostingsFormat.java            |   4 +-
 .../codecs/blocktreeords/FSTOrdsOutputs.java    |   4 +-
 .../blocktreeords/OrdsBlockTreeTermsReader.java |   4 +-
 .../blocktreeords/OrdsBlockTreeTermsWriter.java |   4 +-
 .../codecs/blocktreeords/OrdsFieldReader.java   |   4 +-
 .../blocktreeords/OrdsIntersectTermsEnum.java   |   4 +-
 .../OrdsIntersectTermsEnumFrame.java            |   4 +-
 .../blocktreeords/OrdsSegmentTermsEnum.java     |   4 +-
 .../OrdsSegmentTermsEnumFrame.java              |   4 +-
 .../lucene/codecs/bloom/BloomFilterFactory.java |   2 +-
 .../bloom/BloomFilteringPostingsFormat.java     |   4 +-
 .../codecs/bloom/DefaultBloomFilterFactory.java |   2 +-
 .../apache/lucene/codecs/bloom/FuzzySet.java    |   4 +-
 .../lucene/codecs/bloom/HashFunction.java       |   2 +-
 .../apache/lucene/codecs/bloom/MurmurHash2.java |   2 +-
 .../codecs/memory/DirectDocValuesConsumer.java  |   4 +-
 .../codecs/memory/DirectDocValuesFormat.java    |   4 +-
 .../codecs/memory/DirectDocValuesProducer.java  |   4 +-
 .../codecs/memory/DirectPostingsFormat.java     |   4 +-
 .../codecs/memory/FSTOrdPostingsFormat.java     |   6 +-
 .../lucene/codecs/memory/FSTOrdTermsReader.java |   4 +-
 .../lucene/codecs/memory/FSTOrdTermsWriter.java |   4 +-
 .../lucene/codecs/memory/FSTPostingsFormat.java |   6 +-
 .../lucene/codecs/memory/FSTTermOutputs.java    |   4 +-
 .../lucene/codecs/memory/FSTTermsReader.java    |   4 +-
 .../lucene/codecs/memory/FSTTermsWriter.java    |   4 +-
 .../codecs/memory/MemoryDocValuesConsumer.java  |   4 +-
 .../codecs/memory/MemoryDocValuesFormat.java    |   4 +-
 .../codecs/memory/MemoryDocValuesProducer.java  |   4 +-
 .../codecs/memory/MemoryPostingsFormat.java     |   4 +-
 .../codecs/simpletext/SimpleTextBKDReader.java  |   4 +-
 .../codecs/simpletext/SimpleTextCodec.java      |   4 +-
 .../simpletext/SimpleTextCompoundFormat.java    |   4 +-
 .../simpletext/SimpleTextDocValuesFormat.java   |   4 +-
 .../simpletext/SimpleTextDocValuesReader.java   |   4 +-
 .../simpletext/SimpleTextDocValuesWriter.java   |   4 +-
 .../simpletext/SimpleTextFieldInfosFormat.java  |   4 +-
 .../simpletext/SimpleTextFieldsReader.java      |   4 +-
 .../simpletext/SimpleTextFieldsWriter.java      |   4 +-
 .../simpletext/SimpleTextLiveDocsFormat.java    |   4 +-
 .../simpletext/SimpleTextNormsFormat.java       |   4 +-
 .../simpletext/SimpleTextPointFormat.java       |   4 +-
 .../simpletext/SimpleTextPointReader.java       |  28 +-
 .../simpletext/SimpleTextPointWriter.java       |  47 +-
 .../simpletext/SimpleTextPostingsFormat.java    |   4 +-
 .../simpletext/SimpleTextSegmentInfoFormat.java |   4 +-
 .../SimpleTextStoredFieldsFormat.java           |   4 +-
 .../SimpleTextStoredFieldsReader.java           |   4 +-
 .../SimpleTextStoredFieldsWriter.java           |   4 +-
 .../simpletext/SimpleTextTermVectorsFormat.java |   4 +-
 .../simpletext/SimpleTextTermVectorsReader.java |   4 +-
 .../simpletext/SimpleTextTermVectorsWriter.java |   4 +-
 .../codecs/simpletext/SimpleTextUtil.java       |   4 +-
 .../TestAutoPrefixPostingsFormat.java           |   4 +-
 .../codecs/autoprefix/TestAutoPrefixTerms.java  |   4 +-
 .../blockterms/TestFixedGapPostingsFormat.java  |   4 +-
 ...TestVarGapDocFreqIntervalPostingsFormat.java |   4 +-
 .../TestVarGapFixedIntervalPostingsFormat.java  |   4 +-
 .../codecs/blocktreeords/TestOrdsBlockTree.java |   4 +-
 .../codecs/bloom/TestBloomPostingsFormat.java   |   4 +-
 .../memory/TestDirectDocValuesFormat.java       |   4 +-
 .../codecs/memory/TestDirectPostingsFormat.java |   4 +-
 .../codecs/memory/TestFSTOrdPostingsFormat.java |   4 +-
 .../codecs/memory/TestFSTPostingsFormat.java    |   4 +-
 .../memory/TestMemoryDocValuesFormat.java       |   4 +-
 .../codecs/memory/TestMemoryPostingsFormat.java |   4 +-
 .../TestSimpleTextCompoundFormat.java           |   4 +-
 .../TestSimpleTextDocValuesFormat.java          |   4 +-
 .../TestSimpleTextFieldInfoFormat.java          |   4 +-
 .../simpletext/TestSimpleTextNormsFormat.java   |   4 +-
 .../simpletext/TestSimpleTextPointFormat.java   |  33 +
 .../TestSimpleTextPostingsFormat.java           |   4 +-
 .../TestSimpleTextSegmentInfoFormat.java        |   4 +-
 .../TestSimpleTextStoredFieldsFormat.java       |   4 +-
 .../TestSimpleTextTermVectorsFormat.java        |   4 +-
 lucene/common-build.xml                         |  35 +-
 .../java/org/apache/lucene/LucenePackage.java   |   4 +-
 .../org/apache/lucene/analysis/Analyzer.java    |   4 +-
 .../apache/lucene/analysis/AnalyzerWrapper.java |   4 +-
 .../lucene/analysis/CachingTokenFilter.java     |   4 +-
 .../org/apache/lucene/analysis/CharFilter.java  |   4 +-
 .../analysis/DelegatingAnalyzerWrapper.java     |   8 +-
 .../analysis/LegacyNumericTokenStream.java      |   4 +-
 .../lucene/analysis/ReusableStringReader.java   |   8 +-
 .../java/org/apache/lucene/analysis/Token.java  |   4 +-
 .../org/apache/lucene/analysis/TokenFilter.java |   4 +-
 .../org/apache/lucene/analysis/TokenStream.java |   4 +-
 .../lucene/analysis/TokenStreamToAutomaton.java |   4 +-
 .../org/apache/lucene/analysis/Tokenizer.java   |   4 +-
 .../tokenattributes/BytesTermAttribute.java     |   4 +-
 .../tokenattributes/BytesTermAttributeImpl.java |   4 +-
 .../tokenattributes/CharTermAttribute.java      |   4 +-
 .../tokenattributes/CharTermAttributeImpl.java  |   4 +-
 .../tokenattributes/FlagsAttribute.java         |   4 +-
 .../tokenattributes/FlagsAttributeImpl.java     |   4 +-
 .../tokenattributes/KeywordAttribute.java       |   4 +-
 .../tokenattributes/KeywordAttributeImpl.java   |   4 +-
 .../tokenattributes/OffsetAttribute.java        |   4 +-
 .../tokenattributes/OffsetAttributeImpl.java    |   4 +-
 .../PackedTokenAttributeImpl.java               |   4 +-
 .../tokenattributes/PayloadAttribute.java       |   4 +-
 .../tokenattributes/PayloadAttributeImpl.java   |   4 +-
 .../PositionIncrementAttribute.java             |   4 +-
 .../PositionIncrementAttributeImpl.java         |   4 +-
 .../PositionLengthAttribute.java                |   4 +-
 .../PositionLengthAttributeImpl.java            |   4 +-
 .../TermToBytesRefAttribute.java                |   4 +-
 .../analysis/tokenattributes/TypeAttribute.java |   4 +-
 .../tokenattributes/TypeAttributeImpl.java      |   4 +-
 .../apache/lucene/codecs/BlockTermState.java    |   2 +-
 .../java/org/apache/lucene/codecs/Codec.java    |   4 +-
 .../org/apache/lucene/codecs/CodecUtil.java     |   6 +-
 .../apache/lucene/codecs/CompoundFormat.java    |   4 +-
 .../apache/lucene/codecs/DocValuesConsumer.java |   4 +-
 .../apache/lucene/codecs/DocValuesFormat.java   |   4 +-
 .../apache/lucene/codecs/DocValuesProducer.java |   4 +-
 .../apache/lucene/codecs/FieldInfosFormat.java  |   4 +-
 .../apache/lucene/codecs/FieldsConsumer.java    |   4 +-
 .../apache/lucene/codecs/FieldsProducer.java    |   4 +-
 .../org/apache/lucene/codecs/FilterCodec.java   |   4 +-
 .../apache/lucene/codecs/LiveDocsFormat.java    |   4 +-
 .../lucene/codecs/MultiLevelSkipListReader.java |   4 +-
 .../lucene/codecs/MultiLevelSkipListWriter.java |   4 +-
 .../org/apache/lucene/codecs/NormsConsumer.java |   4 +-
 .../org/apache/lucene/codecs/NormsFormat.java   |   4 +-
 .../org/apache/lucene/codecs/NormsProducer.java |  18 +-
 .../org/apache/lucene/codecs/PointFormat.java   |   4 +-
 .../org/apache/lucene/codecs/PointReader.java   |   4 +-
 .../org/apache/lucene/codecs/PointWriter.java   |   8 +-
 .../apache/lucene/codecs/PostingsFormat.java    |   4 +-
 .../lucene/codecs/PostingsReaderBase.java       |   4 +-
 .../lucene/codecs/PostingsWriterBase.java       |   4 +-
 .../lucene/codecs/PushPostingsWriterBase.java   |   4 +-
 .../apache/lucene/codecs/SegmentInfoFormat.java |   4 +-
 .../lucene/codecs/StoredFieldsFormat.java       |   4 +-
 .../lucene/codecs/StoredFieldsReader.java       |  24 +-
 .../lucene/codecs/StoredFieldsWriter.java       |  26 +-
 .../org/apache/lucene/codecs/TermStats.java     |   4 +-
 .../apache/lucene/codecs/TermVectorsFormat.java |   4 +-
 .../apache/lucene/codecs/TermVectorsReader.java |   4 +-
 .../apache/lucene/codecs/TermVectorsWriter.java |   4 +-
 .../codecs/blocktree/AutoPrefixTermsWriter.java |   4 +-
 .../codecs/blocktree/BitSetPostingsEnum.java    |   4 +-
 .../codecs/blocktree/BitSetTermsEnum.java       |   4 +-
 .../codecs/blocktree/BlockTreeTermsReader.java  |   4 +-
 .../codecs/blocktree/BlockTreeTermsWriter.java  |   4 +-
 .../lucene/codecs/blocktree/FieldReader.java    |   4 +-
 .../codecs/blocktree/IntersectTermsEnum.java    |   4 +-
 .../blocktree/IntersectTermsEnumFrame.java      |   4 +-
 .../codecs/blocktree/SegmentTermsEnum.java      |   4 +-
 .../codecs/blocktree/SegmentTermsEnumFrame.java |   4 +-
 .../apache/lucene/codecs/blocktree/Stats.java   |   4 +-
 .../CompressingStoredFieldsFormat.java          |   4 +-
 .../CompressingStoredFieldsIndexReader.java     |   4 +-
 .../CompressingStoredFieldsIndexWriter.java     |   4 +-
 .../CompressingStoredFieldsReader.java          |   4 +-
 .../CompressingStoredFieldsWriter.java          |   4 +-
 .../CompressingTermVectorsFormat.java           |   4 +-
 .../CompressingTermVectorsReader.java           |   4 +-
 .../CompressingTermVectorsWriter.java           |   4 +-
 .../codecs/compressing/CompressionMode.java     |   4 +-
 .../lucene/codecs/compressing/Compressor.java   |   4 +-
 .../lucene/codecs/compressing/Decompressor.java |   4 +-
 .../GrowableByteArrayDataOutput.java            |   4 +-
 .../apache/lucene/codecs/compressing/LZ4.java   |   4 +-
 .../codecs/compressing/MatchingReaders.java     |   4 +-
 .../apache/lucene/codecs/lucene50/ForUtil.java  |   2 +-
 .../codecs/lucene50/Lucene50CompoundFormat.java |   4 +-
 .../codecs/lucene50/Lucene50CompoundReader.java |  32 +-
 .../lucene50/Lucene50FieldInfosFormat.java      |   4 +-
 .../codecs/lucene50/Lucene50LiveDocsFormat.java |   4 +-
 .../codecs/lucene50/Lucene50PostingsFormat.java |   6 +-
 .../codecs/lucene50/Lucene50PostingsReader.java |   4 +-
 .../codecs/lucene50/Lucene50PostingsWriter.java |   4 +-
 .../lucene50/Lucene50SegmentInfoFormat.java     |   4 +-
 .../codecs/lucene50/Lucene50SkipReader.java     |   4 +-
 .../codecs/lucene50/Lucene50SkipWriter.java     |   4 +-
 .../lucene50/Lucene50StoredFieldsFormat.java    |   4 +-
 .../lucene50/Lucene50TermVectorsFormat.java     |   4 +-
 .../codecs/lucene53/Lucene53NormsConsumer.java  |   4 +-
 .../codecs/lucene53/Lucene53NormsFormat.java    |   4 +-
 .../codecs/lucene53/Lucene53NormsProducer.java  |   4 +-
 .../lucene54/Lucene54DocValuesConsumer.java     |   4 +-
 .../lucene54/Lucene54DocValuesFormat.java       |   4 +-
 .../lucene54/Lucene54DocValuesProducer.java     |   4 +-
 .../lucene/codecs/lucene60/Lucene60Codec.java   |   4 +-
 .../lucene60/Lucene60FieldInfosFormat.java      |   4 +-
 .../codecs/lucene60/Lucene60PointFormat.java    |   7 +-
 .../codecs/lucene60/Lucene60PointReader.java    |  68 +-
 .../codecs/lucene60/Lucene60PointWriter.java    |  91 +-
 .../perfield/PerFieldDocValuesFormat.java       |   4 +-
 .../codecs/perfield/PerFieldPostingsFormat.java |   4 +-
 .../lucene/document/BinaryDocValuesField.java   |   4 +-
 .../org/apache/lucene/document/BinaryPoint.java |   4 +-
 .../lucene/document/CompressionTools.java       |   4 +-
 .../org/apache/lucene/document/DateTools.java   |   4 +-
 .../org/apache/lucene/document/Document.java    |   4 +-
 .../document/DocumentStoredFieldVisitor.java    |   4 +-
 .../lucene/document/DoubleDocValuesField.java   |   4 +-
 .../org/apache/lucene/document/DoublePoint.java |  10 +-
 .../java/org/apache/lucene/document/Field.java  |   7 +-
 .../org/apache/lucene/document/FieldType.java   |   4 +-
 .../lucene/document/FloatDocValuesField.java    |   4 +-
 .../org/apache/lucene/document/FloatPoint.java  |  10 +-
 .../org/apache/lucene/document/IntPoint.java    |  10 +-
 .../lucene/document/LegacyDoubleField.java      |   4 +-
 .../lucene/document/LegacyFloatField.java       |   4 +-
 .../apache/lucene/document/LegacyIntField.java  |   4 +-
 .../apache/lucene/document/LegacyLongField.java |   4 +-
 .../org/apache/lucene/document/LongPoint.java   |  10 +-
 .../lucene/document/NumericDocValuesField.java  |   4 +-
 .../lucene/document/SortedDocValuesField.java   |   4 +-
 .../document/SortedNumericDocValuesField.java   |   4 +-
 .../document/SortedSetDocValuesField.java       |   4 +-
 .../org/apache/lucene/document/StoredField.java |   4 +-
 .../org/apache/lucene/document/StringField.java |   4 +-
 .../org/apache/lucene/document/TextField.java   |   4 +-
 .../apache/lucene/index/AbortingException.java  |   4 +-
 .../apache/lucene/index/AutomatonTermsEnum.java |   4 +-
 .../lucene/index/BaseCompositeReader.java       |   4 +-
 .../apache/lucene/index/BinaryDocValues.java    |   4 +-
 .../index/BinaryDocValuesFieldUpdates.java      |   4 +-
 .../lucene/index/BinaryDocValuesWriter.java     |   4 +-
 .../java/org/apache/lucene/index/BitsSlice.java |   8 +-
 .../apache/lucene/index/BufferedUpdates.java    |   4 +-
 .../lucene/index/BufferedUpdatesStream.java     |   4 +-
 .../apache/lucene/index/ByteSliceReader.java    |   4 +-
 .../apache/lucene/index/ByteSliceWriter.java    |  10 +-
 .../org/apache/lucene/index/CheckIndex.java     |   4 +-
 .../apache/lucene/index/CoalescedUpdates.java   |   4 +-
 .../org/apache/lucene/index/CodecReader.java    |   4 +-
 .../apache/lucene/index/CompositeReader.java    |   4 +-
 .../lucene/index/CompositeReaderContext.java    |   4 +-
 .../lucene/index/ConcurrentMergeScheduler.java  |   4 +-
 .../lucene/index/CorruptIndexException.java     |   4 +-
 .../lucene/index/DefaultIndexingChain.java      |  33 +-
 .../apache/lucene/index/DirectoryReader.java    |   4 +-
 .../org/apache/lucene/index/DocConsumer.java    |   4 +-
 .../java/org/apache/lucene/index/DocValues.java |   4 +-
 .../lucene/index/DocValuesFieldUpdates.java     |   4 +-
 .../org/apache/lucene/index/DocValuesType.java  |   4 +-
 .../apache/lucene/index/DocValuesUpdate.java    |   4 +-
 .../apache/lucene/index/DocValuesWriter.java    |   4 +-
 .../apache/lucene/index/DocumentsWriter.java    |   4 +-
 .../index/DocumentsWriterDeleteQueue.java       |  27 +-
 .../index/DocumentsWriterFlushControl.java      |  10 +-
 .../lucene/index/DocumentsWriterFlushQueue.java |  28 +-
 .../lucene/index/DocumentsWriterPerThread.java  |  10 +-
 .../index/DocumentsWriterPerThreadPool.java     |   8 +-
 .../index/DocumentsWriterStallControl.java      |  10 +-
 .../lucene/index/ExitableDirectoryReader.java   |   4 +-
 .../java/org/apache/lucene/index/FieldInfo.java |   4 +-
 .../org/apache/lucene/index/FieldInfos.java     |   4 +-
 .../apache/lucene/index/FieldTermIterator.java  |   4 +-
 .../java/org/apache/lucene/index/Fields.java    |   4 +-
 .../apache/lucene/index/FilterCodecReader.java  |   4 +-
 .../lucene/index/FilterDirectoryReader.java     |   4 +-
 .../apache/lucene/index/FilterLeafReader.java   |   4 +-
 .../apache/lucene/index/FilteredTermsEnum.java  |   4 +-
 .../lucene/index/FlushByRamOrCountsPolicy.java  |  10 +-
 .../org/apache/lucene/index/FlushPolicy.java    |  10 +-
 .../org/apache/lucene/index/FreqProxFields.java |   4 +-
 .../lucene/index/FreqProxTermsWriter.java       |   4 +-
 .../index/FreqProxTermsWriterPerField.java      |   4 +-
 .../lucene/index/FrozenBufferedUpdates.java     |   4 +-
 .../org/apache/lucene/index/IndexCommit.java    |   5 +-
 .../lucene/index/IndexDeletionPolicy.java       |   4 +-
 .../apache/lucene/index/IndexFileDeleter.java   | 141 +--
 .../org/apache/lucene/index/IndexFileNames.java |   4 +-
 .../index/IndexFormatTooNewException.java       |   1 -
 .../index/IndexFormatTooOldException.java       |   1 -
 .../lucene/index/IndexNotFoundException.java    |   4 +-
 .../org/apache/lucene/index/IndexOptions.java   |   4 +-
 .../org/apache/lucene/index/IndexReader.java    |   8 +-
 .../apache/lucene/index/IndexReaderContext.java |   4 +-
 .../org/apache/lucene/index/IndexUpgrader.java  |   4 +-
 .../org/apache/lucene/index/IndexWriter.java    |  18 +-
 .../apache/lucene/index/IndexWriterConfig.java  |   4 +-
 .../org/apache/lucene/index/IndexableField.java |  12 +-
 .../apache/lucene/index/IndexableFieldType.java |   4 +-
 .../index/KeepOnlyLastCommitDeletionPolicy.java |   4 +-
 .../org/apache/lucene/index/LeafReader.java     |   4 +-
 .../apache/lucene/index/LeafReaderContext.java  |   4 +-
 .../lucene/index/LiveIndexWriterConfig.java     |   4 +-
 .../lucene/index/LogByteSizeMergePolicy.java    |   4 +-
 .../apache/lucene/index/LogDocMergePolicy.java  |   8 +-
 .../org/apache/lucene/index/LogMergePolicy.java |   4 +-
 .../apache/lucene/index/MappedMultiFields.java  |   4 +-
 .../lucene/index/MappingMultiPostingsEnum.java  |   4 +-
 .../org/apache/lucene/index/MergePolicy.java    |  10 +-
 .../apache/lucene/index/MergePolicyWrapper.java |  89 ++
 .../apache/lucene/index/MergeRateLimiter.java   |   4 +-
 .../org/apache/lucene/index/MergeScheduler.java |   4 +-
 .../org/apache/lucene/index/MergeState.java     |   4 +-
 .../org/apache/lucene/index/MergeTrigger.java   |   4 +-
 .../index/MergedPrefixCodedTermsIterator.java   |   4 +-
 .../java/org/apache/lucene/index/MultiBits.java |   8 +-
 .../org/apache/lucene/index/MultiDocValues.java |   4 +-
 .../org/apache/lucene/index/MultiFields.java    |   4 +-
 .../apache/lucene/index/MultiPointValues.java   |   8 +-
 .../apache/lucene/index/MultiPostingsEnum.java  |   4 +-
 .../org/apache/lucene/index/MultiReader.java    |   4 +-
 .../org/apache/lucene/index/MultiTerms.java     |   4 +-
 .../org/apache/lucene/index/MultiTermsEnum.java |   4 +-
 .../apache/lucene/index/NoDeletionPolicy.java   |   4 +-
 .../org/apache/lucene/index/NoMergePolicy.java  |  22 +-
 .../apache/lucene/index/NoMergeScheduler.java   |   4 +-
 .../apache/lucene/index/NormValuesWriter.java   |   4 +-
 .../apache/lucene/index/NumericDocValues.java   |   4 +-
 .../index/NumericDocValuesFieldUpdates.java     |  18 +-
 .../lucene/index/NumericDocValuesWriter.java    |   4 +-
 .../org/apache/lucene/index/OrdTermState.java   |   4 +-
 .../lucene/index/ParallelCompositeReader.java   |   4 +-
 .../apache/lucene/index/ParallelLeafReader.java |   4 +-
 .../lucene/index/ParallelPostingsArray.java     |   4 +-
 .../index/PersistentSnapshotDeletionPolicy.java |  29 +-
 .../org/apache/lucene/index/PointValues.java    |  12 +-
 .../apache/lucene/index/PointValuesWriter.java  |   4 +-
 .../org/apache/lucene/index/PostingsEnum.java   |   4 +-
 .../apache/lucene/index/PrefixCodedTerms.java   |  25 +-
 .../org/apache/lucene/index/QueryTimeout.java   |   4 +-
 .../apache/lucene/index/QueryTimeoutImpl.java   |   4 +-
 .../apache/lucene/index/RandomAccessOrds.java   |   4 +-
 .../org/apache/lucene/index/ReaderManager.java  |   4 +-
 .../org/apache/lucene/index/ReaderSlice.java    |   4 +-
 .../org/apache/lucene/index/ReaderUtil.java     |   4 +-
 .../apache/lucene/index/ReadersAndUpdates.java  |   4 +-
 .../apache/lucene/index/SegmentCommitInfo.java  |   4 +-
 .../apache/lucene/index/SegmentCoreReaders.java |   8 +-
 .../apache/lucene/index/SegmentDocValues.java   |   4 +-
 .../lucene/index/SegmentDocValuesProducer.java  |   4 +-
 .../org/apache/lucene/index/SegmentInfo.java    |   4 +-
 .../org/apache/lucene/index/SegmentInfos.java   |   4 +-
 .../org/apache/lucene/index/SegmentMerger.java  |   4 +-
 .../apache/lucene/index/SegmentReadState.java   |   4 +-
 .../org/apache/lucene/index/SegmentReader.java  |   4 +-
 .../apache/lucene/index/SegmentWriteState.java  |   4 +-
 .../lucene/index/SerialMergeScheduler.java      |   4 +-
 .../lucene/index/SimpleMergedSegmentWarmer.java |   4 +-
 .../apache/lucene/index/SingleTermsEnum.java    |   4 +-
 .../index/SingletonSortedNumericDocValues.java  |   4 +-
 .../index/SingletonSortedSetDocValues.java      |   4 +-
 .../lucene/index/SlowCodecReaderWrapper.java    |   4 +-
 .../index/SlowCompositeReaderWrapper.java       |   4 +-
 .../lucene/index/SnapshotDeletionPolicy.java    |   4 +-
 .../apache/lucene/index/SortedDocValues.java    |   4 +-
 .../lucene/index/SortedDocValuesTermsEnum.java  |   4 +-
 .../lucene/index/SortedDocValuesWriter.java     |   4 +-
 .../lucene/index/SortedNumericDocValues.java    |   4 +-
 .../index/SortedNumericDocValuesWriter.java     |   4 +-
 .../apache/lucene/index/SortedSetDocValues.java |   4 +-
 .../index/SortedSetDocValuesTermsEnum.java      |   4 +-
 .../lucene/index/SortedSetDocValuesWriter.java  |   4 +-
 .../lucene/index/StandardDirectoryReader.java   |   4 +-
 .../apache/lucene/index/StoredFieldVisitor.java |   4 +-
 .../src/java/org/apache/lucene/index/Term.java  |   4 +-
 .../org/apache/lucene/index/TermContext.java    |   4 +-
 .../java/org/apache/lucene/index/TermState.java |   4 +-
 .../lucene/index/TermVectorsConsumer.java       |   4 +-
 .../index/TermVectorsConsumerPerField.java      |   4 +-
 .../src/java/org/apache/lucene/index/Terms.java |   4 +-
 .../java/org/apache/lucene/index/TermsEnum.java |   4 +-
 .../java/org/apache/lucene/index/TermsHash.java |   4 +-
 .../apache/lucene/index/TermsHashPerField.java  |   4 +-
 .../apache/lucene/index/TieredMergePolicy.java  |   8 +-
 .../lucene/index/TrackingIndexWriter.java       |   4 +-
 .../org/apache/lucene/index/TwoPhaseCommit.java |   8 +-
 .../apache/lucene/index/TwoPhaseCommitTool.java |   8 +-
 .../lucene/index/UpgradeIndexMergePolicy.java   |  34 +-
 .../apache/lucene/search/AutomatonQuery.java    |   4 +-
 .../apache/lucene/search/BlendedTermQuery.java  |   4 +-
 .../org/apache/lucene/search/BooleanClause.java |   4 +-
 .../org/apache/lucene/search/BooleanQuery.java  |   4 +-
 .../org/apache/lucene/search/BooleanScorer.java |   4 +-
 .../lucene/search/BooleanTopLevelScorers.java   |   4 +-
 .../org/apache/lucene/search/BooleanWeight.java |   4 +-
 .../apache/lucene/search/BoostAttribute.java    |   4 +-
 .../lucene/search/BoostAttributeImpl.java       |   4 +-
 .../org/apache/lucene/search/BoostQuery.java    |   4 +-
 .../org/apache/lucene/search/BulkScorer.java    |   4 +-
 .../apache/lucene/search/CachingCollector.java  |   4 +-
 .../lucene/search/CollectionStatistics.java     |  10 +-
 .../search/CollectionTerminatedException.java   |   4 +-
 .../org/apache/lucene/search/Collector.java     |   4 +-
 .../apache/lucene/search/CollectorManager.java  |   4 +-
 .../apache/lucene/search/ConjunctionDISI.java   |   4 +-
 .../apache/lucene/search/ConjunctionScorer.java |   4 +-
 .../lucene/search/ConstantScoreQuery.java       |   4 +-
 .../lucene/search/ConstantScoreScorer.java      |   4 +-
 .../lucene/search/ConstantScoreWeight.java      |   4 +-
 .../search/ControlledRealTimeReopenThread.java  |   4 +-
 .../apache/lucene/search/DisiPriorityQueue.java |   4 +-
 .../org/apache/lucene/search/DisiWrapper.java   |   4 +-
 .../search/DisjunctionDISIApproximation.java    |   2 +-
 .../lucene/search/DisjunctionMaxQuery.java      |  14 +-
 .../lucene/search/DisjunctionMaxScorer.java     |  14 +-
 .../apache/lucene/search/DisjunctionScorer.java |   4 +-
 .../lucene/search/DisjunctionSumScorer.java     |   4 +-
 .../java/org/apache/lucene/search/DocIdSet.java |   4 +-
 .../apache/lucene/search/DocIdSetIterator.java  |   4 +-
 .../apache/lucene/search/DocValuesDocIdSet.java |   2 +-
 .../lucene/search/DocValuesRewriteMethod.java   |   4 +-
 .../apache/lucene/search/ExactPhraseScorer.java |   4 +-
 .../apache/lucene/search/ExactPointQuery.java   |   4 +-
 .../org/apache/lucene/search/Explanation.java   |   4 +-
 .../org/apache/lucene/search/FakeScorer.java    |   4 +-
 .../apache/lucene/search/FieldComparator.java   |   4 +-
 .../lucene/search/FieldComparatorSource.java    |   4 +-
 .../java/org/apache/lucene/search/FieldDoc.java |   4 +-
 .../lucene/search/FieldValueHitQueue.java       |   4 +-
 .../apache/lucene/search/FieldValueQuery.java   |   4 +-
 .../apache/lucene/search/FilterCollector.java   |  12 +-
 .../lucene/search/FilterLeafCollector.java      |   4 +-
 .../org/apache/lucene/search/FilterScorer.java  |   4 +-
 .../lucene/search/FilteredDocIdSetIterator.java |   4 +-
 .../org/apache/lucene/search/FuzzyQuery.java    |   4 +-
 .../apache/lucene/search/FuzzyTermsEnum.java    |   4 +-
 .../java/org/apache/lucene/search/HitQueue.java |   4 +-
 .../org/apache/lucene/search/IndexSearcher.java |   4 +-
 .../org/apache/lucene/search/LRUQueryCache.java |   4 +-
 .../org/apache/lucene/search/LeafCollector.java |   4 +-
 .../lucene/search/LeafFieldComparator.java      |   4 +-
 .../lucene/search/LegacyNumericRangeQuery.java  |   4 +-
 .../apache/lucene/search/LiveFieldValues.java   |   4 +-
 .../apache/lucene/search/MatchAllDocsQuery.java |   4 +-
 .../apache/lucene/search/MatchNoDocsQuery.java  |   4 +-
 .../search/MaxNonCompetitiveBoostAttribute.java |   4 +-
 .../MaxNonCompetitiveBoostAttributeImpl.java    |   4 +-
 .../lucene/search/MinShouldMatchSumScorer.java  |   4 +-
 .../apache/lucene/search/MultiCollector.java    |   4 +-
 .../apache/lucene/search/MultiPhraseQuery.java  |   4 +-
 .../apache/lucene/search/MultiTermQuery.java    |   4 +-
 .../MultiTermQueryConstantScoreWrapper.java     |   4 +-
 .../java/org/apache/lucene/search/Multiset.java |   4 +-
 .../apache/lucene/search/NGramPhraseQuery.java  |   4 +-
 .../apache/lucene/search/PhrasePositions.java   |   4 +-
 .../org/apache/lucene/search/PhraseQuery.java   |   4 +-
 .../org/apache/lucene/search/PhraseQueue.java   |   4 +-
 .../apache/lucene/search/PointRangeQuery.java   |   4 +-
 .../search/PositiveScoresOnlyCollector.java     |   4 +-
 .../org/apache/lucene/search/PrefixQuery.java   |   4 +-
 .../java/org/apache/lucene/search/Query.java    |   4 +-
 .../org/apache/lucene/search/QueryCache.java    |   4 +-
 .../lucene/search/QueryCachingPolicy.java       |   4 +-
 .../org/apache/lucene/search/QueryRescorer.java |   4 +-
 .../lucene/search/RandomAccessWeight.java       |   4 +-
 .../apache/lucene/search/ReferenceManager.java  |   4 +-
 .../org/apache/lucene/search/RegexpQuery.java   |   4 +-
 .../apache/lucene/search/ReqExclBulkScorer.java |   4 +-
 .../org/apache/lucene/search/ReqExclScorer.java |   4 +-
 .../apache/lucene/search/ReqOptSumScorer.java   |   2 +-
 .../java/org/apache/lucene/search/Rescorer.java |   4 +-
 .../search/ScoreCachingWrappingScorer.java      |   4 +-
 .../java/org/apache/lucene/search/ScoreDoc.java |   4 +-
 .../java/org/apache/lucene/search/Scorer.java   |   4 +-
 .../apache/lucene/search/ScoringRewrite.java    |   4 +-
 .../apache/lucene/search/SearcherFactory.java   |   4 +-
 .../lucene/search/SearcherLifetimeManager.java  |   4 +-
 .../apache/lucene/search/SearcherManager.java   |   4 +-
 .../apache/lucene/search/SimpleCollector.java   |   4 +-
 .../lucene/search/SimpleFieldComparator.java    |   4 +-
 .../lucene/search/SloppyPhraseScorer.java       |   4 +-
 .../src/java/org/apache/lucene/search/Sort.java |   4 +-
 .../org/apache/lucene/search/SortField.java     |   4 +-
 .../org/apache/lucene/search/SortRescorer.java  |   4 +-
 .../lucene/search/SortedNumericSelector.java    |   4 +-
 .../lucene/search/SortedNumericSortField.java   |   4 +-
 .../apache/lucene/search/SortedSetSelector.java |   4 +-
 .../lucene/search/SortedSetSortField.java       |   4 +-
 .../org/apache/lucene/search/SynonymQuery.java  |   4 +-
 .../lucene/search/TermCollectingRewrite.java    |   4 +-
 .../org/apache/lucene/search/TermQuery.java     |   4 +-
 .../apache/lucene/search/TermRangeQuery.java    |   4 +-
 .../org/apache/lucene/search/TermScorer.java    |   4 +-
 .../apache/lucene/search/TermStatistics.java    |   4 +-
 .../lucene/search/TimeLimitingCollector.java    |   4 +-
 .../java/org/apache/lucene/search/TopDocs.java  |   4 +-
 .../apache/lucene/search/TopDocsCollector.java  |   4 +-
 .../apache/lucene/search/TopFieldCollector.java |   4 +-
 .../org/apache/lucene/search/TopFieldDocs.java  |   4 +-
 .../lucene/search/TopScoreDocCollector.java     |   4 +-
 .../apache/lucene/search/TopTermsRewrite.java   |   4 +-
 .../lucene/search/TotalHitCountCollector.java   |   4 +-
 .../apache/lucene/search/TwoPhaseIterator.java  |   4 +-
 .../search/UsageTrackingQueryCachingPolicy.java |   4 +-
 .../java/org/apache/lucene/search/Weight.java   |   4 +-
 .../org/apache/lucene/search/WildcardQuery.java |   4 +-
 .../lucene/search/similarities/AfterEffect.java |   4 +-
 .../search/similarities/AfterEffectB.java       |   4 +-
 .../search/similarities/AfterEffectL.java       |   4 +-
 .../search/similarities/BM25Similarity.java     |   4 +-
 .../lucene/search/similarities/BasicModel.java  |   4 +-
 .../search/similarities/BasicModelBE.java       |   4 +-
 .../lucene/search/similarities/BasicModelD.java |   4 +-
 .../lucene/search/similarities/BasicModelG.java |   4 +-
 .../search/similarities/BasicModelIF.java       |   4 +-
 .../search/similarities/BasicModelIn.java       |   4 +-
 .../search/similarities/BasicModelIne.java      |   4 +-
 .../lucene/search/similarities/BasicModelP.java |   4 +-
 .../lucene/search/similarities/BasicStats.java  |   4 +-
 .../search/similarities/ClassicSimilarity.java  |   4 +-
 .../search/similarities/DFISimilarity.java      |   4 +-
 .../search/similarities/DFRSimilarity.java      |   4 +-
 .../search/similarities/Distribution.java       |   4 +-
 .../search/similarities/DistributionLL.java     |   4 +-
 .../search/similarities/DistributionSPL.java    |   4 +-
 .../search/similarities/IBSimilarity.java       |   4 +-
 .../search/similarities/Independence.java       |   4 +-
 .../similarities/IndependenceChiSquared.java    |   4 +-
 .../similarities/IndependenceSaturated.java     |   4 +-
 .../similarities/IndependenceStandardized.java  |   4 +-
 .../similarities/LMDirichletSimilarity.java     |   4 +-
 .../similarities/LMJelinekMercerSimilarity.java |   4 +-
 .../search/similarities/LMSimilarity.java       |   4 +-
 .../lucene/search/similarities/Lambda.java      |   4 +-
 .../lucene/search/similarities/LambdaDF.java    |   4 +-
 .../lucene/search/similarities/LambdaTTF.java   |   4 +-
 .../search/similarities/MultiSimilarity.java    |   4 +-
 .../search/similarities/Normalization.java      |   4 +-
 .../search/similarities/NormalizationH1.java    |   4 +-
 .../search/similarities/NormalizationH2.java    |   4 +-
 .../search/similarities/NormalizationH3.java    |   4 +-
 .../search/similarities/NormalizationZ.java     |   4 +-
 .../similarities/PerFieldSimilarityWrapper.java |   4 +-
 .../lucene/search/similarities/Similarity.java  |   4 +-
 .../search/similarities/SimilarityBase.java     |   4 +-
 .../search/similarities/TFIDFSimilarity.java    |   4 +-
 .../lucene/search/spans/ConjunctionSpans.java   |   4 +-
 .../lucene/search/spans/ContainSpans.java       |   4 +-
 .../search/spans/FieldMaskingSpanQuery.java     |   4 +-
 .../apache/lucene/search/spans/FilterSpans.java |   4 +-
 .../lucene/search/spans/NearSpansOrdered.java   |   4 +-
 .../lucene/search/spans/NearSpansUnordered.java |   4 +-
 .../search/spans/ScoringWrapperSpans.java       |   4 +-
 .../lucene/search/spans/SpanBoostQuery.java     |   4 +-
 .../lucene/search/spans/SpanCollector.java      |   4 +-
 .../lucene/search/spans/SpanContainQuery.java   |   4 +-
 .../search/spans/SpanContainingQuery.java       |   4 +-
 .../lucene/search/spans/SpanFirstQuery.java     |   4 +-
 .../search/spans/SpanMultiTermQueryWrapper.java |   4 +-
 .../lucene/search/spans/SpanNearQuery.java      |   4 +-
 .../lucene/search/spans/SpanNotQuery.java       |   4 +-
 .../apache/lucene/search/spans/SpanOrQuery.java |   4 +-
 .../search/spans/SpanPositionCheckQuery.java    |   2 +-
 .../lucene/search/spans/SpanPositionQueue.java  |   4 +-
 .../search/spans/SpanPositionRangeQuery.java    |   2 +-
 .../apache/lucene/search/spans/SpanQuery.java   |   4 +-
 .../apache/lucene/search/spans/SpanScorer.java  |   4 +-
 .../lucene/search/spans/SpanTermQuery.java      |   4 +-
 .../apache/lucene/search/spans/SpanWeight.java  |   4 +-
 .../lucene/search/spans/SpanWithinQuery.java    |   4 +-
 .../org/apache/lucene/search/spans/Spans.java   |   4 +-
 .../apache/lucene/search/spans/TermSpans.java   |  14 +-
 .../lucene/store/AlreadyClosedException.java    |   4 +-
 .../org/apache/lucene/store/BaseDirectory.java  |   4 +-
 .../apache/lucene/store/BufferedChecksum.java   |   4 +-
 .../store/BufferedChecksumIndexInput.java       |   4 +-
 .../apache/lucene/store/BufferedIndexInput.java |   4 +-
 .../apache/lucene/store/ByteArrayDataInput.java |   4 +-
 .../lucene/store/ByteArrayDataOutput.java       |   4 +-
 .../lucene/store/ByteBufferIndexInput.java      |   4 +-
 .../apache/lucene/store/ChecksumIndexInput.java |   8 +-
 .../java/org/apache/lucene/store/DataInput.java |   4 +-
 .../org/apache/lucene/store/DataOutput.java     |   4 +-
 .../java/org/apache/lucene/store/Directory.java |   7 +-
 .../org/apache/lucene/store/FSDirectory.java    | 152 ++-
 .../org/apache/lucene/store/FSLockFactory.java  |   4 +-
 .../lucene/store/FileSwitchDirectory.java       |  10 +-
 .../apache/lucene/store/FilterDirectory.java    |   4 +-
 .../java/org/apache/lucene/store/FlushInfo.java |   4 +-
 .../java/org/apache/lucene/store/IOContext.java |   4 +-
 .../org/apache/lucene/store/IndexInput.java     |   4 +-
 .../org/apache/lucene/store/IndexOutput.java    |   4 +-
 .../lucene/store/InputStreamDataInput.java      |   4 +-
 .../src/java/org/apache/lucene/store/Lock.java  |   4 +-
 .../org/apache/lucene/store/LockFactory.java    |   4 +-
 .../lucene/store/LockObtainFailedException.java |   1 -
 .../store/LockReleaseFailedException.java       |   1 -
 .../org/apache/lucene/store/LockStressTest.java |   4 +-
 .../store/LockValidatingDirectoryWrapper.java   |   4 +-
 .../apache/lucene/store/LockVerifyServer.java   |   4 +-
 .../org/apache/lucene/store/MMapDirectory.java  |   5 +-
 .../java/org/apache/lucene/store/MergeInfo.java |   2 +-
 .../org/apache/lucene/store/NIOFSDirectory.java |  29 +-
 .../lucene/store/NRTCachingDirectory.java       |   6 +-
 .../lucene/store/NativeFSLockFactory.java       |   4 +-
 .../org/apache/lucene/store/NoLockFactory.java  |   4 +-
 .../lucene/store/OutputStreamDataOutput.java    |   4 +-
 .../lucene/store/OutputStreamIndexOutput.java   |   4 +-
 .../org/apache/lucene/store/RAMDirectory.java   |  13 +-
 .../java/org/apache/lucene/store/RAMFile.java   |   4 +-
 .../org/apache/lucene/store/RAMInputStream.java |   4 +-
 .../apache/lucene/store/RAMOutputStream.java    |   6 +-
 .../apache/lucene/store/RandomAccessInput.java  |   4 +-
 .../lucene/store/RateLimitedIndexOutput.java    |   4 +-
 .../org/apache/lucene/store/RateLimiter.java    |   4 +-
 .../apache/lucene/store/SimpleFSDirectory.java  |   5 +-
 .../lucene/store/SimpleFSLockFactory.java       |   4 +-
 .../lucene/store/SingleInstanceLockFactory.java |   4 +-
 .../lucene/store/SleepingLockWrapper.java       |   4 +-
 .../lucene/store/TrackingDirectoryWrapper.java  |   4 +-
 .../lucene/store/VerifyingLockFactory.java      |   4 +-
 .../org/apache/lucene/util/Accountable.java     |   4 +-
 .../org/apache/lucene/util/Accountables.java    |   4 +-
 .../lucene/util/ArrayInPlaceMergeSorter.java    |   4 +-
 .../apache/lucene/util/ArrayIntroSorter.java    |   4 +-
 .../org/apache/lucene/util/ArrayTimSorter.java  |   4 +-
 .../java/org/apache/lucene/util/ArrayUtil.java  |   4 +-
 .../java/org/apache/lucene/util/Attribute.java  |   4 +-
 .../apache/lucene/util/AttributeFactory.java    |   4 +-
 .../org/apache/lucene/util/AttributeImpl.java   |   4 +-
 .../apache/lucene/util/AttributeReflector.java  |   4 +-
 .../org/apache/lucene/util/AttributeSource.java |   4 +-
 .../org/apache/lucene/util/BitDocIdSet.java     |   4 +-
 .../src/java/org/apache/lucene/util/BitSet.java |   4 +-
 .../org/apache/lucene/util/BitSetIterator.java  |   4 +-
 .../java/org/apache/lucene/util/BitUtil.java    |   1 -
 .../src/java/org/apache/lucene/util/Bits.java   |   4 +-
 .../org/apache/lucene/util/ByteBlockPool.java   |   4 +-
 .../java/org/apache/lucene/util/BytesRef.java   |   4 +-
 .../org/apache/lucene/util/BytesRefArray.java   |  27 +-
 .../org/apache/lucene/util/BytesRefBuilder.java |   4 +-
 .../org/apache/lucene/util/BytesRefHash.java    |   4 +-
 .../apache/lucene/util/BytesRefIterator.java    |   4 +-
 .../java/org/apache/lucene/util/CharsRef.java   |   4 +-
 .../org/apache/lucene/util/CharsRefBuilder.java |   4 +-
 .../lucene/util/CloseableThreadLocal.java       |   4 +-
 .../org/apache/lucene/util/CollectionUtil.java  |   4 +-
 .../org/apache/lucene/util/CommandLineUtil.java |   4 +-
 .../java/org/apache/lucene/util/Constants.java  |   4 +-
 .../java/org/apache/lucene/util/Counter.java    |   8 +-
 .../org/apache/lucene/util/DocIdSetBuilder.java |   4 +-
 .../org/apache/lucene/util/FilterIterator.java  |  27 +-
 .../org/apache/lucene/util/FixedBitSet.java     |   4 +-
 .../util/FrequencyTrackingRingBuffer.java       |   4 +-
 .../java/org/apache/lucene/util/IOUtils.java    |  12 +-
 .../apache/lucene/util/InPlaceMergeSorter.java  |   4 +-
 .../java/org/apache/lucene/util/InfoStream.java |   4 +-
 .../apache/lucene/util/IntArrayDocIdSet.java    |   4 +-
 .../org/apache/lucene/util/IntBlockPool.java    |   4 +-
 .../org/apache/lucene/util/IntroSorter.java     |   4 +-
 .../java/org/apache/lucene/util/IntsRef.java    |   8 +-
 .../org/apache/lucene/util/IntsRefBuilder.java  |   4 +-
 .../org/apache/lucene/util/LSBRadixSorter.java  |   4 +-
 .../apache/lucene/util/LegacyNumericUtils.java  |   4 +-
 .../java/org/apache/lucene/util/LongBitSet.java |   4 +-
 .../java/org/apache/lucene/util/LongValues.java |   4 +-
 .../java/org/apache/lucene/util/LongsRef.java   |   8 +-
 .../java/org/apache/lucene/util/MapOfSets.java  |   4 +-
 .../java/org/apache/lucene/util/MathUtil.java   |   4 +-
 .../org/apache/lucene/util/MergedIterator.java  |   4 +-
 .../org/apache/lucene/util/MutableBits.java     |   4 +-
 .../org/apache/lucene/util/NamedSPILoader.java  |   4 +-
 .../apache/lucene/util/NamedThreadFactory.java  |   4 +-
 .../org/apache/lucene/util/NotDocIdSet.java     |   4 +-
 .../org/apache/lucene/util/NumericUtils.java    |   4 +-
 .../org/apache/lucene/util/OfflineSorter.java   |   4 +-
 .../java/org/apache/lucene/util/PagedBytes.java |   4 +-
 .../lucene/util/PrintStreamInfoStream.java      |   4 +-
 .../org/apache/lucene/util/PriorityQueue.java   |  10 +-
 .../org/apache/lucene/util/QueryBuilder.java    |   4 +-
 .../apache/lucene/util/RamUsageEstimator.java   |   4 +-
 .../util/RecyclingByteBlockAllocator.java       |   8 +-
 .../lucene/util/RecyclingIntBlockAllocator.java |   8 +-
 .../java/org/apache/lucene/util/RefCount.java   |  10 +-
 .../org/apache/lucene/util/RoaringDocIdSet.java |   4 +-
 .../org/apache/lucene/util/RollingBuffer.java   |   4 +-
 .../apache/lucene/util/SPIClassIterator.java    |   4 +-
 .../org/apache/lucene/util/SentinelIntSet.java  |   4 +-
 .../java/org/apache/lucene/util/SetOnce.java    |   8 +-
 .../java/org/apache/lucene/util/SloppyMath.java |  10 +-
 .../java/org/apache/lucene/util/SmallFloat.java |  14 +-
 .../src/java/org/apache/lucene/util/Sorter.java |   4 +-
 .../apache/lucene/util/SparseFixedBitSet.java   |   4 +-
 .../lucene/util/StrictStringTokenizer.java      |   4 +-
 .../org/apache/lucene/util/StringHelper.java    |   4 +-
 .../apache/lucene/util/SuppressForbidden.java   |   4 +-
 .../lucene/util/ThreadInterruptedException.java |   9 +-
 .../java/org/apache/lucene/util/TimSorter.java  |   4 +-
 .../org/apache/lucene/util/ToStringUtils.java   |   4 +-
 .../org/apache/lucene/util/UnicodeUtil.java     |   6 +-
 .../java/org/apache/lucene/util/Version.java    |   4 +-
 .../org/apache/lucene/util/VirtualMethod.java   |   4 +-
 .../org/apache/lucene/util/WeakIdentityMap.java |   4 +-
 .../apache/lucene/util/automaton/Automaton.java |   4 +-
 .../lucene/util/automaton/ByteRunAutomaton.java |   4 +-
 .../util/automaton/CharacterRunAutomaton.java   |   4 +-
 .../util/automaton/CompiledAutomaton.java       |   4 +-
 .../automaton/DaciukMihovAutomatonBuilder.java  |   4 +-
 .../util/automaton/FiniteStringsIterator.java   |   4 +-
 .../automaton/Lev1ParametricDescription.java    |   3 +-
 .../automaton/Lev1TParametricDescription.java   |   3 +-
 .../automaton/Lev2ParametricDescription.java    |   3 +-
 .../automaton/Lev2TParametricDescription.java   |   3 +-
 .../util/automaton/LevenshteinAutomata.java     |   4 +-
 .../automaton/LimitedFiniteStringsIterator.java |   4 +-
 .../lucene/util/automaton/SortedIntSet.java     |   4 +-
 .../TooComplexToDeterminizeException.java       |   4 +-
 .../lucene/util/automaton/Transition.java       |   4 +-
 .../lucene/util/automaton/UTF32ToUTF8.java      |   4 +-
 .../lucene/util/automaton/createLevAutomata.py  |  89 +-
 .../org/apache/lucene/util/bkd/BKDReader.java   |   4 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   |   4 +-
 .../apache/lucene/util/bkd/HeapPointReader.java |   4 +-
 .../apache/lucene/util/bkd/HeapPointWriter.java |   4 +-
 .../lucene/util/bkd/OfflinePointReader.java     |   4 +-
 .../lucene/util/bkd/OfflinePointWriter.java     |   5 +-
 .../org/apache/lucene/util/bkd/PointReader.java |   4 +-
 .../org/apache/lucene/util/bkd/PointWriter.java |   4 +-
 .../org/apache/lucene/util/fst/Builder.java     |   4 +-
 .../lucene/util/fst/ByteSequenceOutputs.java    |   4 +-
 .../apache/lucene/util/fst/BytesRefFSTEnum.java |   4 +-
 .../org/apache/lucene/util/fst/BytesStore.java  |   4 +-
 .../lucene/util/fst/CharSequenceOutputs.java    |   4 +-
 .../java/org/apache/lucene/util/fst/FST.java    |   4 +-
 .../org/apache/lucene/util/fst/FSTEnum.java     |   4 +-
 .../lucene/util/fst/ForwardBytesReader.java     |   4 +-
 .../lucene/util/fst/IntSequenceOutputs.java     |   4 +-
 .../apache/lucene/util/fst/IntsRefFSTEnum.java  |   4 +-
 .../org/apache/lucene/util/fst/NoOutputs.java   |   4 +-
 .../org/apache/lucene/util/fst/NodeHash.java    |   4 +-
 .../org/apache/lucene/util/fst/Outputs.java     |   4 +-
 .../org/apache/lucene/util/fst/PairOutputs.java |   4 +-
 .../lucene/util/fst/PositiveIntOutputs.java     |   4 +-
 .../lucene/util/fst/ReverseBytesReader.java     |   4 +-
 .../java/org/apache/lucene/util/fst/Util.java   |   4 +-
 .../util/packed/AbstractBlockPackedWriter.java  |   4 +-
 .../util/packed/AbstractPagedMutable.java       |   4 +-
 .../lucene/util/packed/BlockPackedReader.java   |   4 +-
 .../util/packed/BlockPackedReaderIterator.java  |   4 +-
 .../lucene/util/packed/BlockPackedWriter.java   |   4 +-
 .../lucene/util/packed/BulkOperation.java       |   3 +-
 .../lucene/util/packed/BulkOperationPacked.java |   6 +-
 .../util/packed/BulkOperationPacked1.java       |   3 +-
 .../util/packed/BulkOperationPacked10.java      |   3 +-
 .../util/packed/BulkOperationPacked11.java      |   3 +-
 .../util/packed/BulkOperationPacked12.java      |   3 +-
 .../util/packed/BulkOperationPacked13.java      |   3 +-
 .../util/packed/BulkOperationPacked14.java      |   3 +-
 .../util/packed/BulkOperationPacked15.java      |   3 +-
 .../util/packed/BulkOperationPacked16.java      |   3 +-
 .../util/packed/BulkOperationPacked17.java      |   3 +-
 .../util/packed/BulkOperationPacked18.java      |   3 +-
 .../util/packed/BulkOperationPacked19.java      |   3 +-
 .../util/packed/BulkOperationPacked2.java       |   3 +-
 .../util/packed/BulkOperationPacked20.java      |   3 +-
 .../util/packed/BulkOperationPacked21.java      |   3 +-
 .../util/packed/BulkOperationPacked22.java      |   3 +-
 .../util/packed/BulkOperationPacked23.java      |   3 +-
 .../util/packed/BulkOperationPacked24.java      |   3 +-
 .../util/packed/BulkOperationPacked3.java       |   3 +-
 .../util/packed/BulkOperationPacked4.java       |   3 +-
 .../util/packed/BulkOperationPacked5.java       |   3 +-
 .../util/packed/BulkOperationPacked6.java       |   3 +-
 .../util/packed/BulkOperationPacked7.java       |   3 +-
 .../util/packed/BulkOperationPacked8.java       |   3 +-
 .../util/packed/BulkOperationPacked9.java       |   3 +-
 .../packed/BulkOperationPackedSingleBlock.java  |   4 +-
 .../util/packed/DeltaPackedLongValues.java      |   4 +-
 .../org/apache/lucene/util/packed/Direct16.java |   8 +-
 .../org/apache/lucene/util/packed/Direct32.java |   8 +-
 .../org/apache/lucene/util/packed/Direct64.java |   3 +-
 .../org/apache/lucene/util/packed/Direct8.java  |   8 +-
 .../util/packed/DirectMonotonicReader.java      |   4 +-
 .../util/packed/DirectMonotonicWriter.java      |   4 +-
 .../packed/DirectPacked64SingleBlockReader.java |   4 +-
 .../lucene/util/packed/DirectPackedReader.java  |   4 +-
 .../apache/lucene/util/packed/DirectReader.java |   4 +-
 .../apache/lucene/util/packed/DirectWriter.java |   4 +-
 .../lucene/util/packed/GrowableWriter.java      |   4 +-
 .../util/packed/MonotonicBlockPackedReader.java |   4 +-
 .../util/packed/MonotonicBlockPackedWriter.java |   4 +-
 .../lucene/util/packed/MonotonicLongValues.java |   4 +-
 .../lucene/util/packed/Packed16ThreeBlocks.java |   3 +-
 .../org/apache/lucene/util/packed/Packed64.java |   4 +-
 .../lucene/util/packed/Packed64SingleBlock.java |   3 +-
 .../lucene/util/packed/Packed8ThreeBlocks.java  |   3 +-
 .../lucene/util/packed/PackedDataInput.java     |   4 +-
 .../lucene/util/packed/PackedDataOutput.java    |   4 +-
 .../apache/lucene/util/packed/PackedInts.java   |   4 +-
 .../lucene/util/packed/PackedLongValues.java    |   4 +-
 .../util/packed/PackedReaderIterator.java       |   4 +-
 .../apache/lucene/util/packed/PackedWriter.java |   4 +-
 .../lucene/util/packed/PagedGrowableWriter.java |   4 +-
 .../apache/lucene/util/packed/PagedMutable.java |   4 +-
 .../lucene/util/packed/gen_BulkOperation.py     |  79 +-
 .../org/apache/lucene/util/packed/gen_Direct.py |  29 +-
 .../util/packed/gen_Packed64SingleBlock.py      |  45 +-
 .../lucene/util/packed/gen_PackedThreeBlocks.py |  27 +-
 .../test/org/apache/lucene/TestAssertions.java  |   4 +-
 .../src/test/org/apache/lucene/TestDemo.java    |   4 +-
 .../org/apache/lucene/TestExternalCodecs.java   |   4 +-
 .../lucene/TestMergeSchedulerExternal.java      |   4 +-
 .../src/test/org/apache/lucene/TestSearch.java  |   4 +-
 .../apache/lucene/TestSearchForDuplicates.java  |   4 +-
 .../lucene/analysis/TestCachingTokenFilter.java |   4 +-
 .../apache/lucene/analysis/TestCharFilter.java  |   4 +-
 .../lucene/analysis/TestNumericTokenStream.java |   4 +-
 .../analysis/TestReusableStringReader.java      |  12 +-
 .../org/apache/lucene/analysis/TestToken.java   |   4 +-
 .../tokenattributes/TestBytesRefAttImpl.java    |   4 +-
 .../TestCharTermAttributeImpl.java              |   4 +-
 .../TestPackedTokenAttributeImpl.java           |   4 +-
 .../TestSimpleAttributeImpl.java                |   4 +-
 .../lucene/codecs/TestCodecLoadingDeadlock.java |   4 +-
 .../AbstractTestCompressionMode.java            |   4 +-
 .../AbstractTestLZ4CompressionMode.java         |   4 +-
 .../compressing/TestFastCompressionMode.java    |   4 +-
 .../compressing/TestFastDecompressionMode.java  |   4 +-
 .../TestGrowableByteArrayDataOutput.java        |   4 +-
 .../compressing/TestHighCompressionMode.java    |   4 +-
 .../lucene50/TestBlockPostingsFormat.java       |   4 +-
 .../lucene50/TestBlockPostingsFormat2.java      |   4 +-
 .../lucene50/TestBlockPostingsFormat3.java      |   4 +-
 .../lucene/codecs/lucene50/TestForUtil.java     |   4 +-
 .../lucene50/TestLucene50CompoundFormat.java    |   4 +-
 .../lucene50/TestLucene50FieldInfoFormat.java   |   4 +-
 .../lucene50/TestLucene50SegmentInfoFormat.java |   4 +-
 .../TestLucene50StoredFieldsFormat.java         |   4 +-
 ...cene50StoredFieldsFormatHighCompression.java |   4 +-
 .../lucene50/TestLucene50TermVectorsFormat.java |   4 +-
 .../lucene53/TestLucene53NormsFormat.java       |   4 +-
 .../lucene54/TestLucene54DocValuesFormat.java   |   4 +-
 .../lucene60/TestLucene60PointFormat.java       |  83 ++
 .../perfield/TestPerFieldDocValuesFormat.java   |   4 +-
 .../perfield/TestPerFieldPostingsFormat.java    |   4 +-
 .../perfield/TestPerFieldPostingsFormat2.java   |   4 +-
 .../lucene/document/TestBinaryDocument.java     |   4 +-
 .../apache/lucene/document/TestDateTools.java   |  16 +-
 .../apache/lucene/document/TestDocument.java    |   4 +-
 .../org/apache/lucene/document/TestField.java   |   4 +-
 .../apache/lucene/document/TestFieldType.java   |   4 +-
 .../lucene/index/Test2BBinaryDocValues.java     |   4 +-
 .../lucene/index/Test2BNumericDocValues.java    |   4 +-
 .../apache/lucene/index/Test2BPositions.java    |   4 +-
 .../org/apache/lucene/index/Test2BPostings.java |   4 +-
 .../lucene/index/Test2BPostingsBytes.java       |   4 +-
 .../index/Test2BSortedDocValuesFixedSorted.java |   4 +-
 .../lucene/index/Test2BSortedDocValuesOrds.java |   4 +-
 .../org/apache/lucene/index/Test2BTerms.java    |   4 +-
 .../lucene/index/Test4GBStoredFields.java       |   4 +-
 .../org/apache/lucene/index/TestAddIndexes.java |  26 +-
 .../index/TestAllFilesCheckIndexHeader.java     | 142 +++
 .../index/TestAllFilesDetectTruncation.java     | 126 +++
 .../index/TestAllFilesHaveChecksumFooter.java   |  28 +-
 .../index/TestAllFilesHaveCodecHeader.java      |  35 +-
 .../apache/lucene/index/TestAtomicUpdate.java   |  18 +-
 .../apache/lucene/index/TestBagOfPositions.java |   4 +-
 .../apache/lucene/index/TestBagOfPostings.java  |   4 +-
 .../index/TestBinaryDocValuesUpdates.java       |  36 +-
 .../apache/lucene/index/TestBinaryTerms.java    |   4 +-
 .../org/apache/lucene/index/TestByteSlices.java |  12 +-
 .../org/apache/lucene/index/TestCheckIndex.java |   4 +-
 .../lucene/index/TestCodecHoldsOpenFiles.java   |  19 +-
 .../org/apache/lucene/index/TestCodecUtil.java  |   4 +-
 .../org/apache/lucene/index/TestCodecs.java     |   4 +-
 .../index/TestConcurrentMergeScheduler.java     |   8 +-
 .../index/TestConsistentFieldNumbers.java       |   4 +-
 .../test/org/apache/lucene/index/TestCrash.java |   4 +-
 .../index/TestCrashCausesCorruptIndex.java      |   4 +-
 .../apache/lucene/index/TestCustomNorms.java    |   4 +-
 .../apache/lucene/index/TestDeletionPolicy.java |  38 +-
 .../index/TestDemoParallelLeafReader.java       |  18 +-
 .../lucene/index/TestDirectoryReader.java       | 388 ++++----
 .../lucene/index/TestDirectoryReaderReopen.java |  14 +-
 .../test/org/apache/lucene/index/TestDoc.java   |  12 +-
 .../org/apache/lucene/index/TestDocCount.java   |   4 +-
 .../index/TestDocInverterPerFieldErrorInfo.java |   4 +-
 .../org/apache/lucene/index/TestDocValues.java  |   4 +-
 .../lucene/index/TestDocValuesIndexing.java     |   4 +-
 .../lucene/index/TestDocsAndPositions.java      |   4 +-
 .../apache/lucene/index/TestDocumentWriter.java |   4 +-
 .../index/TestDocumentsWriterDeleteQueue.java   |  29 +-
 .../index/TestDocumentsWriterStallControl.java  |  27 +-
 .../apache/lucene/index/TestDuelingCodecs.java  |   4 +-
 .../lucene/index/TestDuelingCodecsAtNight.java  |   4 +-
 .../lucene/index/TestExceedMaxTermLength.java   |   4 +-
 .../index/TestExitableDirectoryReader.java      |   4 +-
 .../org/apache/lucene/index/TestFieldReuse.java |   6 +-
 .../apache/lucene/index/TestFieldsReader.java   |  69 +-
 .../lucene/index/TestFilterDirectoryReader.java |   4 +-
 .../lucene/index/TestFilterLeafReader.java      |   4 +-
 .../test/org/apache/lucene/index/TestFlex.java  |   4 +-
 .../index/TestFlushByRamOrCountsPolicy.java     |   4 +-
 .../lucene/index/TestForTooMuchCloning.java     |   4 +-
 .../lucene/index/TestForceMergeForever.java     |   4 +-
 .../apache/lucene/index/TestIndexCommit.java    |   4 +-
 .../lucene/index/TestIndexFileDeleter.java      |  35 +-
 .../org/apache/lucene/index/TestIndexInput.java |   4 +-
 .../lucene/index/TestIndexReaderClose.java      |   4 +-
 .../apache/lucene/index/TestIndexWriter.java    | 951 +++++++++---------
 .../lucene/index/TestIndexWriterCommit.java     |  24 +-
 .../lucene/index/TestIndexWriterConfig.java     |   4 +-
 .../lucene/index/TestIndexWriterDelete.java     |  13 +-
 .../index/TestIndexWriterDeleteByQuery.java     |   6 +-
 .../lucene/index/TestIndexWriterExceptions.java |  21 +-
 .../index/TestIndexWriterExceptions2.java       |   7 +-
 .../lucene/index/TestIndexWriterForceMerge.java |   5 +-
 .../lucene/index/TestIndexWriterFromReader.java |  21 +-
 .../index/TestIndexWriterLockRelease.java       |   4 +-
 .../lucene/index/TestIndexWriterMaxDocs.java    |   4 +-
 .../index/TestIndexWriterMergePolicy.java       |   4 +-
 .../lucene/index/TestIndexWriterMerging.java    |  17 +-
 .../index/TestIndexWriterNRTIsCurrent.java      |  28 +-
 .../lucene/index/TestIndexWriterOnDiskFull.java |  10 +-
 .../lucene/index/TestIndexWriterOnJRECrash.java |  28 +-
 .../lucene/index/TestIndexWriterOnVMError.java  |   7 +-
 .../TestIndexWriterOutOfFileDescriptors.java    |   6 +-
 .../lucene/index/TestIndexWriterReader.java     |  14 +-
 .../index/TestIndexWriterThreadsToSegments.java |   4 +-
 .../lucene/index/TestIndexWriterUnicode.java    |   4 +-
 .../index/TestIndexWriterWithThreads.java       |   4 +-
 .../apache/lucene/index/TestIndexableField.java |   7 +-
 .../org/apache/lucene/index/TestInfoStream.java |   4 +-
 .../apache/lucene/index/TestIntBlockPool.java   |   4 +-
 .../org/apache/lucene/index/TestIsCurrent.java  |   4 +-
 .../lucene/index/TestLazyProxSkipping.java      |   4 +-
 .../apache/lucene/index/TestLogMergePolicy.java |   4 +-
 .../apache/lucene/index/TestLongPostings.java   |   4 +-
 .../org/apache/lucene/index/TestManyFields.java |   4 +-
 .../apache/lucene/index/TestMaxPosition.java    |   4 +-
 .../lucene/index/TestMaxTermFrequency.java      |   4 +-
 .../lucene/index/TestMergePolicyWrapper.java    |  37 +
 .../lucene/index/TestMergeRateLimiter.java      |   4 +-
 .../apache/lucene/index/TestMixedCodecs.java    |   4 +-
 .../lucene/index/TestMixedDocValuesUpdates.java |  32 +-
 .../apache/lucene/index/TestMultiDocValues.java |   4 +-
 .../apache/lucene/index/TestMultiFields.java    |   4 +-
 .../lucene/index/TestMultiLevelSkipList.java    |   4 +-
 .../apache/lucene/index/TestMultiTermsEnum.java |   4 +-
 .../lucene/index/TestNRTReaderCleanup.java      |  14 +-
 .../lucene/index/TestNRTReaderWithThreads.java  |   4 +-
 .../org/apache/lucene/index/TestNRTThreads.java |   4 +-
 .../apache/lucene/index/TestNeverDelete.java    |  12 +-
 .../apache/lucene/index/TestNewestSegment.java  |   4 +-
 .../lucene/index/TestNoDeletionPolicy.java      |   4 +-
 .../apache/lucene/index/TestNoMergePolicy.java  |  12 +-
 .../lucene/index/TestNoMergeScheduler.java      |   4 +-
 .../test/org/apache/lucene/index/TestNorms.java |   4 +-
 .../index/TestNumericDocValuesUpdates.java      |  36 +-
 .../org/apache/lucene/index/TestOmitNorms.java  |   4 +-
 .../apache/lucene/index/TestOmitPositions.java  |   9 +-
 .../org/apache/lucene/index/TestOmitTf.java     |   9 +-
 .../org/apache/lucene/index/TestOrdinalMap.java |   4 +-
 .../index/TestParallelCompositeReader.java      |   4 +-
 .../lucene/index/TestParallelLeafReader.java    |   4 +-
 .../index/TestParallelReaderEmptyIndex.java     |   4 +-
 .../lucene/index/TestParallelTermEnum.java      |   4 +-
 .../org/apache/lucene/index/TestPayloads.java   |   4 +-
 .../lucene/index/TestPayloadsOnVectors.java     |   4 +-
 .../lucene/index/TestPerSegmentDeletes.java     |   4 +-
 .../TestPersistentSnapshotDeletionPolicy.java   |  29 +-
 .../apache/lucene/index/TestPointValues.java    | 990 +++----------------
 .../lucene/index/TestPostingsOffsets.java       |   4 +-
 .../lucene/index/TestPrefixCodedTerms.java      |   4 +-
 .../apache/lucene/index/TestReadOnlyIndex.java  |   4 +-
 .../apache/lucene/index/TestReaderClosed.java   |   4 +-
 .../index/TestReaderWrapperDVTypeCheck.java     |   1 -
 .../org/apache/lucene/index/TestRollback.java   |   4 +-
 .../apache/lucene/index/TestRollingUpdates.java |   8 +-
 .../lucene/index/TestSameTokenSamePosition.java |   4 +-
 .../apache/lucene/index/TestSegmentInfos.java   |   4 +-
 .../apache/lucene/index/TestSegmentMerger.java  |   4 +-
 .../apache/lucene/index/TestSegmentReader.java  |   4 +-
 .../lucene/index/TestSegmentTermDocs.java       |   4 +-
 .../lucene/index/TestSegmentTermEnum.java       |   4 +-
 .../lucene/index/TestSizeBoundedForceMerge.java |   4 +-
 .../index/TestSnapshotDeletionPolicy.java       |  26 +-
 .../apache/lucene/index/TestStressAdvance.java  |   4 +-
 .../apache/lucene/index/TestStressDeletes.java  |  14 +-
 .../apache/lucene/index/TestStressIndexing.java |  16 +-
 .../lucene/index/TestStressIndexing2.java       |  18 +-
 .../org/apache/lucene/index/TestStressNRT.java  |   6 +-
 .../org/apache/lucene/index/TestSumDocFreq.java |   4 +-
 .../lucene/index/TestSwappedIndexFiles.java     | 128 +++
 .../test/org/apache/lucene/index/TestTerm.java  |   4 +-
 .../apache/lucene/index/TestTermVectors.java    |   4 +-
 .../lucene/index/TestTermVectorsReader.java     |   4 +-
 .../lucene/index/TestTermVectorsWriter.java     |   4 +-
 .../apache/lucene/index/TestTermdocPerf.java    |  15 +-
 .../test/org/apache/lucene/index/TestTerms.java |   4 +-
 .../org/apache/lucene/index/TestTermsEnum.java  |   4 +-
 .../org/apache/lucene/index/TestTermsEnum2.java |   4 +-
 .../lucene/index/TestThreadedForceMerge.java    |   4 +-
 .../lucene/index/TestTieredMergePolicy.java     |   4 +-
 .../index/TestTragicIndexWriterDeadlock.java    |   4 +-
 .../lucene/index/TestTransactionRollback.java   |   4 +-
 .../apache/lucene/index/TestTransactions.java   |   4 +-
 .../org/apache/lucene/index/TestTryDelete.java  |   4 +-
 .../lucene/index/TestTwoPhaseCommitTool.java    |   4 +-
 .../lucene/index/TestUniqueTermCount.java       |   4 +-
 .../index/TestUpgradeIndexMergePolicy.java      |  26 +
 .../lucene/search/BaseTestRangeFilter.java      |   4 +-
 .../apache/lucene/search/JustCompileSearch.java |   4 +-
 .../lucene/search/MultiCollectorTest.java       |   4 +-
 .../TestApproximationSearchEquivalence.java     |   4 +-
 .../lucene/search/TestAutomatonQuery.java       |   4 +-
 .../search/TestAutomatonQueryUnicode.java       |   4 +-
 .../lucene/search/TestBlendedTermQuery.java     |   4 +-
 .../org/apache/lucene/search/TestBoolean2.java  |   4 +-
 .../apache/lucene/search/TestBooleanCoord.java  |   4 +-
 .../search/TestBooleanMinShouldMatch.java       |   4 +-
 .../org/apache/lucene/search/TestBooleanOr.java |   2 +-
 .../apache/lucene/search/TestBooleanQuery.java  |   4 +-
 .../search/TestBooleanQueryVisitSubscorers.java |   4 +-
 .../lucene/search/TestBooleanRewrites.java      |   4 +-
 .../apache/lucene/search/TestBooleanScorer.java |   4 +-
 .../apache/lucene/search/TestBoostQuery.java    |   4 +-
 .../lucene/search/TestCachingCollector.java     |   4 +-
 .../lucene/search/TestComplexExplanations.java  |   4 +-
 .../TestComplexExplanationsOfNonMatches.java    |   4 +-
 .../lucene/search/TestConjunctionDISI.java      |   4 +-
 .../apache/lucene/search/TestConjunctions.java  |   4 +-
 .../lucene/search/TestConstantScoreQuery.java   |   4 +-
 .../TestControlledRealTimeReopenThread.java     |   4 +-
 .../lucene/search/TestCustomSearcherSort.java   |  14 +-
 .../org/apache/lucene/search/TestDateSort.java  |   4 +-
 .../lucene/search/TestDisjunctionMaxQuery.java  |   4 +-
 .../org/apache/lucene/search/TestDocBoost.java  |   4 +-
 .../search/TestDocValuesRewriteMethod.java      |   4 +-
 .../lucene/search/TestDocValuesScoring.java     |   4 +-
 .../lucene/search/TestEarlyTermination.java     |   4 +-
 .../lucene/search/TestElevationComparator.java  |   4 +-
 .../search/TestFieldCacheRewriteMethod.java     |   4 +-
 .../lucene/search/TestFieldValueQuery.java      |   4 +-
 .../apache/lucene/search/TestFuzzyQuery.java    |   4 +-
 .../apache/lucene/search/TestIndexSearcher.java |   4 +-
 .../apache/lucene/search/TestLRUQueryCache.java |   4 +-
 .../lucene/search/TestLiveFieldValues.java      |   4 +-
 .../lucene/search/TestMinShouldMatch2.java      |   4 +-
 .../lucene/search/TestMultiCollector.java       |   4 +-
 .../lucene/search/TestMultiPhraseEnum.java      |   4 +-
 .../lucene/search/TestMultiPhraseQuery.java     |   4 +-
 .../search/TestMultiTermConstantScore.java      |   4 +-
 .../search/TestMultiTermQueryRewrites.java      |   4 +-
 .../search/TestMultiThreadTermVectors.java      |   4 +-
 .../TestMultiValuedNumericRangeQuery.java       |   4 +-
 .../org/apache/lucene/search/TestMultiset.java  |   4 +-
 .../lucene/search/TestNGramPhraseQuery.java     |   4 +-
 .../apache/lucene/search/TestNeedsScores.java   |   4 +-
 .../test/org/apache/lucene/search/TestNot.java  |   4 +-
 .../lucene/search/TestNumericRangeQuery32.java  |   4 +-
 .../lucene/search/TestNumericRangeQuery64.java  |   4 +-
 .../lucene/search/TestPhrasePrefixQuery.java    |   4 +-
 .../apache/lucene/search/TestPhraseQuery.java   |   4 +-
 .../apache/lucene/search/TestPointQueries.java  |  45 +-
 .../lucene/search/TestPositionIncrement.java    |   4 +-
 .../search/TestPositiveScoresOnlyCollector.java |   4 +-
 .../lucene/search/TestPrefixInBooleanQuery.java |   4 +-
 .../apache/lucene/search/TestPrefixQuery.java   |   4 +-
 .../apache/lucene/search/TestPrefixRandom.java  |   4 +-
 .../lucene/search/TestQueryCachingPolicy.java   |   4 +-
 .../apache/lucene/search/TestQueryRescorer.java |   4 +-
 .../apache/lucene/search/TestRegexpQuery.java   |   4 +-
 .../apache/lucene/search/TestRegexpRandom.java  |   4 +-
 .../apache/lucene/search/TestRegexpRandom2.java |   4 +-
 .../lucene/search/TestReqExclBulkScorer.java    |   4 +-
 .../search/TestSameScoresWithThreads.java       |   4 +-
 .../search/TestScoreCachingWrappingScorer.java  |   4 +-
 .../apache/lucene/search/TestScorerPerf.java    |  32 +-
 .../apache/lucene/search/TestSearchAfter.java   |   4 +-
 .../lucene/search/TestSearchWithThreads.java    |   4 +-
 .../lucene/search/TestSearcherManager.java      |   4 +-
 .../lucene/search/TestShardSearching.java       |   4 +-
 .../apache/lucene/search/TestSimilarity.java    |   4 +-
 .../lucene/search/TestSimilarityProvider.java   |   4 +-
 .../lucene/search/TestSimpleExplanations.java   |   4 +-
 .../TestSimpleExplanationsOfNonMatches.java     |   4 +-
 .../search/TestSimpleSearchEquivalence.java     |  14 +-
 .../lucene/search/TestSloppyPhraseQuery.java    |   4 +-
 .../lucene/search/TestSloppyPhraseQuery2.java   |   4 +-
 .../test/org/apache/lucene/search/TestSort.java |   4 +-
 .../apache/lucene/search/TestSortRandom.java    |   4 +-
 .../apache/lucene/search/TestSortRescorer.java  |   4 +-
 .../search/TestSortedNumericSortField.java      |   4 +-
 .../lucene/search/TestSortedSetSelector.java    |   4 +-
 .../lucene/search/TestSortedSetSortField.java   |   4 +-
 .../lucene/search/TestSubScorerFreqs.java       |   4 +-
 .../apache/lucene/search/TestSynonymQuery.java  |   4 +-
 .../lucene/search/TestTermRangeQuery.java       |   4 +-
 .../apache/lucene/search/TestTermScorer.java    |   4 +-
 .../search/TestTimeLimitingCollector.java       |   4 +-
 .../lucene/search/TestTopDocsCollector.java     |   4 +-
 .../apache/lucene/search/TestTopDocsMerge.java  |   4 +-
 .../lucene/search/TestTopFieldCollector.java    |   4 +-
 .../search/TestTotalHitCountCollector.java      |   4 +-
 .../TestUsageTrackingFilterCachingPolicy.java   |   4 +-
 .../org/apache/lucene/search/TestWildcard.java  |   4 +-
 .../lucene/search/TestWildcardRandom.java       |   4 +-
 .../search/similarities/TestBM25Similarity.java |   4 +-
 .../similarities/TestClassicSimilarity.java     |   4 +-
 .../search/similarities/TestSimilarity2.java    |   4 +-
 .../search/similarities/TestSimilarityBase.java |   4 +-
 .../search/spans/JustCompileSearchSpans.java    |   4 +-
 .../apache/lucene/search/spans/TestBasics.java  |   4 +-
 .../search/spans/TestFieldMaskingSpanQuery.java |   4 +-
 .../lucene/search/spans/TestFilterSpans.java    |  27 +-
 .../search/spans/TestNearSpansOrdered.java      |   4 +-
 .../lucene/search/spans/TestSpanBoostQuery.java |  10 +-
 .../lucene/search/spans/TestSpanCollection.java |   4 +-
 .../search/spans/TestSpanContainQuery.java      |   4 +-
 .../search/spans/TestSpanExplanations.java      |   4 +-
 .../spans/TestSpanExplanationsOfNonMatches.java |   4 +-
 .../lucene/search/spans/TestSpanFirstQuery.java |   4 +-
 .../spans/TestSpanMultiTermQueryWrapper.java    |   4 +-
 .../lucene/search/spans/TestSpanNearQuery.java  |   4 +-
 .../lucene/search/spans/TestSpanNotQuery.java   |   4 +-
 .../lucene/search/spans/TestSpanOrQuery.java    |   4 +-
 .../search/spans/TestSpanSearchEquivalence.java |   4 +-
 .../lucene/search/spans/TestSpanTermQuery.java  |   4 +-
 .../apache/lucene/search/spans/TestSpans.java   |   4 +-
 .../lucene/search/spans/TestSpansEnum.java      |   4 +-
 .../lucene/store/TestBufferedChecksum.java      |   4 +-
 .../lucene/store/TestBufferedIndexInput.java    |  90 +-
 .../lucene/store/TestByteArrayDataInput.java    |   4 +-
 .../org/apache/lucene/store/TestDirectory.java  |  23 +-
 .../lucene/store/TestFileSwitchDirectory.java   |   5 +-
 .../lucene/store/TestFilterDirectory.java       |   4 +-
 .../apache/lucene/store/TestHugeRamFile.java    |   4 +-
 .../apache/lucene/store/TestLockFactory.java    |   4 +-
 .../apache/lucene/store/TestMmapDirectory.java  |   4 +-
 .../org/apache/lucene/store/TestMultiMMap.java  |   4 +-
 .../apache/lucene/store/TestNIOFSDirectory.java |   4 +-
 .../lucene/store/TestNRTCachingDirectory.java   |   4 +-
 .../lucene/store/TestNativeFSLockFactory.java   |  25 +-
 .../apache/lucene/store/TestRAMDirectory.java   |   5 +-
 .../apache/lucene/store/TestRateLimiter.java    |   4 +-
 .../lucene/store/TestSimpleFSDirectory.java     |   4 +-
 .../lucene/store/TestSimpleFSLockFactory.java   |   5 +-
 .../store/TestSingleInstanceLockFactory.java    |   4 +-
 .../lucene/store/TestSleepingLockWrapper.java   |   4 +-
 .../store/TestTrackingDirectoryWrapper.java     |   4 +-
 .../apache/lucene/store/TestWindowsMMap.java    |   4 +-
 .../apache/lucene/util/BaseSortTestCase.java    |   4 +-
 .../lucene/util/StressRamUsageEstimator.java    |   4 +-
 .../apache/lucene/util/Test2BPagedBytes.java    |   4 +-
 .../org/apache/lucene/util/TestArrayUtil.java   |   4 +-
 .../apache/lucene/util/TestAttributeSource.java |   4 +-
 .../apache/lucene/util/TestByteBlockPool.java   |  32 +-
 .../org/apache/lucene/util/TestBytesRef.java    |   4 +-
 .../apache/lucene/util/TestBytesRefArray.java   |  27 +-
 .../apache/lucene/util/TestBytesRefHash.java    |   4 +-
 .../org/apache/lucene/util/TestCharsRef.java    |   8 +-
 .../apache/lucene/util/TestCharsRefBuilder.java |   4 +-
 .../lucene/util/TestCloseableThreadLocal.java   |   1 -
 .../apache/lucene/util/TestCollectionUtil.java  |   4 +-
 .../apache/lucene/util/TestDocIdSetBuilder.java |   4 +-
 .../apache/lucene/util/TestFilterIterator.java  |   1 -
 .../lucene/util/TestFixedBitDocIdSet.java       |   1 -
 .../org/apache/lucene/util/TestFixedBitSet.java |   4 +-
 .../util/TestFrequencyTrackingRingBuffer.java   |   4 +-
 .../org/apache/lucene/util/TestIOUtils.java     |   5 +-
 .../lucene/util/TestInPlaceMergeSorter.java     |   4 +-
 .../lucene/util/TestIntArrayDocIdSet.java       |   4 +-
 .../org/apache/lucene/util/TestIntroSorter.java |   4 +-
 .../org/apache/lucene/util/TestIntsRef.java     |   4 +-
 .../apache/lucene/util/TestLSBRadixSorter.java  |   4 +-
 .../lucene/util/TestLegacyNumericUtils.java     |   4 +-
 .../org/apache/lucene/util/TestLongBitSet.java  |   4 +-
 .../org/apache/lucene/util/TestMathUtil.java    |   4 +-
 .../apache/lucene/util/TestMergedIterator.java  |   4 +-
 .../apache/lucene/util/TestNamedSPILoader.java  |  12 +-
 .../org/apache/lucene/util/TestNotDocIdSet.java |   4 +-
 .../apache/lucene/util/TestOfflineSorter.java   |  32 +-
 .../org/apache/lucene/util/TestPagedBytes.java  |   1 -
 .../apache/lucene/util/TestPriorityQueue.java   |   4 +-
 .../apache/lucene/util/TestQueryBuilder.java    |   4 +-
 .../lucene/util/TestRamUsageEstimator.java      |   4 +-
 .../util/TestRecyclingByteBlockAllocator.java   |  18 +-
 .../util/TestRecyclingIntBlockAllocator.java    |  18 +-
 .../apache/lucene/util/TestRoaringDocIdSet.java |   4 +-
 .../apache/lucene/util/TestRollingBuffer.java   |   8 +-
 .../lucene/util/TestSPIClassIterator.java       |   4 +-
 .../apache/lucene/util/TestSentinelIntSet.java  |   6 +-
 .../org/apache/lucene/util/TestSetOnce.java     |   4 +-
 .../org/apache/lucene/util/TestSloppyMath.java  |   4 +-
 .../org/apache/lucene/util/TestSmallFloat.java  |  14 +-
 .../lucene/util/TestSparseFixedBitDocIdSet.java |   4 +-
 .../lucene/util/TestSparseFixedBitSet.java      |   4 +-
 .../apache/lucene/util/TestStringHelper.java    |   4 +-
 .../org/apache/lucene/util/TestTimSorter.java   |   4 +-
 .../lucene/util/TestTimSorterWorstCase.java     |   4 +-
 .../org/apache/lucene/util/TestUnicodeUtil.java |   4 +-
 .../org/apache/lucene/util/TestVersion.java     |   4 +-
 .../apache/lucene/util/TestVirtualMethod.java   |   4 +-
 .../apache/lucene/util/TestWeakIdentityMap.java |   1 -
 .../automaton/FiniteStringsIteratorTest.java    |   4 +-
 .../LimitedFiniteStringsIteratorTest.java       |   4 +-
 .../lucene/util/automaton/TestAutomaton.java    |   4 +-
 .../util/automaton/TestCompiledAutomaton.java   |   4 +-
 .../lucene/util/automaton/TestDeterminism.java  |   4 +-
 .../util/automaton/TestDeterminizeLexicon.java  |   4 +-
 .../util/automaton/TestLevenshteinAutomata.java |   4 +-
 .../lucene/util/automaton/TestMinimize.java     |   4 +-
 .../lucene/util/automaton/TestOperations.java   |   4 +-
 .../lucene/util/automaton/TestRegExp.java       |   4 +-
 .../lucene/util/automaton/TestUTF32ToUTF8.java  |   4 +-
 .../org/apache/lucene/util/bkd/TestBKD.java     |  10 +-
 .../org/apache/lucene/util/fst/Test2BFST.java   |   5 +-
 .../apache/lucene/util/fst/TestBytesStore.java  |   4 +-
 .../org/apache/lucene/util/fst/TestFSTs.java    |   7 +-
 .../lucene/util/packed/TestDirectMonotonic.java |   4 +-
 .../lucene/util/packed/TestDirectPacked.java    |   4 +-
 .../lucene/util/packed/TestPackedInts.java      |   4 +-
 .../java/org/apache/lucene/demo/IndexFiles.java |   4 +-
 .../org/apache/lucene/demo/SearchFiles.java     |   4 +-
 .../demo/facet/AssociationsFacetsExample.java   |   4 +-
 .../demo/facet/DistanceFacetsExample.java       |   4 +-
 .../ExpressionAggregationFacetsExample.java     |  32 +-
 .../facet/MultiCategoryListsFacetsExample.java  |   4 +-
 .../lucene/demo/facet/RangeFacetsExample.java   |   4 +-
 .../lucene/demo/facet/SimpleFacetsExample.java  |   4 +-
 .../facet/SimpleSortedSetFacetsExample.java     |   4 +-
 .../demo/xmlparser/FormBasedXmlQueryDemo.java   |   1 -
 .../test/org/apache/lucene/demo/TestDemo.java   |   4 +-
 .../facet/TestAssociationsFacetsExample.java    |   4 +-
 .../demo/facet/TestDistanceFacetsExample.java   |   4 +-
 .../TestExpressionAggregationFacetsExample.java |   4 +-
 .../TestMultiCategoryListsFacetsExample.java    |   4 +-
 .../demo/facet/TestRangeFacetsExample.java      |   4 +-
 .../demo/facet/TestSimpleFacetsExample.java     |   4 +-
 .../facet/TestSimpleSortedSetFacetsExample.java |   4 +-
 .../org/apache/lucene/expressions/Bindings.java |   2 +-
 .../apache/lucene/expressions/Expression.java   |   2 +-
 .../expressions/ExpressionComparator.java       |   2 +-
 .../expressions/ExpressionFunctionValues.java   |   2 +-
 .../lucene/expressions/ExpressionRescorer.java  |   4 +-
 .../lucene/expressions/ExpressionSortField.java |   2 +-
 .../expressions/ExpressionValueSource.java      |   2 +-
 .../apache/lucene/expressions/FakeScorer.java   |   4 +-
 .../lucene/expressions/ScoreFunctionValues.java |   2 +-
 .../lucene/expressions/ScoreValueSource.java    |   4 +-
 .../lucene/expressions/SimpleBindings.java      |   4 +-
 .../expressions/js/JavascriptCompiler.java      |   2 +-
 .../js/JavascriptErrorHandlingLexer.java        |   4 +-
 .../js/JavascriptParserErrorStrategy.java       |   4 +-
 .../lucene/expressions/js/VariableContext.java  |   4 +-
 .../lucene/expressions/TestDemoExpressions.java |  32 +-
 .../expressions/TestExpressionRescorer.java     |   4 +-
 .../expressions/TestExpressionSortField.java    |   4 +-
 .../lucene/expressions/TestExpressionSorts.java |   4 +-
 .../expressions/TestExpressionValidation.java   |   4 +-
 .../expressions/TestExpressionValueSource.java  |   4 +-
 .../expressions/js/TestCustomFunctions.java     |   4 +-
 .../expressions/js/TestJavascriptCompiler.java  |   2 +-
 .../expressions/js/TestJavascriptFunction.java  |   2 +-
 .../js/TestJavascriptOperations.java            |   2 +-
 .../expressions/js/TestVariableContext.java     |   4 +-
 .../org/apache/lucene/facet/DrillDownQuery.java |   3 +-
 .../org/apache/lucene/facet/DrillSideways.java  |   3 +-
 .../apache/lucene/facet/DrillSidewaysQuery.java |   4 +-
 .../lucene/facet/DrillSidewaysScorer.java       |   3 +-
 .../org/apache/lucene/facet/FacetField.java     |   3 +-
 .../org/apache/lucene/facet/FacetResult.java    |   3 +-
 .../java/org/apache/lucene/facet/Facets.java    |   3 +-
 .../apache/lucene/facet/FacetsCollector.java    |   3 +-
 .../org/apache/lucene/facet/FacetsConfig.java   |   4 +-
 .../org/apache/lucene/facet/LabelAndValue.java  |   3 +-
 .../org/apache/lucene/facet/MultiFacets.java    |   3 +-
 .../facet/RandomSamplingFacetsCollector.java    |   3 +-
 .../lucene/facet/TopOrdAndFloatQueue.java       |   3 +-
 .../apache/lucene/facet/TopOrdAndIntQueue.java  |   3 +-
 .../apache/lucene/facet/range/DoubleRange.java  |   3 +-
 .../facet/range/DoubleRangeFacetCounts.java     |   3 +-
 .../apache/lucene/facet/range/LongRange.java    |   3 +-
 .../lucene/facet/range/LongRangeCounter.java    |   3 +-
 .../facet/range/LongRangeFacetCounts.java       |   3 +-
 .../org/apache/lucene/facet/range/Range.java    |   3 +-
 .../lucene/facet/range/RangeFacetCounts.java    |   3 +-
 .../DefaultSortedSetDocValuesReaderState.java   |   3 +-
 .../SortedSetDocValuesFacetCounts.java          |   3 +-
 .../sortedset/SortedSetDocValuesFacetField.java |   3 +-
 .../SortedSetDocValuesReaderState.java          |   3 +-
 .../facet/taxonomy/AssociationFacetField.java   |   3 +-
 .../facet/taxonomy/CachedOrdinalsReader.java    |   3 +-
 .../facet/taxonomy/DocValuesOrdinalsReader.java |   4 +-
 .../lucene/facet/taxonomy/FacetLabel.java       |   3 +-
 .../lucene/facet/taxonomy/FakeScorer.java       |   3 +-
 .../facet/taxonomy/FastTaxonomyFacetCounts.java |   3 +-
 .../taxonomy/FloatAssociationFacetField.java    |   3 +-
 .../facet/taxonomy/FloatTaxonomyFacets.java     |   3 +-
 .../taxonomy/IntAssociationFacetField.java      |   3 +-
 .../facet/taxonomy/IntTaxonomyFacets.java       |   3 +-
 .../lucene/facet/taxonomy/LRUHashMap.java       |   3 +-
 .../taxonomy/OrdinalMappingLeafReader.java      |   3 +-
 .../lucene/facet/taxonomy/OrdinalsReader.java   |   3 +-
 .../facet/taxonomy/ParallelTaxonomyArrays.java  |   4 +-
 .../facet/taxonomy/PrintTaxonomyStats.java      |   3 +-
 .../facet/taxonomy/SearcherTaxonomyManager.java |   3 +-
 .../facet/taxonomy/TaxonomyFacetCounts.java     |   3 +-
 .../TaxonomyFacetSumFloatAssociations.java      |   3 +-
 .../TaxonomyFacetSumIntAssociations.java        |   3 +-
 .../taxonomy/TaxonomyFacetSumValueSource.java   |   3 +-
 .../lucene/facet/taxonomy/TaxonomyFacets.java   |   3 +-
 .../facet/taxonomy/TaxonomyMergeUtils.java      |  31 +-
 .../lucene/facet/taxonomy/TaxonomyReader.java   |  17 +-
 .../lucene/facet/taxonomy/TaxonomyWriter.java   |  15 +-
 .../lucene/facet/taxonomy/directory/Consts.java |   7 +-
 .../directory/DirectoryTaxonomyReader.java      |  33 +-
 .../directory/DirectoryTaxonomyWriter.java      |  33 +-
 .../taxonomy/directory/TaxonomyIndexArrays.java |  25 +-
 .../taxonomy/writercache/CategoryPathUtils.java |   7 +-
 .../taxonomy/writercache/CharBlockArray.java    |  21 +-
 .../writercache/Cl2oTaxonomyWriterCache.java    |  15 +-
 .../taxonomy/writercache/CollisionMap.java      |  13 +-
 .../writercache/CompactLabelToOrdinal.java      |   3 +-
 .../taxonomy/writercache/LabelToOrdinal.java    |   7 +-
 .../writercache/LruTaxonomyWriterCache.java     |   9 +-
 .../writercache/NameHashIntCacheLRU.java        |   7 +-
 .../taxonomy/writercache/NameIntCacheLRU.java   |  13 +-
 .../writercache/TaxonomyWriterCache.java        |   9 +-
 .../facet/AssertingSubDocsAtOnceCollector.java  |   3 +-
 .../org/apache/lucene/facet/FacetTestCase.java  |   3 +-
 .../apache/lucene/facet/SlowRAMDirectory.java   |   3 +-
 .../apache/lucene/facet/TestDrillDownQuery.java |   3 +-
 .../apache/lucene/facet/TestDrillSideways.java  |   3 +-
 .../apache/lucene/facet/TestFacetsConfig.java   |   3 +-
 .../lucene/facet/TestMultipleIndexFields.java   |   3 +-
 .../TestRandomSamplingFacetsCollector.java      |  33 +-
 .../facet/range/TestRangeFacetCounts.java       |   3 +-
 .../sortedset/TestSortedSetDocValuesFacets.java |   3 +-
 .../taxonomy/TestCachedOrdinalsReader.java      |   3 +-
 .../lucene/facet/taxonomy/TestFacetLabel.java   |  21 +-
 .../lucene/facet/taxonomy/TestLRUHashMap.java   |   3 +-
 .../taxonomy/TestOrdinalMappingLeafReader.java  |  33 +-
 .../taxonomy/TestSearcherTaxonomyManager.java   |   3 +-
 .../facet/taxonomy/TestTaxonomyCombined.java    |  33 +-
 .../taxonomy/TestTaxonomyFacetAssociations.java |   4 +-
 .../facet/taxonomy/TestTaxonomyFacetCounts.java |   3 +-
 .../taxonomy/TestTaxonomyFacetCounts2.java      |   3 +-
 .../TestTaxonomyFacetSumValueSource.java        |   3 +-
 .../taxonomy/directory/TestAddTaxonomy.java     |  42 +-
 .../TestConcurrentFacetedIndexing.java          |  33 +-
 .../directory/TestDirectoryTaxonomyReader.java  |  33 +-
 .../directory/TestDirectoryTaxonomyWriter.java  |  32 +-
 .../writercache/TestCharBlockArray.java         |  30 +-
 .../writercache/TestCompactLabelToOrdinal.java  |  33 +-
 .../AbstractAllGroupHeadsCollector.java         |   3 +-
 .../grouping/AbstractAllGroupsCollector.java    |   3 +-
 .../AbstractDistinctValuesCollector.java        |   3 +-
 .../AbstractFirstPassGroupingCollector.java     |   3 +-
 .../grouping/AbstractGroupFacetCollector.java   |   3 +-
 .../AbstractSecondPassGroupingCollector.java    |   3 +-
 .../search/grouping/BlockGroupingCollector.java |   3 +-
 .../search/grouping/CollectedSearchGroup.java   |   1 -
 .../lucene/search/grouping/FakeScorer.java      |   3 +-
 .../lucene/search/grouping/GroupDocs.java       |   3 +-
 .../lucene/search/grouping/GroupingSearch.java  |   3 +-
 .../lucene/search/grouping/SearchGroup.java     |   3 +-
 .../lucene/search/grouping/TopGroups.java       |   3 +-
 .../FunctionAllGroupHeadsCollector.java         |   3 +-
 .../function/FunctionAllGroupsCollector.java    |   3 +-
 .../FunctionDistinctValuesCollector.java        |   3 +-
 .../FunctionFirstPassGroupingCollector.java     |   3 +-
 .../FunctionSecondPassGroupingCollector.java    |   3 +-
 .../term/TermAllGroupHeadsCollector.java        |   3 +-
 .../grouping/term/TermAllGroupsCollector.java   |   3 +-
 .../term/TermDistinctValuesCollector.java       |   3 +-
 .../term/TermFirstPassGroupingCollector.java    |   3 +-
 .../grouping/term/TermGroupFacetCollector.java  |   3 +-
 .../term/TermSecondPassGroupingCollector.java   |   3 +-
 .../grouping/AbstractGroupingTestCase.java      |   3 +-
 .../grouping/AllGroupHeadsCollectorTest.java    |   3 +-
 .../search/grouping/AllGroupsCollectorTest.java |   5 +-
 .../grouping/DistinctValuesCollectorTest.java   |   3 +-
 .../grouping/GroupFacetCollectorTest.java       |   3 +-
 .../search/grouping/GroupingSearchTest.java     |   3 +-
 .../lucene/search/grouping/TestGrouping.java    |   1 -
 .../lucene/search/highlight/DefaultEncoder.java |  22 +-
 .../apache/lucene/search/highlight/Encoder.java |  17 +-
 .../lucene/search/highlight/Formatter.java      |   4 +-
 .../lucene/search/highlight/Fragmenter.java     |   3 +-
 .../search/highlight/GradientFormatter.java     |   3 +-
 .../lucene/search/highlight/Highlighter.java    |   3 +-
 .../highlight/InvalidTokenOffsetsException.java |   3 +-
 .../lucene/search/highlight/NullFragmenter.java |   3 +-
 .../highlight/OffsetLimitTokenFilter.java       |   3 +-
 .../lucene/search/highlight/PositionSpan.java   |   3 +-
 .../lucene/search/highlight/QueryScorer.java    |   3 +-
 .../search/highlight/QueryTermExtractor.java    |   3 +-
 .../search/highlight/QueryTermScorer.java       |   3 +-
 .../apache/lucene/search/highlight/Scorer.java  |   3 +-
 .../search/highlight/SimpleFragmenter.java      |   3 +-
 .../search/highlight/SimpleHTMLEncoder.java     |  19 +-
 .../search/highlight/SimpleHTMLFormatter.java   |   3 +-
 .../search/highlight/SimpleSpanFragmenter.java  |   6 +-
 .../search/highlight/SpanGradientFormatter.java |   3 +-
 .../search/highlight/TermVectorLeafReader.java  |   3 +-
 .../lucene/search/highlight/TextFragment.java   |   4 +-
 .../lucene/search/highlight/TokenGroup.java     |   3 +-
 .../lucene/search/highlight/TokenSources.java   |   3 +-
 .../highlight/TokenStreamFromTermVector.java    |   3 +-
 .../search/highlight/WeightedSpanTerm.java      |   6 +-
 .../highlight/WeightedSpanTermExtractor.java    |   3 +-
 .../lucene/search/highlight/WeightedTerm.java   |   3 +-
 .../CustomSeparatorBreakIterator.java           |   9 +-
 .../DefaultPassageFormatter.java                |   3 +-
 .../MultiTermHighlighting.java                  |   3 +-
 .../search/postingshighlight/Passage.java       |   3 +-
 .../postingshighlight/PassageFormatter.java     |   3 +-
 .../search/postingshighlight/PassageScorer.java |   3 +-
 .../postingshighlight/PostingsHighlighter.java  |   3 +-
 .../postingshighlight/WholeBreakIterator.java   |   3 +-
 .../vectorhighlight/BaseFragListBuilder.java    |   3 +-
 .../vectorhighlight/BaseFragmentsBuilder.java   |   3 +-
 .../search/vectorhighlight/BoundaryScanner.java |   3 +-
 .../BreakIteratorBoundaryScanner.java           |   3 +-
 .../vectorhighlight/FastVectorHighlighter.java  |   3 +-
 .../search/vectorhighlight/FieldFragList.java   |   3 +-
 .../search/vectorhighlight/FieldPhraseList.java |   3 +-
 .../search/vectorhighlight/FieldQuery.java      |   3 +-
 .../search/vectorhighlight/FieldTermStack.java  |   3 +-
 .../search/vectorhighlight/FragListBuilder.java |   3 +-
 .../vectorhighlight/FragmentsBuilder.java       |   3 +-
 .../ScoreOrderFragmentsBuilder.java             |   3 +-
 .../vectorhighlight/SimpleBoundaryScanner.java  |   3 +-
 .../vectorhighlight/SimpleFieldFragList.java    |   3 +-
 .../vectorhighlight/SimpleFragListBuilder.java  |   4 +-
 .../vectorhighlight/SimpleFragmentsBuilder.java |   3 +-
 .../vectorhighlight/SingleFragListBuilder.java  |   3 +-
 .../vectorhighlight/WeightedFieldFragList.java  |   3 +-
 .../WeightedFragListBuilder.java                |   3 +-
 .../search/highlight/HighlighterPhraseTest.java |   3 +-
 .../search/highlight/HighlighterTest.java       |   3 +-
 .../lucene/search/highlight/MissesTest.java     |   3 +-
 .../highlight/OffsetLimitTokenFilterTest.java   |   3 +-
 .../search/highlight/TokenSourcesTest.java      |   3 +-
 .../custom/HighlightCustomQueryTest.java        |   4 +-
 .../TestCustomSeparatorBreakIterator.java       |   3 +-
 .../TestMultiTermHighlighting.java              |   3 +-
 .../TestPostingsHighlighter.java                |   3 +-
 .../TestPostingsHighlighterRanking.java         |   3 +-
 .../TestWholeBreakIterator.java                 |   3 +-
 .../vectorhighlight/AbstractTestCase.java       |   3 +-
 .../BreakIteratorBoundaryScannerTest.java       |   3 +-
 .../FastVectorHighlighterTest.java              |   2 +-
 .../vectorhighlight/FieldPhraseListTest.java    |   3 +-
 .../search/vectorhighlight/FieldQueryTest.java  |   3 +-
 .../vectorhighlight/FieldTermStackTest.java     |   3 +-
 .../vectorhighlight/IndexTimeSynonymTest.java   |   3 +-
 .../ScoreOrderFragmentsBuilderTest.java         |   3 +-
 .../SimpleBoundaryScannerTest.java              |   3 +-
 .../SimpleFragListBuilderTest.java              |   3 +-
 .../SimpleFragmentsBuilderTest.java             |   3 +-
 .../SingleFragListBuilderTest.java              |  11 +-
 .../WeightedFragListBuilderTest.java            |   3 +-
 .../search/join/BaseGlobalOrdinalScorer.java    |   3 +-
 .../lucene/search/join/BitSetProducer.java      |   3 +-
 .../lucene/search/join/BlockJoinSelector.java   |   3 +-
 .../lucene/search/join/CheckJoinIndex.java      |   3 +-
 .../search/join/DocValuesTermsCollector.java    |  33 +-
 .../apache/lucene/search/join/FakeScorer.java   |   3 +-
 .../search/join/GenericTermsCollector.java      |  31 +-
 .../search/join/GlobalOrdinalsCollector.java    |   3 +-
 .../lucene/search/join/GlobalOrdinalsQuery.java |   3 +-
 .../join/GlobalOrdinalsWithScoreCollector.java  |   3 +-
 .../join/GlobalOrdinalsWithScoreQuery.java      |   3 +-
 .../org/apache/lucene/search/join/JoinUtil.java |  17 +-
 .../lucene/search/join/QueryBitSetProducer.java |   3 +-
 .../apache/lucene/search/join/ScoreMode.java    |   3 +-
 .../lucene/search/join/TermsCollector.java      |   3 +-
 .../search/join/TermsIncludingScoreQuery.java   |   3 +-
 .../apache/lucene/search/join/TermsQuery.java   |   3 +-
 .../search/join/TermsWithScoreCollector.java    |   9 +-
 .../search/join/ToChildBlockJoinQuery.java      |   3 +-
 .../search/join/ToParentBlockJoinCollector.java |   3 +-
 .../join/ToParentBlockJoinIndexSearcher.java    |   3 +-
 .../search/join/ToParentBlockJoinQuery.java     |   3 +-
 .../search/join/ToParentBlockJoinSortField.java |   3 +-
 .../lucene/search/join/TestBlockJoin.java       |   3 +-
 .../search/join/TestBlockJoinSelector.java      |   4 +-
 .../search/join/TestBlockJoinSorting.java       |   3 +-
 .../search/join/TestBlockJoinValidation.java    |   3 +-
 .../lucene/search/join/TestCheckJoinIndex.java  |   3 +-
 .../apache/lucene/search/join/TestJoinUtil.java |  33 +-
 .../apache/lucene/index/memory/MemoryIndex.java | 110 ++-
 .../lucene/index/memory/TestMemoryIndex.java    |  30 +-
 .../memory/TestMemoryIndexAgainstRAMDir.java    |   3 +-
 .../apache/lucene/document/LazyDocument.java    |  17 +-
 .../apache/lucene/index/MergeReaderWrapper.java |   3 +-
 .../lucene/index/MultiPassIndexSplitter.java    |   3 +-
 .../apache/lucene/index/PKIndexSplitter.java    |   3 +-
 .../java/org/apache/lucene/index/Sorter.java    |   3 +-
 .../apache/lucene/index/SortingLeafReader.java  |   3 +-
 .../apache/lucene/index/SortingMergePolicy.java |  20 +-
 .../org/apache/lucene/misc/GetTermInfo.java     |   3 +-
 .../org/apache/lucene/misc/HighFreqTerms.java   |   3 +-
 .../org/apache/lucene/misc/IndexMergeTool.java  |  14 +-
 .../apache/lucene/misc/SweetSpotSimilarity.java |   1 -
 .../java/org/apache/lucene/misc/TermStats.java  |   3 +-
 .../search/BlockJoinComparatorSource.java       |   3 +-
 .../search/DiversifiedTopDocsCollector.java     |   4 +-
 .../EarlyTerminatingSortingCollector.java       |   3 +-
 .../apache/lucene/store/NativePosixUtil.java    |   3 +-
 .../lucene/store/NativeUnixDirectory.java       |  28 +-
 .../org/apache/lucene/store/RAFDirectory.java   |   4 +-
 .../apache/lucene/store/WindowsDirectory.java   |  27 +-
 .../apache/lucene/uninverting/DocTermOrds.java  |   1 -
 .../apache/lucene/uninverting/FieldCache.java   |   3 +-
 .../lucene/uninverting/FieldCacheImpl.java      |   3 +-
 .../uninverting/FieldCacheSanityChecker.java    |  13 +-
 .../lucene/uninverting/UninvertingReader.java   |   3 +-
 .../apache/lucene/util/fst/ListOfOutputs.java   |   3 +-
 .../util/fst/UpToTwoPositiveIntOutputs.java     |   3 +-
 .../apache/lucene/index/IndexSortingTest.java   |   3 +-
 .../org/apache/lucene/index/SorterTestBase.java |   3 +-
 .../lucene/index/SortingLeafReaderTest.java     |   3 +-
 .../lucene/index/TestBlockJoinSorter.java       |   3 +-
 .../index/TestMultiPassIndexSplitter.java       |   3 +-
 .../lucene/index/TestPKIndexSplitter.java       |  27 +-
 .../lucene/index/TestSortingMergePolicy.java    |  22 +-
 .../lucene/misc/SweetSpotSimilarityTest.java    |   3 -
 .../apache/lucene/misc/TestHighFreqTerms.java   |   3 +-
 .../search/TestDiversifiedTopDocsCollector.java |   3 +-
 .../TestEarlyTerminatingSortingCollector.java   |   3 +-
 .../apache/lucene/store/TestRAFDirectory.java   |   3 +-
 .../lucene/uninverting/TestDocTermOrds.java     |   3 +-
 .../lucene/uninverting/TestFieldCache.java      |  14 +-
 .../uninverting/TestFieldCacheReopen.java       |   3 +-
 .../TestFieldCacheSanityChecker.java            |  14 +-
 .../lucene/uninverting/TestFieldCacheSort.java  |   3 +-
 .../uninverting/TestFieldCacheSortRandom.java   |   3 +-
 .../uninverting/TestFieldCacheVsDocValues.java  |   3 +-
 .../uninverting/TestFieldCacheWithThreads.java  |   3 +-
 .../lucene/uninverting/TestNumericTerms32.java  |   3 +-
 .../lucene/uninverting/TestNumericTerms64.java  |   3 +-
 .../uninverting/TestUninvertingReader.java      |   3 +-
 .../apache/lucene/util/fst/TestFSTsMisc.java    |   4 +-
 .../apache/lucene/queries/BoostingQuery.java    |   3 +-
 .../apache/lucene/queries/CommonTermsQuery.java |   4 +-
 .../lucene/queries/CustomScoreProvider.java     |   3 +-
 .../apache/lucene/queries/CustomScoreQuery.java |   3 +-
 .../org/apache/lucene/queries/TermsQuery.java   |  55 +-
 .../lucene/queries/function/BoostedQuery.java   |   3 +-
 .../lucene/queries/function/FunctionQuery.java  |   3 +-
 .../queries/function/FunctionRangeQuery.java    |   3 +-
 .../lucene/queries/function/FunctionValues.java |   3 +-
 .../lucene/queries/function/ValueSource.java    |   3 +-
 .../queries/function/ValueSourceScorer.java     |   3 +-
 .../function/docvalues/BoolDocValues.java       |   3 +-
 .../docvalues/DocTermsIndexDocValues.java       |   1 -
 .../function/docvalues/DoubleDocValues.java     |   3 +-
 .../function/docvalues/FloatDocValues.java      |   3 +-
 .../function/docvalues/IntDocValues.java        |   3 +-
 .../function/docvalues/LongDocValues.java       |   3 +-
 .../function/docvalues/StrDocValues.java        |   3 +-
 .../function/valuesource/BoolFunction.java      |   1 -
 .../valuesource/BytesRefFieldSource.java        |   3 +-
 .../function/valuesource/ConstNumberSource.java |   1 -
 .../function/valuesource/ConstValueSource.java  |   1 -
 .../function/valuesource/DefFunction.java       |   3 +-
 .../function/valuesource/DivFloatFunction.java  |   1 -
 .../valuesource/DocFreqValueSource.java         |   1 -
 .../valuesource/DoubleConstValueSource.java     |   1 -
 .../function/valuesource/DoubleFieldSource.java |   1 -
 .../function/valuesource/DualFloatFunction.java |   1 -
 .../function/valuesource/EnumFieldSource.java   |   3 +-
 .../function/valuesource/FieldCacheSource.java  |   1 -
 .../function/valuesource/FloatFieldSource.java  |   1 -
 .../function/valuesource/IDFValueSource.java    |   1 -
 .../function/valuesource/IfFunction.java        |   1 -
 .../function/valuesource/IntFieldSource.java    |   1 -
 .../valuesource/JoinDocFreqValueSource.java     |   1 -
 .../valuesource/LinearFloatFunction.java        |   1 -
 .../valuesource/LiteralValueSource.java         |   3 +-
 .../function/valuesource/LongFieldSource.java   |   1 -
 .../function/valuesource/MaxFloatFunction.java  |   1 -
 .../function/valuesource/MinFloatFunction.java  |   1 -
 .../function/valuesource/MultiBoolFunction.java |   1 -
 .../valuesource/MultiFloatFunction.java         |   3 +-
 .../function/valuesource/MultiFunction.java     |   3 +-
 .../function/valuesource/MultiValueSource.java  |   3 +-
 .../function/valuesource/NormValueSource.java   |   1 -
 .../function/valuesource/PowFloatFunction.java  |   1 -
 .../valuesource/ProductFloatFunction.java       |   1 -
 .../function/valuesource/QueryValueSource.java  |   1 -
 .../valuesource/RangeMapFloatFunction.java      |   1 -
 .../valuesource/ReciprocalFloatFunction.java    |   1 -
 .../valuesource/ScaleFloatFunction.java         |   1 -
 .../valuesource/SimpleBoolFunction.java         |   1 -
 .../valuesource/SimpleFloatFunction.java        |   1 -
 .../function/valuesource/SingleFunction.java    |   1 -
 .../valuesource/SortedSetFieldSource.java       |   3 +-
 .../function/valuesource/SumFloatFunction.java  |   1 -
 .../SumTotalTermFreqValueSource.java            |   1 -
 .../function/valuesource/TFValueSource.java     |   3 +-
 .../valuesource/TermFreqValueSource.java        |   1 -
 .../valuesource/TotalTermFreqValueSource.java   |   1 -
 .../function/valuesource/VectorValueSource.java |   3 +-
 .../apache/lucene/queries/mlt/MoreLikeThis.java |  14 +-
 .../lucene/queries/mlt/MoreLikeThisQuery.java   |   6 +-
 .../payloads/AveragePayloadFunction.java        |   4 +-
 .../queries/payloads/MaxPayloadFunction.java    |   4 +-
 .../queries/payloads/MinPayloadFunction.java    |   3 +-
 .../queries/payloads/PayloadFunction.java       |   3 +-
 .../queries/payloads/PayloadScoreQuery.java     |   3 +-
 .../queries/payloads/SpanPayloadCheckQuery.java |   3 +-
 .../lucene/queries/BoostingQueryTest.java       |   3 +-
 .../lucene/queries/CommonTermsQueryTest.java    |   9 +-
 .../apache/lucene/queries/TermsQueryTest.java   |   3 +-
 .../queries/TestCustomScoreExplanations.java    |   3 +-
 .../lucene/queries/TestCustomScoreQuery.java    |   3 +-
 .../queries/function/FunctionTestSetup.java     |  32 +-
 .../queries/function/TestBoostedQuery.java      |  33 +-
 .../function/TestDocValuesFieldSources.java     |   3 +-
 .../queries/function/TestFieldScoreQuery.java   |   3 +-
 .../function/TestFunctionQueryExplanations.java |   3 +-
 .../queries/function/TestFunctionQuerySort.java |   3 +-
 .../function/TestFunctionRangeQuery.java        |   3 +-
 .../function/TestLongNormValueSource.java       |   3 +-
 .../function/TestSortedSetFieldSource.java      |   3 +-
 .../queries/function/TestValueSources.java      |   3 +-
 .../lucene/queries/mlt/TestMoreLikeThis.java    |   3 +-
 .../lucene/queries/payloads/PayloadHelper.java  |   3 +-
 .../queries/payloads/TestPayloadCheckQuery.java |   3 +-
 .../payloads/TestPayloadExplanations.java       |   3 +-
 .../queries/payloads/TestPayloadScoreQuery.java |   3 +-
 .../queries/payloads/TestPayloadSpans.java      |  14 +-
 .../queries/payloads/TestPayloadTermQuery.java  |   3 +-
 .../analyzing/AnalyzingQueryParser.java         |   3 +-
 .../queryparser/classic/FastCharStream.java     |   5 +-
 .../classic/MultiFieldQueryParser.java          |   3 +-
 .../queryparser/classic/QueryParserBase.java    |   1 -
 .../complexPhrase/ComplexPhraseQueryParser.java |   3 +-
 .../queryparser/ext/ExtendableQueryParser.java  |   3 +-
 .../lucene/queryparser/ext/ExtensionQuery.java  |   7 +-
 .../lucene/queryparser/ext/Extensions.java      |   4 +-
 .../lucene/queryparser/ext/ParserExtension.java |   3 +-
 .../flexible/core/QueryNodeError.java           |   3 +-
 .../flexible/core/QueryNodeException.java       |   3 +-
 .../flexible/core/QueryNodeParseException.java  |   3 +-
 .../flexible/core/QueryParserHelper.java        |  16 +-
 .../flexible/core/builders/QueryBuilder.java    |   8 +-
 .../core/builders/QueryTreeBuilder.java         |   3 +-
 .../core/config/AbstractQueryConfig.java        |   3 +-
 .../flexible/core/config/ConfigurationKey.java  |   3 +-
 .../flexible/core/config/FieldConfig.java       |   3 +-
 .../core/config/FieldConfigListener.java        |   3 +-
 .../core/config/QueryConfigHandler.java         |   3 +-
 .../core/messages/QueryParserMessages.java      |   3 +-
 .../flexible/core/nodes/AndQueryNode.java       |   3 +-
 .../flexible/core/nodes/AnyQueryNode.java       |   3 +-
 .../flexible/core/nodes/BooleanQueryNode.java   |   3 +-
 .../flexible/core/nodes/BoostQueryNode.java     |   3 +-
 .../flexible/core/nodes/DeletedQueryNode.java   |   7 +-
 .../flexible/core/nodes/FieldQueryNode.java     |   3 +-
 .../core/nodes/FieldValuePairQueryNode.java     |   3 +-
 .../flexible/core/nodes/FieldableNode.java      |   3 +-
 .../flexible/core/nodes/FuzzyQueryNode.java     |   3 +-
 .../flexible/core/nodes/GroupQueryNode.java     |   3 +-
 .../core/nodes/MatchAllDocsQueryNode.java       |   3 +-
 .../core/nodes/MatchNoDocsQueryNode.java        |   3 +-
 .../flexible/core/nodes/ModifierQueryNode.java  |   3 +-
 .../core/nodes/NoTokenFoundQueryNode.java       |   3 +-
 .../flexible/core/nodes/OpaqueQueryNode.java    |   3 +-
 .../flexible/core/nodes/OrQueryNode.java        |   3 +-
 .../flexible/core/nodes/PathQueryNode.java      |   3 +-
 .../core/nodes/PhraseSlopQueryNode.java         |   3 +-
 .../flexible/core/nodes/ProximityQueryNode.java |   3 +-
 .../flexible/core/nodes/QueryNode.java          |   3 +-
 .../flexible/core/nodes/QueryNodeImpl.java      |   3 +-
 .../core/nodes/QuotedFieldQueryNode.java        |   3 +-
 .../flexible/core/nodes/RangeQueryNode.java     |   7 +-
 .../flexible/core/nodes/SlopQueryNode.java      |   3 +-
 .../flexible/core/nodes/TextableQueryNode.java  |  27 +-
 .../core/nodes/TokenizedPhraseQueryNode.java    |   3 +-
 .../flexible/core/nodes/ValueQueryNode.java     |  27 +-
 .../flexible/core/parser/EscapeQuerySyntax.java |   3 +-
 .../flexible/core/parser/SyntaxParser.java      |   3 +-
 .../NoChildOptimizationQueryNodeProcessor.java  |   3 +-
 .../core/processors/QueryNodeProcessor.java     |   3 +-
 .../core/processors/QueryNodeProcessorImpl.java |   3 +-
 .../processors/QueryNodeProcessorPipeline.java  |   3 +-
 .../RemoveDeletedQueryNodesProcessor.java       |   3 +-
 .../flexible/core/util/QueryNodeOperation.java  |   3 +-
 .../flexible/core/util/StringUtils.java         |   3 +-
 .../core/util/UnescapedCharSequence.java        |   7 +-
 .../queryparser/flexible/messages/Message.java  |   3 +-
 .../flexible/messages/MessageImpl.java          |   3 +-
 .../queryparser/flexible/messages/NLS.java      |   3 +-
 .../flexible/messages/NLSException.java         |   3 +-
 .../precedence/PrecedenceQueryParser.java       |   3 +-
 .../BooleanModifiersQueryNodeProcessor.java     |   3 +-
 .../PrecedenceQueryNodeProcessorPipeline.java   |   3 +-
 .../CommonQueryParserConfiguration.java         |  23 +-
 .../flexible/standard/QueryParserUtil.java      |   3 +-
 .../flexible/standard/StandardQueryParser.java  |   3 +-
 .../standard/builders/AnyQueryNodeBuilder.java  |   3 +-
 .../builders/BooleanQueryNodeBuilder.java       |   3 +-
 .../builders/BoostQueryNodeBuilder.java         |   3 +-
 .../builders/DummyQueryNodeBuilder.java         |   3 +-
 .../builders/FieldQueryNodeBuilder.java         |   3 +-
 .../builders/FuzzyQueryNodeBuilder.java         |   3 +-
 .../builders/GroupQueryNodeBuilder.java         |   3 +-
 .../builders/MatchAllDocsQueryNodeBuilder.java  |   3 +-
 .../builders/MatchNoDocsQueryNodeBuilder.java   |   3 +-
 .../builders/ModifierQueryNodeBuilder.java      |   3 +-
 .../builders/MultiPhraseQueryNodeBuilder.java   |   3 +-
 .../builders/NumericRangeQueryNodeBuilder.java  |   3 +-
 .../builders/PhraseQueryNodeBuilder.java        |   3 +-
 .../PrefixWildcardQueryNodeBuilder.java         |   3 +-
 .../builders/RegexpQueryNodeBuilder.java        |   3 +-
 .../standard/builders/SlopQueryNodeBuilder.java |   3 +-
 .../StandardBooleanQueryNodeBuilder.java        |   3 +-
 .../standard/builders/StandardQueryBuilder.java |   3 +-
 .../builders/StandardQueryTreeBuilder.java      |   3 +-
 .../builders/TermRangeQueryNodeBuilder.java     |   3 +-
 .../builders/WildcardQueryNodeBuilder.java      |   3 +-
 .../config/FieldBoostMapFCListener.java         |   3 +-
 .../config/FieldDateResolutionFCListener.java   |   3 +-
 .../flexible/standard/config/FuzzyConfig.java   |   3 +-
 .../standard/config/NumberDateFormat.java       |   3 +-
 .../flexible/standard/config/NumericConfig.java |   3 +-
 .../config/NumericFieldConfigListener.java      |   3 +-
 .../config/StandardQueryConfigHandler.java      |   3 +-
 .../standard/nodes/AbstractRangeQueryNode.java  |   3 +-
 .../standard/nodes/BooleanModifierNode.java     |   3 +-
 .../standard/nodes/MultiPhraseQueryNode.java    |   3 +-
 .../standard/nodes/NumericQueryNode.java        |   3 +-
 .../standard/nodes/NumericRangeQueryNode.java   |  27 +-
 .../standard/nodes/PrefixWildcardQueryNode.java |   7 +-
 .../standard/nodes/RegexpQueryNode.java         |   3 +-
 .../nodes/StandardBooleanQueryNode.java         |   3 +-
 .../standard/nodes/TermRangeQueryNode.java      |   3 +-
 .../standard/nodes/WildcardQueryNode.java       |   3 +-
 .../standard/parser/EscapeQuerySyntaxImpl.java  |   3 +-
 .../standard/parser/FastCharStream.java         |   5 +-
 .../AllowLeadingWildcardProcessor.java          |   3 +-
 .../processors/AnalyzerQueryNodeProcessor.java  |   3 +-
 .../BooleanQuery2ModifierNodeProcessor.java     |   3 +-
 ...ngleChildOptimizationQueryNodeProcessor.java |   3 +-
 .../processors/BoostQueryNodeProcessor.java     |   3 +-
 .../DefaultPhraseSlopQueryNodeProcessor.java    |   3 +-
 .../processors/FuzzyQueryNodeProcessor.java     |   3 +-
 ...owercaseExpandedTermsQueryNodeProcessor.java |   3 +-
 .../MatchAllDocsQueryNodeProcessor.java         |   3 +-
 .../MultiFieldQueryNodeProcessor.java           |   3 +-
 .../MultiTermRewriteMethodProcessor.java        |   3 +-
 .../processors/NumericQueryNodeProcessor.java   |   3 +-
 .../NumericRangeQueryNodeProcessor.java         |   3 +-
 .../processors/OpenRangeQueryNodeProcessor.java |   3 +-
 .../PhraseSlopQueryNodeProcessor.java           |   3 +-
 .../RemoveEmptyNonLeafQueryNodeProcessor.java   |   3 +-
 .../StandardQueryNodeProcessorPipeline.java     |   3 +-
 .../processors/TermRangeQueryNodeProcessor.java |   3 +-
 .../processors/WildcardQueryNodeProcessor.java  |   3 +-
 .../queryparser/simple/SimpleQueryParser.java   |   3 +-
 .../surround/parser/FastCharStream.java         |   3 +-
 .../queryparser/surround/query/AndQuery.java    |   4 +-
 .../surround/query/BasicQueryFactory.java       |   3 +-
 .../surround/query/ComposedQuery.java           |   3 +-
 .../surround/query/DistanceQuery.java           |   3 +-
 .../surround/query/DistanceRewriteQuery.java    |   3 +-
 .../surround/query/DistanceSubQuery.java        |   3 +-
 .../queryparser/surround/query/FieldsQuery.java |   3 +-
 .../queryparser/surround/query/NotQuery.java    |   3 +-
 .../queryparser/surround/query/OrQuery.java     |   3 +-
 .../surround/query/RewriteQuery.java            |   2 +-
 .../queryparser/surround/query/SimpleTerm.java  |   3 +-
 .../surround/query/SimpleTermRewriteQuery.java  |   2 +-
 .../surround/query/SpanNearClauseFactory.java   |   3 +-
 .../surround/query/SrndBooleanQuery.java        |   3 +-
 .../surround/query/SrndPrefixQuery.java         |   3 +-
 .../queryparser/surround/query/SrndQuery.java   |   3 +-
 .../surround/query/SrndTermQuery.java           |   3 +-
 .../surround/query/SrndTruncQuery.java          |   3 +-
 .../surround/query/TooManyBasicQueries.java     |   3 +-
 .../lucene/queryparser/xml/CoreParser.java      |  27 +-
 .../xml/CorePlusExtensionsParser.java           |  11 +-
 .../queryparser/xml/CorePlusQueriesParser.java  |  13 +-
 .../apache/lucene/queryparser/xml/DOMUtils.java |  19 +-
 .../lucene/queryparser/xml/ParserException.java |   5 +-
 .../lucene/queryparser/xml/QueryBuilder.java    |   8 +-
 .../queryparser/xml/QueryBuilderFactory.java    |  15 +-
 .../queryparser/xml/QueryTemplateManager.java   |  33 +-
 .../xml/builders/BooleanQueryBuilder.java       |  27 +-
 .../xml/builders/BoostingQueryBuilder.java      |  17 +-
 .../xml/builders/BoostingTermBuilder.java       |  23 +-
 .../xml/builders/ConstantScoreQueryBuilder.java |  19 +-
 .../builders/DisjunctionMaxQueryBuilder.java    |   3 +-
 .../xml/builders/FuzzyLikeThisQueryBuilder.java |  25 +-
 .../LegacyNumericRangeQueryBuilder.java         |   3 +-
 .../xml/builders/LikeThisQueryBuilder.java      |  31 +-
 .../xml/builders/MatchAllDocsQueryBuilder.java  |  13 +-
 .../xml/builders/RangeQueryBuilder.java         |   6 +-
 .../xml/builders/SpanBuilderBase.java           |  11 +-
 .../xml/builders/SpanFirstBuilder.java          |  15 +-
 .../xml/builders/SpanNearBuilder.java           |  23 +-
 .../xml/builders/SpanNotBuilder.java            |  15 +-
 .../queryparser/xml/builders/SpanOrBuilder.java |  23 +-
 .../xml/builders/SpanOrTermsBuilder.java        |  33 +-
 .../xml/builders/SpanQueryBuilder.java          |   3 +-
 .../xml/builders/SpanQueryBuilderFactory.java   |  17 +-
 .../xml/builders/SpanTermBuilder.java           |  17 +-
 .../xml/builders/TermQueryBuilder.java          |  19 +-
 .../xml/builders/TermsQueryBuilder.java         |  33 +-
 .../xml/builders/UserInputQueryBuilder.java     |  23 +-
 .../analyzing/TestAnalyzingQueryParser.java     |   3 +-
 .../queryparser/classic/TestMultiAnalyzer.java  |   3 +-
 .../classic/TestMultiFieldQueryParser.java      |   3 +-
 .../classic/TestMultiPhraseQueryParsing.java    |   3 +-
 .../queryparser/classic/TestQueryParser.java    |   3 +-
 .../complexPhrase/TestComplexPhraseQuery.java   |   3 +-
 .../lucene/queryparser/ext/ExtensionStub.java   |  13 +-
 .../ext/TestExtendableQueryParser.java          |   3 +-
 .../lucene/queryparser/ext/TestExtensions.java  |   3 +-
 .../core/builders/TestQueryTreeBuilder.java     |   3 +-
 .../flexible/core/nodes/TestQueryNode.java      |   3 +-
 .../flexible/messages/MessagesTestBundle.java   |  27 +-
 .../queryparser/flexible/messages/TestNLS.java  |   3 +-
 .../precedence/TestPrecedenceQueryParser.java   |   3 +-
 .../flexible/spans/SpanOrQueryNodeBuilder.java  |   3 +-
 .../spans/SpanTermQueryNodeBuilder.java         |   3 +-
 .../flexible/spans/SpansQueryConfigHandler.java |   3 +-
 .../flexible/spans/SpansQueryTreeBuilder.java   |   3 +-
 .../spans/SpansValidatorQueryNodeProcessor.java |   3 +-
 .../flexible/spans/TestSpanQueryParser.java     |   3 +-
 .../spans/TestSpanQueryParserSimpleSample.java  |   3 +-
 .../flexible/spans/UniqueFieldAttribute.java    |   3 +-
 .../spans/UniqueFieldAttributeImpl.java         |   3 +-
 .../spans/UniqueFieldQueryNodeProcessor.java    |   3 +-
 .../standard/TestMultiAnalyzerQPHelper.java     |   3 +-
 .../standard/TestMultiFieldQPHelper.java        |   3 +-
 .../standard/TestNumericQueryParser.java        |   3 +-
 .../flexible/standard/TestQPHelper.java         |   3 +-
 .../flexible/standard/TestStandardQP.java       |   3 +-
 .../simple/TestSimpleQueryParser.java           |   3 +-
 .../surround/query/BooleanQueryTst.java         |   3 +-
 .../surround/query/ExceptionQueryTst.java       |   3 +-
 .../surround/query/SingleFieldTestDb.java       |   3 +-
 .../surround/query/SrndQueryTest.java           |   3 +-
 .../surround/query/Test01Exceptions.java        |   3 +-
 .../surround/query/Test02Boolean.java           |   3 +-
 .../surround/query/Test03Distance.java          |   3 +-
 .../queryparser/util/QueryParserTestBase.java   |   3 +-
 .../lucene/queryparser/xml/TestCoreParser.java  |   9 +-
 .../xml/TestCorePlusExtensionsParser.java       |   9 +-
 .../xml/TestCorePlusQueriesParser.java          |   9 +-
 .../xml/TestQueryTemplateManager.java           |   3 +-
 .../builders/TestNumericRangeQueryBuilder.java  |   3 +-
 .../IndexAndTaxonomyReplicationHandler.java     |   3 +-
 .../replicator/IndexAndTaxonomyRevision.java    |   3 +-
 .../replicator/IndexInputInputStream.java       |   3 +-
 .../replicator/IndexReplicationHandler.java     |   3 +-
 .../apache/lucene/replicator/IndexRevision.java |   3 +-
 .../lucene/replicator/LocalReplicator.java      |   3 +-
 .../replicator/PerSessionDirectoryFactory.java  |   3 +-
 .../lucene/replicator/ReplicationClient.java    |   3 +-
 .../apache/lucene/replicator/Replicator.java    |   3 +-
 .../org/apache/lucene/replicator/Revision.java  |   3 +-
 .../apache/lucene/replicator/RevisionFile.java  |   3 +-
 .../replicator/SessionExpiredException.java     |   3 +-
 .../apache/lucene/replicator/SessionToken.java  |   3 +-
 .../lucene/replicator/http/HttpClientBase.java  |   3 +-
 .../lucene/replicator/http/HttpReplicator.java  |   3 +-
 .../replicator/http/ReplicationService.java     |   3 +-
 .../lucene/replicator/nrt/ReplicaNode.java      |   5 +
 .../IndexAndTaxonomyReplicationClientTest.java  |  20 +-
 .../IndexAndTaxonomyRevisionTest.java           |  11 +-
 .../replicator/IndexReplicationClientTest.java  |   3 +-
 .../lucene/replicator/IndexRevisionTest.java    |   7 +-
 .../lucene/replicator/LocalReplicatorTest.java  |  93 +-
 .../lucene/replicator/ReplicatorTestCase.java   |   3 +-
 .../lucene/replicator/SessionTokenTest.java     |   3 +-
 .../replicator/http/HttpReplicatorTest.java     |   8 +-
 .../replicator/http/ReplicationServlet.java     |   3 +-
 .../replicator/nrt/TestNRTReplication.java      |   6 +-
 lucene/sandbox/build.xml                        |  25 +
 .../idversion/IDVersionPostingsFormat.java      |   3 +-
 .../idversion/IDVersionPostingsReader.java      |   3 +-
 .../idversion/IDVersionPostingsWriter.java      |   3 +-
 .../idversion/IDVersionSegmentTermsEnum.java    |   3 +-
 .../IDVersionSegmentTermsEnumFrame.java         |   3 +-
 .../codecs/idversion/IDVersionTermState.java    |   3 +-
 .../lucene/codecs/idversion/SingleDocsEnum.java |   3 +-
 .../codecs/idversion/SinglePostingsEnum.java    |   3 +-
 .../idversion/VersionBlockTreeTermsReader.java  |   3 +-
 .../idversion/VersionBlockTreeTermsWriter.java  |   3 +-
 .../codecs/idversion/VersionFieldReader.java    |   3 +-
 .../apache/lucene/document/GeoPointField.java   | 132 ---
 .../org/apache/lucene/document/LatLonPoint.java |   5 +-
 .../lucene/payloads/PayloadSpanCollector.java   |   3 +-
 .../apache/lucene/payloads/PayloadSpanUtil.java |   3 +-
 .../sandbox/queries/FuzzyLikeThisQuery.java     |   3 +-
 .../lucene/sandbox/queries/SlowFuzzyQuery.java  |   3 +-
 .../sandbox/queries/SlowFuzzyTermsEnum.java     |   3 +-
 .../lucene/search/DocValuesNumbersQuery.java    |   3 +-
 .../lucene/search/DocValuesRangeQuery.java      |   3 +-
 .../lucene/search/DocValuesTermsQuery.java      |   3 +-
 .../apache/lucene/search/GeoBoundingBox.java    |  47 -
 .../lucene/search/GeoPointDistanceQuery.java    | 171 ----
 .../search/GeoPointDistanceQueryImpl.java       | 132 ---
 .../search/GeoPointDistanceRangeQuery.java      | 103 --
 .../lucene/search/GeoPointInBBoxQuery.java      | 153 ---
 .../lucene/search/GeoPointInBBoxQueryImpl.java  | 163 ---
 .../lucene/search/GeoPointInPolygonQuery.java   | 195 ----
 .../apache/lucene/search/GeoPointTermQuery.java |  71 --
 .../GeoPointTermQueryConstantScoreWrapper.java  | 130 ---
 .../apache/lucene/search/GeoPointTermsEnum.java | 250 -----
 .../lucene/search/PointInPolygonQuery.java      |  11 +-
 .../apache/lucene/search/PointInRectQuery.java  |   5 +-
 .../lucene/search/TermAutomatonQuery.java       |   3 +-
 .../lucene/search/TermAutomatonScorer.java      |   3 +-
 .../search/TokenStreamToTermAutomatonQuery.java |   3 +-
 .../apache/lucene/util/GeoDistanceUtils.java    | 216 ----
 .../org/apache/lucene/util/GeoHashUtils.java    | 274 -----
 .../apache/lucene/util/GeoProjectionUtils.java  | 438 --------
 .../java/org/apache/lucene/util/GeoRect.java    |  67 --
 .../apache/lucene/util/GeoRelationUtils.java    | 428 --------
 .../java/org/apache/lucene/util/GeoUtils.java   | 238 -----
 .../java/org/apache/lucene/util/package.html    |  28 -
 .../codecs/idversion/StringAndPayloadField.java |   7 +-
 .../idversion/TestIDVersionPostingsFormat.java  |   3 +-
 .../lucene/payloads/TestPayloadSpanUtil.java    |   3 +-
 .../sandbox/queries/FuzzyLikeThisQueryTest.java |   3 +-
 .../sandbox/queries/TestSlowFuzzyQuery.java     |   3 +-
 .../sandbox/queries/TestSlowFuzzyQuery2.java    |   3 +-
 .../search/TestDocValuesNumbersQuery.java       |   3 +-
 .../lucene/search/TestDocValuesRangeQuery.java  |   3 +-
 .../lucene/search/TestDocValuesTermsQuery.java  |   3 +-
 .../search/TestFieldCacheTermsFilter.java       |   3 +-
 .../apache/lucene/search/TestGeoPointQuery.java | 351 -------
 .../lucene/search/TestLatLonPointQueries.java   |  61 +-
 .../lucene/search/TestTermAutomatonQuery.java   |   3 +-
 .../lucene/util/BaseGeoPointTestCase.java       | 782 ---------------
 .../org/apache/lucene/util/TestGeoUtils.java    | 537 ----------
 .../apache/lucene/spatial/SpatialStrategy.java  |   3 +-
 .../spatial/bbox/BBoxSimilarityValueSource.java |   3 +-
 .../lucene/spatial/bbox/BBoxStrategy.java       |   3 +-
 .../lucene/spatial/bbox/BBoxValueSource.java    |   3 +-
 .../composite/CompositeSpatialStrategy.java     |   3 +-
 .../spatial/composite/CompositeVerifyQuery.java |   3 +-
 .../composite/IntersectsRPTVerifyQuery.java     |   3 +-
 .../lucene/spatial/document/GeoPointField.java  | 250 +++++
 .../spatial/document/GeoPointTokenStream.java   | 233 +++++
 .../lucene/spatial/document/package-info.java   |  21 +
 .../spatial/prefix/AbstractPrefixTreeQuery.java |   3 +-
 .../prefix/AbstractVisitingPrefixTreeQuery.java |   3 +-
 .../prefix/BytesRefIteratorTokenStream.java     |   3 +-
 .../spatial/prefix/CellToBytesRefIterator.java  |   3 +-
 .../spatial/prefix/ContainsPrefixTreeQuery.java |   3 +-
 .../spatial/prefix/HeatmapFacetCounter.java     |   3 +-
 .../prefix/IntersectsPrefixTreeQuery.java       |   3 +-
 .../prefix/NumberRangePrefixTreeStrategy.java   |   3 +-
 .../PointPrefixTreeFieldCacheProvider.java      |   3 +-
 .../spatial/prefix/PrefixTreeFacetCounter.java  |   3 +-
 .../spatial/prefix/PrefixTreeStrategy.java      |   3 +-
 .../prefix/RecursivePrefixTreeStrategy.java     |   3 +-
 .../prefix/TermQueryPrefixTreeStrategy.java     |   3 +-
 .../spatial/prefix/WithinPrefixTreeQuery.java   |   3 +-
 .../apache/lucene/spatial/prefix/tree/Cell.java |   3 +-
 .../spatial/prefix/tree/CellIterator.java       |   3 +-
 .../prefix/tree/DateRangePrefixTree.java        |   3 +-
 .../spatial/prefix/tree/FilterCellIterator.java |   3 +-
 .../spatial/prefix/tree/GeohashPrefixTree.java  |   3 +-
 .../lucene/spatial/prefix/tree/LegacyCell.java  |   3 +-
 .../spatial/prefix/tree/LegacyPrefixTree.java   |   3 +-
 .../prefix/tree/NumberRangePrefixTree.java      |   3 +-
 .../prefix/tree/PackedQuadPrefixTree.java       |   3 +-
 .../spatial/prefix/tree/QuadPrefixTree.java     |   3 +-
 .../prefix/tree/SingletonCellIterator.java      |   3 +-
 .../spatial/prefix/tree/SpatialPrefixTree.java  |   3 +-
 .../prefix/tree/SpatialPrefixTreeFactory.java   |   1 -
 .../spatial/prefix/tree/TreeCellIterator.java   |   3 +-
 .../lucene/spatial/query/SpatialArgs.java       |   3 +-
 .../lucene/spatial/query/SpatialArgsParser.java |   3 +-
 .../lucene/spatial/query/SpatialOperation.java  |   3 +-
 .../query/UnsupportedSpatialOperation.java      |   3 +-
 .../lucene/spatial/search/GeoBoundingBox.java   |  53 +
 .../spatial/search/GeoPointDistanceQuery.java   | 189 ++++
 .../search/GeoPointDistanceQueryImpl.java       | 107 ++
 .../search/GeoPointDistanceRangeQuery.java      | 116 +++
 .../spatial/search/GeoPointInBBoxQuery.java     | 173 ++++
 .../spatial/search/GeoPointInBBoxQueryImpl.java | 156 +++
 .../spatial/search/GeoPointInPolygonQuery.java  | 153 +++
 .../search/GeoPointInPolygonQueryImpl.java      | 103 ++
 .../spatial/search/GeoPointMultiTermQuery.java  | 166 ++++
 .../search/GeoPointNumericTermsEnum.java        | 161 +++
 .../spatial/search/GeoPointPrefixTermsEnum.java | 237 +++++
 .../GeoPointTermQueryConstantScoreWrapper.java  | 141 +++
 .../spatial/search/GeoPointTermsEnum.java       | 140 +++
 .../lucene/spatial/search/package-info.java     |  21 +
 .../serialized/SerializedDVStrategy.java        |   3 +-
 .../lucene/spatial/spatial4j/Geo3dShape.java    |   3 +-
 .../spatial/util/CachingDoubleValueSource.java  |   1 -
 .../util/DistanceToShapeValueSource.java        |   3 +-
 .../lucene/spatial/util/GeoDistanceUtils.java   | 223 +++++
 .../lucene/spatial/util/GeoEncodingUtils.java   | 157 +++
 .../lucene/spatial/util/GeoHashUtils.java       | 283 ++++++
 .../lucene/spatial/util/GeoProjectionUtils.java | 465 +++++++++
 .../org/apache/lucene/spatial/util/GeoRect.java |  66 ++
 .../lucene/spatial/util/GeoRelationUtils.java   | 520 ++++++++++
 .../apache/lucene/spatial/util/GeoUtils.java    | 187 ++++
 .../spatial/util/ShapeAreaValueSource.java      |   3 +-
 .../lucene/spatial/util/ShapeFieldCache.java    |   1 -
 .../ShapeFieldCacheDistanceValueSource.java     |   3 +-
 .../spatial/util/ShapeFieldCacheProvider.java   |   1 -
 .../spatial/util/ShapePredicateValueSource.java |   3 +-
 .../spatial/vector/DistanceValueSource.java     |   3 +-
 .../spatial/vector/PointVectorStrategy.java     |   3 +-
 .../lucene/spatial/DistanceStrategyTest.java    |   3 +-
 .../apache/lucene/spatial/PortedSolr3Test.java  |   3 +-
 .../lucene/spatial/QueryEqualsHashCodeTest.java |   3 +-
 .../apache/lucene/spatial/SpatialArgsTest.java  |   3 +-
 .../apache/lucene/spatial/SpatialExample.java   |   3 +-
 .../lucene/spatial/SpatialMatchConcern.java     |   1 -
 .../apache/lucene/spatial/SpatialTestCase.java  |   3 +-
 .../apache/lucene/spatial/SpatialTestData.java  |   3 +-
 .../apache/lucene/spatial/SpatialTestQuery.java |   3 +-
 .../apache/lucene/spatial/StrategyTestCase.java |   5 +-
 .../lucene/spatial/TestTestFramework.java       |   3 +-
 .../lucene/spatial/bbox/TestBBoxStrategy.java   |   3 +-
 .../composite/CompositeStrategyTest.java        |   3 +-
 .../prefix/CellToBytesRefIterator50.java        |   3 +-
 .../spatial/prefix/DateNRStrategyTest.java      |   3 +-
 .../spatial/prefix/HeatmapFacetCounterTest.java |   3 +-
 .../lucene/spatial/prefix/JtsPolygonTest.java   |   3 +-
 .../spatial/prefix/NumberRangeFacetsTest.java   |   3 +-
 .../RandomSpatialOpFuzzyPrefixTree50Test.java   |   3 +-
 .../RandomSpatialOpFuzzyPrefixTreeTest.java     |   3 +-
 .../prefix/RandomSpatialOpStrategyTestCase.java |   3 +-
 .../prefix/TestRecursivePrefixTreeStrategy.java |   3 +-
 .../prefix/TestTermQueryPrefixGridStrategy.java |   3 +-
 .../prefix/tree/DateRangePrefixTreeTest.java    |   3 +-
 .../prefix/tree/SpatialPrefixTreeTest.java      |   3 +-
 .../spatial/query/SpatialArgsParserTest.java    |   3 +-
 .../spatial/search/TestGeoPointQuery.java       | 404 ++++++++
 .../serialized/SerializedStrategyTest.java      |   3 +-
 .../lucene/spatial/spatial4j/Geo3dRptTest.java  |   3 +-
 .../Geo3dShapeRectRelationTestCase.java         |   3 +-
 .../Geo3dShapeSphereModelRectRelationTest.java  |   3 +-
 .../Geo3dShapeWGS84ModelRectRelationTest.java   |   3 +-
 .../spatial4j/RandomizedShapeTestCase.java      |   4 +-
 .../spatial/spatial4j/geo3d/GeoPointTest.java   |   3 +-
 .../spatial/util/BaseGeoPointTestCase.java      | 760 ++++++++++++++
 .../lucene/spatial/util/TestGeoUtils.java       | 565 +++++++++++
 .../spatial/vector/TestPointVectorStrategy.java |   3 +-
 .../org/apache/lucene/geo3d/ArcDistance.java    |   3 +-
 .../apache/lucene/geo3d/BasePlanetObject.java   |   3 +-
 .../org/apache/lucene/geo3d/BaseXYZSolid.java   |   3 +-
 .../java/org/apache/lucene/geo3d/Bounds.java    |   3 +-
 .../org/apache/lucene/geo3d/DistanceStyle.java  |   3 +-
 .../org/apache/lucene/geo3d/Geo3DPoint.java     |   3 +-
 .../java/org/apache/lucene/geo3d/Geo3DUtil.java |   3 +-
 .../java/org/apache/lucene/geo3d/GeoArea.java   |   3 +-
 .../org/apache/lucene/geo3d/GeoAreaFactory.java |   3 +-
 .../java/org/apache/lucene/geo3d/GeoBBox.java   |   3 +-
 .../org/apache/lucene/geo3d/GeoBBoxFactory.java |   3 +-
 .../org/apache/lucene/geo3d/GeoBaseBBox.java    |   3 +-
 .../org/apache/lucene/geo3d/GeoBaseCircle.java  |   3 +-
 .../lucene/geo3d/GeoBaseDistanceShape.java      |   3 +-
 .../lucene/geo3d/GeoBaseMembershipShape.java    |   3 +-
 .../org/apache/lucene/geo3d/GeoBasePolygon.java |   3 +-
 .../org/apache/lucene/geo3d/GeoBaseShape.java   |   3 +-
 .../java/org/apache/lucene/geo3d/GeoCircle.java |   3 +-
 .../apache/lucene/geo3d/GeoCircleFactory.java   |   3 +-
 .../geo3d/GeoCompositeMembershipShape.java      |   3 +-
 .../lucene/geo3d/GeoCompositePolygon.java       |   3 +-
 .../apache/lucene/geo3d/GeoConvexPolygon.java   |   3 +-
 .../geo3d/GeoDegenerateHorizontalLine.java      |   3 +-
 .../lucene/geo3d/GeoDegenerateLatitudeZone.java |   3 +-
 .../geo3d/GeoDegenerateLongitudeSlice.java      |   3 +-
 .../apache/lucene/geo3d/GeoDegeneratePoint.java |   5 +-
 .../lucene/geo3d/GeoDegenerateVerticalLine.java |   3 +-
 .../org/apache/lucene/geo3d/GeoDistance.java    |   3 +-
 .../apache/lucene/geo3d/GeoDistanceShape.java   |   3 +-
 .../apache/lucene/geo3d/GeoLatitudeZone.java    |   3 +-
 .../apache/lucene/geo3d/GeoLongitudeSlice.java  |   3 +-
 .../apache/lucene/geo3d/GeoMembershipShape.java |   3 +-
 .../lucene/geo3d/GeoNorthLatitudeZone.java      |   3 +-
 .../apache/lucene/geo3d/GeoNorthRectangle.java  |   3 +-
 .../apache/lucene/geo3d/GeoOutsideDistance.java |   3 +-
 .../java/org/apache/lucene/geo3d/GeoPath.java   |   3 +-
 .../java/org/apache/lucene/geo3d/GeoPoint.java  |   5 +-
 .../org/apache/lucene/geo3d/GeoPolygon.java     |   3 +-
 .../apache/lucene/geo3d/GeoPolygonFactory.java  |   3 +-
 .../org/apache/lucene/geo3d/GeoRectangle.java   |   3 +-
 .../java/org/apache/lucene/geo3d/GeoShape.java  |   3 +-
 .../org/apache/lucene/geo3d/GeoSizeable.java    |   3 +-
 .../lucene/geo3d/GeoSouthLatitudeZone.java      |   3 +-
 .../apache/lucene/geo3d/GeoSouthRectangle.java  |   3 +-
 .../apache/lucene/geo3d/GeoStandardCircle.java  |   3 +-
 .../geo3d/GeoWideDegenerateHorizontalLine.java  |   3 +-
 .../lucene/geo3d/GeoWideLongitudeSlice.java     |   3 +-
 .../lucene/geo3d/GeoWideNorthRectangle.java     |   3 +-
 .../apache/lucene/geo3d/GeoWideRectangle.java   |   3 +-
 .../lucene/geo3d/GeoWideSouthRectangle.java     |   3 +-
 .../java/org/apache/lucene/geo3d/GeoWorld.java  |   3 +-
 .../org/apache/lucene/geo3d/LatLonBounds.java   |   3 +-
 .../org/apache/lucene/geo3d/LinearDistance.java |   3 +-
 .../lucene/geo3d/LinearSquaredDistance.java     |   3 +-
 .../org/apache/lucene/geo3d/Membership.java     |   3 +-
 .../org/apache/lucene/geo3d/NormalDistance.java |   3 +-
 .../lucene/geo3d/NormalSquaredDistance.java     |   3 +-
 .../src/java/org/apache/lucene/geo3d/Plane.java |   3 +-
 .../org/apache/lucene/geo3d/PlanetModel.java    |   3 +-
 .../lucene/geo3d/PointInGeo3DShapeQuery.java    |   3 +-
 .../org/apache/lucene/geo3d/SidedPlane.java     |   3 +-
 .../apache/lucene/geo3d/StandardXYZSolid.java   |   3 +-
 .../src/java/org/apache/lucene/geo3d/Tools.java |   3 +-
 .../java/org/apache/lucene/geo3d/Vector.java    |   3 +-
 .../java/org/apache/lucene/geo3d/XYZBounds.java |   3 +-
 .../java/org/apache/lucene/geo3d/XYZSolid.java  |   3 +-
 .../apache/lucene/geo3d/XYZSolidFactory.java    |   3 +-
 .../java/org/apache/lucene/geo3d/XYdZSolid.java |   3 +-
 .../java/org/apache/lucene/geo3d/XdYZSolid.java |   3 +-
 .../org/apache/lucene/geo3d/XdYdZSolid.java     |   3 +-
 .../java/org/apache/lucene/geo3d/dXYZSolid.java |   3 +-
 .../org/apache/lucene/geo3d/dXYdZSolid.java     |   3 +-
 .../org/apache/lucene/geo3d/dXdYZSolid.java     |   3 +-
 .../org/apache/lucene/geo3d/dXdYdZSolid.java    |   3 +-
 .../org/apache/lucene/geo3d/GeoBBoxTest.java    |   3 +-
 .../org/apache/lucene/geo3d/GeoCircleTest.java  |   3 +-
 .../lucene/geo3d/GeoConvexPolygonTest.java      |   3 +-
 .../org/apache/lucene/geo3d/GeoModelTest.java   |   3 +-
 .../org/apache/lucene/geo3d/GeoPathTest.java    |   3 +-
 .../org/apache/lucene/geo3d/GeoPolygonTest.java |   3 +-
 .../test/org/apache/lucene/geo3d/PlaneTest.java |   3 +-
 .../org/apache/lucene/geo3d/TestGeo3DPoint.java |  14 +-
 .../org/apache/lucene/geo3d/XYZSolidTest.java   |   3 +-
 .../lucene/search/spell/CombineSuggestion.java  |   3 +-
 .../apache/lucene/search/spell/Dictionary.java  |   3 +-
 .../lucene/search/spell/DirectSpellChecker.java |   3 +-
 .../search/spell/HighFrequencyDictionary.java   |   1 -
 .../search/spell/JaroWinklerDistance.java       |   3 +-
 .../lucene/search/spell/LevensteinDistance.java |   3 +-
 .../lucene/search/spell/LuceneDictionary.java   |   3 +-
 .../search/spell/LuceneLevenshteinDistance.java |   3 +-
 .../lucene/search/spell/NGramDistance.java      |  33 +-
 .../search/spell/PlainTextDictionary.java       |   3 +-
 .../lucene/search/spell/SpellChecker.java       |   3 +-
 .../lucene/search/spell/StringDistance.java     |   3 +-
 .../apache/lucene/search/spell/SuggestMode.java |   3 +-
 .../apache/lucene/search/spell/SuggestWord.java |   5 +-
 .../spell/SuggestWordFrequencyComparator.java   |   6 +-
 .../lucene/search/spell/SuggestWordQueue.java   |   5 +-
 .../spell/SuggestWordScoreComparator.java       |   3 +-
 .../search/spell/WordBreakSpellChecker.java     |   3 +-
 .../lucene/search/suggest/BitsProducer.java     |   3 +-
 .../search/suggest/BufferedInputIterator.java   |   3 +-
 .../search/suggest/DocumentDictionary.java      |   3 +-
 .../suggest/DocumentValueSourceDictionary.java  |   3 +-
 .../lucene/search/suggest/FileDictionary.java   |   4 +-
 .../lucene/search/suggest/InMemorySorter.java   |   3 +-
 .../lucene/search/suggest/InputIterator.java    |   3 +-
 .../apache/lucene/search/suggest/Lookup.java    |   3 +-
 .../search/suggest/SortedInputIterator.java     |   3 +-
 .../search/suggest/UnsortedInputIterator.java   |   3 +-
 .../analyzing/AnalyzingInfixSuggester.java      |   6 +-
 .../suggest/analyzing/AnalyzingSuggester.java   |   3 +-
 .../analyzing/BlendedInfixSuggester.java        |   3 +-
 .../search/suggest/analyzing/FSTUtil.java       |   3 +-
 .../suggest/analyzing/FreeTextSuggester.java    |   3 +-
 .../suggest/analyzing/FuzzySuggester.java       |   3 +-
 .../suggest/analyzing/SuggestStopFilter.java    |   3 +-
 .../analyzing/SuggestStopFilterFactory.java     |   3 +-
 .../document/Completion50PostingsFormat.java    |   4 +-
 .../suggest/document/CompletionAnalyzer.java    |   3 +-
 .../document/CompletionFieldsConsumer.java      |   3 +-
 .../document/CompletionFieldsProducer.java      |   3 +-
 .../document/CompletionPostingsFormat.java      |   3 +-
 .../suggest/document/CompletionQuery.java       |   3 +-
 .../suggest/document/CompletionScorer.java      |   3 +-
 .../suggest/document/CompletionTerms.java       |   3 +-
 .../suggest/document/CompletionTokenStream.java |   3 +-
 .../suggest/document/CompletionWeight.java      |   3 +-
 .../document/CompletionsTermsReader.java        |   3 +-
 .../search/suggest/document/ContextQuery.java   |   3 +-
 .../suggest/document/ContextSuggestField.java   |   3 +-
 .../suggest/document/FuzzyCompletionQuery.java  |   3 +-
 .../search/suggest/document/NRTSuggester.java   |   3 +-
 .../suggest/document/NRTSuggesterBuilder.java   |   3 +-
 .../suggest/document/PrefixCompletionQuery.java |   3 +-
 .../suggest/document/RegexCompletionQuery.java  |   3 +-
 .../search/suggest/document/SuggestField.java   |   9 +-
 .../suggest/document/SuggestIndexSearcher.java  |   3 +-
 .../document/SuggestScoreDocPriorityQueue.java  |   3 +-
 .../search/suggest/document/TopSuggestDocs.java |   3 +-
 .../document/TopSuggestDocsCollector.java       |   3 +-
 .../search/suggest/fst/BytesRefSorter.java      |   3 +-
 .../search/suggest/fst/ExternalRefSorter.java   |   4 +-
 .../search/suggest/fst/FSTCompletion.java       |   3 +-
 .../suggest/fst/FSTCompletionBuilder.java       |   3 +-
 .../search/suggest/fst/FSTCompletionLookup.java |   3 +-
 .../suggest/fst/WFSTCompletionLookup.java       |   3 +-
 .../search/suggest/jaspell/JaspellLookup.java   |   3 +-
 .../jaspell/JaspellTernarySearchTrie.java       |   3 +-
 .../search/suggest/tst/TSTAutocomplete.java     |   3 +-
 .../lucene/search/suggest/tst/TSTLookup.java    |   3 +-
 .../search/suggest/tst/TernaryTreeNode.java     |   7 +-
 .../search/spell/TestDirectSpellChecker.java    |   3 +-
 .../search/spell/TestJaroWinklerDistance.java   |   3 +-
 .../search/spell/TestLevenshteinDistance.java   |   3 +-
 .../search/spell/TestLuceneDictionary.java      |   3 +-
 .../lucene/search/spell/TestNGramDistance.java  |   3 +-
 .../search/spell/TestPlainTextDictionary.java   |   3 +-
 .../lucene/search/spell/TestSpellChecker.java   |   3 +-
 .../search/spell/TestWordBreakSpellChecker.java |   3 +-
 .../apache/lucene/search/suggest/Average.java   |   5 +-
 .../search/suggest/DocumentDictionaryTest.java  |  33 +-
 .../DocumentValueSourceDictionaryTest.java      |   3 +-
 .../search/suggest/FileDictionaryTest.java      |  33 +-
 .../org/apache/lucene/search/suggest/Input.java |   3 +-
 .../search/suggest/InputArrayIterator.java      |   3 +-
 .../search/suggest/LookupBenchmarkTest.java     |   3 +-
 .../lucene/search/suggest/PersistenceTest.java  |   6 +-
 .../suggest/TestHighFrequencyDictionary.java    |  27 +-
 .../search/suggest/TestInputIterator.java       |  33 +-
 .../analyzing/AnalyzingInfixSuggesterTest.java  |   3 +-
 .../analyzing/AnalyzingSuggesterTest.java       |   9 +-
 .../analyzing/BlendedInfixSuggesterTest.java    |   3 +-
 .../suggest/analyzing/FuzzySuggesterTest.java   |   9 +-
 .../analyzing/TestFreeTextSuggester.java        |   3 +-
 .../analyzing/TestSuggestStopFilter.java        |   3 +-
 .../analyzing/TestSuggestStopFilterFactory.java |   3 +-
 .../document/CompletionTokenStreamTest.java     |   3 +-
 .../suggest/document/TestContextQuery.java      |   3 +-
 .../document/TestContextSuggestField.java       |   3 +-
 .../document/TestFuzzyCompletionQuery.java      |   3 +-
 .../document/TestPrefixCompletionQuery.java     |   3 +-
 .../document/TestRegexCompletionQuery.java      |   3 +-
 .../suggest/document/TestSuggestField.java      |   3 +-
 .../search/suggest/fst/BytesRefSortersTest.java |   6 +-
 .../search/suggest/fst/FSTCompletionTest.java   |   9 +-
 .../search/suggest/fst/WFSTCompletionTest.java  |   9 +-
 .../analysis/BaseTokenStreamTestCase.java       |   3 +-
 .../analysis/CannedBinaryTokenStream.java       |   3 +-
 .../lucene/analysis/CannedTokenStream.java      |   3 +-
 .../lucene/analysis/CollationTestBase.java      |   4 +-
 .../lucene/analysis/CrankyTokenFilter.java      |   3 +-
 .../lucene/analysis/LookaheadTokenFilter.java   |   3 +-
 .../apache/lucene/analysis/MockAnalyzer.java    |   3 +-
 .../lucene/analysis/MockBytesAnalyzer.java      |   3 +-
 .../apache/lucene/analysis/MockCharFilter.java  |   3 +-
 .../analysis/MockFixedLengthPayloadFilter.java  |   3 +-
 .../lucene/analysis/MockGraphTokenFilter.java   |   3 +-
 .../analysis/MockHoleInjectingTokenFilter.java  |   4 +-
 .../lucene/analysis/MockPayloadAnalyzer.java    |   4 +-
 .../MockRandomLookaheadTokenFilter.java         |   3 +-
 .../lucene/analysis/MockReaderWrapper.java      |   3 +-
 .../apache/lucene/analysis/MockTokenFilter.java |   3 +-
 .../apache/lucene/analysis/MockTokenizer.java   |   3 +-
 .../analysis/MockUTF16TermAttributeImpl.java    |   3 +-
 .../MockVariableLengthPayloadFilter.java        |   3 +-
 .../lucene/analysis/SimplePayloadFilter.java    |   3 +-
 .../lucene/analysis/TokenStreamToDot.java       |   3 +-
 .../lucene/analysis/ValidatingTokenFilter.java  |   3 +-
 .../lucene/analysis/VocabularyAssert.java       |   5 +-
 .../lucene/codecs/MissingOrdRemapper.java       |  11 +-
 .../lucene/codecs/asserting/AssertingCodec.java |   3 +-
 .../asserting/AssertingDocValuesFormat.java     |   3 +-
 .../asserting/AssertingLiveDocsFormat.java      |   3 +-
 .../codecs/asserting/AssertingNormsFormat.java  |   3 +-
 .../codecs/asserting/AssertingPointFormat.java  |   8 +-
 .../asserting/AssertingPostingsFormat.java      |   3 +-
 .../asserting/AssertingStoredFieldsFormat.java  |   3 +-
 .../asserting/AssertingTermVectorsFormat.java   |   3 +-
 .../codecs/blockterms/LuceneFixedGap.java       |   3 +-
 .../blockterms/LuceneVarGapDocFreqInterval.java |   3 +-
 .../blockterms/LuceneVarGapFixedInterval.java   |   3 +-
 .../bloom/TestBloomFilteredLucenePostings.java  |   3 +-
 .../codecs/cheapbastard/CheapBastardCodec.java  |   3 +-
 .../codecs/compressing/CompressingCodec.java    |   3 +-
 .../compressing/FastCompressingCodec.java       |   3 +-
 .../FastDecompressionCompressingCodec.java      |   3 +-
 .../HighCompressionCompressingCodec.java        |   3 +-
 .../dummy/DummyCompressingCodec.java            |   3 +-
 .../lucene/codecs/cranky/CrankyCodec.java       |   9 +-
 .../codecs/cranky/CrankyCompoundFormat.java     |   3 +-
 .../codecs/cranky/CrankyDocValuesFormat.java    |   3 +-
 .../codecs/cranky/CrankyFieldInfosFormat.java   |   3 +-
 .../codecs/cranky/CrankyLiveDocsFormat.java     |   3 +-
 .../lucene/codecs/cranky/CrankyNormsFormat.java |   3 +-
 .../lucene/codecs/cranky/CrankyPointFormat.java | 175 ++++
 .../codecs/cranky/CrankyPostingsFormat.java     |   3 +-
 .../codecs/cranky/CrankySegmentInfoFormat.java  |   3 +-
 .../codecs/cranky/CrankyStoredFieldsFormat.java |   3 +-
 .../codecs/cranky/CrankyTermVectorsFormat.java  |   3 +-
 .../mockrandom/MockRandomPostingsFormat.java    |   3 +-
 .../codecs/ramonly/RAMOnlyPostingsFormat.java   |   3 +-
 .../lucene/index/AlcoholicMergePolicy.java      |   3 +-
 .../lucene/index/AllDeletedFilterReader.java    |   7 +-
 .../lucene/index/AssertingDirectoryReader.java  |   3 +-
 .../lucene/index/AssertingLeafReader.java       |  21 +-
 .../index/BaseCompoundFormatTestCase.java       |   8 +-
 .../BaseCompressingDocValuesFormatTestCase.java |   3 +-
 .../index/BaseDocValuesFormatTestCase.java      |   3 +-
 .../index/BaseFieldInfoFormatTestCase.java      |   3 +-
 .../index/BaseIndexFileFormatTestCase.java      |  19 +-
 .../lucene/index/BaseMergePolicyTestCase.java   |   3 +-
 .../lucene/index/BaseNormsFormatTestCase.java   |   3 +-
 .../lucene/index/BasePointFormatTestCase.java   | 917 +++++++++++++++++
 .../index/BasePostingsFormatTestCase.java       |   5 +-
 .../index/BaseSegmentInfoFormatTestCase.java    |   3 +-
 .../index/BaseStoredFieldsFormatTestCase.java   |   3 +-
 .../index/BaseTermVectorsFormatTestCase.java    |   3 +-
 .../apache/lucene/index/BaseTestCheckIndex.java |   4 +-
 .../java/org/apache/lucene/index/DocHelper.java |   3 +-
 .../lucene/index/FieldFilterLeafReader.java     |   3 +-
 .../apache/lucene/index/ForceMergePolicy.java   |  48 +-
 .../lucene/index/IndexWriterMaxDocsChanger.java |   7 +-
 .../lucene/index/MismatchedDirectoryReader.java |   3 +-
 .../lucene/index/MismatchedLeafReader.java      |   3 +-
 .../lucene/index/MockRandomMergePolicy.java     |   3 +-
 .../apache/lucene/index/PerThreadPKLookup.java  |   3 +-
 .../org/apache/lucene/index/RandomCodec.java    |   3 +-
 .../apache/lucene/index/RandomIndexWriter.java  |   3 +-
 .../lucene/index/RandomPostingsTester.java      |   4 +-
 .../SuppressingConcurrentMergeScheduler.java    |   7 +-
 .../ThreadedIndexingAndSearchingTestCase.java   |   4 +-
 .../apache/lucene/mockfile/DisableFsyncFS.java  |  23 +-
 .../org/apache/lucene/mockfile/ExtrasFS.java    |   3 +-
 .../mockfile/FilterAsynchronousFileChannel.java |   3 +-
 .../lucene/mockfile/FilterDirectoryStream.java  |   3 +-
 .../lucene/mockfile/FilterFileChannel.java      |   3 +-
 .../apache/lucene/mockfile/FilterFileStore.java |   3 +-
 .../lucene/mockfile/FilterFileSystem.java       |   3 +-
 .../mockfile/FilterFileSystemProvider.java      |   3 +-
 .../lucene/mockfile/FilterInputStream2.java     |   3 +-
 .../lucene/mockfile/FilterOutputStream2.java    |   3 +-
 .../org/apache/lucene/mockfile/FilterPath.java  |   3 +-
 .../mockfile/FilterSeekableByteChannel.java     |   3 +-
 .../apache/lucene/mockfile/HandleLimitFS.java   |   3 +-
 .../lucene/mockfile/HandleTrackingFS.java       |   3 +-
 .../java/org/apache/lucene/mockfile/LeakFS.java |   3 +-
 .../lucene/mockfile/MockFileSystemTestCase.java |   3 +-
 .../org/apache/lucene/mockfile/ShuffleFS.java   |   3 +-
 .../org/apache/lucene/mockfile/VerboseFS.java   |   3 +-
 .../apache/lucene/mockfile/VirusCheckingFS.java |  80 ++
 .../org/apache/lucene/mockfile/WindowsFS.java   |   3 +-
 .../lucene/search/AssertingBulkScorer.java      |   3 +-
 .../lucene/search/AssertingCollector.java       |   3 +-
 .../lucene/search/AssertingIndexSearcher.java   |   3 +-
 .../lucene/search/AssertingLeafCollector.java   |   3 +-
 .../apache/lucene/search/AssertingQuery.java    |   3 +-
 .../apache/lucene/search/AssertingScorer.java   |   3 +-
 .../apache/lucene/search/AssertingWeight.java   |   3 +-
 .../lucene/search/BaseExplanationTestCase.java  |   3 +-
 .../lucene/search/BulkScorerWrapperScorer.java  |   3 +-
 .../org/apache/lucene/search/CheckHits.java     |   3 +-
 .../org/apache/lucene/search/QueryUtils.java    |   3 +-
 .../lucene/search/RandomApproximationQuery.java |   3 +-
 .../lucene/search/ScorerIndexSearcher.java      |   3 +-
 .../search/SearchEquivalenceTestBase.java       |   3 +-
 .../lucene/search/ShardSearchingTestBase.java   |   3 +-
 .../search/similarities/RandomSimilarity.java   |   3 +-
 .../lucene/search/spans/AssertingSpanQuery.java |   3 +-
 .../search/spans/AssertingSpanWeight.java       |   3 +-
 .../lucene/search/spans/AssertingSpans.java     |   3 +-
 .../lucene/search/spans/MultiSpansWrapper.java  |   3 +-
 .../lucene/search/spans/SpanTestUtil.java       |   3 +-
 .../lucene/store/BaseDirectoryTestCase.java     |  77 +-
 .../lucene/store/BaseDirectoryWrapper.java      |   3 +-
 .../lucene/store/BaseLockFactoryTestCase.java   |  49 +-
 .../lucene/store/MockDirectoryWrapper.java      | 229 +----
 .../lucene/store/MockIndexInputWrapper.java     |  13 +-
 .../lucene/store/MockIndexOutputWrapper.java    |   3 +-
 .../lucene/store/RawDirectoryWrapper.java       |   3 +-
 .../store/SlowClosingMockIndexInputWrapper.java |   3 +-
 .../store/SlowOpeningMockIndexInputWrapper.java |   3 +-
 .../lucene/util/AbstractBeforeAfterRule.java    |  23 +-
 .../apache/lucene/util/BaseBitSetTestCase.java  |   3 +-
 .../lucene/util/BaseDocIdSetTestCase.java       |   3 +-
 .../apache/lucene/util/CloseableDirectory.java  |  15 +-
 .../java/org/apache/lucene/util/English.java    |   9 +-
 .../util/FailOnNonBulkMergesInfoStream.java     |   3 +-
 .../org/apache/lucene/util/FailureMarker.java   |  13 +-
 .../org/apache/lucene/util/LineFileDocs.java    |   3 +-
 .../lucene/util/LuceneJUnit3MethodProvider.java |   3 +-
 .../org/apache/lucene/util/LuceneTestCase.java  | 121 ++-
 .../org/apache/lucene/util/NullInfoStream.java  |   3 +-
 .../lucene/util/QuickPatchThreadsFilter.java    |   7 +-
 .../org/apache/lucene/util/RamUsageTester.java  |   3 +-
 .../org/apache/lucene/util/RemoveUponClose.java |  19 +-
 .../java/org/apache/lucene/util/Rethrow.java    |   3 +-
 .../util/RunListenerPrintReproduceInfo.java     |  33 +-
 .../lucene/util/TestRuleAssertionsRequired.java |   3 +-
 .../apache/lucene/util/TestRuleDelegate.java    |  15 +-
 .../util/TestRuleIgnoreAfterMaxFailures.java    |  21 +-
 .../lucene/util/TestRuleIgnoreTestSuites.java   |  11 +-
 .../lucene/util/TestRuleLimitSysouts.java       |  33 +-
 .../apache/lucene/util/TestRuleMarkFailure.java |  19 +-
 .../util/TestRuleRestoreSystemProperties.java   |  17 +-
 .../util/TestRuleSetupAndRestoreClassEnv.java   |   3 +-
 .../TestRuleSetupAndRestoreInstanceEnv.java     |   7 +-
 .../util/TestRuleSetupTeardownChained.java      |  13 +-
 .../lucene/util/TestRuleStoreClassName.java     |   3 +-
 .../util/TestRuleTemporaryFilesCleanup.java     |   5 +-
 .../lucene/util/TestRuleThreadAndTestName.java  |  11 +-
 .../apache/lucene/util/TestSecurityManager.java |   9 +-
 .../java/org/apache/lucene/util/TestUtil.java   | 105 +-
 .../lucene/util/ThrottledIndexOutput.java       |   9 +-
 .../java/org/apache/lucene/util/TimeUnits.java  |   3 +-
 .../util/automaton/AutomatonTestUtil.java       |   3 +-
 .../org/apache/lucene/util/fst/FSTTester.java   |   9 +-
 .../lucene/analysis/TestGraphTokenizers.java    |   3 +-
 .../analysis/TestLookaheadTokenFilter.java      |   3 +-
 .../lucene/analysis/TestMockAnalyzer.java       |   3 +-
 .../lucene/analysis/TestMockCharFilter.java     |   9 +-
 .../apache/lucene/analysis/TestPosition.java    |   5 +-
 .../lucene/analysis/TrivialLookaheadFilter.java |   3 +-
 .../asserting/TestAssertingDocValuesFormat.java |   3 +-
 .../asserting/TestAssertingNormsFormat.java     |   3 +-
 .../asserting/TestAssertingPointFormat.java     |  30 +
 .../asserting/TestAssertingPostingsFormat.java  |   3 +-
 .../TestAssertingStoredFieldsFormat.java        |   3 +-
 .../TestAssertingTermVectorsFormat.java         |   3 +-
 .../TestCompressingStoredFieldsFormat.java      |   7 +-
 .../TestCompressingTermVectorsFormat.java       |  33 +-
 .../lucene/index/TestAssertingLeafReader.java   |   3 +-
 .../lucene/index/TestForceMergePolicy.java      |   3 +-
 .../lucene/mockfile/TestDisableFsyncFS.java     |   3 +-
 .../apache/lucene/mockfile/TestExtrasFS.java    |   3 +-
 .../lucene/mockfile/TestHandleLimitFS.java      |   3 +-
 .../lucene/mockfile/TestHandleTrackingFS.java   |   3 +-
 .../org/apache/lucene/mockfile/TestLeakFS.java  |   3 +-
 .../apache/lucene/mockfile/TestShuffleFS.java   |   3 +-
 .../apache/lucene/mockfile/TestVerboseFS.java   |   3 +-
 .../lucene/mockfile/TestVirusCheckingFS.java    |  62 ++
 .../apache/lucene/mockfile/TestWindowsFS.java   |   5 +-
 .../lucene/store/TestMockDirectoryWrapper.java  |   4 +-
 .../test/org/apache/lucene/util/SorePoint.java  |   3 +-
 .../test/org/apache/lucene/util/SoreType.java   |   3 +-
 .../lucene/util/TestBeforeAfterOverrides.java   |  17 +-
 .../apache/lucene/util/TestCodecReported.java   |  15 +-
 .../util/TestExceptionInBeforeClassHooks.java   |   3 +-
 .../util/TestFailIfDirectoryNotClosed.java      |   3 +-
 .../util/TestFailIfUnreferencedFiles.java       |   3 +-
 .../apache/lucene/util/TestGroupFiltering.java  |  21 +-
 .../apache/lucene/util/TestJUnitRuleOrder.java  |   3 +-
 .../apache/lucene/util/TestMaxFailuresRule.java |   3 +-
 .../util/TestRamUsageTesterOnWildAnimals.java   |   3 +-
 .../lucene/util/TestReproduceMessage.java       |   3 +-
 .../util/TestReproduceMessageWithRepeated.java  |   3 +-
 .../util/TestRunWithRestrictedPermissions.java  |   3 +-
 .../lucene/util/TestSeedFromUncaught.java       |   3 +-
 .../lucene/util/TestSetupTeardownChaining.java  |   3 +-
 .../lucene/util/TestWorstCaseTestBehavior.java  |  15 +-
 .../org/apache/lucene/util/WithNestedTests.java |   3 +-
 .../dependencies/GetMavenDependenciesTask.java  |   3 +-
 .../dependencies/InterpolatedProperties.java    |   3 +-
 .../lucene/validation/LibVersionsCheckTask.java |   3 +-
 .../lucene/validation/LicenseCheckTask.java     |   3 +-
 .../apache/lucene/validation/LicenseType.java   |   5 +-
 .../lucene/validation/ivyde/IvyNodeElement.java |  28 +-
 .../validation/ivyde/IvyNodeElementAdapter.java |  28 +-
 solr/CHANGES.txt                                |  34 +
 .../apache/solr/schema/ICUCollationField.java   |   3 +-
 .../TestFoldingMultitermExtrasQuery.java        |   3 +-
 .../solr/schema/TestICUCollationField.java      |   1 -
 .../schema/TestICUCollationFieldDocValues.java  |   1 -
 .../schema/TestICUCollationFieldOptions.java    |   3 +-
 .../analytics/accumulator/BasicAccumulator.java |   1 -
 .../accumulator/FacetingAccumulator.java        |   1 -
 .../analytics/accumulator/ValueAccumulator.java |   1 -
 .../facet/FacetValueAccumulator.java            |   1 -
 .../facet/FieldFacetAccumulator.java            |   1 -
 .../facet/QueryFacetAccumulator.java            |   1 -
 .../facet/RangeFacetAccumulator.java            |   1 -
 .../analytics/expression/BaseExpression.java    |   1 -
 .../expression/DualDelegateExpression.java      |   1 -
 .../solr/analytics/expression/Expression.java   |   1 -
 .../analytics/expression/ExpressionFactory.java |   1 -
 .../expression/MultiDelegateExpression.java     |   1 -
 .../expression/SingleDelegateExpression.java    |   1 -
 .../plugin/AnalyticsStatisticsCollector.java    |   1 -
 .../request/AbstractFieldFacetRequest.java      |   1 -
 .../request/AnalyticsContentHandler.java        |   1 -
 .../analytics/request/AnalyticsRequest.java     |   1 -
 .../request/AnalyticsRequestFactory.java        |   1 -
 .../solr/analytics/request/AnalyticsStats.java  |   1 -
 .../analytics/request/ExpressionRequest.java    |   1 -
 .../solr/analytics/request/FacetRequest.java    |   1 -
 .../analytics/request/FieldFacetRequest.java    |   1 -
 .../analytics/request/QueryFacetRequest.java    |   1 -
 .../analytics/request/RangeFacetRequest.java    |   1 -
 .../AbstractDelegatingStatsCollector.java       |   1 -
 .../statistics/MedianStatsCollector.java        |   1 -
 .../statistics/MinMaxStatsCollector.java        |   1 -
 .../statistics/NumericStatsCollector.java       |   1 -
 .../statistics/PercentileStatsCollector.java    |   1 -
 .../analytics/statistics/StatsCollector.java    |   1 -
 .../StatsCollectorSupplierFactory.java          |   1 -
 .../statistics/UniqueStatsCollector.java        |   1 -
 .../solr/analytics/util/AnalyticsParams.java    |   1 -
 .../solr/analytics/util/AnalyticsParsers.java   |   1 -
 .../solr/analytics/util/MedianCalculator.java   |   1 -
 .../analytics/util/PercentileCalculator.java    |   1 -
 .../analytics/util/RangeEndpointCalculator.java |   1 -
 .../AbsoluteValueDoubleFunction.java            |   1 -
 .../util/valuesource/AddDoubleFunction.java     |   1 -
 .../util/valuesource/ConcatStringFunction.java  |   1 -
 .../util/valuesource/ConstDateSource.java       |   1 -
 .../util/valuesource/ConstDoubleSource.java     |   1 -
 .../util/valuesource/ConstStringSource.java     |   1 -
 .../util/valuesource/DateFieldSource.java       |   1 -
 .../util/valuesource/DateMathFunction.java      |   1 -
 .../util/valuesource/DivDoubleFunction.java     |   1 -
 .../util/valuesource/DualDoubleFunction.java    |   1 -
 .../util/valuesource/FilterFieldSource.java     |   1 -
 .../util/valuesource/LogDoubleFunction.java     |   1 -
 .../util/valuesource/MultiDateFunction.java     |   1 -
 .../util/valuesource/MultiDoubleFunction.java   |   1 -
 .../util/valuesource/MultiStringFunction.java   |   1 -
 .../valuesource/MultiplyDoubleFunction.java     |   1 -
 .../util/valuesource/NegateDoubleFunction.java  |   1 -
 .../util/valuesource/PowDoubleFunction.java     |   1 -
 .../util/valuesource/ReverseStringFunction.java |   1 -
 .../util/valuesource/SingleDoubleFunction.java  |   1 -
 .../util/valuesource/SingleStringFunction.java  |   1 -
 .../handler/component/AnalyticsComponent.java   |   1 -
 .../analytics/AbstractAnalyticsStatsTest.java   |   1 -
 .../org/apache/solr/analytics/NoFacetTest.java  |   1 -
 .../analytics/expression/ExpressionTest.java    |   1 -
 .../facet/AbstractAnalyticsFacetTest.java       |   1 -
 .../analytics/facet/FieldFacetExtrasTest.java   |   1 -
 .../solr/analytics/facet/FieldFacetTest.java    |   1 -
 .../solr/analytics/facet/QueryFacetTest.java    |   1 -
 .../solr/analytics/facet/RangeFacetTest.java    |   1 -
 .../util/valuesource/FunctionTest.java          |   1 -
 .../handler/clustering/ClusteringComponent.java |   3 +-
 .../handler/clustering/ClusteringEngine.java    |   2 +-
 .../handler/clustering/ClusteringParams.java    |   3 +-
 .../clustering/DocumentClusteringEngine.java    |   3 +-
 .../clustering/SearchClusteringEngine.java      |   3 +-
 .../carrot2/CarrotClusteringEngine.java         |   3 +-
 .../clustering/carrot2/CarrotParams.java        |  10 +-
 .../carrot2/LuceneCarrot2StemmerFactory.java    |   7 +-
 .../carrot2/LuceneCarrot2TokenizerFactory.java  |   3 +-
 .../clustering/carrot2/SolrResourceLocator.java |   1 -
 .../SolrStopwordsCarrot2LexicalDataFactory.java |   3 +-
 .../clustering/AbstractClusteringTestCase.java  |   3 +-
 .../clustering/ClusteringComponentTest.java     |   3 +-
 .../DistributedClusteringComponentTest.java     |   3 +-
 .../MockDocumentClusteringEngine.java           |   2 +-
 .../carrot2/CarrotClusteringEngineTest.java     |   3 +-
 .../carrot2/DuplicatingStemmerFactory.java      |   3 +-
 .../carrot2/DuplicatingTokenizerFactory.java    |   3 +-
 .../carrot2/EchoClusteringAlgorithm.java        |   2 +-
 .../carrot2/EchoStemsClusteringAlgorithm.java   |   4 +-
 .../carrot2/EchoTokensClusteringAlgorithm.java  |   4 +-
 ...exicalResourcesCheckClusteringAlgorithm.java |   4 +-
 .../carrot2/MockClusteringAlgorithm.java        |   2 +-
 .../handler/dataimport/CachePropertyUtil.java   |   3 +-
 .../apache/solr/handler/dataimport/Context.java |   1 -
 .../solr/handler/dataimport/ContextImpl.java    |   4 +-
 .../solr/handler/dataimport/DIHCache.java       |   3 +-
 .../handler/dataimport/DIHCacheSupport.java     |   3 +-
 .../solr/handler/dataimport/DIHLogLevels.java   |   3 +-
 .../solr/handler/dataimport/DIHProperties.java  |   5 +-
 .../solr/handler/dataimport/DIHWriter.java      |   2 +-
 .../solr/handler/dataimport/DIHWriterBase.java  |   3 +-
 .../dataimport/DataImportHandlerException.java  |   1 -
 .../solr/handler/dataimport/DataImporter.java   |   1 -
 .../solr/handler/dataimport/DataSource.java     |   1 -
 .../handler/dataimport/DateFormatEvaluator.java |  33 +-
 .../dataimport/DateFormatTransformer.java       |   1 -
 .../solr/handler/dataimport/DebugLogger.java    |   3 +-
 .../solr/handler/dataimport/DocBuilder.java     |   1 -
 .../solr/handler/dataimport/JdbcDataSource.java |  70 +-
 .../dataimport/NumberFormatTransformer.java     |   3 +-
 .../dataimport/SimplePropertiesWriter.java      |   3 +-
 .../handler/dataimport/SolrEntityProcessor.java |   3 +-
 .../dataimport/SolrQueryEscapingEvaluator.java  |  15 +-
 .../dataimport/SortedMapBackedCache.java        |   3 +-
 .../dataimport/SqlEscapingEvaluator.java        |  11 +-
 .../solr/handler/dataimport/UrlEvaluator.java   |  17 +-
 .../apache/solr/handler/dataimport/Zipper.java  |  27 +-
 .../dataimport/config/DIHConfiguration.java     |  33 +-
 .../solr/handler/dataimport/config/Entity.java  |  26 +-
 .../handler/dataimport/config/EntityField.java  |   1 -
 .../solr/handler/dataimport/config/Field.java   |   1 -
 .../dataimport/config/PropertyWriter.java       |   9 +-
 .../solr/handler/dataimport/config/Script.java  |  25 +-
 .../dataimport/AbstractDIHCacheTestCase.java    |   3 +-
 .../dataimport/AbstractDIHJdbcTestCase.java     |  30 +-
 .../AbstractSqlEntityProcessorTestCase.java     |  33 +-
 .../dataimport/AddAColumnTransformer.java       |   7 +-
 .../handler/dataimport/DestroyCountCache.java   |   3 +-
 .../dataimport/MockSolrEntityProcessor.java     |   3 +-
 .../handler/dataimport/TestEphemeralCache.java  |   3 +-
 .../handler/dataimport/TestErrorHandling.java   |   3 +-
 .../dataimport/TestFileListEntityProcessor.java |   3 +-
 .../TestFileListWithLineEntityProcessor.java    |  19 +-
 .../dataimport/TestHierarchicalDocBuilder.java  |   3 +-
 .../TestJdbcDataSourceConvertType.java          |   3 +-
 .../dataimport/TestLineEntityProcessor.java     |   3 +-
 .../handler/dataimport/TestNestedChildren.java  |  15 +-
 .../dataimport/TestSimplePropertiesWriter.java  |  33 +-
 .../TestSolrEntityProcessorEndToEnd.java        |   3 +-
 .../dataimport/TestSolrEntityProcessorUnit.java |   2 +-
 .../dataimport/TestSortedMapBackedCache.java    |   7 +-
 .../dataimport/TestSqlEntityProcessor.java      |   9 +-
 .../dataimport/TestSqlEntityProcessorDelta.java |  21 +-
 .../TestVariableResolverEndToEnd.java           |  31 +-
 .../dataimport/TestXPathRecordReader.java       |   2 +-
 .../dataimport/TestZKPropertiesWriter.java      |   7 +-
 .../dataimport/TripleThreatTransformer.java     |  16 +-
 .../handler/extraction/ParseContextConfig.java  |   3 +-
 .../extraction/RegexRulesPasswordProvider.java  |   1 -
 .../handler/extraction/SolrContentHandler.java  |   1 -
 .../ExtractingRequestHandlerTest.java           |   3 +-
 .../extraction/ParseContextConfigTest.java      |   3 +-
 .../solr/update/processor/DetectedLanguage.java |   3 +-
 ...DetectLanguageIdentifierUpdateProcessor.java |   3 +-
 ...anguageIdentifierUpdateProcessorFactory.java |   3 +-
 .../solr/update/processor/LangIdParams.java     |   3 +-
 .../LanguageIdentifierUpdateProcessor.java      |   3 +-
 .../TikaLanguageIdentifierUpdateProcessor.java  |   3 +-
 ...anguageIdentifierUpdateProcessorFactory.java |   3 +-
 ...ageIdentifierUpdateProcessorFactoryTest.java |   3 +-
 ...dentifierUpdateProcessorFactoryTestCase.java |   1 -
 ...ageIdentifierUpdateProcessorFactoryTest.java |   3 +-
 .../solr/hadoop/AlphaNumericComparator.java     |   1 -
 .../solr/hadoop/DataInputInputStream.java       |   1 -
 .../hadoop/UnbufferedDataInputInputStream.java  |   1 -
 .../apache/solr/hadoop/ZooKeeperInspector.java  |   1 -
 .../solr/hadoop/MorphlineGoLiveMiniMRTest.java  |   1 -
 .../solr/AbstractSolrMorphlineZkTestBase.java   |   1 -
 .../uima/processor/FieldMappingException.java   |   3 +-
 .../uima/processor/SolrUIMAConfiguration.java   |   3 +-
 .../processor/SolrUIMAConfigurationReader.java  |   3 +-
 .../solr/uima/processor/UIMAToSolrMapper.java   |   3 +-
 .../processor/UIMAUpdateRequestProcessor.java   |   3 +-
 .../UIMAUpdateRequestProcessorFactory.java      |   3 +-
 .../UIMATokenizersSolrIntegrationTest.java      |   3 +-
 .../UIMAUpdateRequestProcessorTest.java         |   3 +-
 .../uima/processor/an/DummyEntityAnnotator.java |  17 +-
 .../processor/an/DummyExceptionAnnotator.java   |  11 +-
 .../processor/an/DummySentimentAnnotator.java   |  21 +-
 .../java/org/apache/solr/response/PageTool.java |   1 -
 .../solr/response/SolrParamResourceLoader.java  |   1 -
 .../solr/response/SolrVelocityLogger.java       |   3 +-
 .../solr/response/VelocityResponseWriter.java   |   1 -
 .../test/org/apache/solr/velocity/MockTool.java |   3 +-
 .../velocity/VelocityResponseWriterTest.java    |   1 -
 .../solr/analysis/ReversedWildcardFilter.java   |   3 +-
 .../analysis/ReversedWildcardFilterFactory.java |   3 +-
 .../org/apache/solr/analysis/SolrAnalyzer.java  |   1 -
 .../apache/solr/analysis/TokenizerChain.java    |   1 -
 .../solrj/embedded/EmbeddedSolrServer.java      |   1 -
 .../solr/client/solrj/embedded/JettyConfig.java |   1 -
 .../client/solrj/embedded/JettySolrRunner.java  |   1 -
 .../solr/client/solrj/embedded/SSLConfig.java   |   3 +-
 .../org/apache/solr/cloud/ActionThrottle.java   |   7 +-
 .../src/java/org/apache/solr/cloud/Assign.java  |  27 +-
 .../solr/cloud/CloudConfigSetService.java       |   1 -
 .../org/apache/solr/cloud/CloudDescriptor.java  |   3 +-
 .../java/org/apache/solr/cloud/CloudUtil.java   |   3 +-
 .../cloud/CurrentCoreDescriptorProvider.java    |   3 +-
 .../org/apache/solr/cloud/DistributedMap.java   |   3 +-
 .../org/apache/solr/cloud/DistributedQueue.java |   4 +-
 .../org/apache/solr/cloud/ElectionContext.java  |   3 +-
 .../org/apache/solr/cloud/LeaderElector.java    |   3 +-
 .../cloud/LeaderInitiatedRecoveryThread.java    |  33 +-
 .../java/org/apache/solr/cloud/Overseer.java    |  27 +-
 .../OverseerAutoReplicaFailoverThread.java      |   3 +-
 .../OverseerCollectionConfigSetProcessor.java   |   3 +-
 .../cloud/OverseerCollectionMessageHandler.java |   3 +-
 .../cloud/OverseerConfigSetMessageHandler.java  |   3 +-
 .../solr/cloud/OverseerMessageHandler.java      |   3 +-
 .../solr/cloud/OverseerNodePrioritizer.java     |   7 +-
 .../apache/solr/cloud/OverseerSolrResponse.java |   3 +-
 .../solr/cloud/OverseerTaskProcessor.java       |   3 +-
 .../apache/solr/cloud/OverseerTaskQueue.java    |   7 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java |   3 +-
 .../solr/cloud/SizeLimitedDistributedMap.java   |   3 +-
 .../org/apache/solr/cloud/SolrZkServer.java     |  27 +-
 .../org/apache/solr/cloud/SyncStrategy.java     |  19 +-
 .../src/java/org/apache/solr/cloud/ZkCLI.java   |  33 +-
 .../org/apache/solr/cloud/ZkController.java     |   3 +-
 .../apache/solr/cloud/ZkSolrResourceLoader.java |   3 +-
 .../cloud/overseer/ClusterStateMutator.java     |   7 +-
 .../solr/cloud/overseer/CollectionMutator.java  |   7 +-
 .../solr/cloud/overseer/OverseerAction.java     |   3 +-
 .../solr/cloud/overseer/ReplicaMutator.java     |   7 +-
 .../solr/cloud/overseer/SliceMutator.java       |   7 +-
 .../solr/cloud/overseer/ZkStateWriter.java      |   3 +-
 .../solr/cloud/overseer/ZkWriteCommand.java     |   3 +-
 .../apache/solr/cloud/rule/ImplicitSnitch.java  |   4 +-
 .../apache/solr/cloud/rule/RemoteCallback.java  |   3 +-
 .../apache/solr/cloud/rule/ReplicaAssigner.java |   7 +-
 .../java/org/apache/solr/cloud/rule/Rule.java   |   4 +-
 .../java/org/apache/solr/cloud/rule/Snitch.java |   3 +-
 .../apache/solr/cloud/rule/SnitchContext.java   |   3 +-
 .../solr/core/AbstractSolrEventListener.java    |   1 -
 .../solr/core/CachingDirectoryFactory.java      |   3 +-
 .../java/org/apache/solr/core/CloseHook.java    |   4 +-
 .../java/org/apache/solr/core/CloudConfig.java  |   3 +-
 .../java/org/apache/solr/core/CodecFactory.java |   3 +-
 .../src/java/org/apache/solr/core/Config.java   |   1 -
 .../org/apache/solr/core/ConfigOverlay.java     |   3 +-
 .../java/org/apache/solr/core/ConfigSet.java    |   1 -
 .../apache/solr/core/ConfigSetProperties.java   |   1 -
 .../org/apache/solr/core/ConfigSetService.java  |   1 -
 .../org/apache/solr/core/CoreContainer.java     |  59 +-
 .../org/apache/solr/core/CoreDescriptor.java    |   1 -
 .../apache/solr/core/CorePropertiesLocator.java |   3 +-
 .../java/org/apache/solr/core/CoresLocator.java |   3 +-
 .../java/org/apache/solr/core/Diagnostics.java  |   4 +-
 .../org/apache/solr/core/DirectoryFactory.java  |   3 +-
 .../solr/core/EphemeralDirectoryFactory.java    |   3 +-
 .../apache/solr/core/HdfsDirectoryFactory.java  |   3 +-
 .../org/apache/solr/core/ImplicitPlugins.java   |   4 +-
 .../solr/core/IndexDeletionPolicyWrapper.java   |   3 +-
 .../apache/solr/core/IndexReaderFactory.java    |   3 +-
 .../java/org/apache/solr/core/InitParams.java   |   3 +-
 .../org/apache/solr/core/JarRepository.java     |   3 +-
 .../apache/solr/core/MMapDirectoryFactory.java  |   4 +-
 .../org/apache/solr/core/MapSerializable.java   |   4 +-
 .../org/apache/solr/core/MemClassLoader.java    |   3 +-
 .../apache/solr/core/NIOFSDirectoryFactory.java |   3 +-
 .../solr/core/NRTCachingDirectoryFactory.java   |   1 -
 .../java/org/apache/solr/core/NodeConfig.java   |   3 +-
 .../java/org/apache/solr/core/PluginBag.java    |   4 +-
 .../apache/solr/core/QuerySenderListener.java   |   1 -
 .../apache/solr/core/RAMDirectoryFactory.java   |   1 -
 .../org/apache/solr/core/RequestHandlers.java   |   1 -
 .../org/apache/solr/core/RequestParams.java     |   3 +-
 .../apache/solr/core/RunExecutableListener.java |   1 -
 .../apache/solr/core/SchemaCodecFactory.java    |  33 +-
 .../solr/core/SimpleFSDirectoryFactory.java     |   3 +-
 .../java/org/apache/solr/core/SolrConfig.java   |   1 -
 .../src/java/org/apache/solr/core/SolrCore.java |   1 -
 .../java/org/apache/solr/core/SolrCores.java    |   3 +-
 .../apache/solr/core/SolrDeletionPolicy.java    |   3 +-
 .../org/apache/solr/core/SolrEventListener.java |   1 -
 .../org/apache/solr/core/SolrInfoMBean.java     |   1 -
 .../apache/solr/core/SolrResourceLoader.java    |   1 -
 .../core/SolrResourceNotFoundException.java     |   1 -
 .../org/apache/solr/core/SolrXmlConfig.java     |   3 +-
 .../solr/core/StandardDirectoryFactory.java     |   3 +-
 .../solr/core/StandardIndexReaderFactory.java   |   3 +-
 .../java/org/apache/solr/core/ZkContainer.java  |   3 +-
 .../handler/AnalysisRequestHandlerBase.java     |   1 -
 .../org/apache/solr/handler/BlobHandler.java    |   3 +-
 .../apache/solr/handler/CdcrBufferManager.java  |   3 +-
 .../solr/handler/CdcrBufferStateManager.java    |   3 +-
 .../solr/handler/CdcrLeaderStateManager.java    |   7 +-
 .../org/apache/solr/handler/CdcrParams.java     |   3 +-
 .../solr/handler/CdcrProcessStateManager.java   |   7 +-
 .../org/apache/solr/handler/CdcrReplicator.java |   3 +-
 .../solr/handler/CdcrReplicatorManager.java     |   3 +-
 .../solr/handler/CdcrReplicatorScheduler.java   |   3 +-
 .../solr/handler/CdcrReplicatorState.java       |   3 +-
 .../apache/solr/handler/CdcrRequestHandler.java |   3 +-
 .../apache/solr/handler/CdcrStateManager.java   |   3 +-
 .../solr/handler/CdcrUpdateLogSynchronizer.java |  15 +-
 .../solr/handler/ContentStreamHandlerBase.java  |   3 +-
 .../solr/handler/ContentStreamLoader.java       |   4 +-
 .../handler/DocumentAnalysisRequestHandler.java |   1 -
 .../apache/solr/handler/DumpRequestHandler.java |   1 -
 .../handler/FieldAnalysisRequestHandler.java    |   1 -
 .../solr/handler/MoreLikeThisHandler.java       |   1 -
 .../solr/handler/NestedRequestHandler.java      |   3 +-
 .../solr/handler/NotFoundRequestHandler.java    |   3 +-
 .../apache/solr/handler/OldBackupDirectory.java |   3 +-
 .../apache/solr/handler/PingRequestHandler.java |   1 -
 .../apache/solr/handler/RealTimeGetHandler.java |   1 -
 .../apache/solr/handler/RequestHandlerBase.java |   1 -
 .../solr/handler/RequestHandlerUtils.java       |   1 -
 .../org/apache/solr/handler/RestoreCore.java    |   4 +-
 .../org/apache/solr/handler/SQLHandler.java     | 200 +++-
 .../org/apache/solr/handler/SchemaHandler.java  |   4 +-
 .../apache/solr/handler/SolrConfigHandler.java  |   4 +-
 .../solr/handler/StandardRequestHandler.java    |   1 -
 .../org/apache/solr/handler/StreamHandler.java  |   1 -
 .../solr/handler/UpdateRequestHandler.java      |   1 -
 .../solr/handler/admin/AdminHandlers.java       |   1 -
 .../solr/handler/admin/ClusterStatus.java       |   4 +-
 .../solr/handler/admin/ConfigSetsHandler.java   |   7 +-
 .../solr/handler/admin/CoreAdminHandler.java    |   1 -
 .../solr/handler/admin/CoreAdminOperation.java  |   3 +-
 .../apache/solr/handler/admin/InfoHandler.java  |   3 +-
 .../solr/handler/admin/LoggingHandler.java      |   3 +-
 .../solr/handler/admin/LukeRequestHandler.java  |  21 +-
 .../solr/handler/admin/PluginInfoHandler.java   |   1 -
 .../handler/admin/PropertiesRequestHandler.java |   1 -
 .../solr/handler/admin/RebalanceLeaders.java    |  33 +-
 .../solr/handler/admin/SecurityConfHandler.java |   3 +-
 .../admin/SegmentsInfoRequestHandler.java       |  33 +-
 .../handler/admin/ShowFileRequestHandler.java   |   1 -
 .../handler/admin/SolrInfoMBeanHandler.java     |   3 +-
 .../solr/handler/admin/SystemInfoHandler.java   |   1 -
 .../solr/handler/admin/ThreadDumpHandler.java   |   1 -
 .../handler/admin/ZookeeperInfoHandler.java     |   1 -
 .../handler/component/DateFacetProcessor.java   |   3 +-
 .../solr/handler/component/DebugComponent.java  |   1 -
 .../solr/handler/component/ExpandComponent.java |   1 -
 .../solr/handler/component/FacetComponent.java  |   1 -
 .../solr/handler/component/FieldFacetStats.java |   3 +-
 .../handler/component/HighlightComponent.java   |   1 -
 .../handler/component/HttpShardHandler.java     |   3 +-
 .../component/HttpShardHandlerFactory.java      |   3 +-
 .../component/IterativeMergeStrategy.java       |  31 +-
 .../solr/handler/component/MergeStrategy.java   |  31 +-
 .../component/MoreLikeThisComponent.java        |   1 -
 .../solr/handler/component/PivotFacet.java      |   1 -
 .../solr/handler/component/PivotFacetField.java |   1 -
 .../PivotFacetFieldValueCollection.java         |   3 +-
 .../handler/component/PivotFacetHelper.java     |   1 -
 .../handler/component/PivotFacetProcessor.java  |   3 +-
 .../solr/handler/component/PivotFacetValue.java |   3 +-
 .../solr/handler/component/QueryComponent.java  |  49 +-
 .../component/QueryElevationComponent.java      |   1 -
 .../handler/component/RangeFacetProcessor.java  |   3 +-
 .../handler/component/RangeFacetRequest.java    |   7 +-
 .../handler/component/RealTimeGetComponent.java |  11 +-
 .../solr/handler/component/ResponseBuilder.java |   5 +-
 .../handler/component/ResponseLogComponent.java |   3 +-
 .../solr/handler/component/SearchComponent.java |   1 -
 .../solr/handler/component/SearchHandler.java   |   1 -
 .../apache/solr/handler/component/ShardDoc.java | 166 +---
 .../component/ShardFieldSortedHitQueue.java     | 178 ++++
 .../solr/handler/component/ShardHandler.java    |   4 +-
 .../handler/component/ShardHandlerFactory.java  |   4 +-
 .../solr/handler/component/ShardResponse.java   |   3 +-
 .../handler/component/SpatialHeatmapFacets.java |   3 +-
 .../handler/component/SpellCheckComponent.java  |   1 -
 .../handler/component/SpellCheckMergeData.java  |   3 +-
 .../solr/handler/component/StatsComponent.java  |   1 -
 .../solr/handler/component/StatsField.java      |   1 -
 .../solr/handler/component/StatsValues.java     |   2 -
 .../handler/component/StatsValuesFactory.java   |   3 +-
 .../handler/component/SuggestComponent.java     |   3 +-
 .../handler/component/TermVectorComponent.java  |  34 +-
 .../solr/handler/component/TermsComponent.java  |   3 +-
 .../apache/solr/handler/loader/CSVLoader.java   |   1 -
 .../solr/handler/loader/CSVLoaderBase.java      |   1 -
 .../handler/loader/ContentStreamLoader.java     |   4 +-
 .../solr/handler/loader/JavabinLoader.java      |   1 -
 .../apache/solr/handler/loader/JsonLoader.java  |   3 +-
 .../apache/solr/handler/loader/XMLLoader.java   |   3 +-
 .../highlight/BreakIteratorBoundaryScanner.java |   1 -
 .../apache/solr/highlight/DefaultEncoder.java   |   1 -
 .../solr/highlight/HighlightingPluginBase.java  |   1 -
 .../org/apache/solr/highlight/HtmlEncoder.java  |   1 -
 .../solr/highlight/PostingsSolrHighlighter.java |   3 +-
 .../highlight/ScoreOrderFragmentsBuilder.java   |   1 -
 .../solr/highlight/SimpleBoundaryScanner.java   |   1 -
 .../solr/highlight/SimpleFragListBuilder.java   |   1 -
 .../solr/highlight/SimpleFragmentsBuilder.java  |   1 -
 .../solr/highlight/SingleFragListBuilder.java   |   1 -
 .../solr/highlight/SolrBoundaryScanner.java     |   1 -
 .../org/apache/solr/highlight/SolrEncoder.java  |   1 -
 .../apache/solr/highlight/SolrFormatter.java    |   1 -
 .../solr/highlight/SolrFragListBuilder.java     |   1 -
 .../apache/solr/highlight/SolrFragmenter.java   |   1 -
 .../solr/highlight/SolrFragmentsBuilder.java    |   1 -
 .../apache/solr/highlight/SolrHighlighter.java  |   3 +-
 .../solr/highlight/WeightedFragListBuilder.java |   1 -
 .../apache/solr/index/hdfs/CheckHdfsIndex.java  |   3 +-
 .../org/apache/solr/internal/csv/CSVParser.java |   6 +-
 .../apache/solr/internal/csv/CSVPrinter.java    |   6 +-
 .../apache/solr/internal/csv/CSVStrategy.java   |   6 +-
 .../org/apache/solr/internal/csv/CSVUtils.java  |   6 +-
 .../apache/solr/internal/csv/CharBuffer.java    |  26 +-
 .../internal/csv/ExtendedBufferedReader.java    |   6 +-
 .../solr/internal/csv/writer/CSVConfig.java     |  26 +-
 .../internal/csv/writer/CSVConfigGuesser.java   |  26 +-
 .../solr/internal/csv/writer/CSVField.java      |  26 +-
 .../solr/internal/csv/writer/CSVWriter.java     |  26 +-
 .../org/apache/solr/logging/CircularList.java   |   1 -
 .../org/apache/solr/logging/ListenerConfig.java |   5 +-
 .../org/apache/solr/logging/LogWatcher.java     |   1 -
 .../apache/solr/logging/LogWatcherConfig.java   |   3 +-
 .../org/apache/solr/logging/LoggerInfo.java     |   1 -
 .../apache/solr/logging/MDCLoggingContext.java  |   3 +-
 .../org/apache/solr/parser/FastCharStream.java  |   5 +-
 .../java/org/apache/solr/parser/QueryParser.jj  |   2 +-
 .../solr/parser/QueryParserTokenManager.java    | 426 ++++----
 .../apache/solr/parser/SolrQueryParserBase.java |   1 -
 .../java/org/apache/solr/query/FilterQuery.java |   3 +-
 .../org/apache/solr/query/SolrRangeQuery.java   |   3 +-
 .../apache/solr/request/DocValuesFacets.java    |   3 +-
 .../org/apache/solr/request/DocValuesStats.java |   3 +-
 .../org/apache/solr/request/IntervalFacets.java |  33 +-
 .../solr/request/LocalSolrQueryRequest.java     |   1 -
 .../org/apache/solr/request/NumericFacets.java  |   3 +-
 .../request/PerSegmentSingleValuedFaceting.java |   1 -
 .../org/apache/solr/request/SimpleFacets.java   |   1 -
 .../apache/solr/request/SolrQueryRequest.java   |   1 -
 .../solr/request/SolrQueryRequestBase.java      |   1 -
 .../apache/solr/request/SolrRequestHandler.java |   1 -
 .../apache/solr/request/SolrRequestInfo.java    |   1 -
 .../org/apache/solr/request/json/JSONUtil.java  |   4 +-
 .../apache/solr/request/json/ObjectUtil.java    |   4 +-
 .../apache/solr/request/json/RequestUtil.java   |   4 +-
 .../solr/request/macro/MacroExpander.java       |   3 +-
 .../solr/response/BasicResultContext.java       |   3 +-
 .../response/BinaryQueryResponseWriter.java     |   4 +-
 .../apache/solr/response/CSVResponseWriter.java |   1 -
 .../org/apache/solr/response/DocsStreamer.java  |   3 +-
 .../solr/response/JSONResponseWriter.java       |   1 -
 .../apache/solr/response/PHPResponseWriter.java |   1 -
 .../response/PHPSerializedResponseWriter.java   |   1 -
 .../solr/response/PythonResponseWriter.java     |   1 -
 .../solr/response/QueryResponseWriter.java      |   1 -
 .../solr/response/QueryResponseWriterUtil.java  |   1 -
 .../apache/solr/response/RawResponseWriter.java |   1 -
 .../org/apache/solr/response/ResultContext.java |   1 -
 .../solr/response/SchemaXmlResponseWriter.java  |   1 -
 .../apache/solr/response/SchemaXmlWriter.java   |   3 +-
 .../solr/response/SmileResponseWriter.java      |   4 +-
 .../apache/solr/response/SolrQueryResponse.java |   1 -
 .../solr/response/SortingResponseWriter.java    | 101 +-
 .../solr/response/TextResponseWriter.java       |   1 -
 .../apache/solr/response/XMLResponseWriter.java |   1 -
 .../org/apache/solr/response/XMLWriter.java     |   1 -
 .../solr/response/XSLTResponseWriter.java       |   1 -
 .../transform/BaseEditorialTransformer.java     |   5 +-
 .../solr/response/transform/DocTransformer.java |   1 -
 .../response/transform/DocTransformers.java     |   1 -
 .../transform/ElevatedMarkerFactory.java        |   3 +-
 .../transform/ExcludedMarkerFactory.java        |   3 +-
 .../response/transform/TransformerFactory.java  |   1 -
 .../org/apache/solr/rest/BaseSolrResource.java  |   3 +-
 .../java/org/apache/solr/rest/DELETEable.java   |   3 +-
 .../src/java/org/apache/solr/rest/GETable.java  |   3 +-
 .../org/apache/solr/rest/ManagedResource.java   |   3 +-
 .../solr/rest/ManagedResourceObserver.java      |   3 +-
 .../solr/rest/ManagedResourceStorage.java       |   3 +-
 .../src/java/org/apache/solr/rest/POSTable.java |   3 +-
 .../src/java/org/apache/solr/rest/PUTable.java  |   3 +-
 .../java/org/apache/solr/rest/RestManager.java  |   3 +-
 .../org/apache/solr/rest/SolrSchemaRestApi.java |   3 +-
 .../solr/rest/schema/BaseFieldResource.java     |   3 +-
 .../solr/rest/schema/BaseFieldTypeResource.java |   3 +-
 .../schema/CopyFieldCollectionResource.java     |   4 +-
 .../schema/DynamicFieldCollectionResource.java  |   3 +-
 .../solr/rest/schema/DynamicFieldResource.java  |   4 +-
 .../rest/schema/FieldCollectionResource.java    |   4 +-
 .../apache/solr/rest/schema/FieldResource.java  |   3 +-
 .../schema/FieldTypeCollectionResource.java     |   3 +-
 .../solr/rest/schema/FieldTypeResource.java     |   3 +-
 .../solr/rest/schema/FieldTypeXmlAdapter.java   |   3 +-
 .../analysis/BaseManagedTokenFilterFactory.java |   3 +-
 .../analysis/ManagedStopFilterFactory.java      |   3 +-
 .../analysis/ManagedSynonymFilterFactory.java   |   3 +-
 .../schema/analysis/ManagedWordSetResource.java |   3 +-
 .../solr/schema/AbstractSpatialFieldType.java   |   3 +-
 .../AbstractSpatialPrefixTreeFieldType.java     |   3 +-
 .../solr/schema/AbstractSubTypeFieldType.java   |   3 +-
 .../java/org/apache/solr/schema/BBoxField.java  |   3 +-
 .../org/apache/solr/schema/BinaryField.java     |   1 -
 .../java/org/apache/solr/schema/BoolField.java  |   1 -
 .../solr/schema/ClassicIndexSchemaFactory.java  |   3 +-
 .../org/apache/solr/schema/CollationField.java  |   3 +-
 .../apache/solr/schema/CoordinateFieldType.java |   1 -
 .../java/org/apache/solr/schema/CopyField.java  |   1 -
 .../org/apache/solr/schema/CurrencyField.java   |   3 +-
 .../org/apache/solr/schema/DateRangeField.java  |   3 +-
 .../apache/solr/schema/DateValueFieldType.java  |   1 -
 .../solr/schema/DoubleValueFieldType.java       |   1 -
 .../java/org/apache/solr/schema/EnumField.java  |   3 +-
 .../solr/schema/ExchangeRateProvider.java       |   3 +-
 .../solr/schema/ExternalFileFieldReloader.java  |   1 -
 .../org/apache/solr/schema/FieldProperties.java |   1 -
 .../java/org/apache/solr/schema/FieldType.java  |   1 -
 .../solr/schema/FieldTypePluginLoader.java      |   1 -
 .../apache/solr/schema/FloatValueFieldType.java |   1 -
 .../org/apache/solr/schema/GeoHashField.java    |   1 -
 .../org/apache/solr/schema/IndexSchema.java     |   1 -
 .../apache/solr/schema/IndexSchemaFactory.java  |   3 +-
 .../apache/solr/schema/IntValueFieldType.java   |   1 -
 .../solr/schema/JsonPreAnalyzedParser.java      |  33 +-
 .../java/org/apache/solr/schema/LatLonType.java |   3 +-
 .../apache/solr/schema/LongValueFieldType.java  |   1 -
 .../apache/solr/schema/ManagedIndexSchema.java  |   7 +-
 .../solr/schema/ManagedIndexSchemaFactory.java  |   3 +-
 .../solr/schema/NumericValueFieldType.java      |   1 -
 .../schema/OpenExchangeRatesOrgProvider.java    |   3 +-
 .../java/org/apache/solr/schema/PointType.java  |   1 -
 .../apache/solr/schema/PreAnalyzedField.java    |   3 +-
 .../apache/solr/schema/PrimitiveFieldType.java  |   1 -
 .../org/apache/solr/schema/RandomSortField.java |   1 -
 .../schema/RptWithGeometrySpatialField.java     |   3 +-
 .../org/apache/solr/schema/SchemaAware.java     |   4 +-
 .../org/apache/solr/schema/SchemaField.java     |   1 -
 .../org/apache/solr/schema/SchemaManager.java   |   4 +-
 .../apache/solr/schema/SimilarityFactory.java   |   3 +-
 .../solr/schema/SimplePreAnalyzedParser.java    |   3 +-
 .../schema/SpatialPointVectorFieldType.java     |   3 +-
 .../apache/solr/schema/SpatialQueryable.java    |   4 +-
 .../SpatialRecursivePrefixTreeFieldType.java    |   3 +-
 .../SpatialTermQueryPrefixTreeFieldType.java    |   1 -
 .../java/org/apache/solr/schema/StrField.java   |   1 -
 .../org/apache/solr/schema/StrFieldSource.java  |   1 -
 .../java/org/apache/solr/schema/TextField.java  |   1 -
 .../org/apache/solr/schema/TrieDateField.java   |   1 -
 .../org/apache/solr/schema/TrieDoubleField.java |   1 -
 .../org/apache/solr/schema/TrieFloatField.java  |   1 -
 .../org/apache/solr/schema/TrieIntField.java    |   1 -
 .../org/apache/solr/schema/TrieLongField.java   |   1 -
 .../java/org/apache/solr/schema/UUIDField.java  |   3 +-
 .../apache/solr/schema/ZkIndexSchemaReader.java |   3 +-
 .../org/apache/solr/search/AnalyticsQuery.java  |  31 +-
 .../java/org/apache/solr/search/BitDocSet.java  |   1 -
 .../solr/search/BitsFilteredDocIdSet.java       |   3 +-
 .../solr/search/BitsFilteredPostingsEnum.java   |   3 +-
 .../apache/solr/search/BoostQParserPlugin.java  |   4 -
 .../org/apache/solr/search/CacheConfig.java     |   1 -
 .../apache/solr/search/CacheRegenerator.java    |   1 -
 .../solr/search/CollapsingQParserPlugin.java    |   5 -
 .../solr/search/ComplexPhraseQParserPlugin.java |   4 +-
 .../java/org/apache/solr/search/CursorMark.java |   1 -
 .../apache/solr/search/DelegatingCollector.java |   1 -
 .../apache/solr/search/DisMaxQParserPlugin.java |   4 -
 .../org/apache/solr/search/DocIterator.java     |   1 -
 .../java/org/apache/solr/search/DocList.java    |   1 -
 .../org/apache/solr/search/DocListAndSet.java   |   1 -
 .../src/java/org/apache/solr/search/DocSet.java |   1 -
 .../java/org/apache/solr/search/DocSetBase.java |   1 -
 .../org/apache/solr/search/DocSetBuilder.java   |   3 +-
 .../org/apache/solr/search/DocSetCollector.java |   3 +-
 .../org/apache/solr/search/DocSetProducer.java  |   3 +-
 .../java/org/apache/solr/search/DocSetUtil.java |   4 +-
 .../java/org/apache/solr/search/DocSlice.java   |   1 -
 .../solr/search/EarlyTerminatingCollector.java  |   3 +-
 .../EarlyTerminatingCollectorException.java     |   4 +-
 .../apache/solr/search/ExportQParserPlugin.java |   4 -
 .../solr/search/ExtendedDismaxQParser.java      |   1 -
 .../search/ExtendedDismaxQParserPlugin.java     |   5 -
 .../org/apache/solr/search/ExtendedQuery.java   |   1 -
 .../apache/solr/search/ExtendedQueryBase.java   |   1 -
 .../org/apache/solr/search/FastLRUCache.java    |   3 +-
 .../org/apache/solr/search/FieldParams.java     |   3 +-
 .../apache/solr/search/FieldQParserPlugin.java  |   4 -
 .../src/java/org/apache/solr/search/Filter.java |   3 +-
 .../apache/solr/search/FilteredDocIdSet.java    |   3 +-
 .../solr/search/FunctionQParserPlugin.java      |   4 -
 .../solr/search/FunctionRangeQParserPlugin.java |   4 -
 .../apache/solr/search/FunctionRangeQuery.java  |   1 -
 .../java/org/apache/solr/search/Grouping.java   |   1 -
 .../java/org/apache/solr/search/HashDocSet.java |   1 -
 .../apache/solr/search/HashQParserPlugin.java   |   5 -
 .../java/org/apache/solr/search/Insanity.java   |   3 +-
 .../apache/solr/search/JoinQParserPlugin.java   |   4 -
 .../java/org/apache/solr/search/LFUCache.java   |   3 +-
 .../java/org/apache/solr/search/LRUCache.java   |   1 -
 .../apache/solr/search/LuceneQParserPlugin.java |   4 -
 .../apache/solr/search/NestedQParserPlugin.java |   4 -
 .../org/apache/solr/search/NoOpRegenerator.java |   3 +-
 .../solr/search/OldLuceneQParserPlugin.java     |   4 -
 .../java/org/apache/solr/search/PostFilter.java |   1 -
 .../apache/solr/search/PrefixQParserPlugin.java |   4 -
 .../org/apache/solr/search/QParserPlugin.java   |   4 +
 .../org/apache/solr/search/QueryCommand.java    |   3 +-
 .../org/apache/solr/search/QueryContext.java    |   3 +-
 .../org/apache/solr/search/QueryParsing.java    |   1 -
 .../org/apache/solr/search/QueryResult.java     |   3 +-
 .../org/apache/solr/search/QueryResultKey.java  |   1 -
 .../java/org/apache/solr/search/QueryUtils.java |   1 -
 .../apache/solr/search/QueryWrapperFilter.java  |   3 +-
 .../java/org/apache/solr/search/RankQuery.java  |  31 +-
 .../apache/solr/search/RawQParserPlugin.java    |   4 -
 .../apache/solr/search/ReRankQParserPlugin.java |  29 +-
 .../org/apache/solr/search/ScoreFilter.java     |   1 -
 .../apache/solr/search/SimpleQParserPlugin.java |   8 +-
 .../java/org/apache/solr/search/SolrCache.java  |   1 -
 .../org/apache/solr/search/SolrCacheBase.java   |   1 -
 .../solr/search/SolrConstantScoreQuery.java     |  31 +-
 .../org/apache/solr/search/SolrCoreParser.java  |   3 +-
 .../apache/solr/search/SolrFieldCacheMBean.java |   1 -
 .../java/org/apache/solr/search/SolrFilter.java |   1 -
 .../apache/solr/search/SolrIndexSearcher.java   |   1 -
 .../org/apache/solr/search/SolrQueryParser.java |   1 -
 .../solr/search/SolrQueryTimeoutImpl.java       |   3 +-
 .../java/org/apache/solr/search/SortSpec.java   |   1 -
 .../org/apache/solr/search/SortSpecParsing.java |  31 +-
 .../org/apache/solr/search/SortedIntDocSet.java |   1 -
 .../java/org/apache/solr/search/Sorting.java    |   1 -
 .../solr/search/SpatialBoxQParserPlugin.java    |   6 -
 .../solr/search/SpatialFilterQParser.java       |   4 +-
 .../solr/search/SpatialFilterQParserPlugin.java |   8 +-
 .../org/apache/solr/search/SpatialOptions.java  |   3 +-
 .../java/org/apache/solr/search/StrParser.java  |   7 +-
 .../solr/search/SurroundQParserPlugin.java      |   8 +-
 .../apache/solr/search/SwitchQParserPlugin.java |   4 -
 .../org/apache/solr/search/SyntaxError.java     |   1 -
 .../apache/solr/search/TermQParserPlugin.java   |   4 -
 .../apache/solr/search/TermsQParserPlugin.java  |   7 +-
 .../org/apache/solr/search/WrappedQuery.java    |   1 -
 .../apache/solr/search/XmlQParserPlugin.java    |   7 +-
 .../solr/search/facet/AggValueSource.java       |   3 +-
 .../org/apache/solr/search/facet/AvgAgg.java    |   3 +-
 .../org/apache/solr/search/facet/BlockJoin.java |   4 +-
 .../org/apache/solr/search/facet/CountAgg.java  |   3 +-
 .../solr/search/facet/FacetDebugInfo.java       |   1 -
 .../apache/solr/search/facet/FacetField.java    |   3 +-
 .../search/facet/FacetFieldProcessorDV.java     |   3 +-
 .../facet/FacetFieldProcessorNumeric.java       |   3 +-
 .../apache/solr/search/facet/FacetMerger.java   |   3 +-
 .../apache/solr/search/facet/FacetModule.java   |   3 +-
 .../solr/search/facet/FacetProcessor.java       |   3 +-
 .../apache/solr/search/facet/FacetQuery.java    |   3 +-
 .../apache/solr/search/facet/FacetRange.java    |   3 +-
 .../apache/solr/search/facet/FacetRequest.java  |  13 +-
 .../org/apache/solr/search/facet/FieldUtil.java |   3 +-
 .../org/apache/solr/search/facet/HLLAgg.java    |   3 +-
 .../apache/solr/search/facet/LegacyFacet.java   |   4 +-
 .../org/apache/solr/search/facet/MaxAgg.java    |   3 +-
 .../org/apache/solr/search/facet/MinAgg.java    |   3 +-
 .../apache/solr/search/facet/PercentileAgg.java |   3 +-
 .../solr/search/facet/SimpleAggValueSource.java |   3 +-
 .../org/apache/solr/search/facet/SlotAcc.java   |   3 +-
 .../solr/search/facet/StrAggValueSource.java    |   3 +-
 .../org/apache/solr/search/facet/SumAgg.java    |   3 +-
 .../org/apache/solr/search/facet/SumsqAgg.java  |   3 +-
 .../solr/search/facet/UnInvertedField.java      |   1 -
 .../org/apache/solr/search/facet/UniqueAgg.java |   3 +-
 .../apache/solr/search/facet/UniqueSlotAcc.java |   3 +-
 .../search/function/CollapseScoreFunction.java  |   1 -
 .../solr/search/function/OrdFieldSource.java    |   1 -
 .../search/function/ReverseOrdFieldSource.java  |   1 -
 .../search/function/ValueSourceRangeFilter.java |   1 -
 .../distance/GeoDistValueSourceParser.java      |   3 +-
 .../function/distance/GeohashFunction.java      |   3 +-
 .../distance/GeohashHaversineFunction.java      |   4 +-
 .../distance/HaversineConstFunction.java        |   3 +-
 .../function/distance/HaversineFunction.java    |   3 +-
 .../distance/SquaredEuclideanFunction.java      |   3 +-
 .../distance/StringDistanceFunction.java        |   3 +-
 .../distance/VectorDistanceFunction.java        |   3 +-
 .../apache/solr/search/grouping/Command.java    |   3 +-
 .../solr/search/grouping/CommandHandler.java    |   3 +-
 .../search/grouping/GroupingSpecification.java  |   3 +-
 .../grouping/collector/FilterCollector.java     |   3 +-
 .../distributed/ShardRequestFactory.java        |   3 +-
 .../distributed/ShardResponseProcessor.java     |   3 +-
 .../distributed/command/GroupConverter.java     |   3 +-
 .../distributed/command/QueryCommand.java       |   3 +-
 .../distributed/command/QueryCommandResult.java |   3 +-
 .../command/SearchGroupsFieldCommand.java       |   3 +-
 .../command/SearchGroupsFieldCommandResult.java |   3 +-
 .../command/TopGroupsFieldCommand.java          |   3 +-
 .../SearchGroupsRequestFactory.java             |   3 +-
 .../StoredFieldsShardRequestFactory.java        |   3 +-
 .../TopGroupsShardRequestFactory.java           |   3 +-
 .../SearchGroupShardResponseProcessor.java      |   3 +-
 .../StoredFieldsShardResponseProcessor.java     |   3 +-
 .../TopGroupsShardResponseProcessor.java        |   3 +-
 .../SearchGroupsResultTransformer.java          |   3 +-
 .../ShardResultTransformer.java                 |   3 +-
 .../TopGroupsResultTransformer.java             |   3 +-
 .../EndResultTransformer.java                   |   3 +-
 .../GroupedEndResultTransformer.java            |   3 +-
 .../MainEndResultTransformer.java               |   3 +-
 .../SimpleEndResultTransformer.java             |   3 +-
 .../apache/solr/search/join/BitSetSlice.java    |   1 -
 .../solr/search/join/BlockJoinChildQParser.java |   1 -
 .../join/BlockJoinChildQParserPlugin.java       |   1 -
 .../join/BlockJoinDocSetFacetComponent.java     |  33 +-
 .../search/join/BlockJoinFacetCollector.java    |   3 +-
 .../search/join/BlockJoinFacetComponent.java    |   3 +-
 .../solr/search/join/BlockJoinFacetFilter.java  |   3 +-
 .../join/BlockJoinFieldFacetAccumulator.java    |   3 +-
 .../search/join/BlockJoinParentQParser.java     |   1 -
 .../join/BlockJoinParentQParserPlugin.java      |   5 -
 .../apache/solr/search/join/FrontierQuery.java  |   3 +-
 .../solr/search/join/GraphQParserPlugin.java    |   7 +-
 .../org/apache/solr/search/join/GraphQuery.java |  92 +-
 .../solr/search/join/GraphQueryParser.java      |   3 +-
 .../solr/search/join/GraphTermsCollector.java   |   5 +-
 .../search/join/ScoreJoinQParserPlugin.java     |   5 -
 .../solr/search/join/ScoreModeParser.java       |  19 +-
 .../apache/solr/search/mlt/CloudMLTQParser.java |   3 +-
 .../solr/search/mlt/MLTQParserPlugin.java       |   8 +-
 .../solr/search/mlt/SimpleMLTQParser.java       |   3 +-
 .../similarities/BM25SimilarityFactory.java     |   3 +-
 .../similarities/ClassicSimilarityFactory.java  |   3 +-
 .../similarities/DFISimilarityFactory.java      |   3 +-
 .../similarities/DFRSimilarityFactory.java      |   3 +-
 .../similarities/IBSimilarityFactory.java       |   3 +-
 .../LMDirichletSimilarityFactory.java           |   3 +-
 .../LMJelinekMercerSimilarityFactory.java       |   3 +-
 .../similarities/SchemaSimilarityFactory.java   |   3 +-
 .../SweetSpotSimilarityFactory.java             |   3 +-
 .../solr/search/stats/CachedSearcherStats.java  |   3 +-
 .../solr/search/stats/CollectionStats.java      |   5 +-
 .../search/stats/ExactSharedStatsCache.java     |   3 +-
 .../solr/search/stats/ExactStatsCache.java      |   3 +-
 .../apache/solr/search/stats/LRUStatsCache.java |   3 +-
 .../solr/search/stats/LocalStatsCache.java      |   7 +-
 .../solr/search/stats/LocalStatsSource.java     |   3 +-
 .../apache/solr/search/stats/StatsCache.java    |   3 +-
 .../apache/solr/search/stats/StatsSource.java   |   3 +-
 .../org/apache/solr/search/stats/StatsUtil.java |   7 +-
 .../org/apache/solr/search/stats/TermStats.java |   3 +-
 .../solr/security/AuthenticationPlugin.java     |  33 +-
 .../solr/security/AuthorizationContext.java     |   3 +-
 .../solr/security/AuthorizationPlugin.java      |   3 +-
 .../solr/security/AuthorizationResponse.java    |   3 +-
 .../apache/solr/security/BasicAuthPlugin.java   |   4 +-
 .../solr/security/ConfigEditablePlugin.java     |   3 +-
 .../security/HttpClientInterceptorPlugin.java   |   4 +-
 .../apache/solr/security/KerberosFilter.java    |  25 +-
 .../apache/solr/security/KerberosPlugin.java    |  33 +-
 .../solr/security/PKIAuthenticationPlugin.java  |   3 +-
 .../security/RuleBasedAuthorizationPlugin.java  |   3 +-
 .../solr/security/SecurityPluginHolder.java     |   4 +-
 .../security/Sha256AuthenticationProvider.java  |   3 +-
 .../org/apache/solr/servlet/BaseSolrFilter.java |   1 -
 .../apache/solr/servlet/BaseSolrServlet.java    |   1 -
 .../solr/servlet/CheckLoggingConfiguration.java |   1 -
 .../solr/servlet/DirectSolrConnection.java      |   1 -
 .../org/apache/solr/servlet/HttpSolrCall.java   |   7 +-
 .../apache/solr/servlet/LoadAdminUiServlet.java |   1 -
 .../apache/solr/servlet/RedirectServlet.java    |   1 -
 .../org/apache/solr/servlet/ResponseUtils.java  |   3 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |   1 -
 .../apache/solr/servlet/SolrRequestParsers.java |   1 -
 .../solr/servlet/cache/HttpCacheHeaderUtil.java |   1 -
 .../org/apache/solr/servlet/cache/Method.java   |   1 -
 .../spelling/AbstractLuceneSpellChecker.java    |   5 +-
 .../spelling/ConjunctionSolrSpellChecker.java   |   3 +-
 .../solr/spelling/DirectSolrSpellChecker.java   |   3 +-
 .../solr/spelling/IndexBasedSpellChecker.java   |   3 +-
 .../solr/spelling/PossibilityIterator.java      |   3 +-
 .../apache/solr/spelling/QueryConverter.java    |   3 +-
 .../org/apache/solr/spelling/ResultEntry.java   |   3 +-
 .../apache/solr/spelling/SolrSpellChecker.java  |   3 +-
 .../solr/spelling/SpellCheckCollation.java      |   3 +-
 .../solr/spelling/SpellCheckCollator.java       |   3 +-
 .../solr/spelling/SpellCheckCorrection.java     |   3 +-
 .../apache/solr/spelling/SpellingOptions.java   |  17 +-
 .../solr/spelling/SpellingQueryConverter.java   |   1 -
 .../apache/solr/spelling/SpellingResult.java    |   3 +-
 .../solr/spelling/SuggestQueryConverter.java    |   3 +-
 .../spelling/WordBreakSolrSpellChecker.java     |   3 +-
 .../spelling/suggest/DictionaryFactory.java     |   3 +-
 .../suggest/DocumentDictionaryFactory.java      |   3 +-
 .../DocumentExpressionDictionaryFactory.java    |   3 +-
 .../spelling/suggest/FileDictionaryFactory.java |   3 +-
 .../suggest/HighFrequencyDictionaryFactory.java |   3 +-
 .../solr/spelling/suggest/LookupFactory.java    |   3 +-
 .../solr/spelling/suggest/SolrSuggester.java    |   3 +-
 .../apache/solr/spelling/suggest/Suggester.java |   1 -
 .../solr/spelling/suggest/SuggesterOptions.java |   3 +-
 .../solr/spelling/suggest/SuggesterParams.java  |   3 +-
 .../solr/spelling/suggest/SuggesterResult.java  |   3 +-
 .../fst/AnalyzingInfixLookupFactory.java        |   3 +-
 .../suggest/fst/AnalyzingLookupFactory.java     |   3 +-
 .../suggest/fst/BlendedInfixLookupFactory.java  |   3 +-
 .../spelling/suggest/fst/FSTLookupFactory.java  |   3 +-
 .../suggest/fst/FreeTextLookupFactory.java      |  23 +-
 .../suggest/fst/FuzzyLookupFactory.java         |   3 +-
 .../spelling/suggest/fst/WFSTLookupFactory.java |   3 +-
 .../suggest/jaspell/JaspellLookupFactory.java   |   7 +-
 .../spelling/suggest/tst/TSTLookupFactory.java  |   3 +-
 .../solr/store/blockcache/BlockCache.java       |   3 +-
 .../solr/store/blockcache/BlockCacheKey.java    |   4 +-
 .../store/blockcache/BlockCacheLocation.java    |   3 +-
 .../solr/store/blockcache/BlockDirectory.java   |   5 +-
 .../store/blockcache/BlockDirectoryCache.java   |   3 +-
 .../solr/store/blockcache/BlockLocks.java       |   3 +-
 .../solr/store/blockcache/BufferStore.java      |   3 +-
 .../org/apache/solr/store/blockcache/Cache.java |   3 +-
 .../store/blockcache/CachedIndexOutput.java     |   3 +-
 .../blockcache/CustomBufferedIndexInput.java    |   3 +-
 .../apache/solr/store/blockcache/Metrics.java   |   3 +-
 .../blockcache/ReusedBufferedIndexOutput.java   |   3 +-
 .../org/apache/solr/store/blockcache/Store.java |   3 +-
 .../apache/solr/store/hdfs/HdfsDirectory.java   |   3 +-
 .../apache/solr/store/hdfs/HdfsFileReader.java  |   3 +-
 .../apache/solr/store/hdfs/HdfsFileWriter.java  |   3 +-
 .../solr/store/hdfs/HdfsLocalityReporter.java   |   3 +-
 .../apache/solr/store/hdfs/HdfsLockFactory.java |   3 +-
 .../apache/solr/update/AddUpdateCommand.java    |   1 -
 .../apache/solr/update/CdcrTransactionLog.java  |   3 +-
 .../org/apache/solr/update/CdcrUpdateLog.java   |   3 +-
 .../org/apache/solr/update/CommitTracker.java   |   7 +-
 .../apache/solr/update/CommitUpdateCommand.java |   1 -
 .../solr/update/DefaultSolrCoreState.java       |   3 +-
 .../solr/update/DeleteByQueryWrapper.java       |   3 +-
 .../apache/solr/update/DeleteUpdateCommand.java |   1 -
 .../solr/update/DirectUpdateHandler2.java       |   1 -
 .../org/apache/solr/update/DocumentBuilder.java |   1 -
 .../apache/solr/update/HdfsTransactionLog.java  |  23 +-
 .../org/apache/solr/update/HdfsUpdateLog.java   |   1 -
 .../apache/solr/update/IndexFingerprint.java    | 200 ++++
 .../apache/solr/update/LoggingInfoStream.java   |   1 -
 .../org/apache/solr/update/MemOutputStream.java |   1 -
 .../apache/solr/update/MergeIndexesCommand.java |   1 -
 .../java/org/apache/solr/update/PeerSync.java   |  79 +-
 .../solr/update/RollbackUpdateCommand.java      |   1 -
 .../apache/solr/update/SolrCmdDistributor.java  |   3 +-
 .../org/apache/solr/update/SolrCoreState.java   |   3 +-
 .../org/apache/solr/update/SolrIndexConfig.java |   1 -
 .../apache/solr/update/SolrIndexSplitter.java   |   1 -
 .../org/apache/solr/update/SolrIndexWriter.java |   1 -
 .../apache/solr/update/SplitIndexCommand.java   |   1 -
 .../solr/update/StreamingSolrClients.java       |   3 +-
 .../org/apache/solr/update/TransactionLog.java  |   1 -
 .../org/apache/solr/update/UpdateCommand.java   |   1 -
 .../org/apache/solr/update/UpdateHandler.java   |   1 -
 .../java/org/apache/solr/update/UpdateLog.java  |   7 +-
 .../apache/solr/update/UpdateShardHandler.java  |   3 +-
 .../solr/update/UpdateShardHandlerConfig.java   |   3 +-
 .../org/apache/solr/update/VersionBucket.java   |   1 -
 .../org/apache/solr/update/VersionInfo.java     |   1 -
 ...tractDefaultValueUpdateProcessorFactory.java |   1 -
 .../AddSchemaFieldsUpdateProcessorFactory.java  |   1 -
 ...aluesOrNoneFieldMutatingUpdateProcessor.java |   1 -
 .../processor/AtomicUpdateDocumentMerger.java   |   7 +-
 .../update/processor/CdcrUpdateProcessor.java   |   3 +-
 .../processor/CdcrUpdateProcessorFactory.java   |   3 +-
 .../ConcatFieldUpdateProcessorFactory.java      |   1 -
 .../CountFieldValuesUpdateProcessorFactory.java |   1 -
 .../DefaultValueUpdateProcessorFactory.java     |   1 -
 .../processor/DistributedUpdateProcessor.java   |   7 +-
 .../DistributedUpdateProcessorFactory.java      |   3 +-
 .../DistributingUpdateProcessorFactory.java     |   1 -
 ...BasedVersionConstraintsProcessorFactory.java |   1 -
 .../DocExpirationUpdateProcessorFactory.java    |   1 -
 .../FieldLengthUpdateProcessorFactory.java      |   1 -
 .../processor/FieldMutatingUpdateProcessor.java |   1 -
 .../FieldMutatingUpdateProcessorFactory.java    |   1 -
 ...FieldNameMutatingUpdateProcessorFactory.java |   4 +-
 .../FieldValueMutatingUpdateProcessor.java      |   1 -
 .../FieldValueSubsetUpdateProcessorFactory.java |   1 -
 .../FirstFieldValueUpdateProcessorFactory.java  |   1 -
 .../HTMLStripFieldUpdateProcessorFactory.java   |   1 -
 ...oreCommitOptimizeUpdateProcessorFactory.java |   1 -
 .../IgnoreFieldUpdateProcessorFactory.java      |   1 -
 .../LastFieldValueUpdateProcessorFactory.java   |   1 -
 .../processor/LogUpdateProcessorFactory.java    |   1 -
 .../solr/update/processor/Lookup3Signature.java |   3 +-
 .../solr/update/processor/MD5Signature.java     |   3 +-
 .../MaxFieldValueUpdateProcessorFactory.java    |   1 -
 .../MinFieldValueUpdateProcessorFactory.java    |   1 -
 .../NoOpDistributingUpdateProcessorFactory.java |   1 -
 ...ParseBooleanFieldUpdateProcessorFactory.java |   1 -
 .../ParseDateFieldUpdateProcessorFactory.java   |   1 -
 .../ParseDoubleFieldUpdateProcessorFactory.java |   1 -
 .../ParseFloatFieldUpdateProcessorFactory.java  |   1 -
 .../ParseIntFieldUpdateProcessorFactory.java    |   1 -
 .../ParseLongFieldUpdateProcessorFactory.java   |   1 -
 ...ParseNumericFieldUpdateProcessorFactory.java |   1 -
 .../PreAnalyzedUpdateProcessorFactory.java      |  33 +-
 .../RemoveBlankFieldUpdateProcessorFactory.java |   1 -
 .../processor/RunUpdateProcessorFactory.java    |   1 -
 .../processor/ScriptEngineCustomizer.java       |   1 -
 .../apache/solr/update/processor/Signature.java |   3 +-
 .../SignatureUpdateProcessorFactory.java        |   3 +-
 .../processor/SimpleUpdateProcessorFactory.java |   3 +-
 .../StatelessScriptUpdateProcessorFactory.java  |   3 +-
 .../update/processor/TextProfileSignature.java  |   3 +-
 .../TimestampUpdateProcessorFactory.java        |   1 -
 .../TrimFieldUpdateProcessorFactory.java        |   1 -
 .../TruncateFieldUpdateProcessorFactory.java    |   1 -
 .../processor/UUIDUpdateProcessorFactory.java   |   1 -
 .../UniqFieldsUpdateProcessorFactory.java       |   3 +-
 .../processor/UpdateRequestProcessor.java       |   1 -
 .../processor/UpdateRequestProcessorChain.java  |   1 -
 .../UpdateRequestProcessorFactory.java          |   1 -
 .../apache/solr/util/AdjustableSemaphore.java   |   3 +-
 .../org/apache/solr/util/BoundedTreeSet.java    |   2 -
 .../org/apache/solr/util/CommandOperation.java  |   3 +-
 .../apache/solr/util/ConcurrentLFUCache.java    |   3 +-
 .../apache/solr/util/ConcurrentLRUCache.java    |   3 +-
 .../java/org/apache/solr/util/CryptoKeys.java   |   3 +-
 .../src/java/org/apache/solr/util/DOMUtil.java  |   3 +-
 .../org/apache/solr/util/DateFormatUtil.java    |   3 +-
 .../org/apache/solr/util/DateMathParser.java    |   1 -
 .../solr/util/DefaultSolrThreadFactory.java     |   3 +-
 .../org/apache/solr/util/DistanceUnits.java     |  19 +-
 .../java/org/apache/solr/util/FSHDFSUtils.java  |   3 +-
 .../java/org/apache/solr/util/FastWriter.java   |   3 +-
 .../java/org/apache/solr/util/FileUtils.java    |   3 +-
 .../src/java/org/apache/solr/util/HdfsUtil.java |  17 +-
 .../org/apache/solr/util/LongPriorityQueue.java |   7 +-
 .../java/org/apache/solr/util/MapListener.java  |   3 +-
 .../java/org/apache/solr/util/NumberUtils.java  |   1 -
 .../org/apache/solr/util/PivotListEntry.java    |   3 +-
 .../java/org/apache/solr/util/PrimUtils.java    |   3 +-
 .../apache/solr/util/PropertiesInputStream.java |   3 +-
 .../solr/util/PropertiesOutputStream.java       |   3 +-
 .../org/apache/solr/util/PropertiesUtil.java    |   3 +-
 .../src/java/org/apache/solr/util/RTimer.java   |   3 +-
 .../java/org/apache/solr/util/RTimerTree.java   |   3 +-
 .../apache/solr/util/RecordingJSONParser.java   |   4 +-
 .../java/org/apache/solr/util/RefCounted.java   |   1 -
 .../org/apache/solr/util/RegexFileFilter.java   |   3 +-
 .../org/apache/solr/util/SimplePostTool.java    |   3 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  |   3 +-
 .../org/apache/solr/util/SolrLogLayout.java     |  33 +-
 .../org/apache/solr/util/SolrPluginUtils.java   |   1 -
 .../java/org/apache/solr/util/SpatialUtils.java |   3 +-
 .../org/apache/solr/util/SystemIdResolver.java  |   3 +-
 .../org/apache/solr/util/TestInjection.java     |   3 +-
 .../src/java/org/apache/solr/util/TimeOut.java  |   3 +-
 .../org/apache/solr/util/TimeZoneUtils.java     |   1 -
 .../org/apache/solr/util/VersionedFile.java     |   1 -
 .../hll/BigEndianAscendingWordDeserializer.java |   3 +-
 .../hll/BigEndianAscendingWordSerializer.java   |   3 +-
 .../java/org/apache/solr/util/hll/BitUtil.java  |   3 +-
 .../org/apache/solr/util/hll/BitVector.java     |   3 +-
 .../src/java/org/apache/solr/util/hll/HLL.java  |   3 +-
 .../org/apache/solr/util/hll/HLLMetadata.java   |   3 +-
 .../java/org/apache/solr/util/hll/HLLType.java  |   3 +-
 .../java/org/apache/solr/util/hll/HLLUtil.java  |   3 +-
 .../org/apache/solr/util/hll/IHLLMetadata.java  |   3 +-
 .../apache/solr/util/hll/ISchemaVersion.java    |   3 +-
 .../apache/solr/util/hll/IWordDeserializer.java |   3 +-
 .../apache/solr/util/hll/IWordSerializer.java   |   3 +-
 .../org/apache/solr/util/hll/LongIterator.java  |   3 +-
 .../org/apache/solr/util/hll/NumberUtil.java    |   3 +-
 .../apache/solr/util/hll/SchemaVersionOne.java  |   3 +-
 .../apache/solr/util/hll/SerializationUtil.java |   3 +-
 .../solr/util/plugin/AbstractPluginLoader.java  |   1 -
 .../solr/util/plugin/MapInitializedPlugin.java  |   1 -
 .../solr/util/plugin/MapPluginLoader.java       |   1 -
 .../util/plugin/NamedListInitializedPlugin.java |   1 -
 .../solr/util/plugin/NamedListPluginLoader.java |   1 -
 .../apache/solr/util/plugin/SolrCoreAware.java  |   1 -
 .../java/org/apache/solr/util/stats/Clock.java  |   1 -
 .../java/org/apache/solr/util/stats/EWMA.java   |   1 -
 .../util/stats/ExponentiallyDecayingSample.java |   1 -
 .../org/apache/solr/util/stats/Histogram.java   |   1 -
 .../java/org/apache/solr/util/stats/Meter.java  |   1 -
 .../java/org/apache/solr/util/stats/Sample.java |   1 -
 .../org/apache/solr/util/stats/Snapshot.java    |   1 -
 .../java/org/apache/solr/util/stats/Timer.java  |   1 -
 .../apache/solr/util/stats/TimerContext.java    |   1 -
 .../apache/solr/util/stats/UniformSample.java   |   1 -
 .../solr/util/xslt/TransformerProvider.java     |   1 -
 .../runtimecode/RuntimeLibReqHandler.java       |   4 +-
 .../runtimecode/RuntimeLibResponseWriter.java   |  18 +-
 .../runtimecode/RuntimeLibSearchComponent.java  |   4 +-
 .../solr/AnalysisAfterCoreReloadTest.java       |   3 +-
 .../org/apache/solr/BasicFunctionalityTest.java |   1 -
 .../org/apache/solr/ConvertedLegacyTest.java    |   1 -
 .../test/org/apache/solr/CursorPagingTest.java  |   1 -
 .../apache/solr/DisMaxRequestHandlerTest.java   |   1 -
 .../solr/DistributedIntervalFacetingTest.java   |  28 +-
 .../test/org/apache/solr/EchoParamsTest.java    |   1 -
 .../test/org/apache/solr/MinimalSchemaTest.java |   1 -
 .../test/org/apache/solr/OutputWriterTest.java  |   1 -
 .../src/test/org/apache/solr/SampleTest.java    |   1 -
 .../org/apache/solr/SolrTestCaseJ4Test.java     |   3 +-
 .../test/org/apache/solr/TestCrossCoreJoin.java |   1 -
 .../solr/TestCursorMarkWithoutUniqueKey.java    |   1 -
 .../apache/solr/TestDistributedGrouping.java    |   3 +-
 .../apache/solr/TestDistributedMissingSort.java |   1 -
 .../org/apache/solr/TestDistributedSearch.java  |   1 -
 .../org/apache/solr/TestDocumentBuilder.java    |   4 +-
 .../org/apache/solr/TestGroupingSearch.java     |   1 -
 .../apache/solr/TestHighlightDedupGrouping.java |   1 -
 .../core/src/test/org/apache/solr/TestJoin.java |   1 -
 .../org/apache/solr/TestRandomDVFaceting.java   |   1 -
 .../org/apache/solr/TestRandomFaceting.java     |   1 -
 .../solr/TestSimpleTrackingShardHandler.java    |   1 -
 .../org/apache/solr/TestTolerantSearch.java     |  33 +-
 .../PathHierarchyTokenizerFactoryTest.java      |   1 -
 .../apache/solr/analysis/TestCharFilters.java   |   3 +-
 .../TestReversedWildcardFilterFactory.java      |   3 +-
 .../TestWordDelimiterFilterFactory.java         |   3 +-
 .../ThrowingMockTokenFilterFactory.java         |   3 +-
 .../solr/client/solrj/ConnectionReuseTest.java  |   5 +-
 .../TestEmbeddedSolrServerConstructors.java     |   3 +-
 .../solrj/embedded/TestJettySolrRunner.java     |   3 +-
 .../apache/solr/cloud/ActionThrottleTest.java   |   3 +-
 .../apache/solr/cloud/AliasIntegrationTest.java |   3 +-
 .../test/org/apache/solr/cloud/AssignTest.java  |   3 +-
 .../AsyncCallRequestStatusResponseTest.java     |   3 +-
 .../solr/cloud/AsyncMigrateRouteKeyTest.java    |   3 +-
 .../solr/cloud/BaseCdcrDistributedZkTest.java   |   3 +-
 .../solr/cloud/BasicDistributedZk2Test.java     |   3 +-
 .../solr/cloud/BasicDistributedZkTest.java      |   3 +-
 .../test/org/apache/solr/cloud/BasicZkTest.java |   3 +-
 .../cloud/CdcrReplicationDistributedZkTest.java |   3 +-
 .../solr/cloud/CdcrReplicationHandlerTest.java  |   3 +-
 .../solr/cloud/CdcrRequestHandlerTest.java      |   3 +-
 .../solr/cloud/CdcrVersionReplicationTest.java  |   3 +-
 .../cloud/ChaosMonkeyNothingIsSafeTest.java     |   3 +-
 .../solr/cloud/ChaosMonkeySafeLeaderTest.java   |   3 +-
 .../solr/cloud/ChaosMonkeyShardSplitTest.java   |   3 +-
 .../apache/solr/cloud/CleanupOldIndexTest.java  |   3 +-
 .../cloud/CloudExitableDirectoryReaderTest.java |  34 +-
 .../org/apache/solr/cloud/ClusterStateTest.java |  27 +-
 .../solr/cloud/ClusterStateUpdateTest.java      |   3 +-
 .../apache/solr/cloud/CollectionReloadTest.java |   3 +-
 .../solr/cloud/CollectionStateFormat2Test.java  |   3 +-
 .../cloud/CollectionTooManyReplicasTest.java    |   3 +-
 .../CollectionsAPIAsyncDistributedZkTest.java   |   3 +-
 .../cloud/CollectionsAPIDistributedZkTest.java  |   3 +-
 .../solr/cloud/CollectionsAPISolrJTests.java    |   3 +-
 ...ConcurrentDeleteAndCreateCollectionTest.java |   3 +-
 .../apache/solr/cloud/ConfigSetsAPITest.java    |   3 +-
 .../solr/cloud/ConnectionManagerTest.java       |  27 +-
 .../apache/solr/cloud/CustomCollectionTest.java |   3 +-
 .../solr/cloud/DeleteInactiveReplicaTest.java   |   3 +-
 .../DeleteLastCustomShardedReplicaTest.java     |   3 +-
 .../apache/solr/cloud/DeleteReplicaTest.java    |   3 +-
 .../org/apache/solr/cloud/DeleteShardTest.java  |   3 +-
 .../cloud/DistribJoinFromCollectionTest.java    |   3 +-
 .../apache/solr/cloud/DistributedQueueTest.java |  22 +-
 .../solr/cloud/DistributedVersionInfoTest.java  |   3 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java  |   3 +-
 .../cloud/FullSolrCloudDistribCmdsTest.java     |   3 +-
 .../apache/solr/cloud/HttpPartitionTest.java    |   3 +-
 .../org/apache/solr/cloud/KerberosTestUtil.java |  29 +-
 .../cloud/LeaderElectionIntegrationTest.java    |   3 +-
 .../apache/solr/cloud/LeaderElectionTest.java   |  28 +-
 .../cloud/LeaderFailoverAfterPartitionTest.java |   3 +-
 .../LeaderInitiatedRecoveryOnCommitTest.java    |   3 +-
 ...aderInitiatedRecoveryOnShardRestartTest.java |   3 +-
 .../apache/solr/cloud/MigrateRouteKeyTest.java  |   3 +-
 .../apache/solr/cloud/MultiThreadedOCPTest.java |   3 +-
 ...utOfBoxZkACLAndCredentialsProvidersTest.java |  33 +-
 ...rriddenZkACLAndCredentialsProvidersTest.java |  33 +-
 ...verseerCollectionConfigSetProcessorTest.java |   3 +-
 .../apache/solr/cloud/OverseerRolesTest.java    |   4 +-
 .../apache/solr/cloud/OverseerStatusTest.java   |   3 +-
 .../solr/cloud/OverseerTaskQueueTest.java       |  22 +-
 .../org/apache/solr/cloud/OverseerTest.java     |   3 +-
 .../solr/cloud/RecoveryAfterSoftCommitTest.java |   3 +-
 .../org/apache/solr/cloud/RecoveryZkTest.java   |   3 +-
 .../apache/solr/cloud/RemoteQueryErrorTest.java |   3 +-
 .../solr/cloud/ReplicaPropertiesBase.java       |   3 +-
 .../solr/cloud/ReplicationFactorTest.java       |   3 +-
 .../solr/cloud/RestartWhileUpdatingTest.java    |   3 +-
 .../apache/solr/cloud/RollingRestartTest.java   |   3 +-
 .../org/apache/solr/cloud/SSLMigrationTest.java |   3 +-
 .../solr/cloud/SaslZkACLProviderTest.java       |  33 +-
 .../solr/cloud/ShardRoutingCustomTest.java      |   3 +-
 .../org/apache/solr/cloud/ShardRoutingTest.java |   3 +-
 .../org/apache/solr/cloud/ShardSplitTest.java   |   3 +-
 .../cloud/SharedFSAutoReplicaFailoverTest.java  |   3 +-
 .../SharedFSAutoReplicaFailoverUtilsTest.java   |   3 +-
 .../cloud/SimpleCollectionCreateDeleteTest.java |   3 +-
 .../org/apache/solr/cloud/SliceStateTest.java   |  27 +-
 .../apache/solr/cloud/SolrCloudExampleTest.java |   3 +-
 .../org/apache/solr/cloud/SolrXmlInZkTest.java  |  22 +-
 .../org/apache/solr/cloud/SyncSliceTest.java    |   3 +-
 .../solr/cloud/TestAuthenticationFramework.java |   3 +-
 .../apache/solr/cloud/TestCloudInspectUtil.java |   7 +-
 .../apache/solr/cloud/TestCollectionAPI.java    |   4 +-
 .../apache/solr/cloud/TestConfigSetsAPI.java    |   1 -
 .../cloud/TestConfigSetsAPIExclusivity.java     |   1 -
 .../solr/cloud/TestConfigSetsAPIZkFailure.java  |   1 -
 .../org/apache/solr/cloud/TestCryptoKeys.java   |   3 +-
 .../solr/cloud/TestDistribDocBasedVersion.java  |   3 +-
 .../solr/cloud/TestDownShardTolerantSearch.java |   3 +-
 .../TestExclusionRuleCollectionAccess.java      |   3 +-
 .../apache/solr/cloud/TestHashPartitioner.java  |  27 +-
 .../solr/cloud/TestLeaderElectionZkExpiry.java  |   3 +-
 .../TestLeaderInitiatedRecoveryThread.java      |  31 +-
 .../solr/cloud/TestMiniSolrCloudCluster.java    |   3 +-
 .../cloud/TestMiniSolrCloudClusterBase.java     |   4 +-
 .../cloud/TestMiniSolrCloudClusterKerberos.java |   3 +-
 .../solr/cloud/TestMiniSolrCloudClusterSSL.java |   3 +-
 .../cloud/TestRandomRequestDistribution.java    |   3 +-
 .../apache/solr/cloud/TestRebalanceLeaders.java |   3 +-
 .../solr/cloud/TestReplicaProperties.java       |   4 +-
 .../solr/cloud/TestRequestForwarding.java       |   3 +-
 .../cloud/TestRequestStatusCollectionAPI.java   |   3 +-
 .../solr/cloud/TestShortCircuitedRequests.java  |   3 +-
 .../cloud/TestSolrCloudWithKerberosAlt.java     |   3 +-
 .../org/apache/solr/cloud/TestZkChroot.java     |   3 +-
 .../TlogReplayBufferedWhileIndexingTest.java    | 136 +++
 .../cloud/TriLevelCompositeIdRoutingTest.java   |   3 +-
 .../solr/cloud/UnloadDistributedZkTest.java     |   3 +-
 ...MParamsZkACLAndCredentialsProvidersTest.java |  33 +-
 .../test/org/apache/solr/cloud/ZkCLITest.java   |   3 +-
 .../org/apache/solr/cloud/ZkControllerTest.java |  28 +-
 .../org/apache/solr/cloud/ZkNodePropsTest.java  |  27 +-
 .../org/apache/solr/cloud/ZkSolrClientTest.java |  27 +-
 .../cloud/hdfs/HdfsBasicDistributedZk2Test.java |   3 +-
 .../cloud/hdfs/HdfsBasicDistributedZkTest.java  |   3 +-
 .../hdfs/HdfsChaosMonkeyNothingIsSafeTest.java  |   3 +-
 .../hdfs/HdfsChaosMonkeySafeLeaderTest.java     |   3 +-
 .../HdfsCollectionsAPIDistributedZkTest.java    |   3 +-
 .../solr/cloud/hdfs/HdfsNNFailoverTest.java     |   1 -
 .../solr/cloud/hdfs/HdfsRecoverLeaseTest.java   |   3 +-
 .../solr/cloud/hdfs/HdfsRecoveryZkTest.java     |   3 +-
 .../hdfs/HdfsRestartWhileUpdatingTest.java      |   3 +-
 .../solr/cloud/hdfs/HdfsSyncSliceTest.java      |   3 +-
 .../apache/solr/cloud/hdfs/HdfsTestUtil.java    |  33 +-
 .../solr/cloud/hdfs/HdfsThreadLeakTest.java     |   3 +-
 ...HdfsTlogReplayBufferedWhileIndexingTest.java |  63 ++
 .../cloud/hdfs/HdfsUnloadDistributedZkTest.java |   3 +-
 .../HdfsWriteToMultipleCollectionsTest.java     |   1 -
 .../apache/solr/cloud/hdfs/StressHdfsTest.java  |   1 -
 .../cloud/overseer/TestClusterStateMutator.java |   3 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java  |   3 +-
 .../solr/cloud/overseer/ZkStateWriterTest.java  |   3 +-
 .../apache/solr/cloud/rule/RuleEngineTest.java  |   3 +-
 .../org/apache/solr/cloud/rule/RulesTest.java   |   7 +-
 .../solr/core/BlobStoreTestRequestHandler.java  |   4 +-
 .../solr/core/CachingDirectoryFactoryTest.java  |  33 +-
 .../solr/core/CountUsageValueSourceParser.java  |   3 +-
 .../apache/solr/core/DirectoryFactoryTest.java  |   1 -
 .../solr/core/DummyValueSourceParser.java       |   3 +-
 .../solr/core/ExitableDirectoryReaderTest.java  |   4 +-
 .../solr/core/HdfsDirectoryFactoryTest.java     |   1 -
 .../org/apache/solr/core/MockInfoMBean.java     |  13 +-
 .../solr/core/MockShardHandlerFactory.java      |   3 +-
 .../solr/core/OpenCloseCoreStressTest.java      |   1 -
 .../org/apache/solr/core/PluginInfoTest.java    |   3 +-
 .../apache/solr/core/QueryResultKeyTest.java    |   1 -
 .../solr/core/RAMDirectoryFactoryTest.java      |   1 -
 .../apache/solr/core/RequestHandlersTest.java   |   1 -
 .../apache/solr/core/ResourceLoaderTest.java    |   1 -
 .../test/org/apache/solr/core/SOLR749Test.java  |   3 +-
 .../core/SolrCoreCheckLockOnStartupTest.java    |   3 +-
 .../test/org/apache/solr/core/SolrCoreTest.java |   1 -
 .../org/apache/solr/core/TestBadConfig.java     |   1 -
 .../org/apache/solr/core/TestCodecSupport.java  |   3 +-
 .../test/org/apache/solr/core/TestConfig.java   |   1 -
 .../org/apache/solr/core/TestConfigOverlay.java |   3 +-
 .../solr/core/TestConfigSetImmutable.java       |   3 +-
 .../solr/core/TestConfigSetProperties.java      |   3 +-
 .../org/apache/solr/core/TestConfigSets.java    |   3 +-
 .../org/apache/solr/core/TestCoreContainer.java |   1 -
 .../org/apache/solr/core/TestCoreDiscovery.java |   3 +-
 .../apache/solr/core/TestDynamicLoading.java    |   4 +-
 .../solr/core/TestImplicitCoreProperties.java   |  32 +-
 .../apache/solr/core/TestInfoStreamLogging.java |   3 +-
 .../org/apache/solr/core/TestInitParams.java    |   3 +-
 .../org/apache/solr/core/TestLazyCores.java     |   3 +-
 .../apache/solr/core/TestMergePolicyConfig.java |   3 +-
 .../test/org/apache/solr/core/TestNRTOpen.java  |   3 +-
 .../solr/core/TestQuerySenderListener.java      |   1 -
 .../solr/core/TestQuerySenderNoQuery.java       |   3 +-
 .../solr/core/TestReloadAndDeleteDocs.java      |   1 -
 .../solr/core/TestShardHandlerFactory.java      |   1 -
 .../apache/solr/core/TestSolrConfigHandler.java |   4 +-
 .../apache/solr/core/TestSolrIndexConfig.java   |   3 +-
 .../test/org/apache/solr/core/TestSolrXml.java  |   3 +-
 .../apache/solr/core/TestXIncludeConfig.java    |   3 +-
 .../handler/AnalysisRequestHandlerTestBase.java |   1 -
 .../solr/handler/CSVRequestHandlerTest.java     |   1 -
 .../apache/solr/handler/CheckBackupStatus.java  |   3 +-
 .../DocumentAnalysisRequestHandlerTest.java     |   1 -
 .../FieldAnalysisRequestHandlerTest.java        |   1 -
 .../org/apache/solr/handler/JsonLoaderTest.java |   1 -
 .../solr/handler/MoreLikeThisHandlerTest.java   |   1 -
 .../solr/handler/PingRequestHandlerTest.java    |   1 -
 .../apache/solr/handler/RequestLoggingTest.java |   3 +-
 .../handler/StandardRequestHandlerTest.java     |   1 -
 .../apache/solr/handler/TestBlobHandler.java    |   3 +-
 .../org/apache/solr/handler/TestCSVLoader.java  |   1 -
 .../apache/solr/handler/TestConfigReload.java   |   3 +-
 .../handler/TestReplicationHandlerBackup.java   |   3 +-
 .../apache/solr/handler/TestReqParamsAPI.java   |  33 +-
 .../apache/solr/handler/TestRestoreCore.java    |   4 +-
 .../org/apache/solr/handler/TestSQLHandler.java |   6 +-
 .../handler/TestSolrConfigHandlerCloud.java     |   4 +-
 .../TestSolrConfigHandlerConcurrent.java        |   3 +-
 .../handler/ThrowErrorOnInitRequestHandler.java |   1 -
 .../admin/CoreAdminCreateDiscoverTest.java      |   1 -
 .../handler/admin/CoreAdminHandlerTest.java     |  62 +-
 .../admin/CoreAdminRequestStatusTest.java       |   3 +-
 .../admin/CoreMergeIndexesAdminHandlerTest.java |   3 +-
 .../solr/handler/admin/InfoHandlerTest.java     |   1 -
 .../solr/handler/admin/LoggingHandlerTest.java  |   1 -
 .../handler/admin/LukeRequestHandlerTest.java   |   1 -
 .../solr/handler/admin/MBeansHandlerTest.java   |   1 -
 .../handler/admin/SecurityConfHandlerTest.java  |   4 +-
 .../admin/SegmentsInfoRequestHandlerTest.java   |   3 +-
 .../admin/ShowFileRequestHandlerTest.java       |   3 +-
 .../handler/admin/SystemInfoHandlerTest.java    |   1 -
 .../handler/component/BadComponentTest.java     |   5 +-
 .../handler/component/DebugComponentTest.java   |   3 +-
 .../DistributedDebugComponentTest.java          |  33 +-
 .../DistributedExpandComponentTest.java         |   3 +-
 .../DistributedFacetPivotLargeTest.java         |   3 +-
 .../DistributedFacetPivotLongTailTest.java      |   3 +-
 .../DistributedFacetPivotSmallAdvancedTest.java |   3 +-
 .../DistributedFacetPivotSmallTest.java         |   3 +-
 .../DistributedFacetPivotWhiteBoxTest.java      |   3 +-
 .../component/DistributedMLTComponentTest.java  |   3 +-
 ...DistributedQueryComponentCustomSortTest.java |   3 +-
 ...stributedQueryComponentOptimizationTest.java |   3 +-
 .../DistributedQueryElevationComponentTest.java |   3 +-
 .../DistributedSpellCheckComponentTest.java     |   3 +-
 .../DistributedSuggestComponentTest.java        |   3 +-
 .../DistributedTermsComponentTest.java          |   3 +-
 .../component/DummyCustomParamSpellChecker.java |  26 +-
 .../handler/component/FacetPivotSmallTest.java  |   3 +-
 .../component/QueryElevationComponentTest.java  |   1 -
 .../component/ResponseLogComponentTest.java     |  17 +-
 .../handler/component/SearchHandlerTest.java    |   1 -
 .../component/SpatialHeatmapFacetsTest.java     |   3 +-
 .../component/SpellCheckComponentTest.java      |   1 -
 .../handler/component/StatsComponentTest.java   |   3 +-
 .../SuggestComponentContextFilterQueryTest.java |   3 +-
 .../handler/component/SuggestComponentTest.java |   3 +-
 .../TermVectorComponentDistributedTest.java     |   1 -
 .../component/TermVectorComponentTest.java      |  18 +-
 .../handler/component/TermsComponentTest.java   |   3 +-
 ...estDistributedStatsComponentCardinality.java |   7 +-
 .../handler/component/TestExpandComponent.java  |  31 +-
 .../handler/component/TestPivotHelperCode.java  |   5 +-
 .../TestTrackingShardHandlerFactory.java        |   3 +-
 .../solr/handler/loader/JavabinLoaderTest.java  |   3 +-
 .../highlight/FastVectorHighlighterTest.java    |   1 -
 .../highlight/HighlighterMaxOffsetTest.java     |   3 +-
 .../apache/solr/highlight/HighlighterTest.java  |   1 -
 .../highlight/TestPostingsSolrHighlighter.java  |   3 +-
 .../solr/index/hdfs/CheckHdfsIndexTest.java     |   3 +-
 .../apache/solr/internal/csv/CSVParserTest.java |   6 +-
 .../solr/internal/csv/CSVPrinterTest.java       |   6 +-
 .../solr/internal/csv/CSVStrategyTest.java      |   6 +-
 .../apache/solr/internal/csv/CSVUtilsTest.java  |   6 +-
 .../solr/internal/csv/CharBufferTest.java       |  26 +-
 .../csv/ExtendedBufferedReaderTest.java         |   6 +-
 .../csv/writer/CSVConfigGuesserTest.java        |  26 +-
 .../solr/internal/csv/writer/CSVConfigTest.java |  26 +-
 .../solr/internal/csv/writer/CSVFieldTest.java  |  26 +-
 .../solr/internal/csv/writer/CSVWriterTest.java |  26 +-
 .../org/apache/solr/logging/TestLogWatcher.java |   3 +-
 .../org/apache/solr/request/JSONWriterTest.java |   1 -
 .../apache/solr/request/SimpleFacetsTest.java   |   1 -
 .../apache/solr/request/SmileWriterTest.java    |   4 +-
 .../org/apache/solr/request/TestFaceting.java   |   1 -
 .../solr/request/TestIntervalFaceting.java      |   1 -
 .../solr/request/TestRemoteStreaming.java       |   3 +-
 .../org/apache/solr/request/TestWriterPerf.java |   1 -
 .../apache/solr/request/macro/TestMacros.java   |   4 +-
 .../solr/response/TestCSVResponseWriter.java    |   1 -
 .../solr/response/TestChildDocTransformer.java  |   3 +-
 .../solr/response/TestCustomDocTransformer.java |   3 +-
 .../TestPHPSerializedResponseWriter.java        |   1 -
 .../solr/response/TestRawResponseWriter.java    |   1 -
 .../solr/response/TestRawTransformer.java       |   3 +-
 .../solr/response/TestSolrQueryResponse.java    |   3 +-
 .../response/TestSortingResponseWriter.java     |   6 +-
 .../apache/solr/rest/SolrRestletTestBase.java   |   3 +-
 .../apache/solr/rest/TestManagedResource.java   |   3 +-
 .../solr/rest/TestManagedResourceStorage.java   |   3 +-
 .../org/apache/solr/rest/TestRestManager.java   |   3 +-
 .../solr/rest/schema/TestBulkSchemaAPI.java     |  86 +-
 .../rest/schema/TestClassNameShortening.java    |   3 +-
 .../schema/TestCopyFieldCollectionResource.java |   3 +-
 .../schema/TestDefaultSearchFieldResource.java  |   3 +-
 .../TestDynamicFieldCollectionResource.java     |   3 +-
 .../rest/schema/TestDynamicFieldResource.java   |   3 +-
 .../schema/TestFieldCollectionResource.java     |   3 +-
 .../solr/rest/schema/TestFieldResource.java     |   3 +-
 .../schema/TestFieldTypeCollectionResource.java |   3 +-
 .../solr/rest/schema/TestFieldTypeResource.java |   4 +-
 .../TestManagedSchemaDynamicFieldResource.java  |   3 +-
 .../schema/TestManagedSchemaFieldResource.java  |   3 +-
 .../TestManagedSchemaFieldTypeResource.java     |   3 +-
 .../schema/TestRemoveLastDynamicCopyField.java  |   4 +-
 .../rest/schema/TestSchemaNameResource.java     |   3 +-
 .../solr/rest/schema/TestSchemaResource.java    |   4 +-
 .../schema/TestSchemaSimilarityResource.java    |   3 +-
 .../rest/schema/TestSchemaVersionResource.java  |   3 +-
 .../TestSerializedLuceneMatchVersion.java       |   3 +-
 ...tSolrQueryParserDefaultOperatorResource.java |   3 +-
 .../schema/TestSolrQueryParserResource.java     |   3 +-
 .../rest/schema/TestUniqueKeyFieldResource.java |   3 +-
 .../analysis/TestManagedStopFilterFactory.java  |   3 +-
 .../TestManagedSynonymFilterFactory.java        |   3 +-
 .../solr/schema/AbstractCurrencyFieldTest.java  |   3 +-
 .../apache/solr/schema/BadCopyFieldTest.java    |   1 -
 .../apache/solr/schema/BadIndexSchemaTest.java  |   1 -
 .../solr/schema/ChangedSchemaMergeTest.java     |   1 -
 .../org/apache/solr/schema/CopyFieldTest.java   |   1 -
 .../schema/CurrencyFieldOpenExchangeTest.java   |   3 +-
 .../solr/schema/CurrencyFieldXmlFileTest.java   |   3 +-
 .../solr/schema/CustomAnalyzerStrField.java     |   3 +-
 .../org/apache/solr/schema/DateFieldTest.java   |   1 -
 .../apache/solr/schema/DateRangeFieldTest.java  |   5 +-
 .../solr/schema/DocValuesMissingTest.java       |   3 +-
 .../apache/solr/schema/DocValuesMultiTest.java  |   3 +-
 .../org/apache/solr/schema/DocValuesTest.java   |   3 +-
 .../org/apache/solr/schema/EnumFieldTest.java   |   3 +-
 .../solr/schema/ExternalFileFieldSortTest.java  |  19 +-
 .../schema/IndexSchemaRuntimeFieldTest.java     |   3 +-
 .../org/apache/solr/schema/IndexSchemaTest.java |   1 -
 .../solr/schema/MockExchangeRateProvider.java   |   3 +-
 .../org/apache/solr/schema/MultiTermTest.java   |   3 +-
 .../apache/solr/schema/MyCrazyCustomField.java  |   3 +-
 .../solr/schema/NotRequiredUniqueKeyTest.java   |   1 -
 .../apache/solr/schema/NumericFieldsTest.java   |   1 -
 .../OpenExchangeRatesOrgProviderTest.java       |   3 +-
 .../org/apache/solr/schema/PolyFieldTest.java   |   3 +-
 .../solr/schema/PreAnalyzedFieldTest.java       |   3 +-
 .../solr/schema/PrimitiveFieldTypeTest.java     |   1 -
 .../apache/solr/schema/RequiredFieldsTest.java  |   1 -
 .../SchemaVersionSpecificBehaviorTest.java      |   1 -
 .../apache/solr/schema/SortableBinaryField.java |   3 +-
 .../solr/schema/SpatialRPTFieldTypeTest.java    |   3 +-
 .../solr/schema/SynonymTokenizerTest.java       |   3 +-
 .../org/apache/solr/schema/TestBinaryField.java |   1 -
 .../solr/schema/TestBulkSchemaConcurrent.java   |   4 +-
 .../solr/schema/TestCloudManagedSchema.java     |   3 +-
 .../TestCloudManagedSchemaConcurrent.java       |   3 +-
 .../apache/solr/schema/TestCloudSchemaless.java |   3 +-
 .../apache/solr/schema/TestCollationField.java  |   1 -
 .../schema/TestCollationFieldDocValues.java     |   1 -
 .../apache/solr/schema/TestManagedSchema.java   |   3 +-
 .../apache/solr/schema/TestOmitPositions.java   |   3 +-
 .../apache/solr/schema/TestSchemaManager.java   |   3 +-
 .../solr/schema/TestUseDocValuesAsStored.java   |   3 +-
 .../solr/schema/TestUseDocValuesAsStored2.java  |   3 +-
 .../solr/schema/ThrowErrorOnInitFieldType.java  |   3 +-
 .../TrieIntPrefixActsAsRangeQueryFieldType.java |   3 +-
 .../org/apache/solr/schema/WrappedIntField.java |   3 +-
 .../solr/search/AnalyticsMergeStrategyTest.java |   3 +-
 .../apache/solr/search/AnalyticsQueryTest.java  |   1 -
 .../org/apache/solr/search/CursorMarkTest.java  |   1 -
 .../solr/search/DelayingSearchComponent.java    |   3 +-
 .../test/org/apache/solr/search/DocSetPerf.java |   1 -
 .../apache/solr/search/FooQParserPlugin.java    |   5 -
 .../apache/solr/search/MergeStrategyTest.java   |   3 +-
 .../apache/solr/search/MockSearchComponent.java |   3 +-
 .../apache/solr/search/QueryEqualityTest.java   |  16 +-
 .../apache/solr/search/QueryParsingTest.java    |   3 +-
 .../org/apache/solr/search/RankQueryTest.java   |   1 -
 .../apache/solr/search/ReturnFieldsTest.java    |   1 -
 .../apache/solr/search/SortSpecParsingTest.java |   3 +-
 .../apache/solr/search/SpatialFilterTest.java   |   4 +-
 .../solr/search/TestAddFieldRealTimeGet.java    |   3 +-
 .../solr/search/TestAnalyticsQParserPlugin.java |   5 -
 .../solr/search/TestCollapseQParserPlugin.java  |   1 -
 .../search/TestComplexPhraseQParserPlugin.java  |   3 +-
 .../apache/solr/search/TestComponentsName.java  |   3 +-
 .../org/apache/solr/search/TestCustomSort.java  |   3 +-
 .../test/org/apache/solr/search/TestDocSet.java |   1 -
 .../solr/search/TestElisionMultitermQuery.java  |  11 +-
 .../solr/search/TestExtendedDismaxParser.java   |   1 -
 .../apache/solr/search/TestFieldSortValues.java |   3 +-
 .../solr/search/TestFilteredDocIdSet.java       |   3 +-
 .../org/apache/solr/search/TestFiltering.java   |   1 -
 .../solr/search/TestFoldingMultitermQuery.java  |   3 +-
 .../solr/search/TestHashQParserPlugin.java      |   1 -
 .../org/apache/solr/search/TestInitQParser.java |   3 +-
 .../org/apache/solr/search/TestLFUCache.java    |   3 +-
 .../org/apache/solr/search/TestLRUCache.java    |   3 +-
 .../solr/search/TestMaxScoreQueryParser.java    |   3 +-
 .../apache/solr/search/TestMissingGroups.java   |   1 -
 .../apache/solr/search/TestNoOpRegenerator.java |   3 +-
 ...OverriddenPrefixQueryForCustomFieldType.java |   3 +-
 .../solr/search/TestPseudoReturnFields.java     |   1 -
 .../org/apache/solr/search/TestQueryUtils.java  |   1 -
 .../solr/search/TestQueryWrapperFilter.java     |   4 +-
 .../search/TestRandomCollapseQParserPlugin.java |   1 -
 .../apache/solr/search/TestRankQueryPlugin.java |   5 -
 .../solr/search/TestReRankQParserPlugin.java    |  64 +-
 .../apache/solr/search/TestReloadDeadlock.java  |   5 +-
 .../org/apache/solr/search/TestSearchPerf.java  |   1 -
 .../solr/search/TestSimpleQParserPlugin.java    |   3 +-
 .../apache/solr/search/TestSmileRequest.java    |   4 +-
 .../apache/solr/search/TestSolr4Spatial.java    |   3 +-
 .../apache/solr/search/TestSolr4Spatial2.java   |   3 +-
 .../test/org/apache/solr/search/TestSolrJ.java  |   1 -
 .../apache/solr/search/TestSolrQueryParser.java | 231 ++---
 .../test/org/apache/solr/search/TestSort.java   |   1 -
 .../solr/search/TestStandardQParsers.java       |   3 +-
 .../solr/search/TestStressUserVersions.java     |   5 +-
 .../solr/search/TestSurroundQueryParser.java    |   3 +-
 .../org/apache/solr/search/TestTrieFacet.java   |   1 -
 .../solr/search/TestValueSourceCache.java       |   3 +-
 .../org/apache/solr/search/TestXmlQParser.java  |   5 +-
 .../solr/search/facet/TestJsonFacets.java       |  25 +-
 .../search/function/NvlValueSourceParser.java   |   1 -
 .../search/function/SortByFunctionTest.java     |   3 +-
 .../solr/search/function/TestFunctionQuery.java |   1 -
 .../function/TestMinMaxOnMultiValuedField.java  |   1 -
 .../solr/search/function/TestOrdValues.java     |   3 +-
 .../function/TestSortByMinMaxFunction.java      |   3 +-
 .../function/distance/DistanceFunctionTest.java |   3 +-
 .../apache/solr/search/join/BJQParserTest.java  |   1 -
 .../search/join/BlockJoinFacetDistribTest.java  |   3 +-
 .../search/join/BlockJoinFacetRandomTest.java   |   3 +-
 .../search/join/BlockJoinFacetSimpleTest.java   |   3 +-
 .../apache/solr/search/join/GraphQueryTest.java |  26 +-
 .../search/join/TestScoreJoinQPNoScore.java     |   1 -
 .../solr/search/join/TestScoreJoinQPScore.java  |   1 -
 .../solr/search/json/TestJsonRequest.java       |   3 +-
 .../solr/search/mlt/CloudMLTQParserTest.java    |   3 +-
 .../solr/search/mlt/SimpleMLTQParserTest.java   |   3 +-
 .../similarities/BaseSimilarityTestCase.java    |   3 +-
 .../similarities/TestBM25SimilarityFactory.java |   3 +-
 .../TestClassicSimilarityFactory.java           |   3 +-
 .../similarities/TestDFISimilarityFactory.java  |   3 +-
 .../similarities/TestDFRSimilarityFactory.java  |   3 +-
 .../similarities/TestIBSimilarityFactory.java   |   3 +-
 .../TestLMDirichletSimilarityFactory.java       |   3 +-
 .../TestLMJelinekMercerSimilarityFactory.java   |   3 +-
 .../TestNonDefinedSimilarityFactory.java        |   3 +-
 .../similarities/TestPerFieldSimilarity.java    |   3 +-
 .../TestPerFieldSimilarityClassic.java          |   3 +-
 ...stPerFieldSimilarityWithDefaultOverride.java |   3 +-
 .../TestSweetSpotSimilarityFactory.java         |   3 +-
 .../solr/search/stats/TestBaseStatsCache.java   |   3 +-
 .../search/stats/TestDefaultStatsCache.java     |   3 +-
 .../solr/search/stats/TestDistribIDF.java       |   3 +-
 .../search/stats/TestExactSharedStatsCache.java |   4 +-
 .../solr/search/stats/TestExactStatsCache.java  |   4 +-
 .../solr/search/stats/TestLRUStatsCache.java    |   4 +-
 .../solr/security/BasicAuthIntegrationTest.java |  66 +-
 .../solr/security/MockAuthenticationPlugin.java |   4 +-
 .../solr/security/MockAuthorizationPlugin.java  |   3 +-
 .../PKIAuthenticationIntegrationTest.java       |   4 +-
 .../security/TestAuthorizationFramework.java    |   7 +-
 .../security/TestPKIAuthenticationPlugin.java   |   3 +-
 .../TestRuleBasedAuthorizationPlugin.java       |   3 +-
 .../TestSha256AuthenticationProvider.java       |   3 +-
 .../solr/servlet/DirectSolrConnectionTest.java  |   1 -
 .../apache/solr/servlet/ResponseHeaderTest.java |   3 +-
 .../solr/servlet/SolrRequestParserTest.java     |   1 -
 .../ConjunctionSolrSpellCheckerTest.java        |  23 +-
 .../spelling/DirectSolrSpellCheckerTest.java    |   3 +-
 .../spelling/FileBasedSpellCheckerTest.java     |   1 -
 .../apache/solr/spelling/SampleComparator.java  |   3 +-
 .../solr/spelling/SpellCheckCollatorTest.java   |   3 +-
 .../spelling/SpellPossibilityIteratorTest.java  |   3 +-
 .../spelling/SpellingQueryConverterTest.java    |   1 -
 .../spelling/TestSuggestSpellingConverter.java  |   3 +-
 .../spelling/WordBreakSolrSpellCheckerTest.java |   3 +-
 .../solr/spelling/suggest/SuggesterFSTTest.java |   3 +-
 .../solr/spelling/suggest/SuggesterTSTTest.java |   3 +-
 .../solr/spelling/suggest/SuggesterTest.java    |   1 -
 .../spelling/suggest/SuggesterWFSTTest.java     |   3 +-
 .../suggest/TestAnalyzeInfixSuggestions.java    |  11 +-
 .../suggest/TestAnalyzedSuggestions.java        |   3 +-
 .../suggest/TestBlendedInfixSuggestions.java    |   3 +-
 .../suggest/TestFileDictionaryLookup.java       |   3 +-
 .../suggest/TestFreeTextSuggestions.java        |   3 +-
 .../suggest/TestFuzzyAnalyzedSuggestions.java   |  11 +-
 .../TestHighFrequencyDictionaryFactory.java     |   3 +-
 .../spelling/suggest/TestPhraseSuggestions.java |   3 +-
 .../solr/store/blockcache/BlockCacheTest.java   |   3 +-
 .../store/blockcache/BlockDirectoryTest.java    |   3 +-
 .../solr/store/blockcache/BufferStoreTest.java  |   3 +-
 .../solr/store/hdfs/HdfsDirectoryTest.java      |   3 +-
 .../solr/store/hdfs/HdfsLockFactoryTest.java    |   3 +-
 .../apache/solr/update/AddBlockUpdateTest.java  |  25 +-
 .../solr/update/AnalysisErrorHandlingTest.java  |   3 +-
 .../org/apache/solr/update/AutoCommitTest.java  |   1 -
 .../apache/solr/update/CdcrUpdateLogTest.java   |   3 +-
 .../solr/update/DataDrivenBlockJoinTest.java    |  23 +-
 .../update/DirectUpdateHandlerOptimizeTest.java |   3 +-
 .../solr/update/DirectUpdateHandlerTest.java    |   1 -
 .../apache/solr/update/DocumentBuilderTest.java |   1 -
 .../apache/solr/update/DummyMergePolicy.java    |   3 +-
 .../apache/solr/update/HardAutoCommitTest.java  |   1 -
 .../solr/update/MockStreamingSolrClients.java   |   3 +-
 .../org/apache/solr/update/PeerSyncTest.java    |  26 +-
 .../apache/solr/update/SoftAutoCommitTest.java  |   1 -
 .../solr/update/SolrCmdDistributorTest.java     |   3 +-
 .../apache/solr/update/SolrIndexConfigTest.java |   3 +-
 .../solr/update/SolrIndexSplitterTest.java      |   3 +-
 .../update/TestDocBasedVersionConstraints.java  |   1 -
 .../solr/update/TestExceedMaxTermLength.java    |   3 +-
 .../apache/solr/update/TestHdfsUpdateLog.java   |   3 +-
 .../solr/update/TestIndexingPerformance.java    |   1 -
 .../apache/solr/update/UpdateParamsTest.java    |   1 -
 .../org/apache/solr/update/VersionInfoTest.java |   3 +-
 ...dSchemaFieldsUpdateProcessorFactoryTest.java |   1 -
 .../update/processor/AtomicUpdatesTest.java     |  31 +-
 .../CloneFieldUpdateProcessorFactoryTest.java   |   1 -
 .../processor/CustomUpdateRequestProcessor.java |   1 -
 .../CustomUpdateRequestProcessorFactory.java    |   1 -
 .../DefaultValueUpdateProcessorTest.java        |   1 -
 ...DocExpirationUpdateProcessorFactoryTest.java |   1 -
 .../FieldMutatingUpdateProcessorTest.java       |   1 -
 ...ommitOptimizeUpdateProcessorFactoryTest.java |   3 +-
 .../ParsingFieldUpdateProcessorsTest.java       |   1 -
 .../PreAnalyzedUpdateProcessorTest.java         |  13 +-
 .../RecordingUpdateProcessorFactory.java        |   1 -
 .../solr/update/processor/RuntimeUrp.java       |   3 +-
 .../solr/update/processor/ScriptEngineTest.java |   1 -
 .../SignatureUpdateProcessorFactoryTest.java    |   1 -
 ...atelessScriptUpdateProcessorFactoryTest.java |   3 +-
 .../processor/TestNamedUpdateProcessors.java    |   4 +-
 .../TestPartialUpdateDeduplication.java         |   3 +-
 .../UUIDUpdateProcessorFallbackTest.java        |   3 +-
 .../UniqFieldsUpdateProcessorFactoryTest.java   |   1 -
 .../processor/UpdateProcessorTestBase.java      |   3 +-
 .../UpdateRequestProcessorFactoryTest.java      |   1 -
 .../test/org/apache/solr/util/BitSetPerf.java   |   1 -
 .../org/apache/solr/util/CircularListTest.java  |   1 -
 .../test/org/apache/solr/util/DOMUtilTest.java  |   3 +-
 .../apache/solr/util/DateMathParserTest.java    |   1 -
 .../org/apache/solr/util/DistanceUnitsTest.java |   9 +-
 .../org/apache/solr/util/FileUtilsTest.java     |   3 +-
 .../org/apache/solr/util/MockCoreContainer.java |   3 +-
 .../org/apache/solr/util/PrimUtilsTest.java     |  14 +-
 .../apache/solr/util/SimplePostToolTest.java    |   3 +-
 .../apache/solr/util/SolrPluginUtilsTest.java   |   1 -
 .../apache/solr/util/TestFastOutputStream.java  |   1 -
 .../org/apache/solr/util/TestFastWriter.java    |   1 -
 .../solr/util/TestObjectReleaseTracker.java     |   3 +-
 .../org/apache/solr/util/TestRTimerTree.java    |   3 +-
 .../solr/util/TestRandomForceMergePolicy.java   |  36 -
 .../apache/solr/util/TestRandomMergePolicy.java |  74 --
 .../apache/solr/util/TestSolrCLIRunExample.java |   3 +-
 .../apache/solr/util/TestSystemIdResolver.java  |   3 +-
 .../org/apache/solr/util/TestTestInjection.java |   3 +-
 .../test/org/apache/solr/util/TestUtils.java    |   1 -
 .../org/apache/solr/util/TimeZoneUtilsTest.java |   1 -
 .../BigEndianAscendingWordDeserializerTest.java |   1 -
 .../BigEndianAscendingWordSerializerTest.java   |   1 -
 .../org/apache/solr/util/hll/BitVectorTest.java |   1 -
 .../apache/solr/util/hll/ExplicitHLLTest.java   |   1 -
 .../org/apache/solr/util/hll/FullHLLTest.java   |   1 -
 .../solr/util/hll/HLLSerializationTest.java     |   1 -
 .../org/apache/solr/util/hll/HLLUtilTest.java   |   1 -
 .../solr/util/hll/IntegrationTestGenerator.java |   1 -
 .../solr/util/hll/ProbabilisticTestUtil.java    |   1 -
 .../org/apache/solr/util/hll/SparseHLLTest.java |   1 -
 .../solr/client/solrj/ResponseParser.java       |   1 -
 .../apache/solr/client/solrj/SolrClient.java    |   1 -
 .../org/apache/solr/client/solrj/SolrQuery.java |   1 -
 .../apache/solr/client/solrj/SolrRequest.java   |   1 -
 .../apache/solr/client/solrj/SolrResponse.java  |   1 -
 .../solr/client/solrj/SolrServerException.java  |   1 -
 .../client/solrj/StreamingResponseCallback.java |   1 -
 .../client/solrj/beans/BindingException.java    |   3 +-
 .../solr/client/solrj/impl/CloudSolrClient.java |   3 +-
 .../solrj/impl/ConcurrentUpdateSolrClient.java  |   1 -
 .../client/solrj/impl/HttpClientConfigurer.java |   4 +-
 .../solr/client/solrj/impl/HttpSolrClient.java  |   2 +-
 .../solrj/impl/InputStreamResponseParser.java   |   3 +-
 .../solrj/impl/Krb5HttpClientConfigurer.java    |   3 +-
 .../client/solrj/impl/NoOpResponseParser.java   |   3 +-
 .../solrj/impl/SolrHttpRequestRetryHandler.java |   3 +-
 .../impl/SolrPortAwareCookieSpecFactory.java    |  33 +-
 .../client/solrj/impl/XMLResponseParser.java    |   1 -
 .../solr/client/solrj/io/SolrClientCache.java   |   3 +-
 .../org/apache/solr/client/solrj/io/Tuple.java  |   3 +-
 .../client/solrj/io/comp/ComparatorLambda.java  |  11 +-
 .../client/solrj/io/comp/ComparatorOrder.java   |   7 +-
 .../client/solrj/io/comp/FieldComparator.java   |   1 -
 .../solr/client/solrj/io/comp/HashKey.java      |  14 +-
 .../solrj/io/comp/MultipleFieldComparator.java  |   1 -
 .../client/solrj/io/comp/StreamComparator.java  |   1 -
 .../solr/client/solrj/io/eq/Equalitor.java      |   5 +-
 .../solr/client/solrj/io/eq/FieldEqualitor.java |   1 -
 .../solrj/io/eq/MultipleFieldEqualitor.java     |   1 -
 .../client/solrj/io/eq/StreamEqualitor.java     |   1 -
 .../client/solrj/io/ops/ConcatOperation.java    |  23 +-
 .../client/solrj/io/ops/DistinctOperation.java  |  33 +-
 .../client/solrj/io/ops/GroupOperation.java     |  33 +-
 .../client/solrj/io/ops/ReduceOperation.java    |   6 +-
 .../client/solrj/io/ops/ReplaceOperation.java   |  21 +-
 .../solrj/io/ops/ReplaceWithFieldOperation.java |  23 +-
 .../solrj/io/ops/ReplaceWithValueOperation.java |  23 +-
 .../client/solrj/io/ops/StreamOperation.java    |  19 +-
 .../client/solrj/io/sql/ConnectionImpl.java     |  46 +-
 .../solrj/io/sql/DatabaseMetaDataImpl.java      |  16 +-
 .../solr/client/solrj/io/sql/DriverImpl.java    |   7 +-
 .../solr/client/solrj/io/sql/ResultSetImpl.java | 245 ++++-
 .../solrj/io/sql/ResultSetMetaDataImpl.java     |  55 +-
 .../solr/client/solrj/io/sql/StatementImpl.java |  41 +-
 .../client/solrj/io/stream/BiJoinStream.java    |   1 -
 .../client/solrj/io/stream/CloudSolrStream.java |  13 +-
 .../solrj/io/stream/ComplementStream.java       |   1 -
 .../client/solrj/io/stream/DaemonStream.java    |   2 -
 .../client/solrj/io/stream/ExceptionStream.java |   1 -
 .../client/solrj/io/stream/FacetStream.java     |   3 +-
 .../client/solrj/io/stream/HashJoinStream.java  |   1 -
 .../client/solrj/io/stream/InnerJoinStream.java |   1 -
 .../client/solrj/io/stream/IntersectStream.java |   1 -
 .../solr/client/solrj/io/stream/JDBCStream.java |   1 -
 .../client/solrj/io/stream/JSONTupleStream.java |  33 +-
 .../solr/client/solrj/io/stream/JoinStream.java |  33 +-
 .../solrj/io/stream/LeftOuterJoinStream.java    |   1 -
 .../client/solrj/io/stream/MergeStream.java     |   1 -
 .../solrj/io/stream/OuterHashJoinStream.java    |   1 -
 .../client/solrj/io/stream/ParallelStream.java  |   1 -
 .../client/solrj/io/stream/PushBackStream.java  |   1 -
 .../solr/client/solrj/io/stream/RankStream.java |   1 -
 .../client/solrj/io/stream/ReducerStream.java   |   1 -
 .../client/solrj/io/stream/RollupStream.java    |   8 +-
 .../client/solrj/io/stream/SelectStream.java    |   1 -
 .../solr/client/solrj/io/stream/SolrStream.java |   1 -
 .../client/solrj/io/stream/StatsStream.java     |   3 +-
 .../client/solrj/io/stream/StreamContext.java   |   1 -
 .../client/solrj/io/stream/TupleStream.java     |   1 -
 .../client/solrj/io/stream/UniqueStream.java    |   1 -
 .../solrj/io/stream/expr/Expressible.java       |   7 +-
 .../solrj/io/stream/expr/StreamExpression.java  |   9 +-
 .../expr/StreamExpressionNamedParameter.java    |   7 +-
 .../stream/expr/StreamExpressionParameter.java  |   3 +-
 .../io/stream/expr/StreamExpressionParser.java  |  22 +-
 .../io/stream/expr/StreamExpressionValue.java   |   3 +-
 .../solrj/io/stream/expr/StreamFactory.java     |  33 +-
 .../client/solrj/io/stream/metrics/Bucket.java  |  11 +-
 .../solrj/io/stream/metrics/CountMetric.java    |   3 +-
 .../solrj/io/stream/metrics/MaxMetric.java      |   3 +-
 .../solrj/io/stream/metrics/MeanMetric.java     |   3 +-
 .../client/solrj/io/stream/metrics/Metric.java  |   3 +-
 .../solrj/io/stream/metrics/MinMetric.java      |   3 +-
 .../solrj/io/stream/metrics/SumMetric.java      |   3 +-
 .../solrj/request/AbstractUpdateRequest.java    |   3 +-
 .../solrj/request/CollectionAdminRequest.java   |   1 -
 .../solrj/request/ConfigSetAdminRequest.java    |   1 -
 .../request/ContentStreamUpdateRequest.java     |   4 +-
 .../client/solrj/request/CoreAdminRequest.java  |   1 -
 .../client/solrj/request/DirectXmlRequest.java  |   1 -
 .../solrj/request/DocumentAnalysisRequest.java  |   1 -
 .../solrj/request/FieldAnalysisRequest.java     |   1 -
 .../solrj/request/GenericSolrRequest.java       |   3 +-
 .../client/solrj/request/IsUpdateRequest.java   |   4 +-
 .../solr/client/solrj/request/LukeRequest.java  |   1 -
 .../solr/client/solrj/request/QueryRequest.java |   1 -
 .../client/solrj/request/RequestWriter.java     |   1 -
 .../solr/client/solrj/request/SolrPing.java     |   1 -
 .../client/solrj/request/UpdateRequest.java     |   2 +-
 .../request/schema/AbstractSchemaRequest.java   |   3 +-
 .../request/schema/AnalyzerDefinition.java      |   3 +-
 .../request/schema/FieldTypeDefinition.java     |   3 +-
 .../solrj/request/schema/SchemaRequest.java     |   3 +-
 .../solrj/response/AnalysisResponseBase.java    |   1 -
 .../solr/client/solrj/response/Cluster.java     |   3 +-
 .../solrj/response/ClusteringResponse.java      |   3 +-
 .../solrj/response/CollectionAdminResponse.java |   1 -
 .../solrj/response/ConfigSetAdminResponse.java  |   1 -
 .../solrj/response/CoreAdminResponse.java       |   1 -
 .../response/DocumentAnalysisResponse.java      |   1 -
 .../solr/client/solrj/response/FacetField.java  |   1 -
 .../solrj/response/FieldAnalysisResponse.java   |   1 -
 .../solr/client/solrj/response/Group.java       |   3 +-
 .../client/solrj/response/GroupCommand.java     |   3 +-
 .../client/solrj/response/GroupResponse.java    |   3 +-
 .../client/solrj/response/IntervalFacet.java    |   8 +-
 .../client/solrj/response/LukeResponse.java     |   1 -
 .../solr/client/solrj/response/PivotField.java  |   1 -
 .../client/solrj/response/QueryResponse.java    |   1 -
 .../solr/client/solrj/response/RangeFacet.java  |   3 +-
 .../solrj/response/SimpleSolrResponse.java      |   4 +-
 .../client/solrj/response/SolrPingResponse.java |   1 -
 .../client/solrj/response/SolrResponseBase.java |   1 -
 .../solrj/response/SpellCheckResponse.java      |   3 +-
 .../solrj/response/SuggesterResponse.java       |   3 +-
 .../solr/client/solrj/response/Suggestion.java  |   2 +-
 .../client/solrj/response/TermsResponse.java    |   3 +-
 .../client/solrj/response/UpdateResponse.java   |   1 -
 .../schema/FieldTypeRepresentation.java         |   3 +-
 .../response/schema/SchemaRepresentation.java   |   3 +-
 .../solrj/response/schema/SchemaResponse.java   |   3 +-
 .../solr/client/solrj/util/ClientUtils.java     |   1 -
 .../java/org/apache/solr/common/Callable.java   |   3 +-
 .../apache/solr/common/EmptyEntityResolver.java |   3 +-
 .../org/apache/solr/common/EnumFieldValue.java  |   3 +-
 .../org/apache/solr/common/SolrDocument.java    |   1 -
 .../apache/solr/common/SolrDocumentBase.java    |  13 +-
 .../apache/solr/common/SolrDocumentList.java    |   1 -
 .../org/apache/solr/common/SolrException.java   |   1 -
 .../apache/solr/common/SolrInputDocument.java   |   1 -
 .../org/apache/solr/common/SolrInputField.java  |   1 -
 .../org/apache/solr/common/StringUtils.java     |   3 +-
 .../org/apache/solr/common/cloud/Aliases.java   |   3 +-
 .../solr/common/cloud/BeforeReconnect.java      |  27 +-
 .../solr/common/cloud/ClosableThread.java       |   3 +-
 .../apache/solr/common/cloud/ClusterState.java  |   3 +-
 .../solr/common/cloud/ClusterStateUtil.java     |  31 +-
 .../solr/common/cloud/CompositeIdRouter.java    |   3 +-
 .../solr/common/cloud/ConnectionManager.java    |   7 +-
 .../common/cloud/DefaultConnectionStrategy.java |  27 +-
 .../solr/common/cloud/DefaultZkACLProvider.java |  13 +-
 .../cloud/DefaultZkCredentialsProvider.java     |   9 +-
 .../apache/solr/common/cloud/DocCollection.java |   3 +-
 .../org/apache/solr/common/cloud/DocRouter.java |   3 +-
 .../solr/common/cloud/HashBasedRouter.java      |   3 +-
 .../solr/common/cloud/ImplicitDocRouter.java    |   3 +-
 .../apache/solr/common/cloud/OnReconnect.java   |  27 +-
 .../apache/solr/common/cloud/PlainIdRouter.java |   5 +-
 .../org/apache/solr/common/cloud/Replica.java   |   3 +-
 .../apache/solr/common/cloud/RoutingRule.java   |   3 +-
 .../solr/common/cloud/SaslZkACLProvider.java    |   3 +-
 .../org/apache/solr/common/cloud/Slice.java     |   3 +-
 .../apache/solr/common/cloud/SolrZkClient.java  |  23 +-
 .../apache/solr/common/cloud/SolrZooKeeper.java |   3 +-
 ...ParamsAllAndReadonlyDigestZkACLProvider.java |  23 +-
 ...tCredentialsDigestZkCredentialsProvider.java |  17 +-
 .../apache/solr/common/cloud/ZkACLProvider.java |  11 +-
 .../cloud/ZkClientConnectionStrategy.java       |  27 +-
 .../apache/solr/common/cloud/ZkCmdExecutor.java |   3 +-
 .../solr/common/cloud/ZkConfigManager.java      |   1 -
 .../solr/common/cloud/ZkCoreNodeProps.java      |  27 +-
 .../common/cloud/ZkCredentialsProvider.java     |   7 +-
 .../apache/solr/common/cloud/ZkNodeProps.java   |   3 +-
 .../apache/solr/common/cloud/ZkOperation.java   |   6 +-
 .../apache/solr/common/cloud/ZkStateReader.java |   5 +-
 .../solr/common/cloud/ZooKeeperException.java   |   4 +-
 .../org/apache/solr/common/luke/FieldFlag.java  |   4 +-
 .../solr/common/params/AnalysisParams.java      |   1 -
 .../solr/common/params/AppendedSolrParams.java  |   1 -
 .../solr/common/params/CollectionParams.java    |   3 +-
 .../solr/common/params/CommonAdminParams.java   |   1 -
 .../apache/solr/common/params/CommonParams.java |   1 -
 .../solr/common/params/ConfigSetParams.java     |   3 +-
 .../solr/common/params/CoreAdminParams.java     |   1 -
 .../solr/common/params/CursorMarkParams.java    |   1 -
 .../solr/common/params/DefaultSolrParams.java   |   1 -
 .../apache/solr/common/params/DisMaxParams.java |   1 -
 .../apache/solr/common/params/EventParams.java  |   4 +-
 .../apache/solr/common/params/ExpandParams.java |   1 -
 .../apache/solr/common/params/FacetParams.java  |   1 -
 .../apache/solr/common/params/GroupParams.java  |   1 -
 .../solr/common/params/HighlightParams.java     |   1 -
 .../solr/common/params/MapSolrParams.java       |   1 -
 .../common/params/ModifiableSolrParams.java     |   1 -
 .../solr/common/params/MoreLikeThisParams.java  |   1 -
 .../solr/common/params/MultiMapSolrParams.java  |   1 -
 .../common/params/QueryElevationParams.java     |   4 +-
 .../solr/common/params/RequiredSolrParams.java  |   1 -
 .../apache/solr/common/params/ShardParams.java  |   1 -
 .../apache/solr/common/params/SimpleParams.java |   3 +-
 .../apache/solr/common/params/SolrParams.java   |   1 -
 .../solr/common/params/SpatialParams.java       |   4 +-
 .../solr/common/params/SpellingParams.java      |   1 -
 .../apache/solr/common/params/StatsParams.java  |   1 -
 .../solr/common/params/TermVectorParams.java    |   4 +-
 .../apache/solr/common/params/TermsParams.java  |   1 -
 .../apache/solr/common/params/UpdateParams.java |   1 -
 .../org/apache/solr/common/util/ByteUtils.java  |   1 -
 .../java/org/apache/solr/common/util/Cache.java |   3 +-
 .../apache/solr/common/util/ContentStream.java  |   1 -
 .../solr/common/util/ContentStreamBase.java     |   1 -
 .../solr/common/util/DataInputInputStream.java  |   1 -
 .../org/apache/solr/common/util/DateUtil.java   |   3 +-
 .../apache/solr/common/util/ExecutorUtil.java   |  11 +-
 .../solr/common/util/FastInputStream.java       |   1 -
 .../solr/common/util/FastOutputStream.java      |   1 -
 .../java/org/apache/solr/common/util/Hash.java  |   3 +-
 .../org/apache/solr/common/util/IOUtils.java    |  15 +-
 .../apache/solr/common/util/IteratorChain.java  |   1 -
 .../org/apache/solr/common/util/NamedList.java  |   1 -
 .../solr/common/util/ObjectReleaseTracker.java  |   3 +-
 .../java/org/apache/solr/common/util/Pair.java  |   3 +-
 .../org/apache/solr/common/util/RetryUtil.java  |  11 +-
 .../solr/common/util/SimpleOrderedMap.java      |   3 +-
 .../common/util/SolrjNamedThreadFactory.java    |   9 +-
 .../org/apache/solr/common/util/StrUtils.java   |   1 -
 .../solr/common/util/SuppressForbidden.java     |   3 +-
 .../org/apache/solr/common/util/URLUtil.java    |   1 -
 .../java/org/apache/solr/common/util/Utils.java |   4 +-
 .../java/org/apache/solr/common/util/XML.java   |   1 -
 .../apache/solr/common/util/XMLErrorLogger.java |   1 -
 ...ollectionAdminRequestRequiredParamsTest.java |   3 +-
 .../apache/solr/client/solrj/GetByIdTest.java   |   3 +-
 .../solr/client/solrj/LargeVolumeTestBase.java  |   1 -
 .../solrj/MergeIndexesExampleTestBase.java      |   1 -
 .../client/solrj/SolrExampleBinaryTest.java     |   1 -
 .../solr/client/solrj/SolrExampleTestBase.java  |   1 -
 .../solr/client/solrj/SolrExampleTests.java     |   1 -
 .../solr/client/solrj/SolrExampleTestsBase.java |   1 -
 .../solr/client/solrj/SolrExampleXMLTest.java   |   1 -
 .../solr/client/solrj/SolrExceptionTest.java    |   1 -
 .../apache/solr/client/solrj/SolrQueryTest.java |   1 -
 .../client/solrj/SolrSchemalessExampleTest.java |   1 -
 .../solr/client/solrj/StartSolrJetty.java       |   1 -
 .../solr/client/solrj/TestLBHttpSolrClient.java |   1 -
 .../client/solrj/TestSolrJErrorHandling.java    |   7 +-
 .../AbstractEmbeddedSolrServerTestCase.java     |   3 +-
 .../client/solrj/embedded/JettyWebappTest.java  |   1 -
 .../solrj/embedded/LargeVolumeEmbeddedTest.java |   1 -
 .../solrj/embedded/LargeVolumeJettyTest.java    |   1 -
 .../embedded/MergeIndexesEmbeddedTest.java      |   1 -
 .../solrj/embedded/SolrExampleEmbeddedTest.java |   1 -
 .../solrj/embedded/SolrExampleJettyTest.java    |   1 -
 .../SolrExampleStreamingBinaryTest.java         |   3 +-
 .../embedded/SolrExampleStreamingTest.java      |   1 -
 .../solrj/embedded/TestEmbeddedSolrServer.java  |   3 +-
 .../solrj/embedded/TestSolrProperties.java      |   1 -
 .../solrj/impl/BasicHttpSolrClientTest.java     |   7 +-
 .../CloudSolrClientMultiConstructorTest.java    |  21 +-
 .../client/solrj/impl/CloudSolrClientTest.java  |   3 +-
 .../impl/ConcurrentUpdateSolrClientTest.java    |   3 -
 .../solrj/impl/ExternalHttpClientTest.java      |   3 +-
 .../client/solrj/impl/LBHttpSolrClientTest.java |   1 -
 .../solrj/impl/SolrPortAwareCookieSpecTest.java |   3 +-
 .../impl/TestCloudSolrClientConnections.java    |   3 +-
 .../client/solrj/io/sql/JdbcDriverTest.java     |   3 +-
 .../solr/client/solrj/io/sql/JdbcTest.java      | 349 ++++++-
 .../client/solrj/io/stream/JDBCStreamTest.java  |   3 +-
 .../solrj/io/stream/RecordCountStream.java      |   2 +-
 .../solrj/io/stream/StreamExpressionTest.java   |  66 +-
 .../stream/StreamExpressionToExpessionTest.java |  22 +-
 .../client/solrj/io/stream/StreamingTest.java   |   3 +-
 .../stream/expr/StreamExpressionParserTest.java |   3 +-
 .../io/stream/ops/ConcatOperationTest.java      |   3 +-
 .../solrj/io/stream/ops/OperationsTest.java     |   3 +-
 .../solr/client/solrj/request/SchemaTest.java   |   3 +-
 .../solr/client/solrj/request/SolrPingTest.java |   3 +-
 .../request/TestConfigSetAdminRequest.java      |   1 -
 .../client/solrj/request/TestCoreAdmin.java     |   1 -
 .../client/solrj/request/TestUpdateRequest.java |   3 +-
 .../solrj/response/AnlysisResponseBaseTest.java |   1 -
 .../response/DocumentAnalysisResponseTest.java  |   1 -
 .../client/solrj/response/FacetFieldTest.java   |   3 +-
 .../response/FieldAnalysisResponseTest.java     |   1 -
 .../solrj/response/NoOpResponseParserTest.java  |   3 +-
 .../solrj/response/QueryResponseTest.java       |   1 -
 .../solrj/response/TermsResponseTest.java       |   3 +-
 .../solrj/response/TestClusteringResponse.java  |   3 +-
 .../solrj/response/TestSpellCheckResponse.java  |   3 +-
 .../solrj/response/TestSuggesterResponse.java   |   3 +-
 .../solr/client/solrj/util/ClientUtilsTest.java |   1 -
 .../apache/solr/common/SolrDocumentTest.java    |   1 -
 .../solr/common/cloud/SolrZkClientTest.java     |   3 +-
 .../solr/common/cloud/TestZkConfigManager.java  |   1 -
 .../common/params/CommonAdminParamsTest.java    |   1 -
 .../solr/common/params/CommonParamsTest.java    |   1 -
 .../common/params/ModifiableSolrParamsTest.java |  25 +-
 .../solr/common/params/ShardParamsTest.java     |   3 +-
 .../solr/common/params/SolrParamTest.java       |   1 -
 .../solr/common/util/ContentStreamTest.java     |   1 -
 .../solr/common/util/IteratorChainTest.java     |   1 -
 .../apache/solr/common/util/NamedListTest.java  |   1 -
 .../apache/solr/common/util/TestDateUtil.java   |   3 +-
 .../org/apache/solr/common/util/TestHash.java   |   3 +-
 .../solr/common/util/TestJavaBinCodec.java      |   3 +-
 .../solr/common/util/TestJsonRecordReader.java  |   3 +-
 .../apache/solr/common/util/TestRetryUtil.java  |   3 +-
 .../solr/common/util/TestXMLEscaping.java       |   1 -
 .../apache/solr/common/util/URLUtilTest.java    |   1 -
 .../solr/BaseDistributedSearchTestCase.java     |   3 +-
 .../src/java/org/apache/solr/JSONTestUtil.java  |   1 -
 .../apache/solr/SolrIgnoredThreadsFilter.java   |  13 +-
 .../java/org/apache/solr/SolrJettyTestBase.java |   3 +-
 .../java/org/apache/solr/SolrTestCaseHS.java    |   4 +-
 .../java/org/apache/solr/SolrTestCaseJ4.java    |   1 -
 .../solr/analysis/MockCharFilterFactory.java    |   3 +-
 .../solr/analysis/MockTokenFilterFactory.java   |   3 +-
 .../solr/analysis/MockTokenizerFactory.java     |   3 +-
 .../analysis/StringMockSolrResourceLoader.java  |   3 +-
 .../solr/cloud/AbstractDistribZkTestBase.java   |   3 +-
 .../apache/solr/cloud/AbstractZkTestCase.java   |   3 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java |   7 +-
 .../org/apache/solr/cloud/CloudInspectUtil.java |  33 +-
 .../java/org/apache/solr/cloud/IpTables.java    |   4 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java |   3 +-
 .../org/apache/solr/cloud/MockSolrZkClient.java |   3 +-
 .../apache/solr/cloud/MockZkStateReader.java    |   3 +-
 .../java/org/apache/solr/cloud/SocketProxy.java |   3 +-
 .../solr/cloud/StoppableIndexingThread.java     |  25 +-
 .../solr/cloud/StoppableSearchThread.java       |   7 +-
 .../org/apache/solr/cloud/ZkTestServer.java     |  27 +-
 .../solr/core/AbstractBadConfigTestBase.java    |   1 -
 .../apache/solr/core/MockDirectoryFactory.java  |   6 +-
 .../solr/core/MockFSDirectoryFactory.java       |   4 +-
 .../component/TrackingShardHandlerFactory.java  |   3 +-
 .../processor/BufferingRequestProcessor.java    |   1 -
 .../apache/solr/util/AbstractSolrTestCase.java  |   2 -
 .../apache/solr/util/BadHdfsThreadsFilter.java  |   3 +-
 .../solr/util/BadMrClusterThreadsFilter.java    |   3 +-
 .../solr/util/BadZookeeperThreadsFilter.java    |   3 +-
 .../org/apache/solr/util/BaseTestHarness.java   |   3 +-
 .../org/apache/solr/util/DOMUtilTestBase.java   |   1 -
 .../org/apache/solr/util/ExternalPaths.java     |   3 +-
 .../apache/solr/util/RESTfulServerProvider.java |   3 +-
 .../solr/util/RandomForceMergePolicy.java       |   1 -
 .../org/apache/solr/util/RandomMergePolicy.java |  52 +-
 .../apache/solr/util/ReadOnlyCoresLocator.java  |   3 +-
 .../java/org/apache/solr/util/RestTestBase.java |   3 +-
 .../org/apache/solr/util/RestTestHarness.java   |   3 +-
 .../util/RevertDefaultThreadHandlerRule.java    |  23 +-
 .../org/apache/solr/util/SSLTestConfig.java     |   3 +-
 .../java/org/apache/solr/util/TestHarness.java  |   1 -
 5346 files changed, 22854 insertions(+), 22215 deletions(-)
----------------------------------------------------------------------



[12/15] lucene-solr git commit: add more non-stress test cases

Posted by mi...@apache.org.
add more non-stress test cases


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: b1b2c799aa1d6598daa4ea8c63a5fa5484b5052d
Parents: 82ecccf
Author: Mike McCandless <mi...@apache.org>
Authored: Sat Feb 6 09:04:15 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Feb 6 09:04:15 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/replicator/nrt/Node.java  |   3 +-
 .../lucene/replicator/nrt/ReplicaNode.java      |   2 +
 .../lucene/replicator/nrt/NodeProcess.java      |  17 +-
 .../replicator/nrt/SimplePrimaryNode.java       |  20 +-
 .../replicator/nrt/SimpleReplicaNode.java       |  12 +-
 .../lucene/replicator/nrt/SimpleServer.java     |   5 +-
 .../replicator/nrt/TestNRTReplication.java      | 717 +++++++++++++++----
 .../nrt/TestStressNRTReplication.java           |  13 +-
 8 files changed, 640 insertions(+), 149 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
index 742b19f..e54c01e 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
@@ -49,7 +49,8 @@ import org.apache.lucene.util.StringHelper;
 abstract class Node implements Closeable {
 
   static boolean VERBOSE_FILES = true;
-  static boolean VERBOSE_CONNECTIONS = false;
+  // nocommit
+  static boolean VERBOSE_CONNECTIONS = true;
 
   // Keys we store into IndexWriter's commit user data:
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/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 c7af429..133992f 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
@@ -86,6 +86,8 @@ abstract class ReplicaNode extends Node {
 
       // Obtain a write lock on this index since we "act like" an IndexWriter, to prevent any other IndexWriter or ReplicaNode from using it:
       writeFileLock = dir.obtainLock(IndexWriter.WRITE_LOCK_NAME);
+      
+      // nocommit must check for no pending deletes here, like IW does
 
       state = "init";
       deleter = new ReplicaFileDeleter(this, dir);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index be0b3df3..9d8b764 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -77,6 +77,7 @@ class NodeProcess implements Closeable {
       isOpen = false;
       p.destroy();
       try {
+        p.waitFor();
         pumper.join();
       } catch (InterruptedException ie) {
         Thread.currentThread().interrupt();
@@ -95,6 +96,7 @@ class NodeProcess implements Closeable {
       }
       return true;
     } catch (Throwable t) {
+      // nocommit throw this
       // Something wrong with this replica; skip it:
       System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
       return false;
@@ -106,6 +108,7 @@ class NodeProcess implements Closeable {
       c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
       c.flush();
     } catch (Throwable t) {
+      // nocommit throw this
       // Something wrong with this replica; skip it:
       System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
     }
@@ -118,6 +121,7 @@ class NodeProcess implements Closeable {
       c.s.shutdownOutput();
       return c.in.readVLong();
     } catch (Throwable t) {
+      // nocommit throw this
       // Something wrong with this replica; skip it:
       System.out.println("PARENT: top: hit SocketException during getSearchingVersion with R" + id + "; skipping");
       return -1L;
@@ -145,11 +149,11 @@ class NodeProcess implements Closeable {
   public synchronized boolean shutdown() {
     lock.lock();
     try {
-      System.out.println("PARENT: now shutdown node=" + id + " isOpen=" + isOpen);
+      //System.out.println("PARENT: now shutdown node=" + id + " isOpen=" + isOpen);
       if (isOpen) {
         // Ask the child process to shutdown gracefully:
         isOpen = false;
-        System.out.println("PARENT: send CMD_CLOSE to node=" + id);
+        //System.out.println("PARENT: send CMD_CLOSE to node=" + id);
         try (Connection c = new Connection(tcpPort)) {
           c.out.writeByte(SimplePrimaryNode.CMD_CLOSE);
           c.flush();
@@ -174,6 +178,15 @@ class NodeProcess implements Closeable {
     }
   }
 
+  public void newNRTPoint(long version, int primaryTCPPort) throws IOException {
+    try (Connection c = new Connection(tcpPort)) {
+      c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
+      c.out.writeVLong(version);
+      c.out.writeInt(primaryTCPPort);
+      c.flush();
+    }
+  }
+
   public void addOrUpdateDocument(Connection c, Document doc, boolean isUpdate) throws IOException {
     if (isPrimary == false) {
       throw new IllegalStateException("only primary can index");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
index b9ecced..7f5634c 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -76,8 +76,8 @@ class SimplePrimaryNode extends PrimaryNode {
   final Random random;
 
   // These are updated by parent test process whenever replicas change:
-  int[] replicaTCPPorts;
-  int[] replicaIDs;
+  int[] replicaTCPPorts = new int[0];
+  int[] replicaIDs = new int[0];
 
   // So we only flip a bit once per file name:
   final Set<String> bitFlipped = Collections.synchronizedSet(new HashSet<>());
@@ -115,8 +115,8 @@ class SimplePrimaryNode extends PrimaryNode {
   }
 
   public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory,
-                           boolean doFlipBitsDuringCopy) throws IOException {
-    super(initWriter(id, random, indexPath), id, primaryGen, forcePrimaryVersion, searcherFactory);
+                           boolean doFlipBitsDuringCopy, boolean doCheckIndexOnClose) throws IOException {
+    super(initWriter(id, random, indexPath, doCheckIndexOnClose), id, primaryGen, forcePrimaryVersion, searcherFactory);
     this.tcpPort = tcpPort;
     this.random = new Random(random.nextLong());
     this.doFlipBitsDuringCopy = doFlipBitsDuringCopy;
@@ -129,8 +129,8 @@ class SimplePrimaryNode extends PrimaryNode {
     this.replicaTCPPorts = replicaTCPPorts;
   }
 
-  private static IndexWriter initWriter(int id, Random random, Path indexPath) throws IOException {
-    Directory dir = SimpleReplicaNode.getDirectory(random, id, indexPath);
+  private static IndexWriter initWriter(int id, Random random, Path indexPath, boolean doCheckIndexOnClose) throws IOException {
+    Directory dir = SimpleReplicaNode.getDirectory(random, id, indexPath, doCheckIndexOnClose);
 
     MockAnalyzer analyzer = new MockAnalyzer(random);
     analyzer.setMaxTokenLength(TestUtil.nextInt(random, 1, IndexWriter.MAX_TERM_LENGTH));
@@ -599,13 +599,15 @@ class SimplePrimaryNode extends PrimaryNode {
           IndexSearcher searcher = mgr.acquire();
           try {
             long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
-            int hitCount = searcher.search(new TermQuery(new Term("body", "the")), 1).totalHits;
+            int hitCount = searcher.count(new TermQuery(new Term("body", "the")));
             //message("version=" + version + " searcher=" + searcher);
             out.writeVLong(version);
             out.writeVInt(hitCount);
+            bos.flush();
           } finally {
             mgr.release(searcher);
           }
+          bos.flush();
         }
         continue outer;
 
@@ -615,10 +617,11 @@ class SimplePrimaryNode extends PrimaryNode {
           IndexSearcher searcher = mgr.acquire();
           try {
             long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
-            int hitCount = searcher.search(new MatchAllDocsQuery(), 1).totalHits;
+            int hitCount = searcher.count(new MatchAllDocsQuery());
             //message("version=" + version + " searcher=" + searcher);
             out.writeVLong(version);
             out.writeVInt(hitCount);
+            bos.flush();
           } finally {
             mgr.release(searcher);
           }
@@ -630,6 +633,7 @@ class SimplePrimaryNode extends PrimaryNode {
           Thread.currentThread().setName("msearch");
           int expectedAtLeastCount = in.readVInt();
           verifyAtLeastMarkerCount(expectedAtLeastCount, out);
+          bos.flush();
         }
         continue outer;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
index bc8bb03..2510c40 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -64,8 +64,8 @@ class SimpleReplicaNode extends ReplicaNode {
   /** Changes over time, as primary node crashes and moves around */
   int curPrimaryTCPPort;
 
-  public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory) throws IOException {
-    super(id, getDirectory(random, id, indexPath), searcherFactory);
+  public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory, boolean doCheckIndexOnClose) throws IOException {
+    super(id, getDirectory(random, id, indexPath, doCheckIndexOnClose), searcherFactory);
     this.tcpPort = tcpPort;
     this.random = new Random(random.nextLong());
 
@@ -131,13 +131,13 @@ class SimpleReplicaNode extends ReplicaNode {
     return new SimpleCopyJob(reason, c, copyState, this, files, highPriority, onceDone);
   }
 
-  static Directory getDirectory(Random random, int id, Path path) throws IOException {
+  static Directory getDirectory(Random random, int id, Path path, boolean doCheckIndexOnClose) throws IOException {
     MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path);
     
     dir.setAssertNoUnrefencedFilesOnClose(true);
-    // This is very costly (takes more time to check than it did to index); we do this ourselves in the end instead of each time a replica
-    // is restarted:
-    dir.setCheckIndexOnClose(false);
+    if (doCheckIndexOnClose) {
+      dir.setCheckIndexOnClose(false);
+    }
 
     // Corrupt any index files not referenced by current commit point; this is important (increases test evilness) because we may have done
     // a hard crash of the previous JVM writing to this directory and so MDW's corrupt-unknown-files-on-close never ran:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index 3fdc45f..72e33d7 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -238,6 +238,7 @@ public class SimpleServer extends LuceneTestCase {
     boolean doRandomCrash = "true".equals(System.getProperty("tests.nrtreplication.doRandomCrash"));
     boolean doRandomClose = "true".equals(System.getProperty("tests.nrtreplication.doRandomClose"));
     boolean doFlipBitsDuringCopy = "true".equals(System.getProperty("tests.nrtreplication.doFlipBitsDuringCopy"));
+    boolean doCheckIndexOnClose = "true".equals(System.getProperty("tests.nrtreplication.checkonclose"));
 
     // Create server socket that we listen for incoming requests on:
     try (final ServerSocket ss = new ServerSocket(0, 0, InetAddress.getLoopbackAddress())) {
@@ -246,11 +247,11 @@ public class SimpleServer extends LuceneTestCase {
       System.out.println("\nPORT: " + tcpPort);
       final Node node;
       if (isPrimary) {
-        node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy);
+        node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy, doCheckIndexOnClose);
         System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
       } else {
         try {
-          node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+          node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null, doCheckIndexOnClose);
         } catch (RuntimeException re) {
           if (re.getMessage().startsWith("replica cannot start")) {
             // this is "OK": it means MDW's refusal to delete a segments_N commit point means we cannot start:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 7ba3bc2..15e9c8c 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -17,15 +17,6 @@ package org.apache.lucene.replicator.nrt;
  * limitations under the License.
  */
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LineFileDocs;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-import org.apache.lucene.util.LuceneTestCase;
-
-import com.carrotsearch.randomizedtesting.SeedUtils;
-
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -38,6 +29,16 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
 // nocommit make some explicit failure tests
 
 // MockRandom's .sd file has no index header/footer:
@@ -49,9 +50,12 @@ public class TestNRTReplication extends LuceneTestCase {
   private Path childTempDir;
 
   final AtomicLong nodeStartCounter = new AtomicLong();
+  private long nextPrimaryGen;
+  private long lastPrimaryGen;
+  LineFileDocs docs;
 
   /** Launches a child "server" (separate JVM), which is either primary or replica node */
-  NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
+  private NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, long forcePrimaryVersion, boolean willCrash) throws IOException {
     List<String> cmd = new ArrayList<>();
 
     cmd.add(System.getProperty("java.home") 
@@ -61,26 +65,30 @@ public class TestNRTReplication extends LuceneTestCase {
         + "java");
     cmd.add("-Xmx512m");
 
+    long myPrimaryGen;
     if (primaryTCPPort != -1) {
+      // I am a replica
       cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + primaryTCPPort);
-    } else if (isPrimary == false) {
-      // We cannot start a replica when there is no primary:
-      return null;
+      myPrimaryGen = lastPrimaryGen;
+    } else {
+      myPrimaryGen = nextPrimaryGen++;
+      lastPrimaryGen = myPrimaryGen;
     }
+    cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
     cmd.add("-Dtests.nrtreplication.closeorcrash=false");
 
     cmd.add("-Dtests.nrtreplication.node=true");
     cmd.add("-Dtests.nrtreplication.nodeid=" + id);
     cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
     cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
-    if (isPrimary) {
+    cmd.add("-Dtests.nrtreplication.checkonclose=true");
+
+    if (primaryTCPPort == -1) {
+      // We are the primary node
       cmd.add("-Dtests.nrtreplication.isPrimary=true");
       cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
     }
 
-    long myPrimaryGen = 0;
-    cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
-
     // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
     // new node:
     long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
@@ -112,7 +120,6 @@ public class TestNRTReplication extends LuceneTestCase {
     long initCommitVersion = -1;
     long initInfosVersion = -1;
     Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
-    boolean willCrash = false;
     boolean sawExistingSegmentsFile = false;
 
     while (true) {
@@ -169,7 +176,7 @@ public class TestNRTReplication extends LuceneTestCase {
                                            message("done wait for process " + p);
                                            int exitValue = p.exitValue();
                                            message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
-                                           if (exitValue != 0) {
+                                           if (exitValue != 0 && finalWillCrash == false) {
                                              // should fail test
                                              throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
                                            }
@@ -178,32 +185,33 @@ public class TestNRTReplication extends LuceneTestCase {
     pumper.setName("pump" + id);
 
     message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
-    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion, nodeClosing);
+    return new NodeProcess(p, id, tcpPort, pumper, primaryTCPPort == -1, initCommitVersion, initInfosVersion, nodeClosing);
   }
 
-  public void testReplicateDeleteAllDocuments() throws Exception {
-
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
     Node.globalStartNS = System.nanoTime();
     childTempDir = createTempDir("child");
+    docs = new LineFileDocs(random());
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    docs.close();
+  }
+
+  public void testReplicateDeleteAllDocuments() throws Exception {
 
-    message("change thread name from " + Thread.currentThread().getName());
-    Thread.currentThread().setName("main");
-    
     Path primaryPath = createTempDir("primary");
-    NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 
     Path replicaPath = createTempDir("replica");
-    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 
     // Tell primary current replicas:
-    try (Connection c = new Connection(primary.tcpPort)) {
-      c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
-      c.out.writeVInt(1);
-      c.out.writeVInt(replica.id);
-      c.out.writeVInt(replica.tcpPort);
-      c.flush();
-      c.in.readByte();
-    }
+    sendReplicasToPrimary(primary, replica);
 
     // Index 10 docs into primary:
     LineFileDocs docs = new LineFileDocs(random());
@@ -215,33 +223,14 @@ public class TestNRTReplication extends LuceneTestCase {
     }
 
     // Nothing in replica index yet
-    Connection replicaC = new Connection(replica.tcpPort);
-    replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
-    replicaC.flush();
-    long version1 = replicaC.in.readVLong();
-    assertEquals(0L, version1);
-    int hitCount = replicaC.in.readVInt();
-    assertEquals(0, hitCount);
+    assertVersionAndHits(replica, 0, 0);
 
     // Refresh primary, which also pushes to replica:
     long primaryVersion1 = primary.flush(0);
     assertTrue(primaryVersion1 > 0);
 
-    long version2;
-
     // Wait for replica to show the change
-    while (true) {
-      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
-      replicaC.flush();
-      version2 = replicaC.in.readVLong();
-      hitCount = replicaC.in.readVInt();
-      if (version2 == primaryVersion1) {
-        assertEquals(10, hitCount);
-        // good!
-        break;
-      }
-      Thread.sleep(10);
-    }
+    waitForVersionAndHits(replica, primaryVersion1, 10);
 
     // Delete all docs from primary
     if (random().nextBoolean()) {
@@ -255,32 +244,14 @@ public class TestNRTReplication extends LuceneTestCase {
     }
 
     // Replica still shows 10 docs:
-    replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
-    replicaC.flush();
-    long version3 = replicaC.in.readVLong();
-    assertEquals(version2, version3);
-    hitCount = replicaC.in.readVInt();
-    assertEquals(10, hitCount);
+    assertVersionAndHits(replica, primaryVersion1, 10);
     
     // Refresh primary, which also pushes to replica:
     long primaryVersion2 = primary.flush(0);
     assertTrue(primaryVersion2 > primaryVersion1);
 
     // Wait for replica to show the change
-    long version4;
-    while (true) {
-      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
-      replicaC.flush();
-      version4 = replicaC.in.readVLong();
-      hitCount = replicaC.in.readVInt();
-      if (version4 == primaryVersion2) {
-        assertTrue(version4 > version3);
-        assertEquals(0, hitCount);
-        // good!
-        break;
-      }
-      Thread.sleep(10);
-    }
+    waitForVersionAndHits(replica, primaryVersion2, 0);
 
     // Index 10 docs again:
     for(int i=0;i<10;i++) {
@@ -293,21 +264,8 @@ public class TestNRTReplication extends LuceneTestCase {
     assertTrue(primaryVersion3 > primaryVersion2);
 
     // Wait for replica to show the change
-    while (true) {
-      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
-      replicaC.flush();
-      long version5 = replicaC.in.readVLong();
-      hitCount = replicaC.in.readVInt();
-      if (version5 == primaryVersion3) {
-        assertEquals(10, hitCount);
-        assertTrue(version5 > version4);
-        // good!
-        break;
-      }
-      Thread.sleep(10);
-    }
+    waitForVersionAndHits(replica, primaryVersion3, 10);
 
-    replicaC.close();
     primaryC.close();
 
     replica.close();
@@ -316,27 +274,13 @@ public class TestNRTReplication extends LuceneTestCase {
 
   public void testReplicateForceMerge() throws Exception {
 
-    Node.globalStartNS = System.nanoTime();
-    childTempDir = createTempDir("child");
-
-    message("change thread name from " + Thread.currentThread().getName());
-    Thread.currentThread().setName("main");
-    
     Path primaryPath = createTempDir("primary");
-    NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 
     Path replicaPath = createTempDir("replica");
-    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 
-    // Tell primary current replicas:
-    try (Connection c = new Connection(primary.tcpPort)) {
-      c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
-      c.out.writeVInt(1);
-      c.out.writeVInt(replica.id);
-      c.out.writeVInt(replica.tcpPort);
-      c.flush();
-      c.in.readByte();
-    }
+    sendReplicasToPrimary(primary, replica);
 
     // Index 10 docs into primary:
     LineFileDocs docs = new LineFileDocs(random());
@@ -367,29 +311,560 @@ public class TestNRTReplication extends LuceneTestCase {
     long primaryVersion3 = primary.flush(0);
     assertTrue(primaryVersion3 > primaryVersion2);
 
-    Connection replicaC = new Connection(replica.tcpPort);
-
     // Wait for replica to show the change
-    while (true) {
-      replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
-      replicaC.flush();
-      long version = replicaC.in.readVLong();
-      int hitCount = replicaC.in.readVInt();
-      if (version == primaryVersion3) {
-        assertEquals(20, hitCount);
-        // good!
-        break;
+    waitForVersionAndHits(replica, primaryVersion3, 20);
+
+    primaryC.close();
+
+    replica.close();
+    primary.close();
+  }
+
+  // Start up, index 10 docs, replicate, but crash and restart the replica without committing it:
+  public void testReplicaCrashNoCommit() throws Exception {
+
+    Path primaryPath = createTempDir("primary");
+    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+    Path replicaPath = createTempDir("replica");
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
       }
-      Thread.sleep(10);
     }
 
-    replicaC.close();
-    primaryC.close();
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    // Crash replica:
+    replica.crash();
+
+    // Restart replica:
+    replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+    // On startup the replica searches the last commit (empty here):
+    assertVersionAndHits(replica, 0, 0);
+
+    // Ask replica to sync:
+    replica.newNRTPoint(primaryVersion1, primary.tcpPort);
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    replica.close();
+    primary.close();
+  }
+
+  // Start up, index 10 docs, replicate, commit, crash and restart the replica
+  public void testReplicaCrashWithCommit() throws Exception {
+
+    Path primaryPath = createTempDir("primary");
+    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+    Path replicaPath = createTempDir("replica");
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    // Commit and crash replica:
+    replica.commit();
+    replica.crash();
+
+    // Restart replica:
+    replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+    // On startup the replica searches the last commit:
+    assertVersionAndHits(replica, primaryVersion1, 10);
 
     replica.close();
     primary.close();
   }
 
+  // Start up, index 10 docs, replicate, commit, crash, index more docs, replicate, then restart the replica
+  public void testIndexingWhileReplicaIsDown() throws Exception {
+
+    Path primaryPath = createTempDir("primary");
+    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+    Path replicaPath = createTempDir("replica");
+    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    // Commit and crash replica:
+    replica.commit();
+    replica.crash();
+
+    sendReplicasToPrimary(primary);
+
+    // Index 10 more docs, while replica is down
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // And flush:
+    long primaryVersion2 = primary.flush(0);
+    assertTrue(primaryVersion2 > primaryVersion1);
+
+    // Now restart replica:
+    replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // On startup the replica still searches its last commit:
+    assertVersionAndHits(replica, primaryVersion1, 10);
+
+    // Now ask replica to sync:
+    replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+
+    waitForVersionAndHits(replica, primaryVersion2, 20);
+
+    replica.close();
+    primary.close();
+  }
+ 
+  // Crash primary and promote a replica
+  public void testCrashPrimary1() throws Exception {
+
+    Path path1 = createTempDir("1");
+    NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+    Path path2 = createTempDir("2");
+    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    // Crash primary:
+    primary.crash();
+
+    // Promote replica:
+    replica.commit();
+    replica.close();
+    
+    primary = startNode(-1, 1, path2, -1, false);
+
+    // Should still see 10 docs:
+    assertVersionAndHits(primary, primaryVersion1, 10);
+
+    primary.close();
+  }
+
+  // Crash primary and then restart it
+  public void testCrashPrimary2() throws Exception {
+
+    Path path1 = createTempDir("1");
+    NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+    Path path2 = createTempDir("2");
+    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    primary.commit();
+
+    // Index 10 docs, but crash before replicating or committing:
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Crash primary:
+    primary.crash();
+
+    // Restart it:
+    primary = startNode(-1, 0, path1, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 more docs
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    long primaryVersion2 = primary.flush(0);
+    assertTrue(primaryVersion2 > primaryVersion1);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion2, 20);
+
+    primary.close();
+    replica.close();
+  }
+
+  // Crash primary and then restart it, while a replica node is down, then bring replica node back up and make sure it properly "unforks" itself
+  public void testCrashPrimary3() throws Exception {
+
+    Path path1 = createTempDir("1");
+    NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+    Path path2 = createTempDir("2");
+    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    replica.commit();
+
+    replica.close();
+    primary.crash();
+
+    // At this point replica is "in the future": it has 10 docs committed, but the primary crashed before committing so it has 0 docs
+
+    // Restart primary:
+    primary = startNode(-1, 0, path1, -1, true);
+
+    // Index 20 docs into primary:
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<20;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Flush primary, but there are no replicas to sync to:
+    long primaryVersion2 = primary.flush(0);
+
+    // Now restart replica, which on init should detect on a "lost branch" because its 10 docs that were committed came from a different
+    // primary node:
+    replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    assertVersionAndHits(replica, primaryVersion2, 20);
+
+    primary.close();
+    replica.close();
+  }
+
+  public void testCrashPrimaryWhileCopying() throws Exception {
+
+    Path path1 = createTempDir("1");
+    NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+    Path path2 = createTempDir("2");
+    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 100 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<100;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes (async) to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    Thread.sleep(TestUtil.nextInt(random(), 1, 30));
+
+    // Crash primary, likely/hopefully while replica is still copying
+    primary.crash();
+
+    // Could see either 100 docs (replica finished before crash) or 0 docs:
+    try (Connection c = new Connection(replica.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+      c.flush();
+      long version = c.in.readVLong();
+      int hitCount = c.in.readVInt();
+      if (version == 0) {
+        assertEquals(0, hitCount);
+      } else {
+        assertEquals(primaryVersion1, version);
+        assertEquals(100, hitCount);
+      }
+    }
+
+    primary.close();
+    replica.close();
+  }
+
+  public void testCrashReplica() throws Exception {
+
+    Path path1 = createTempDir("1");
+    NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+    Path path2 = createTempDir("2");
+    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Index 10 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Refresh primary, which also pushes to replica:
+    long primaryVersion1 = primary.flush(0);
+    assertTrue(primaryVersion1 > 0);
+
+    // Wait for replica to sync up:
+    waitForVersionAndHits(replica, primaryVersion1, 10);
+
+    // Crash replica
+    replica.crash();
+
+    sendReplicasToPrimary(primary);
+
+    // Lots of new flushes while replica is down:
+    long primaryVersion2 = 0;
+    for(int iter=0;iter<10;iter++) {
+      // Index 10 docs into primary:
+      try (Connection c = new Connection(primary.tcpPort)) {
+        c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+        for(int i=0;i<10;i++) {
+          Document doc = docs.nextDoc();
+          primary.addOrUpdateDocument(c, doc, false);
+        }
+      }
+      primaryVersion2 = primary.flush(0);
+    }
+
+    // Start up replica again:
+    replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    sendReplicasToPrimary(primary, replica);
+
+    // Now ask replica to sync:
+    replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+
+    // Make sure it sees all docs that were indexed while it was down:
+    assertVersionAndHits(primary, primaryVersion2, 110);
+
+    replica.close();
+    primary.close();
+  }
+
+  public void testFullClusterCrash() throws Exception {
+
+    Path path1 = createTempDir("1");
+    NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+    Path path2 = createTempDir("2");
+    NodeProcess replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
+
+    Path path3 = createTempDir("3");
+    NodeProcess replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
+
+    sendReplicasToPrimary(primary, replica1, replica2);
+
+    // Index 50 docs into primary:
+    LineFileDocs docs = new LineFileDocs(random());
+    long primaryVersion1 = 0;
+    for (int iter=0;iter<5;iter++) {
+      try (Connection c = new Connection(primary.tcpPort)) {
+        c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+        for(int i=0;i<10;i++) {
+          Document doc = docs.nextDoc();
+          primary.addOrUpdateDocument(c, doc, false);
+        }
+      }
+
+      // Refresh primary, which also pushes to replicas:
+      primaryVersion1 = primary.flush(0);
+      assertTrue(primaryVersion1 > 0);
+    }
+
+    // Wait for replicas to sync up:
+    waitForVersionAndHits(replica1, primaryVersion1, 50);
+    waitForVersionAndHits(replica2, primaryVersion1, 50);
+
+    primary.commit();
+    replica1.commit();
+    replica2.commit();
+
+    // Index 10 more docs, but don't sync to replicas:
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      for(int i=0;i<10;i++) {
+        Document doc = docs.nextDoc();
+        primary.addOrUpdateDocument(c, doc, false);
+      }
+    }
+
+    // Full cluster crash
+    primary.crash();
+    replica1.crash();
+    replica2.crash();
+
+    // Full cluster restart
+    primary = startNode(-1, 0, path1, -1, true);
+    replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
+    replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
+
+    // Only 50 because we didn't commit primary before the crash:
+    
+    // It's -1 because it's unpredictable how IW changes segments version on init:
+    assertVersionAndHits(primary, -1, 50);
+    assertVersionAndHits(replica1, primaryVersion1, 50);
+    assertVersionAndHits(replica2, primaryVersion1, 50);
+
+    primary.close();
+    replica1.close();
+    replica2.close();
+  }
+
+  /** Tell primary current replicas. */
+  private void sendReplicasToPrimary(NodeProcess primary, NodeProcess... replicas) throws IOException {
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+      c.out.writeVInt(replicas.length);
+      for(int id=0;id<replicas.length;id++) {
+        NodeProcess replica = replicas[id];
+        c.out.writeVInt(replica.id);
+        c.out.writeVInt(replica.tcpPort);
+      }
+      c.flush();
+      c.in.readByte();
+    }
+  }
+
+  /** Verifies this node is currently searching the specified version with the specified total hit count, or that it eventually does when
+   *  keepTrying is true. */
+  private void assertVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
+    try (Connection c = new Connection(node.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+      c.flush();
+      long version = c.in.readVLong();
+      int hitCount = c.in.readVInt();
+      if (expectedVersion != -1) {
+        assertEquals("hitCount=" + hitCount, expectedVersion, version);
+      }
+      assertEquals(expectedHitCount, hitCount);
+    }
+  }
+
+  private void waitForVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
+    try (Connection c = new Connection(node.tcpPort)) {
+      while (true) {
+        c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+        c.flush();
+        long version = c.in.readVLong();
+        int hitCount = c.in.readVInt();
+
+        if (version == expectedVersion) {
+          assertEquals(expectedHitCount, hitCount);
+          break;
+        }
+
+        assertTrue(version < expectedVersion);
+        Thread.sleep(10);
+      }
+    }
+  }
+
   static void message(String message) {
     long now = System.nanoTime();
     System.out.println(String.format(Locale.ROOT,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
index 63ff12a..04bbdc1 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
@@ -518,6 +518,10 @@ public class TestStressNRTReplication extends LuceneTestCase {
       return null;
     }
 
+    // This is very costly (takes more time to check than it did to index); we do this ourselves in the end instead of each time a replica
+    // is restarted:
+    // cmd.add("-Dtests.nrtreplication.checkonclose=true");
+
     cmd.add("-Dtests.nrtreplication.node=true");
     cmd.add("-Dtests.nrtreplication.nodeid=" + id);
     cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
@@ -590,7 +594,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
     long initInfosVersion = -1;
     Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
     boolean willCrash = false;
-    boolean sawExistingSegmentsFile = false;
 
     while (true) {
       String l = r.readLine();
@@ -609,12 +612,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
 
         // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
         if (isPrimary == false) {
-          if (sawExistingSegmentsFile) {
-            // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
-            // and retry again later:
-            message("failed to remove segments_N; skipping");
-            return null;
-          }
           for(int i=0;i<100;i++) {
             NodeProcess primary2 = primary;
             if (primaryGen != myPrimaryGen || primary2 == null || primary2.nodeIsClosing.get()) {
@@ -658,8 +655,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
         willCrash = true;
       } else if (l.startsWith("NODE STARTED")) {
         break;
-      } else if (l.contains("replica cannot start: existing segments file=")) {
-        sawExistingSegmentsFile = true;
       }
     }
 


[09/15] lucene-solr git commit: fix compilation errors

Posted by mi...@apache.org.
fix compilation errors


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

Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: d9f20f53edd9dc5895ad18e1844fb4ae9652189e
Parents: 8889469
Author: Michael McCandless <ma...@mikemccandless.com>
Authored: Tue Feb 2 06:16:34 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 2 06:16:34 2016 -0500

----------------------------------------------------------------------
 .../replicator/nrt/TestNRTReplication.java      | 36 +++++++++++---------
 1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9f20f53/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index b2240eb..e2824af 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -17,6 +17,15 @@ package org.apache.lucene.replicator.nrt;
  * limitations under the License.
  */
 
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -25,18 +34,10 @@ import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LineFileDocs;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-
-import com.carrotsearch.randomizedtesting.SeedUtils;
-
 // MockRandom's .sd file has no index header/footer:
 @SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
 @SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
@@ -151,6 +152,7 @@ public class TestNRTReplication extends LuceneTestCase {
     final boolean finalWillCrash = willCrash;
 
     // Baby sits the child process, pulling its stdout and printing to our stdout:
+    AtomicBoolean nodeClosing = new AtomicBoolean();
     Thread pumper = ThreadPumper.start(
                                        new Runnable() {
                                          @Override
@@ -170,11 +172,11 @@ public class TestNRTReplication extends LuceneTestCase {
                                              throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
                                            }
                                          }
-                                       }, r, System.out, null);
+                                       }, r, System.out, null, nodeClosing);
     pumper.setName("pump" + id);
 
     message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
-    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion, nodeClosing);
   }
 
   public void testReplicateDeleteAllDocuments() throws Exception {
@@ -220,7 +222,7 @@ public class TestNRTReplication extends LuceneTestCase {
     assertEquals(0, hitCount);
 
     // Refresh primary, which also pushes to replica:
-    long primaryVersion1 = primary.flush();
+    long primaryVersion1 = primary.flush(0);
     assertTrue(primaryVersion1 > 0);
 
     long version2;
@@ -259,7 +261,7 @@ public class TestNRTReplication extends LuceneTestCase {
     assertEquals(10, hitCount);
     
     // Refresh primary, which also pushes to replica:
-    long primaryVersion2 = primary.flush();
+    long primaryVersion2 = primary.flush(0);
     assertTrue(primaryVersion2 > primaryVersion1);
 
     // Wait for replica to show the change
@@ -285,7 +287,7 @@ public class TestNRTReplication extends LuceneTestCase {
     }
 
     // Refresh primary, which also pushes to replica:
-    long primaryVersion3 = primary.flush();
+    long primaryVersion3 = primary.flush(0);
     assertTrue(primaryVersion3 > primaryVersion2);
 
     // Wait for replica to show the change
@@ -344,7 +346,7 @@ public class TestNRTReplication extends LuceneTestCase {
     }
 
     // Refresh primary, which also pushes to replica:
-    long primaryVersion1 = primary.flush();
+    long primaryVersion1 = primary.flush(0);
     assertTrue(primaryVersion1 > 0);
 
     // Index 10 more docs into primary:
@@ -354,13 +356,13 @@ public class TestNRTReplication extends LuceneTestCase {
     }
 
     // Refresh primary, which also pushes to replica:
-    long primaryVersion2 = primary.flush();
+    long primaryVersion2 = primary.flush(0);
     assertTrue(primaryVersion2 > primaryVersion1);
 
     primary.forceMerge(primaryC);
 
     // Refresh primary, which also pushes to replica:
-    long primaryVersion3 = primary.flush();
+    long primaryVersion3 = primary.flush(0);
     assertTrue(primaryVersion3 > primaryVersion2);
 
     Connection replicaC = new Connection(replica.tcpPort);


[14/15] lucene-solr git commit: Merge branch 'master' into nrt_replicas

Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --cc lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index 133992f,0000000..a7adbe2
mode 100644,000000..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
@@@ -1,763 -1,0 +1,768 @@@
 +package org.apache.lucene.replicator.nrt;
 +
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.List;
 +import java.util.Locale;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.concurrent.ConcurrentHashMap;
 +import java.util.concurrent.ConcurrentMap;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +
 +import org.apache.lucene.index.DirectoryReader;
 +import org.apache.lucene.index.IndexFileNames;
 +import org.apache.lucene.index.IndexWriter;
 +import org.apache.lucene.index.SegmentInfos;
 +import org.apache.lucene.index.Term;
 +import org.apache.lucene.search.IndexSearcher;
 +import org.apache.lucene.search.SearcherFactory;
 +import org.apache.lucene.search.TermQuery;
 +import org.apache.lucene.store.AlreadyClosedException;
 +import org.apache.lucene.store.BufferedChecksumIndexInput;
 +import org.apache.lucene.store.ByteArrayIndexInput;
 +import org.apache.lucene.store.Directory;
++import org.apache.lucene.store.FSDirectory;
 +import org.apache.lucene.store.IOContext;
 +import org.apache.lucene.store.IndexOutput;
 +import org.apache.lucene.store.Lock;
 +import org.apache.lucene.util.IOUtils;
 +
 +/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */
 +
 +abstract class ReplicaNode extends Node {
 +
 +  ReplicaFileDeleter deleter;
 +
 +  /** IncRef'd files in the current commit point: */
 +  private final Collection<String> lastCommitFiles = new HashSet<>();
 +
 +  /** IncRef'd files in the current NRT point: */
 +  protected final Collection<String> lastNRTFiles = new HashSet<>();
 +
 +  /** Currently running merge pre-copy jobs */
 +  protected final Set<CopyJob> mergeCopyJobs = Collections.synchronizedSet(new HashSet<>());
 +
 +  /** Non-null when we are currently copying files from a new NRT point: */
 +  protected CopyJob curNRTCopy;
 +
 +  /** We hold this to ensure an external IndexWriter cannot also open on our directory: */
 +  private final Lock writeFileLock;
 +
 +  /** Merged segment files that we pre-copied, but have not yet made visible in a new NRT point. */
 +  final Set<String> pendingMergeFiles = Collections.synchronizedSet(new HashSet<String>());
 +
 +  /** Primary gen last time we successfully replicated: */
 +  protected long lastPrimaryGen;
 +
 +  public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory) throws IOException {
 +    super(id, dir, searcherFactory);
 +
++    if (dir instanceof FSDirectory && ((FSDirectory) dir).checkPendingDeletions()) {
++      throw new IllegalArgumentException("Directory " + dir + " still has pending deleted files; cannot initialize IndexWriter");
++    }
++
 +    boolean success = false;
 +
 +    try {
 +      message("top: init replica dir=" + dir);
 +
 +      // Obtain a write lock on this index since we "act like" an IndexWriter, to prevent any other IndexWriter or ReplicaNode from using it:
 +      writeFileLock = dir.obtainLock(IndexWriter.WRITE_LOCK_NAME);
 +      
 +      // nocommit must check for no pending deletes here, like IW does
 +
 +      state = "init";
 +      deleter = new ReplicaFileDeleter(this, dir);
 +    } catch (Throwable t) {
 +      message("exc on init:");
 +      t.printStackTrace(System.out);
 +      throw t;
 +    } finally {
 +      if (success == false) {
 +        IOUtils.closeWhileHandlingException(this);
 +      }
 +    }
 +  }
 +
 +  /** Start up this replica, which possibly requires heavy copying of files from the primary node, if we were down for a long time */
 +  protected synchronized void start(long curPrimaryGen) throws IOException {
 +
 +    if (state.equals("init") == false) {
 +      throw new IllegalStateException("already started");
 +    }
 +
 +    message("top: now start");
 +    try {
 +
 +      // Figure out what state our local index is in now:
 +      String segmentsFileName = SegmentInfos.getLastCommitSegmentsFileName(dir);
 +
 +      // Also look for any pending_segments_N, in case we crashed mid-commit.  We must "inflate" our infos gen to at least this, since
 +      // otherwise we may wind up re-using the pending_segments_N file name on commit, and then our deleter can get angry because it still
 +      // wants to delete this file:
 +      long maxPendingGen = -1;
 +      for(String fileName : dir.listAll()) {
 +        if (fileName.startsWith(IndexFileNames.PENDING_SEGMENTS)) {
 +          long gen = Long.parseLong(fileName.substring(IndexFileNames.PENDING_SEGMENTS.length()+1), Character.MAX_RADIX);
 +          if (gen > maxPendingGen) {
 +            maxPendingGen = gen;
 +          }
 +        }
 +      }
 +
 +      SegmentInfos infos;
 +      if (segmentsFileName == null) {
 +        // No index here yet:
 +        infos = new SegmentInfos();
 +        message("top: init: no segments in index");
 +      } else {
 +        message("top: init: read existing segments commit " + segmentsFileName);
 +        infos = SegmentInfos.readCommit(dir, segmentsFileName);
 +        message("top: init: segments: " + infos.toString() + " version=" + infos.getVersion());
 +        Collection<String> indexFiles = infos.files(false);
 +
 +        lastCommitFiles.add(segmentsFileName);
 +        lastCommitFiles.addAll(indexFiles);
 +
 +        // Always protect the last commit:
 +        deleter.incRef(lastCommitFiles);
 +
 +        lastNRTFiles.addAll(indexFiles);
 +        deleter.incRef(lastNRTFiles);
 +        message("top: commitFiles=" + lastCommitFiles);
 +        message("top: nrtFiles=" + lastNRTFiles);
 +      }
 +
 +      message("top: delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
 +      deleter.deleteUnknownFiles(segmentsFileName);
 +      message("top: done delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
 +
 +      String s = infos.getUserData().get(PRIMARY_GEN_KEY);
 +      long myPrimaryGen;
 +      if (s == null) {
 +        assert infos.size() == 0;
 +        myPrimaryGen = -1;
 +      } else {
 +        myPrimaryGen = Long.parseLong(s);
 +      }
 +      message("top: myPrimaryGen=" + myPrimaryGen);
 +
 +      boolean doCommit;
 +
 +      if (infos.size() > 0 && myPrimaryGen != -1 && myPrimaryGen != curPrimaryGen) {
 +
 +        assert myPrimaryGen < curPrimaryGen;
 +
 +        // Primary changed while we were down.  In this case, we must sync from primary before opening a reader, because it's possible current
 +        // files we have will need to be overwritten with different ones (if index rolled back and "forked"), and we can't overwrite open
 +        // files on Windows:
 +
 +        final long initSyncStartNS = System.nanoTime();
 +
 +        message("top: init: primary changed while we were down myPrimaryGen=" + myPrimaryGen +
 +                " vs curPrimaryGen=" + curPrimaryGen +
 +                "; sync now before mgr init");
 +
 +        // Try until we succeed in copying over the latest NRT point:
 +        CopyJob job = null;
 +
 +        // We may need to overwrite files referenced by our latest commit, either right now on initial sync, or on a later sync.  To make
 +        // sure the index is never even in an "apparently" corrupt state (where an old segments_N references invalid files) we forcefully
 +        // remove the commit now, and refuse to start the replica if this delete fails:
 +        message("top: now delete starting commit point " + segmentsFileName);
 +
 +        // If this throws exc (e.g. due to virus checker), we cannot start this replica:
 +        assert deleter.getRefCount(segmentsFileName) == 1;
 +        deleter.decRef(Collections.singleton(segmentsFileName));
 +        if (deleter.isPending(segmentsFileName)) {
 +          // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else we can cause corruption:
 +          throw new RuntimeException("replica cannot start: existing segments file=" + segmentsFileName + " must be removed in order to start, but the file delete failed");
 +        }
 +        // So we don't later try to decRef it (illegally) again:
 +        boolean didRemove = lastCommitFiles.remove(segmentsFileName);
 +        assert didRemove;
 +
 +        while (true) {
 +          job = newCopyJob("sync on startup replica=" + name() + " myVersion=" + infos.getVersion(),
 +                           null,
 +                           null,
 +                           true,
 +                           null);
 +          job.start();
 +
 +          message("top: init: sync sis.version=" + job.getCopyState().version);
 +
 +          Collection<String> fileNamesToCopy = job.getFileNamesToCopy();
 +
 +          // Force this copy job to finish while we wait, now.  Note that this can be very time consuming!
 +          // NOTE: newNRTPoint detects we are still in init (mgr is null) and does not cancel our copy if a flush happens
 +          try {
 +            job.runBlocking();
 +            job.finish();
 +
 +            // Success!
 +            break;
 +          } catch (IOException ioe) {
 +            job.cancel("startup failed", ioe);
 +            if (ioe.getMessage().contains("checksum mismatch after file copy")) {
 +              // OK-ish
 +              message("top: failed to copy: " + ioe + "; retrying");
 +            } else {
 +              throw ioe;
 +            }
 +          }
 +        }
 +
 +        lastPrimaryGen = job.getCopyState().primaryGen;
 +        byte[] infosBytes = job.getCopyState().infosBytes;
 +
 +        SegmentInfos syncInfos = SegmentInfos.readCommit(dir,
 +                                                         new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", job.getCopyState().infosBytes)),
 +                                                         job.getCopyState().gen);
 +
 +        // Must always commit to a larger generation than what's currently in the index:
 +        syncInfos.updateGeneration(infos);
 +        infos = syncInfos;
 +
 +        assert infos.getVersion() == job.getCopyState().version;
 +        message("  version=" + infos.getVersion() + " segments=" + infos.toString());
 +        message("top: init: incRef nrtFiles=" + job.getFileNames());
 +        deleter.incRef(job.getFileNames());
 +        message("top: init: decRef lastNRTFiles=" + lastNRTFiles);
 +        deleter.decRef(lastNRTFiles);
 +
 +        lastNRTFiles.clear();
 +        lastNRTFiles.addAll(job.getFileNames());
 +
 +        message("top: init: set lastNRTFiles=" + lastNRTFiles);
 +        lastFileMetaData = job.getCopyState().files;
 +        message(String.format(Locale.ROOT, "top: %d: start: done sync: took %.3fs for %s, opened NRT reader version=%d",
 +                              id,
 +                              (System.nanoTime()-initSyncStartNS)/1000000000.0,
 +                              bytesToString(job.getTotalBytesCopied()),
 +                              job.getCopyState().version));
 +
 +        doCommit = true;
 +      } else {
 +        doCommit = false;
 +        lastPrimaryGen = curPrimaryGen;
 +        message("top: same primary as before");
 +      }
 +
 +      if (infos.getGeneration() < maxPendingGen) {
 +        message("top: move infos generation from " + infos.getGeneration() + " to " + maxPendingGen);
 +        infos.setNextWriteGeneration(maxPendingGen);
 +      }
 +
 +      // Notify primary we started, to give it a chance to send any warming merges our way to reduce NRT latency of first sync:
 +      sendNewReplica();
 +
 +      // Finally, we are open for business, since our index now "agrees" with the primary:
 +      mgr = new SegmentInfosSearcherManager(dir, this, infos, searcherFactory);
 +
 +      IndexSearcher searcher = mgr.acquire();
 +      try {
 +        // TODO: this is test specific:
 +        int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
 +        message("top: marker count=" + hitCount + " version=" + ((DirectoryReader) searcher.getIndexReader()).getVersion());
 +      } finally {
 +        mgr.release(searcher);
 +      }
 +
 +      // Must commit after init mgr:
 +      if (doCommit) {
 +        // Very important to commit what we just sync'd over, because we removed the pre-existing commit point above if we had to
 +        // overwrite any files it referenced:
 +        commit();
 +      }
 +
 +      message("top: done start");
 +      state = "idle";
 +    } catch (Throwable t) {
 +      if (t.getMessage().startsWith("replica cannot start") == false) {
 +        message("exc on start:");
 +        t.printStackTrace(System.out);
 +      } else {
 +        dir.close();
 +      }
 +      IOUtils.reThrow(t);
 +    }
 +  }
 +  
 +  final Object commitLock = new Object();
 +
 +  @Override
 +  public void commit() throws IOException {
 +
 +    synchronized(commitLock) {
 +
 +      SegmentInfos infos;
 +      Collection<String> indexFiles;
 +
 +      synchronized (this) {
 +        infos = ((SegmentInfosSearcherManager) mgr).getCurrentInfos();
 +        indexFiles = infos.files(false);
 +        deleter.incRef(indexFiles);
 +      }
 +
 +      message("top: commit primaryGen=" + lastPrimaryGen + " infos=" + infos.toString() + " files=" + indexFiles);
 +
 +      // fsync all index files we are now referencing
 +      dir.sync(indexFiles);
 +
 +      Map<String,String> commitData = new HashMap<>();
 +      commitData.put(PRIMARY_GEN_KEY, Long.toString(lastPrimaryGen));
 +      commitData.put(VERSION_KEY, Long.toString(getCurrentSearchingVersion()));
 +      infos.setUserData(commitData, false);
 +
 +      // write and fsync a new segments_N
 +      infos.commit(dir);
 +
 +      // Notify current infos (which may have changed while we were doing dir.sync above) what generation we are up to; this way future
 +      // commits are guaranteed to go to the next (unwritten) generations:
 +      if (mgr != null) {
 +        ((SegmentInfosSearcherManager) mgr).getCurrentInfos().updateGeneration(infos);
 +      }
 +      String segmentsFileName = infos.getSegmentsFileName();
 +      message("top: commit wrote segments file " + segmentsFileName + " version=" + infos.getVersion() + " sis=" + infos.toString() + " commitData=" + commitData);
 +      deleter.incRef(Collections.singletonList(segmentsFileName));
 +      message("top: commit decRef lastCommitFiles=" + lastCommitFiles);
 +      deleter.decRef(lastCommitFiles);
 +      lastCommitFiles.clear();
 +      lastCommitFiles.addAll(indexFiles);
 +      lastCommitFiles.add(segmentsFileName);
 +      message("top: commit version=" + infos.getVersion() + " files now " + lastCommitFiles);
 +    }
 +  }
 +
 +  void finishNRTCopy(CopyJob job, long startNS) throws IOException {
 +    CopyState copyState = job.getCopyState();
 +    message("top: finishNRTCopy: version=" + copyState.version + (job.getFailed() ? " FAILED" : "") + " job=" + job);
 +
 +    // NOTE: if primary crashed while we were still copying then the job will hit an exc trying to read bytes for the files from the primary node,
 +    // and the job will be marked as failed here:
 +
 +    synchronized (this) {
 +
 +      if ("syncing".equals(state)) {
 +        state = "idle";
 +      }
 +
 +      if (curNRTCopy == job) {
 +        message("top: now clear curNRTCopy; job=" + job);
 +        curNRTCopy = null;
 +      } else {
 +        assert job.getFailed();
 +        message("top: skip clear curNRTCopy: we were cancelled; job=" + job);
 +      }
 +
 +      if (job.getFailed()) {
 +        return;
 +      }
 +
 +      // Does final file renames:
 +      job.finish();
 +
 +      // Turn byte[] back to SegmentInfos:
 +      byte[] infosBytes = copyState.infosBytes;
 +      SegmentInfos infos = SegmentInfos.readCommit(dir,
 +                                                   new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", copyState.infosBytes)),
 +                                                   copyState.gen);
 +      assert infos.getVersion() == copyState.version;
 +
 +      message("  version=" + infos.getVersion() + " segments=" + infos.toString());
 +
 +      // Cutover to new searcher:
 +      if (mgr != null) {
 +        ((SegmentInfosSearcherManager) mgr).setCurrentInfos(infos);
 +      }
 +
 +      // Must first incRef new NRT files, then decRef old ones, to make sure we don't remove an NRT file that's in common to both:
 +      Collection<String> newFiles = copyState.files.keySet();
 +      message("top: incRef newNRTFiles=" + newFiles);
 +      deleter.incRef(newFiles);
 +
 +      // If any of our new files were previously copied merges, we clear them now, so we don't try to later delete a non-existent file:
 +      pendingMergeFiles.removeAll(newFiles);
 +      message("top: after remove from pending merges pendingMergeFiles=" + pendingMergeFiles);
 +
 +      message("top: decRef lastNRTFiles=" + lastNRTFiles);
 +      deleter.decRef(lastNRTFiles);
 +      lastNRTFiles.clear();
 +      lastNRTFiles.addAll(newFiles);
 +      message("top: set lastNRTFiles=" + lastNRTFiles);
 +
 +      // At this point we can remove any completed merge segment files that we still do not reference.  This can happen when a merge
 +      // finishes, copies its files out to us, but is then merged away (or dropped due to 100% deletions) before we ever cutover to it
 +      // in an NRT point:
 +      if (copyState.completedMergeFiles.isEmpty() == false) {
 +        message("now remove-if-not-ref'd completed merge files: " + copyState.completedMergeFiles);
 +        for(String fileName : copyState.completedMergeFiles) {
 +          if (pendingMergeFiles.contains(fileName)) {
 +            pendingMergeFiles.remove(fileName);
 +            deleter.deleteIfNoRef(fileName);
 +          }
 +        }
 +      }
 +
 +      lastFileMetaData = copyState.files;
 +
 +      // It's a good time to delete pending files, since we just refreshed and some previously open files are now closed:
 +      deleter.deletePending();
 +    }
 +
 +    int markerCount;
 +    IndexSearcher s = mgr.acquire();
 +    try {
 +      markerCount = s.count(new TermQuery(new Term("marker", "marker")));
 +    } finally {
 +      mgr.release(s);
 +    }
 +
 +    message(String.format(Locale.ROOT, "top: done sync: took %.3fs for %s, opened NRT reader version=%d markerCount=%d",
 +                          (System.nanoTime()-startNS)/1000000000.0,
 +                          bytesToString(job.getTotalBytesCopied()),
 +                          copyState.version,
 +                          markerCount));
 +  }
 +
 +  /** Start a background copying job, to copy the specified files from the current primary node.  If files is null then the latest copy
 +   *  state should be copied.  If prevJob is not null, then the new copy job is replacing it and should 1) cancel the previous one, and
 +   *  2) optionally salvage e.g. partially copied and, shared with the new copy job, files. */
 +  protected abstract CopyJob newCopyJob(String reason, Map<String,FileMetaData> files, Map<String,FileMetaData> prevFiles,
 +                                        boolean highPriority, CopyJob.OnceDone onceDone) throws IOException;
 +
 +  /** Runs this job async'd */
 +  protected abstract void launch(CopyJob job);
 +
 +  /** Tell primary we (replica) just started, so primary can tell us to warm any already warming merges.  This lets us keep low nrt refresh
 +   *  time for the first nrt sync after we started. */
 +  protected abstract void sendNewReplica() throws IOException;
 +
 +  /** Call this to notify this replica node that a new NRT infos is available on the primary.
 +   *  We kick off a job (runs in the background) to copy files across, and open a new reader once that's done. */
 +  public synchronized CopyJob newNRTPoint(long version) throws IOException {
 +
 +    if (isClosed()) {
 +      throw new AlreadyClosedException("this replica is closed: state=" + state);
 +    }
 +
 +    // Caller should not "publish" us until we have finished .start():
 +    assert mgr != null;
 +
 +    if ("idle".equals(state)) {
 +      state = "syncing";
 +    }
 +
 +    long curVersion = getCurrentSearchingVersion();
 +
 +    message("top: start sync sis.version=" + version);
 +
 +    if (version == curVersion) {
 +      // Caller releases the CopyState:
 +      message("top: new NRT point has same version as current; skipping");
 +      return null;
 +    }
 +
 +    if (version < curVersion) {
 +      // This can happen, if two syncs happen close together, and due to thread scheduling, the incoming older version runs after the newer version
 +      message("top: new NRT point (version=" + version + ") is older than current (version=" + version + "); skipping");
 +      return null;
 +    }
 +
 +    final long startNS = System.nanoTime();
 +
 +    message("top: newNRTPoint");
 +    CopyJob job = null;
 +    try {
 +      job = newCopyJob("NRT point sync version=" + version,
 +                       null,
 +                       lastFileMetaData,
 +                       true,
 +                       new CopyJob.OnceDone() {
 +                         @Override
 +                         public void run(CopyJob job) {
 +                           try {
 +                             finishNRTCopy(job, startNS);
 +                           } catch (IOException ioe) {
 +                             throw new RuntimeException(ioe);
 +                           }
 +                         }
 +                       });
 +    } catch (NodeCommunicationException nce) {
 +      // E.g. primary could crash/close when we are asking it for the copy state:
 +      message("top: ignoring communication exception creating CopyJob: " + nce);
 +      //nce.printStackTrace(System.out);
 +      if (state.equals("syncing")) {
 +        state = "idle";
 +      }
 +      return null;
 +    }
 +
 +    Collection<String> newNRTFiles = job.getFileNames();
 +    long newPrimaryGen = job.getCopyState().primaryGen;
 +    maybeNewPrimary(newPrimaryGen);
 +
 +    message("top: newNRTPoint: job files=" + newNRTFiles);
 +
 +    if (curNRTCopy != null) {
 +      job.transferAndCancel(curNRTCopy);
 +      assert curNRTCopy.getFailed();
 +    }
 +
 +    curNRTCopy = job;
 +
 +    for(String fileName : curNRTCopy.getFileNamesToCopy()) {
 +      assert lastCommitFiles.contains(fileName) == false: "fileName=" + fileName + " is in lastCommitFiles and is being copied?";
 +      synchronized (mergeCopyJobs) {
 +        for (CopyJob mergeJob : mergeCopyJobs) {
 +          if (mergeJob.getFileNames().contains(fileName)) {
 +            // TODO: we could maybe transferAndCancel here?  except CopyJob can't transferAndCancel more than one currently
 +            message("top: now cancel merge copy job=" + mergeJob + ": file " + fileName + " is now being copied via NRT point");
 +            mergeJob.cancel("newNRTPoint is copying over the same file", null);
 +          }
 +        }
 +      }
 +    }
 +
 +    try {
 +      job.start();
 +    } catch (NodeCommunicationException nce) {
 +      // E.g. primary could crash/close when we are asking it for the copy state:
 +      message("top: ignoring exception starting CopyJob: " + nce);
 +      nce.printStackTrace(System.out);
 +      if (state.equals("syncing")) {
 +        state = "idle";
 +      }
 +      return null;
 +    }
 +
 +    // Runs in the background jobs thread, maybe slowly/throttled, and calls finishSync once it's done:
 +    launch(curNRTCopy);
 +    return curNRTCopy;
 +  }
 +
 +  public synchronized boolean isCopying() {
 +    return curNRTCopy != null;
 +  }
 +
 +  @Override
 +  public boolean isClosed() {
 +    return "closed".equals(state) || "closing".equals(state) || "crashing".equals(state) || "crashed".equals(state);
 +  }
 +
 +  @Override
 +  public void close() throws IOException {
 +    message("top: now close");
 +
 +    synchronized (this) {
 +      state = "closing";
 +      if (curNRTCopy != null) {
 +        curNRTCopy.cancel("closing", null);
 +      }
 +    }
 +
 +    synchronized (this) {
 +      message("top: close mgr");
 +      mgr.close();
 +
 +      message("top: decRef lastNRTFiles=" + lastNRTFiles);
 +      deleter.decRef(lastNRTFiles);
 +      lastNRTFiles.clear();
 +
 +      // NOTE: do not decRef these!
 +      lastCommitFiles.clear();
 +
 +      message("top: delete if no ref pendingMergeFiles=" + pendingMergeFiles);
 +      for(String fileName : pendingMergeFiles) {
 +        deleter.deleteIfNoRef(fileName);
 +      }
 +      pendingMergeFiles.clear();
 +    
 +      message("top: close dir");
 +      IOUtils.close(writeFileLock, dir);
 +    }
 +    message("top: done close");
 +    state = "closed";
 +  }
 +
 +  /** Called when the primary changed */
 +  protected synchronized void maybeNewPrimary(long newPrimaryGen) {
 +    if (newPrimaryGen != lastPrimaryGen) {
 +      message("top: now change lastPrimaryGen from " + lastPrimaryGen + " to " + newPrimaryGen + " pendingMergeFiles=" + pendingMergeFiles);
 +      assert newPrimaryGen > lastPrimaryGen: "newPrimaryGen=" + newPrimaryGen + " vs lastPrimaryGen=" + lastPrimaryGen;
 +      lastPrimaryGen = newPrimaryGen;
 +      pendingMergeFiles.clear();
 +    } else {
 +      message("top: keep current lastPrimaryGen=" + lastPrimaryGen);
 +    }
 +  }
 +
 +  protected synchronized CopyJob launchPreCopyMerge(AtomicBoolean finished, long newPrimaryGen, Map<String,FileMetaData> files) throws IOException {
 +
 +    CopyJob job;
 +
 +    maybeNewPrimary(newPrimaryGen);
 +    final long primaryGenStart = lastPrimaryGen;
 +    Set<String> fileNames = files.keySet();
 +    message("now pre-copy warm merge files=" + fileNames + " primaryGen=" + newPrimaryGen);
 +
 +    for(String fileName : fileNames) {
 +      assert pendingMergeFiles.contains(fileName) == false: "file \"" + fileName + "\" is already being warmed!";
 +      assert lastNRTFiles.contains(fileName) == false: "file \"" + fileName + "\" is already NRT visible!";
 +    }
 +
 +    job = newCopyJob("warm merge on " + name() + " filesNames=" + fileNames,
 +                     files, null, false,
 +                     new CopyJob.OnceDone() {
 +
 +                       @Override
 +                       public void run(CopyJob job) throws IOException {
 +                         // Signals that this replica has finished
 +                         mergeCopyJobs.remove(job);
 +                         message("done warming merge " + fileNames + " failed?=" + job.getFailed());
 +                         synchronized(this) {
 +                           if (job.getFailed() == false) {
 +                             if (lastPrimaryGen != primaryGenStart) {
 +                               message("merge pre copy finished but primary has changed; cancelling job files=" + fileNames);
 +                               job.cancel("primary changed during merge copy", null);
 +                             } else {
 +                               boolean abort = false;
 +                               for (String fileName : fileNames) {
 +                                 if (lastNRTFiles.contains(fileName)) {
 +                                   message("abort merge finish: file " + fileName + " is referenced by last NRT point");
 +                                   abort = true;
 +                                 }
 +                                 if (lastCommitFiles.contains(fileName)) {
 +                                   message("abort merge finish: file " + fileName + " is referenced by last commit point");
 +                                   abort = true;
 +                                 }
 +                               }
 +                               if (abort) {
 +                                 // Even though in newNRTPoint we have similar logic, which cancels any merge copy jobs if an NRT point
 +                                 // shows up referencing the files we are warming (because primary got impatient and gave up on us), we also
 +                                 // need it here in case replica is way far behind and fails to even receive the merge pre-copy request
 +                                 // until after the newNRTPoint referenced those files:
 +                                 job.cancel("merged segment was separately copied via NRT point", null);
 +                               } else {
 +                                 job.finish();
 +                                 message("merge pre copy finished files=" + fileNames);
 +                                 for(String fileName : fileNames) {
 +                                   assert pendingMergeFiles.contains(fileName) == false : "file \"" + fileName + "\" is already in pendingMergeFiles";
 +                                   message("add file " + fileName + " to pendingMergeFiles");
 +                                   pendingMergeFiles.add(fileName);
 +                                 }
 +                               }
 +                             }
 +                           } else {
 +                             message("merge copy finished with failure");
 +                           }
 +                         }
 +                         finished.set(true);
 +                       }
 +                     });
 +
 +    job.start();
 +
 +    // When warming a merge we better not already have any of these files copied!
 +    assert job.getFileNamesToCopy().size() == files.size();
 +
 +    mergeCopyJobs.add(job);
 +    launch(job);
 +
 +    return job;
 +  }
 +
 +  public IndexOutput createTempOutput(String prefix, String suffix, IOContext ioContext) throws IOException {
 +    return dir.createTempOutput(prefix, suffix, IOContext.DEFAULT);
 +  }
 +
 +  /** Compares incoming per-file identity (id, checksum, header, footer) versus what we have locally and returns the subset of the incoming
 +   *  files that need copying */
 +  public List<Map.Entry<String,FileMetaData>> getFilesToCopy(Map<String,FileMetaData> files) throws IOException {
 +
 +    boolean doCopyCommitFiles = false;
 +    List<Map.Entry<String,FileMetaData>> toCopy = new ArrayList<>();
 +    for (Map.Entry<String,FileMetaData> ent : files.entrySet()) {
 +      String fileName = ent.getKey();
 +      FileMetaData fileMetaData = ent.getValue();
 +      if (fileIsIdentical(fileName, fileMetaData) == false) {
 +        toCopy.add(ent);
 +      }
 +    }
 +
 +    return toCopy;
 +  }
 +
 +  /** Carefully determine if the file on the primary, identified by its {@code String fileName} along with the {@link FileMetaData}
 +   * "summarizing" its contents, is precisely the same file that we have locally.  If the file does not exist locally, or if its its header
 +   * (inclues the segment id), length, footer (including checksum) differ, then this returns false, else true. */
 +  private boolean fileIsIdentical(String fileName, FileMetaData srcMetaData) throws IOException {
 +
 +    if (deleter.isPending(fileName)) {
 +      // This was a file we had wanted to delete yet a virus checker prevented us, and now we need to overwrite it.
 +      // Such files are in an unknown state, and even if their header and footer and length all
 +      // match, since they may not have been fsync'd by the previous node instance on this directory,
 +      // they could in theory have corruption internally.  So we always force ourselves to copy them here:
 +      if (Node.VERBOSE_FILES) {
 +        message("file " + fileName + ": will copy [we had wanted to delete this file on init, but failed]");
 +      }
 +      return false;
 +    }
 +
 +    FileMetaData destMetaData = readLocalFileMetaData(fileName);
 +    if (destMetaData == null) {
 +      // Something went wrong in reading the file (it's corrupt, truncated, does not exist, etc.):
 +      return false;
 +    }
 +
 +    if (Arrays.equals(destMetaData.header, srcMetaData.header) == false ||
 +        Arrays.equals(destMetaData.footer, srcMetaData.footer) == false) {
 +      // Segment name was reused!  This is rare but possible and otherwise devastating:
 +      if (Node.VERBOSE_FILES) {
 +        message("file " + fileName + ": will copy [header/footer is different]");
 +      }
 +      return false;
 +    } else {
 +      return true;
 +    }
 +  }
 +
 +  private ConcurrentMap<String,Boolean> copying = new ConcurrentHashMap<>();
 +
 +  // Used only to catch bugs, ensuring a given file name is only ever being copied bye one job:
 +  public void startCopyFile(String name) {
 +    if (copying.putIfAbsent(name, Boolean.TRUE) != null) {
 +      throw new IllegalStateException("file " + name + " is being copied in two places!");
 +    }
 +  }
 +
 +  public void finishCopyFile(String name) {
 +    if (copying.remove(name) == null) {
 +      throw new IllegalStateException("file " + name + " was not actually being copied?");
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --cc lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 15e9c8c,0000000..cd98b48
mode 100644,000000..100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@@ -1,876 -1,0 +1,876 @@@
 +package org.apache.lucene.replicator.nrt;
 +
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +import java.io.BufferedReader;
 +import java.io.IOException;
 +import java.io.InputStreamReader;
 +import java.io.UnsupportedEncodingException;
 +import java.nio.file.Path;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.Locale;
 +import java.util.concurrent.atomic.AtomicBoolean;
 +import java.util.concurrent.atomic.AtomicLong;
 +import java.util.regex.Pattern;
 +
 +import org.apache.lucene.document.Document;
 +import org.apache.lucene.util.IOUtils;
 +import org.apache.lucene.util.LineFileDocs;
 +import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 +import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 +import org.apache.lucene.util.LuceneTestCase;
 +import org.apache.lucene.util.TestUtil;
 +
 +import com.carrotsearch.randomizedtesting.SeedUtils;
 +
 +// nocommit make some explicit failure tests
 +
 +// MockRandom's .sd file has no index header/footer:
 +@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
 +@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
 +public class TestNRTReplication extends LuceneTestCase {
 +
 +  /** cwd where we start each child (server) node */
 +  private Path childTempDir;
 +
 +  final AtomicLong nodeStartCounter = new AtomicLong();
 +  private long nextPrimaryGen;
 +  private long lastPrimaryGen;
 +  LineFileDocs docs;
 +
 +  /** Launches a child "server" (separate JVM), which is either primary or replica node */
 +  private NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, long forcePrimaryVersion, boolean willCrash) throws IOException {
 +    List<String> cmd = new ArrayList<>();
 +
 +    cmd.add(System.getProperty("java.home") 
 +        + System.getProperty("file.separator")
 +        + "bin"
 +        + System.getProperty("file.separator")
 +        + "java");
 +    cmd.add("-Xmx512m");
 +
 +    long myPrimaryGen;
 +    if (primaryTCPPort != -1) {
 +      // I am a replica
 +      cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + primaryTCPPort);
 +      myPrimaryGen = lastPrimaryGen;
 +    } else {
 +      myPrimaryGen = nextPrimaryGen++;
 +      lastPrimaryGen = myPrimaryGen;
 +    }
 +    cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
 +    cmd.add("-Dtests.nrtreplication.closeorcrash=false");
 +
 +    cmd.add("-Dtests.nrtreplication.node=true");
 +    cmd.add("-Dtests.nrtreplication.nodeid=" + id);
 +    cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
 +    cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
 +    cmd.add("-Dtests.nrtreplication.checkonclose=true");
 +
 +    if (primaryTCPPort == -1) {
 +      // We are the primary node
 +      cmd.add("-Dtests.nrtreplication.isPrimary=true");
 +      cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
 +    }
 +
 +    // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
 +    // new node:
 +    long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
 +
 +    cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(seed));
 +    cmd.add("-ea");
 +    cmd.add("-cp");
 +    cmd.add(System.getProperty("java.class.path"));
 +    cmd.add("org.junit.runner.JUnitCore");
 +    cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
 +
 +    message("child process command: " + cmd);
 +    ProcessBuilder pb = new ProcessBuilder(cmd);
 +    pb.redirectErrorStream(true);
 +
 +    // Important, so that the scary looking hs_err_<pid>.log appear under our test temp dir:
 +    pb.directory(childTempDir.toFile());
 +
 +    Process p = pb.start();
 +
 +    BufferedReader r;
 +    try {
 +      r = new BufferedReader(new InputStreamReader(p.getInputStream(), IOUtils.UTF_8));
 +    } catch (UnsupportedEncodingException uee) {
 +      throw new RuntimeException(uee);
 +    }
 +
 +    int tcpPort = -1;
 +    long initCommitVersion = -1;
 +    long initInfosVersion = -1;
 +    Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
 +    boolean sawExistingSegmentsFile = false;
 +
 +    while (true) {
 +      String l = r.readLine();
 +      if (l == null) {
 +        message("top: node=" + id + " failed to start");
 +        try {
 +          p.waitFor();
 +        } catch (InterruptedException ie) {
 +          throw new RuntimeException(ie);
 +        }
 +        message("exit value=" + p.exitValue());
 +        message("top: now fail test replica R" + id + " failed to start");
 +        throw new RuntimeException("replica R" + id + " failed to start");
 +      }
 +
 +      if (logTimeStart.matcher(l).matches()) {
 +        // Already a well-formed log output:
 +        System.out.println(l);
 +      } else {
 +        message(l);
 +      }
 +
 +      if (l.startsWith("PORT: ")) {
 +        tcpPort = Integer.parseInt(l.substring(6).trim());
 +      } else if (l.startsWith("COMMIT VERSION: ")) {
 +        initCommitVersion = Integer.parseInt(l.substring(16).trim());
 +      } else if (l.startsWith("INFOS VERSION: ")) {
 +        initInfosVersion = Integer.parseInt(l.substring(15).trim());
 +      } else if (l.contains("will crash after")) {
 +        willCrash = true;
 +      } else if (l.startsWith("NODE STARTED")) {
 +        break;
 +      } else if (l.contains("replica cannot start: existing segments file=")) {
 +        sawExistingSegmentsFile = true;
 +      }
 +    }
 +
 +    final boolean finalWillCrash = willCrash;
 +
 +    // Baby sits the child process, pulling its stdout and printing to our stdout:
 +    AtomicBoolean nodeClosing = new AtomicBoolean();
 +    Thread pumper = ThreadPumper.start(
 +                                       new Runnable() {
 +                                         @Override
 +                                         public void run() {
 +                                           message("now wait for process " + p);
 +                                           try {
 +                                             p.waitFor();
 +                                           } catch (Throwable t) {
 +                                             throw new RuntimeException(t);
 +                                           }
 +
 +                                           message("done wait for process " + p);
 +                                           int exitValue = p.exitValue();
 +                                           message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
 +                                           if (exitValue != 0 && finalWillCrash == false) {
 +                                             // should fail test
 +                                             throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
 +                                           }
 +                                         }
 +                                       }, r, System.out, null, nodeClosing);
 +    pumper.setName("pump" + id);
 +
 +    message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
 +    return new NodeProcess(p, id, tcpPort, pumper, primaryTCPPort == -1, initCommitVersion, initInfosVersion, nodeClosing);
 +  }
 +
 +  @Override
 +  public void setUp() throws Exception {
 +    super.setUp();
 +    Node.globalStartNS = System.nanoTime();
 +    childTempDir = createTempDir("child");
 +    docs = new LineFileDocs(random());
 +  }
 +
 +  @Override
 +  public void tearDown() throws Exception {
 +    super.tearDown();
 +    docs.close();
 +  }
 +
 +  public void testReplicateDeleteAllDocuments() throws Exception {
 +
 +    Path primaryPath = createTempDir("primary");
 +    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 +
 +    Path replicaPath = createTempDir("replica");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 +
 +    // Tell primary current replicas:
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    Connection primaryC = new Connection(primary.tcpPort);
 +    primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +    for(int i=0;i<10;i++) {
 +      Document doc = docs.nextDoc();
 +      primary.addOrUpdateDocument(primaryC, doc, false);
 +    }
 +
 +    // Nothing in replica index yet
 +    assertVersionAndHits(replica, 0, 0);
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to show the change
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Delete all docs from primary
 +    if (random().nextBoolean()) {
 +      // Inefficiently:
 +      for(int id=0;id<10;id++) {
 +        primary.deleteDocument(primaryC, Integer.toString(id));
 +      }
 +    } else {
 +      // Efficiently:
 +      primary.deleteAllDocuments(primaryC);
 +    }
 +
 +    // Replica still shows 10 docs:
 +    assertVersionAndHits(replica, primaryVersion1, 10);
 +    
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion2 = primary.flush(0);
 +    assertTrue(primaryVersion2 > primaryVersion1);
 +
 +    // Wait for replica to show the change
 +    waitForVersionAndHits(replica, primaryVersion2, 0);
 +
 +    // Index 10 docs again:
 +    for(int i=0;i<10;i++) {
 +      Document doc = docs.nextDoc();
 +      primary.addOrUpdateDocument(primaryC, doc, false);
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion3 = primary.flush(0);
 +    assertTrue(primaryVersion3 > primaryVersion2);
 +
 +    // Wait for replica to show the change
 +    waitForVersionAndHits(replica, primaryVersion3, 10);
 +
 +    primaryC.close();
 +
 +    replica.close();
 +    primary.close();
 +  }
 +
 +  public void testReplicateForceMerge() throws Exception {
 +
 +    Path primaryPath = createTempDir("primary");
 +    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 +
 +    Path replicaPath = createTempDir("replica");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    Connection primaryC = new Connection(primary.tcpPort);
 +    primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +    for(int i=0;i<10;i++) {
 +      Document doc = docs.nextDoc();
 +      primary.addOrUpdateDocument(primaryC, doc, false);
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Index 10 more docs into primary:
 +    for(int i=0;i<10;i++) {
 +      Document doc = docs.nextDoc();
 +      primary.addOrUpdateDocument(primaryC, doc, false);
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion2 = primary.flush(0);
 +    assertTrue(primaryVersion2 > primaryVersion1);
 +
 +    primary.forceMerge(primaryC);
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion3 = primary.flush(0);
 +    assertTrue(primaryVersion3 > primaryVersion2);
 +
 +    // Wait for replica to show the change
 +    waitForVersionAndHits(replica, primaryVersion3, 20);
 +
 +    primaryC.close();
 +
 +    replica.close();
 +    primary.close();
 +  }
 +
 +  // Start up, index 10 docs, replicate, but crash and restart the replica without committing it:
 +  public void testReplicaCrashNoCommit() throws Exception {
 +
 +    Path primaryPath = createTempDir("primary");
 +    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 +
 +    Path replicaPath = createTempDir("replica");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Crash replica:
 +    replica.crash();
 +
 +    // Restart replica:
 +    replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 +
 +    // On startup the replica searches the last commit (empty here):
 +    assertVersionAndHits(replica, 0, 0);
 +
 +    // Ask replica to sync:
 +    replica.newNRTPoint(primaryVersion1, primary.tcpPort);
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    replica.close();
 +    primary.close();
 +  }
 +
 +  // Start up, index 10 docs, replicate, commit, crash and restart the replica
 +  public void testReplicaCrashWithCommit() throws Exception {
 +
 +    Path primaryPath = createTempDir("primary");
 +    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 +
 +    Path replicaPath = createTempDir("replica");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Commit and crash replica:
 +    replica.commit();
 +    replica.crash();
 +
 +    // Restart replica:
 +    replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 +
 +    // On startup the replica searches the last commit:
 +    assertVersionAndHits(replica, primaryVersion1, 10);
 +
 +    replica.close();
 +    primary.close();
 +  }
 +
 +  // Start up, index 10 docs, replicate, commit, crash, index more docs, replicate, then restart the replica
 +  public void testIndexingWhileReplicaIsDown() throws Exception {
 +
 +    Path primaryPath = createTempDir("primary");
 +    NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
 +
 +    Path replicaPath = createTempDir("replica");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Commit and crash replica:
 +    replica.commit();
 +    replica.crash();
 +
 +    sendReplicasToPrimary(primary);
 +
 +    // Index 10 more docs, while replica is down
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // And flush:
 +    long primaryVersion2 = primary.flush(0);
 +    assertTrue(primaryVersion2 > primaryVersion1);
 +
 +    // Now restart replica:
 +    replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // On startup the replica still searches its last commit:
 +    assertVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Now ask replica to sync:
 +    replica.newNRTPoint(primaryVersion2, primary.tcpPort);
 +
 +    waitForVersionAndHits(replica, primaryVersion2, 20);
 +
 +    replica.close();
 +    primary.close();
 +  }
 + 
 +  // Crash primary and promote a replica
 +  public void testCrashPrimary1() throws Exception {
 +
 +    Path path1 = createTempDir("1");
 +    NodeProcess primary = startNode(-1, 0, path1, -1, true);
 +
 +    Path path2 = createTempDir("2");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Crash primary:
 +    primary.crash();
 +
 +    // Promote replica:
 +    replica.commit();
 +    replica.close();
 +    
 +    primary = startNode(-1, 1, path2, -1, false);
 +
 +    // Should still see 10 docs:
 +    assertVersionAndHits(primary, primaryVersion1, 10);
 +
 +    primary.close();
 +  }
 +
 +  // Crash primary and then restart it
 +  public void testCrashPrimary2() throws Exception {
 +
 +    Path path1 = createTempDir("1");
 +    NodeProcess primary = startNode(-1, 0, path1, -1, true);
 +
 +    Path path2 = createTempDir("2");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    primary.commit();
 +
 +    // Index 10 docs, but crash before replicating or committing:
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Crash primary:
 +    primary.crash();
 +
 +    // Restart it:
 +    primary = startNode(-1, 0, path1, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 more docs
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    long primaryVersion2 = primary.flush(0);
 +    assertTrue(primaryVersion2 > primaryVersion1);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion2, 20);
 +
 +    primary.close();
 +    replica.close();
 +  }
 +
 +  // Crash primary and then restart it, while a replica node is down, then bring replica node back up and make sure it properly "unforks" itself
 +  public void testCrashPrimary3() throws Exception {
 +
 +    Path path1 = createTempDir("1");
 +    NodeProcess primary = startNode(-1, 0, path1, -1, true);
 +
 +    Path path2 = createTempDir("2");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    replica.commit();
 +
 +    replica.close();
 +    primary.crash();
 +
 +    // At this point replica is "in the future": it has 10 docs committed, but the primary crashed before committing so it has 0 docs
 +
 +    // Restart primary:
 +    primary = startNode(-1, 0, path1, -1, true);
 +
 +    // Index 20 docs into primary:
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<20;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Flush primary, but there are no replicas to sync to:
 +    long primaryVersion2 = primary.flush(0);
 +
 +    // Now restart replica, which on init should detect on a "lost branch" because its 10 docs that were committed came from a different
 +    // primary node:
 +    replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    assertVersionAndHits(replica, primaryVersion2, 20);
 +
 +    primary.close();
 +    replica.close();
 +  }
 +
 +  public void testCrashPrimaryWhileCopying() throws Exception {
 +
 +    Path path1 = createTempDir("1");
 +    NodeProcess primary = startNode(-1, 0, path1, -1, true);
 +
 +    Path path2 = createTempDir("2");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 100 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<100;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes (async) to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    Thread.sleep(TestUtil.nextInt(random(), 1, 30));
 +
 +    // Crash primary, likely/hopefully while replica is still copying
 +    primary.crash();
 +
 +    // Could see either 100 docs (replica finished before crash) or 0 docs:
 +    try (Connection c = new Connection(replica.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
 +      c.flush();
 +      long version = c.in.readVLong();
 +      int hitCount = c.in.readVInt();
 +      if (version == 0) {
 +        assertEquals(0, hitCount);
 +      } else {
 +        assertEquals(primaryVersion1, version);
 +        assertEquals(100, hitCount);
 +      }
 +    }
 +
 +    primary.close();
 +    replica.close();
 +  }
 +
 +  public void testCrashReplica() throws Exception {
 +
 +    Path path1 = createTempDir("1");
 +    NodeProcess primary = startNode(-1, 0, path1, -1, true);
 +
 +    Path path2 = createTempDir("2");
 +    NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Index 10 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Refresh primary, which also pushes to replica:
 +    long primaryVersion1 = primary.flush(0);
 +    assertTrue(primaryVersion1 > 0);
 +
 +    // Wait for replica to sync up:
 +    waitForVersionAndHits(replica, primaryVersion1, 10);
 +
 +    // Crash replica
 +    replica.crash();
 +
 +    sendReplicasToPrimary(primary);
 +
 +    // Lots of new flushes while replica is down:
 +    long primaryVersion2 = 0;
 +    for(int iter=0;iter<10;iter++) {
 +      // Index 10 docs into primary:
 +      try (Connection c = new Connection(primary.tcpPort)) {
 +        c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +        for(int i=0;i<10;i++) {
 +          Document doc = docs.nextDoc();
 +          primary.addOrUpdateDocument(c, doc, false);
 +        }
 +      }
 +      primaryVersion2 = primary.flush(0);
 +    }
 +
 +    // Start up replica again:
 +    replica = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica);
 +
 +    // Now ask replica to sync:
 +    replica.newNRTPoint(primaryVersion2, primary.tcpPort);
 +
 +    // Make sure it sees all docs that were indexed while it was down:
 +    assertVersionAndHits(primary, primaryVersion2, 110);
 +
 +    replica.close();
 +    primary.close();
 +  }
 +
 +  public void testFullClusterCrash() throws Exception {
 +
 +    Path path1 = createTempDir("1");
 +    NodeProcess primary = startNode(-1, 0, path1, -1, true);
 +
 +    Path path2 = createTempDir("2");
 +    NodeProcess replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
 +
 +    Path path3 = createTempDir("3");
 +    NodeProcess replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
 +
 +    sendReplicasToPrimary(primary, replica1, replica2);
 +
 +    // Index 50 docs into primary:
 +    LineFileDocs docs = new LineFileDocs(random());
 +    long primaryVersion1 = 0;
 +    for (int iter=0;iter<5;iter++) {
 +      try (Connection c = new Connection(primary.tcpPort)) {
 +        c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +        for(int i=0;i<10;i++) {
 +          Document doc = docs.nextDoc();
 +          primary.addOrUpdateDocument(c, doc, false);
 +        }
 +      }
 +
 +      // Refresh primary, which also pushes to replicas:
 +      primaryVersion1 = primary.flush(0);
 +      assertTrue(primaryVersion1 > 0);
 +    }
 +
 +    // Wait for replicas to sync up:
 +    waitForVersionAndHits(replica1, primaryVersion1, 50);
 +    waitForVersionAndHits(replica2, primaryVersion1, 50);
 +
 +    primary.commit();
 +    replica1.commit();
 +    replica2.commit();
 +
 +    // Index 10 more docs, but don't sync to replicas:
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
 +      for(int i=0;i<10;i++) {
 +        Document doc = docs.nextDoc();
 +        primary.addOrUpdateDocument(c, doc, false);
 +      }
 +    }
 +
 +    // Full cluster crash
 +    primary.crash();
 +    replica1.crash();
 +    replica2.crash();
 +
 +    // Full cluster restart
 +    primary = startNode(-1, 0, path1, -1, true);
 +    replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
 +    replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
 +
 +    // Only 50 because we didn't commit primary before the crash:
 +    
 +    // It's -1 because it's unpredictable how IW changes segments version on init:
 +    assertVersionAndHits(primary, -1, 50);
-     assertVersionAndHits(replica1, primaryVersion1, 50);
-     assertVersionAndHits(replica2, primaryVersion1, 50);
++    assertVersionAndHits(replica1, primary.initInfosVersion, 50);
++    assertVersionAndHits(replica2, primary.initInfosVersion, 50);
 +
 +    primary.close();
 +    replica1.close();
 +    replica2.close();
 +  }
 +
 +  /** Tell primary current replicas. */
 +  private void sendReplicasToPrimary(NodeProcess primary, NodeProcess... replicas) throws IOException {
 +    try (Connection c = new Connection(primary.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
 +      c.out.writeVInt(replicas.length);
 +      for(int id=0;id<replicas.length;id++) {
 +        NodeProcess replica = replicas[id];
 +        c.out.writeVInt(replica.id);
 +        c.out.writeVInt(replica.tcpPort);
 +      }
 +      c.flush();
 +      c.in.readByte();
 +    }
 +  }
 +
 +  /** Verifies this node is currently searching the specified version with the specified total hit count, or that it eventually does when
 +   *  keepTrying is true. */
 +  private void assertVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
 +    try (Connection c = new Connection(node.tcpPort)) {
 +      c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
 +      c.flush();
 +      long version = c.in.readVLong();
 +      int hitCount = c.in.readVInt();
 +      if (expectedVersion != -1) {
-         assertEquals("hitCount=" + hitCount, expectedVersion, version);
++        assertEquals("wrong searcher version, with hitCount=" + hitCount, expectedVersion, version);
 +      }
 +      assertEquals(expectedHitCount, hitCount);
 +    }
 +  }
 +
 +  private void waitForVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
 +    try (Connection c = new Connection(node.tcpPort)) {
 +      while (true) {
 +        c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
 +        c.flush();
 +        long version = c.in.readVLong();
 +        int hitCount = c.in.readVInt();
 +
 +        if (version == expectedVersion) {
 +          assertEquals(expectedHitCount, hitCount);
 +          break;
 +        }
 +
 +        assertTrue(version < expectedVersion);
 +        Thread.sleep(10);
 +      }
 +    }
 +  }
 +
 +  static void message(String message) {
 +    long now = System.nanoTime();
 +    System.out.println(String.format(Locale.ROOT,
 +                                     "%5.3fs       :     parent [%11s] %s",
 +                                     (now-Node.globalStartNS)/1000000000.,
 +                                     Thread.currentThread().getName(),
 +                                     message));
 +  }
 +}


[06/15] lucene-solr git commit: fix more nocommits; add separate test that deleteAll can replicate

Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
new file mode 100644
index 0000000..271c5d2
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
@@ -0,0 +1,1160 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
+/*
+  TODO
+    - fangs
+      - sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
+    - why do we do the "rename temp to actual" all at the end...?  what really does that buy us?
+    - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
+    - test should not print scary exceptions and then succeed!
+    - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
+    - are the pre-copied-completed-merged files not being cleared in primary?
+      - hmm the logic isn't right today?  a replica may skip pulling a given copy state, that recorded the finished merged segments?
+    - beast & fix bugs
+    - graceful cluster restart
+    - better translog integration
+    - get "graceful primary shutdown" working
+    - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion
+    - clean up how version is persisted in commit data
+    - why am i not using hashes here?  how does ES use them?
+    - get all other "single shard" functions working too: this cluster should "act like" a single shard
+      - SLM
+      - controlled nrt reopen thread / returning long gen on write
+      - live field values
+      - add indexes
+    - make cluster level APIs to search, index, that deal w/ primary failover, etc.
+    - must prune xlog
+      - refuse to start primary unless we have quorum
+    - later
+      - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback?
+      - back pressure on indexing if replicas can't keep up?
+      - get xlog working on top?  needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit
+        quorum
+        - maybe fix IW to return "gen" or "seq id" or "segment name" or something?
+      - replica can copy files from other replicas too / use multicast / rsync / something
+      - each replica could also pre-open a SegmentReader after pre-copy when warming a merge
+      - we can pre-copy newly flushed files too, for cases where reopen rate is low vs IW's flushing because RAM buffer is full
+      - opto: pre-copy files as they are written; if they will become CFS, we can build CFS on the replica?
+      - what about multiple commit points?
+      - fix primary to init directly from an open replica, instead of having to commit/close the replica first
+*/
+
+// Tricky cases:
+//   - we are pre-copying a merge, then replica starts up part way through, so it misses that pre-copy and must do it on next nrt point
+//   - a down replica starts up, but it's "from the future" vs the current primary, and must copy over file names with different contents
+//     but referenced by its latest commit point, so it must fully remove that commit ... which is a hazardous window
+//   - replica comes up just as the primary is crashing / moving
+//   - electing a new primary when a replica is just finishing its nrt sync: we need to wait for it so we are sure to get the "most up to
+//     date" replica
+//   - replica comes up after merged segment finished so it doesn't copy over the merged segment "promptly" (i.e. only sees it on NRT refresh)
+
+/**
+ * Test case showing how to implement NRT replication.  This test spawns a sub-process per-node, running TestNRTReplicationChild.
+ *
+ * One node is primary, and segments are periodically flushed there, then concurrently the N replica nodes copy the new files over and open new readers, while
+ * primary also opens a new reader.
+ *
+ * Nodes randomly crash and are restarted.  If the primary crashes, a replica is promoted.
+ *
+ * Merges are currently first finished on the primary and then pre-copied out to replicas with a merged segment warmer so they don't block
+ * ongoing NRT reopens.  Probably replicas could do their own merging instead, but this is more complex and may not be better overall
+ * (merging takes a lot of IO resources).
+ *
+ * Slow network is simulated with a RateLimiter.
+ */
+
+// MockRandom's .sd file has no index header/footer:
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class TestStressNRTReplication extends LuceneTestCase {
+
+  // Test evilness controls:
+
+  /** Randomly crash the current primary (losing data!) and promote the "next best" replica. */
+  static final boolean DO_CRASH_PRIMARY = true;
+
+  /** Randomly crash (JVM core dumps) a replica; it will later randomly be restarted and sync itself. */
+  static final boolean DO_CRASH_REPLICA = true;
+
+  /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
+  static final boolean DO_CLOSE_REPLICA = true;
+
+  /** If false, all child + parent output is interleaved into single stdout/err */
+  static final boolean SEPARATE_CHILD_OUTPUT = false;
+
+  // nocommit DO_CLOSE_PRIMARY?
+
+  /** Randomly crash whole cluster and then restart it */
+  static final boolean DO_FULL_CLUSTER_CRASH = true;
+
+  /** True if we randomly flip a bit while copying files out */
+  static final boolean DO_BIT_FLIPS_DURING_COPY = true;
+
+  /** Set to a non-null value to force exactly that many nodes; else, it's random. */
+  static final Integer NUM_NODES = null;
+
+  static final boolean DO_RANDOM_XLOG_REPLAY = false;
+
+  final AtomicBoolean failed = new AtomicBoolean();
+
+  final AtomicBoolean stop = new AtomicBoolean();
+
+  /** cwd where we start each child (server) node */
+  private Path childTempDir;
+
+  long primaryGen;
+
+  volatile long lastPrimaryVersion;
+
+  volatile NodeProcess primary;
+  volatile NodeProcess[] nodes;
+  volatile long[] nodeTimeStamps;
+  volatile boolean[] starting;
+  
+  Path[] indexPaths;
+
+  Path transLogPath;
+  SimpleTransLog transLog;
+  final AtomicInteger markerUpto = new AtomicInteger();
+
+  /** Maps searcher version to how many hits the query body:the matched. */
+  final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
+
+  /** Maps searcher version to how many marker documents matched.  This should only ever grow (we never delete marker documents). */
+  final Map<Long,Integer> versionToMarker = new ConcurrentHashMap<>();
+
+  /** Maps searcher version to xlog location when refresh of this version started. */
+  final Map<Long,Long> versionToTransLogLocation = new ConcurrentHashMap<>();
+
+  final AtomicLong nodeStartCounter = new AtomicLong();
+
+  final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
+
+  public void test() throws Exception {
+
+    Node.globalStartNS = System.nanoTime();
+
+    message("change thread name from " + Thread.currentThread().getName());
+    Thread.currentThread().setName("main");
+
+    childTempDir = createTempDir("child");
+
+    // We are parent process:
+
+    // Silly bootstrapping:
+    versionToTransLogLocation.put(0L, 0L);
+    versionToTransLogLocation.put(1L, 0L);
+
+    int numNodes;
+
+    if (NUM_NODES == null) {
+      numNodes = TestUtil.nextInt(random(), 2, 10);
+    } else {
+      numNodes = NUM_NODES.intValue();
+    }
+
+    System.out.println("TEST: using " + numNodes + " nodes");
+
+    transLogPath = createTempDir("NRTReplication").resolve("translog");
+    transLog = new SimpleTransLog(transLogPath);
+
+    //state.rateLimiters = new RateLimiter[numNodes];
+    indexPaths = new Path[numNodes];
+    nodes = new NodeProcess[numNodes];
+    nodeTimeStamps = new long[numNodes];
+    Arrays.fill(nodeTimeStamps, Node.globalStartNS);
+    starting = new boolean[numNodes];
+    
+    for(int i=0;i<numNodes;i++) {
+      indexPaths[i] = createTempDir("index" + i);
+    }
+
+    Thread[] indexers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+    System.out.println("TEST: launch " + indexers.length + " indexer threads");
+    for(int i=0;i<indexers.length;i++) {
+      indexers[i] = new IndexThread();
+      indexers[i].setName("indexer" + i);
+      indexers[i].setDaemon(true);
+      indexers[i].start();
+    }
+
+    Thread[] searchers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+    System.out.println("TEST: launch " + searchers.length + " searcher threads");
+    for(int i=0;i<searchers.length;i++) {
+      searchers[i] = new SearchThread();
+      searchers[i].setName("searcher" + i);
+      searchers[i].setDaemon(true);
+      searchers[i].start();
+    }
+
+    Thread restarter = new RestartThread();
+    restarter.setName("restarter");
+    restarter.setDaemon(true);
+    restarter.start();
+
+    int runTimeSec;
+    if (TEST_NIGHTLY) {
+      runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 120, 240);
+    } else {
+      runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 45, 120);
+    }
+
+    System.out.println("TEST: will run for " + runTimeSec + " sec");
+
+    long endTime = System.nanoTime() + runTimeSec*1000000000L;
+
+    sendReplicasToPrimary();
+
+    while (failed.get() == false && System.nanoTime() < endTime) {
+
+      // Wait a bit:
+      Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
+      if (primary != null && random().nextBoolean()) {
+        message("top: now flush primary");
+        NodeProcess curPrimary = primary;
+        if (curPrimary != null) {
+
+          // Save these before we start flush:
+          long nextTransLogLoc = transLog.getNextLocation();
+          int markerUptoSav = markerUpto.get();
+
+          long result;
+          try {
+            result = primary.flush();
+          } catch (Throwable t) {
+            message("top: flush failed; skipping: " + t.getMessage());
+            result = -1;
+          }
+          if (result > 0) {
+            // There were changes
+            lastPrimaryVersion = result;
+            addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
+            addVersionMarker(lastPrimaryVersion, markerUptoSav);
+          }
+        }
+      }
+
+      StringBuilder sb = new StringBuilder();
+      int liveCount = 0;
+      for(int i=0;i<nodes.length;i++) {
+        NodeProcess node = nodes[i];
+        if (node != null) {
+          if (sb.length() != 0) {
+            sb.append(" ");
+          }
+          liveCount++;
+          if (node.isPrimary) {
+            sb.append('P');
+          } else {
+            sb.append('R');
+          }
+          sb.append(i);
+        }
+      }
+
+      message("PG=" + (primary == null ? "X" : primaryGen) + " " + liveCount + " (of " + nodes.length + ") nodes running: " + sb);
+
+      // Commit a random node, primary or replica
+
+      {
+        NodeProcess node = nodes[random().nextInt(nodes.length)];
+        if (node != null) {
+          // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
+          // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
+          message("top: now commit node=" + node);
+          node.commitAsync();
+        }
+      }
+    }
+
+    message("TEST: top: test done, now close");
+    stop.set(true);
+    for(Thread thread : indexers) {
+      thread.join();
+    }
+    for(Thread thread : searchers) {
+      thread.join();
+    }
+    restarter.join();
+
+    // Close replicas before primary so we cancel any in-progres replications:
+    System.out.println("TEST: top: now close replicas");
+    List<Closeable> toClose = new ArrayList<>();
+    for(NodeProcess node : nodes) {
+      if (node != primary && node != null) {
+        toClose.add(node);
+      }
+    }
+    IOUtils.close(toClose);
+    IOUtils.close(primary);
+    IOUtils.close(transLog);
+
+    if (failed.get() == false) {
+      message("TEST: top: now checkIndex");    
+      for(Path path : indexPaths) {
+        message("TEST: check " + path);
+        MockDirectoryWrapper dir = newMockFSDirectory(path);
+        // Just too slow otherwise
+        dir.setCrossCheckTermVectorsOnClose(false);
+        dir.close();
+      }
+    } else {
+      message("TEST: failed; skip checkIndex");
+    }
+  }
+
+  private boolean anyNodesStarting() {
+    for(int id=0;id<nodes.length;id++) {
+      if (starting[id]) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /** Picks a replica and promotes it as new primary. */
+  private void promoteReplica() throws IOException {
+    message("top: primary crashed; now pick replica to promote");
+    long maxSearchingVersion = -1;
+    NodeProcess replicaToPromote = null;
+
+    // We must promote the most current replica, because otherwise file name reuse can cause a replication to fail when it needs to copy
+    // over a file currently held open for searching.  This also minimizes recovery work since the most current replica means less xlog
+    // replay to catch up:
+    for (NodeProcess node : nodes) {
+      if (node != null) {
+        message("ask " + node + " for its current searching version");
+        long searchingVersion = node.getSearchingVersion();
+        message(node + " has searchingVersion=" + searchingVersion);
+        if (searchingVersion > maxSearchingVersion) {
+          maxSearchingVersion = searchingVersion;
+          replicaToPromote = node;
+        }
+      }
+    }
+
+    if (replicaToPromote == null) {
+      message("top: no replicas running; skipping primary promotion");
+      return;
+    }
+
+    message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
+    if (replicaToPromote.commit() == false) {
+      message("top: commit failed; skipping primary promotion");
+      return;
+    }
+
+    message("top: now shutdown " + replicaToPromote);
+    if (replicaToPromote.shutdown() == false) {
+      message("top: shutdown failed for R" + replicaToPromote.id + "; skipping primary promotion");
+      return;
+    }
+
+    int id = replicaToPromote.id;
+    message("top: now startPrimary " + replicaToPromote);
+    startPrimary(replicaToPromote.id);
+  }
+
+  void startPrimary(int id) throws IOException {
+    message(id + ": top: startPrimary lastPrimaryVersion=" + lastPrimaryVersion);
+    assert nodes[id] == null;
+
+    // Force version of new primary to advance beyond where old primary was, so we never re-use versions.  It may have
+    // already advanced beyond newVersion, e.g. if it flushed new segments while during xlog replay:
+
+    // First start node as primary (it opens an IndexWriter) but do not publish it for searching until we replay xlog:
+    NodeProcess newPrimary = startNode(id, indexPaths[id], true, lastPrimaryVersion+1);
+    if (newPrimary == null) {
+      message("top: newPrimary failed to start; abort");
+      return;
+    }
+
+    // Get xlog location that this node was guaranteed to already have indexed through; this may replay some ops already indexed but it's OK
+    // because the ops are idempotent: we updateDocument (by docid) on replay even for original addDocument:
+    Long startTransLogLoc;
+    Integer markerCount;
+    if (newPrimary.initCommitVersion == 0) {
+      startTransLogLoc = 0L;
+      markerCount = 0;
+    } else {
+      startTransLogLoc = versionToTransLogLocation.get(newPrimary.initCommitVersion);
+      markerCount = versionToMarker.get(newPrimary.initCommitVersion);
+    }
+    assert startTransLogLoc != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToTransLogLocation: keys=" + versionToTransLogLocation.keySet();
+    assert markerCount != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToMarker: keys=" + versionToMarker.keySet();
+
+    // When the primary starts, the userData in its latest commit point tells us which version it had indexed up to, so we know where to
+    // replay from in the xlog.  However, we forcefuly advance the version, and then IW on init (or maybe getReader) also adds 1 to it.
+    // Since we publish the primary in this state (before xlog replay is done), a replica can start up at this point and pull this version,
+    // and possibly later be chosen as a primary, causing problems if the version is known recorded in the translog map.  So we record it
+    // here:
+
+    addTransLogLoc(newPrimary.initInfosVersion, startTransLogLoc);
+    addVersionMarker(newPrimary.initInfosVersion, markerCount);
+
+    assert newPrimary.initInfosVersion >= lastPrimaryVersion;
+    message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
+    lastPrimaryVersion = newPrimary.initInfosVersion;
+
+    // Publish new primary, before replaying xlog.  This means other indexing ops can come in at the same time as we catch up indexing
+    // previous ops.  Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
+    // translog) concurrently with new incoming ops.
+    nodes[id] = newPrimary;
+    primary = newPrimary;
+
+    sendReplicasToPrimary();
+
+    long nextTransLogLoc = transLog.getNextLocation();
+    int nextMarkerUpto = markerUpto.get();
+    message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
+    try {
+      transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
+    } catch (IOException ioe) {
+      message("top: replay xlog failed; abort");
+      return;
+    }
+    message("top: done replay trans log");
+  }
+
+  /** Launches a child "server" (separate JVM), which is either primary or replica node */
+  NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
+    nodeTimeStamps[id] = System.nanoTime();
+    List<String> cmd = new ArrayList<>();
+
+    NodeProcess curPrimary = primary;
+
+    cmd.add(System.getProperty("java.home") 
+        + System.getProperty("file.separator")
+        + "bin"
+        + System.getProperty("file.separator")
+        + "java");
+    cmd.add("-Xmx512m");
+
+    if (curPrimary != null) {
+      cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + curPrimary.tcpPort);
+    } else if (isPrimary == false) {
+      // We cannot start a replica when there is no primary:
+      return null;
+    }
+
+    cmd.add("-Dtests.nrtreplication.node=true");
+    cmd.add("-Dtests.nrtreplication.nodeid=" + id);
+    cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
+    cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
+    if (isPrimary) {
+      cmd.add("-Dtests.nrtreplication.isPrimary=true");
+      cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
+      if (DO_CRASH_PRIMARY) {
+        cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
+      }
+    } else {
+      if (DO_CRASH_REPLICA) {
+        cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
+      }
+      if (DO_CLOSE_REPLICA) {
+        cmd.add("-Dtests.nrtreplication.doRandomClose=true");
+      }
+    }
+
+    if (DO_BIT_FLIPS_DURING_COPY) {
+      cmd.add("-Dtests.nrtreplication.doFlipBitsDuringCopy=true");
+    }
+
+    long myPrimaryGen = primaryGen;
+    cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
+
+    // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
+    // new node:
+    long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
+
+    cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(seed));
+    cmd.add("-ea");
+    cmd.add("-cp");
+    cmd.add(System.getProperty("java.class.path"));
+    cmd.add("org.junit.runner.JUnitCore");
+    cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
+
+    Writer childLog;
+
+    if (SEPARATE_CHILD_OUTPUT) {
+      Path childOut = childTempDir.resolve(id + ".log");
+      message("logging to " + childOut);
+      childLog = Files.newBufferedWriter(childOut, StandardCharsets.UTF_8, StandardOpenOption.APPEND, StandardOpenOption.CREATE);
+      childLog.write("\n\nSTART NEW CHILD:\n");
+    } else {
+      childLog = null;
+    }
+
+    message("child process command: " + cmd);
+    ProcessBuilder pb = new ProcessBuilder(cmd);
+    pb.redirectErrorStream(true);
+
+    // Important, so that the scary looking hs_err_<pid>.log appear under our test temp dir:
+    pb.directory(childTempDir.toFile());
+
+    Process p = pb.start();
+
+    BufferedReader r;
+    try {
+      r = new BufferedReader(new InputStreamReader(p.getInputStream(), IOUtils.UTF_8));
+    } catch (UnsupportedEncodingException uee) {
+      throw new RuntimeException(uee);
+    }
+
+    int tcpPort = -1;
+    long initCommitVersion = -1;
+    long initInfosVersion = -1;
+    Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+    boolean willCrash = false;
+    boolean sawExistingSegmentsFile = false;
+
+    while (true) {
+      String l = r.readLine();
+      if (l == null) {
+        message("top: node=" + id + " failed to start");
+        try {
+          p.waitFor();
+        } catch (InterruptedException ie) {
+          throw new RuntimeException(ie);
+        }
+        message("exit value=" + p.exitValue());
+
+        // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
+        if (isPrimary == false) {
+          if (sawExistingSegmentsFile) {
+            // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
+            // and retry again later:
+            message("failed to remove segments_N; skipping");
+            return null;
+          }
+          for(int i=0;i<10;i++) {
+            if (primaryGen != myPrimaryGen || primary == null) {
+              // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
+              message("primary crashed/closed while replica R" + id + " tried to start; skipping");
+              return null;
+            } else {
+              try {
+                Thread.sleep(10);
+              } catch (InterruptedException ie) {
+                throw new ThreadInterruptedException(ie);
+              }
+            }
+          }
+        }
+
+        // Should fail the test:
+        message("top: now fail test replica R" + id + " failed to start");
+        failed.set(true);
+        throw new RuntimeException("replica R" + id + " failed to start");
+      }
+
+      if (childLog != null) {
+        childLog.write(l);
+        childLog.write("\n");
+        childLog.flush();
+      } else if (logTimeStart.matcher(l).matches()) {
+        // Already a well-formed log output:
+        System.out.println(l);
+      } else {
+        message(l);
+      }
+
+      if (l.startsWith("PORT: ")) {
+        tcpPort = Integer.parseInt(l.substring(6).trim());
+      } else if (l.startsWith("COMMIT VERSION: ")) {
+        initCommitVersion = Integer.parseInt(l.substring(16).trim());
+      } else if (l.startsWith("INFOS VERSION: ")) {
+        initInfosVersion = Integer.parseInt(l.substring(15).trim());
+      } else if (l.contains("will crash after")) {
+        willCrash = true;
+      } else if (l.startsWith("NODE STARTED")) {
+        break;
+      } else if (l.contains("replica cannot start: existing segments file=")) {
+        sawExistingSegmentsFile = true;
+      }
+    }
+
+    final boolean finalWillCrash = willCrash;
+
+    // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
+    Thread pumper = ThreadPumper.start(
+                                       new Runnable() {
+                                         @Override
+                                         public void run() {
+                                           message("now wait for process " + p);
+                                           try {
+                                             p.waitFor();
+                                           } catch (Throwable t) {
+                                             throw new RuntimeException(t);
+                                           }
+
+                                           message("done wait for process " + p);
+                                           int exitValue = p.exitValue();
+                                           message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
+                                           if (childLog != null) {
+                                             try {
+                                               childLog.write("process done; exitValue=" + exitValue + "\n");
+                                               childLog.close();
+                                             } catch (IOException ioe) {
+                                               throw new RuntimeException(ioe);
+                                             }
+                                           }
+                                           if (exitValue != 0 && finalWillCrash == false && crashingNodes.remove(id) == false) {
+                                             // should fail test
+                                             failed.set(true);
+                                             if (childLog != null) {
+                                               throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue + "; see " + childLog + " for details");
+                                             } else {
+                                               throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
+                                             }
+                                           }
+                                           nodeClosed(id);
+                                         }
+                                       }, r, System.out, childLog);
+    pumper.setName("pump" + id);
+
+    message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
+    return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+  }
+
+  private void nodeClosed(int id) {
+    NodeProcess oldNode = nodes[id];
+    if (primary != null && oldNode == primary) {
+      message("top: " + primary + ": primary process finished");
+      primary = null;
+      primaryGen++;
+    } else {
+      message("top: " + oldNode + ": replica process finished");
+    }
+    if (oldNode != null) {
+      oldNode.isOpen = false;
+    }
+    nodes[id] = null;
+    nodeTimeStamps[id] = System.nanoTime();
+
+    sendReplicasToPrimary();
+  }
+
+  /** Sends currently alive replicas to primary, which uses this to know who to notify when it does a refresh */
+  private void sendReplicasToPrimary() {
+    NodeProcess curPrimary = primary;
+    if (curPrimary != null) {
+      List<NodeProcess> replicas = new ArrayList<>();
+      for (NodeProcess node : nodes) {
+        if (node != null && node.isPrimary == false) {
+          replicas.add(node);
+        }
+      }
+
+      message("top: send " + replicas.size() + " replicas to primary");
+
+      try (Connection c = new Connection(curPrimary.tcpPort)) {
+        c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+        c.out.writeVInt(replicas.size());        
+        for(NodeProcess replica : replicas) {
+          c.out.writeVInt(replica.id);
+          c.out.writeVInt(replica.tcpPort);
+        }
+        c.flush();
+        c.in.readByte();
+      } catch (Throwable t) {
+        message("top: ignore exc sending replicas to primary: " + t);
+      }
+    }
+  }
+
+  void addVersionMarker(long version, int count) {
+    //System.out.println("ADD VERSION MARKER version=" + version + " count=" + count);
+    if (versionToMarker.containsKey(version)) {
+      int curCount = versionToMarker.get(version);
+      if (curCount != count) {
+        message("top: wrong marker count version=" + version + " count=" + count + " curCount=" + curCount);
+        throw new IllegalStateException("version=" + version + " count=" + count + " curCount=" + curCount);
+      }
+    } else {
+      message("top: record marker count: version=" + version + " count=" + count);
+      versionToMarker.put(version, count);
+    }
+  }
+
+  void addTransLogLoc(long version, long loc) {
+    message("top: record transLogLoc: version=" + version + " loc=" + loc);
+    versionToTransLogLocation.put(version, loc);
+  }
+
+  // Periodically wakes up and starts up any down nodes:
+  private class RestartThread extends Thread {
+    @Override
+    public void run() {
+
+      List<Thread> startupThreads = Collections.synchronizedList(new ArrayList<>());
+
+      try {
+        while (stop.get() == false) {
+          Thread.sleep(TestUtil.nextInt(random(), 50, 500));
+          message("top: restarter cycle");
+
+          // Randomly crash full cluster:
+          if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
+            message("top: full cluster crash");
+            for(int i=0;i<nodes.length;i++) {
+              if (starting[i]) {
+                message("N" + i + ": top: wait for startup so we can crash...");
+                while (starting[i]) {
+                  Thread.sleep(10);
+                }
+                message("N" + i + ": top: done wait for startup");
+              }
+              NodeProcess node = nodes[i];
+              if (node != null) {
+                crashingNodes.add(i);
+                message("top: N" + node.id + ": top: now crash node");
+                node.crash();
+                message("top: N" + node.id + ": top: done crash node");
+              }
+            }
+          }
+
+          List<Integer> downNodes = new ArrayList<>();
+          StringBuilder b = new StringBuilder();
+          long nowNS = System.nanoTime();
+          for(int i=0;i<nodes.length;i++) {
+            b.append(' ');
+            double sec = (nowNS - nodeTimeStamps[i])/1000000000.0;
+            String prefix;
+            if (nodes[i] == null) {
+              downNodes.add(i);
+              if (starting[i]) {
+                prefix = "s";
+              } else {
+                prefix = "x";
+              }
+            } else {
+              prefix = "";
+            }
+            if (primary != null && nodes[i] == primary) {
+              prefix += "p";
+            }
+            b.append(String.format(Locale.ROOT, "%s%d(%.1fs)", prefix, i, sec));
+          }
+          message("node status" + b.toString());
+          message("downNodes=" + downNodes);
+
+          // If primary is down, promote a replica:
+          if (primary == null) {
+            if (anyNodesStarting()) {
+              message("top: skip promote replica: nodes are still starting");
+              continue;
+            }
+            promoteReplica();
+          }
+
+          // Randomly start up a down a replica:
+
+          // Stop or start a replica
+          if (downNodes.isEmpty() == false) {
+            int idx = downNodes.get(random().nextInt(downNodes.size()));
+            if (starting[idx] == false) {
+              if (primary == null) {
+                if (downNodes.size() == nodes.length) {
+                  // Cold start: entire cluster is down, start this node up as the new primary
+                  message("N" + idx + ": top: cold start as primary");
+                  startPrimary(idx);
+                }
+              } else if (random().nextDouble() < ((double) downNodes.size())/nodes.length) {
+                // Start up replica:
+                starting[idx] = true;
+                message("N" + idx + ": top: start up: launch thread");
+                Thread t = new Thread() {
+                    @Override
+                    public void run() {
+                      try {
+                        message("N" + idx + ": top: start up thread");
+                        nodes[idx] = startNode(idx, indexPaths[idx], false, -1);
+                        sendReplicasToPrimary();
+                      } catch (Throwable t) {
+                        failed.set(true);
+                        stop.set(true);
+                        throw new RuntimeException(t);
+                      } finally {
+                        starting[idx] = false;
+                        startupThreads.remove(Thread.currentThread());
+                      }
+                    }
+                  };
+                t.setName("start R" + idx);
+                t.start();
+                startupThreads.add(t);
+              }
+            } else {
+              message("node " + idx + " still starting");
+            }
+          }
+        }
+
+        System.out.println("Restarter: now stop: join " + startupThreads.size() + " startup threads");
+
+        while (startupThreads.size() > 0) {
+          Thread.sleep(10);
+        }
+
+      } catch (Throwable t) {
+        failed.set(true);
+        stop.set(true);
+        throw new RuntimeException(t);
+      }
+    }
+  }
+
+  /** Randomly picks a node and runs a search against it */
+  private class SearchThread extends Thread {
+
+    @Override
+    public void run() {
+      // Maps version to number of hits for silly 'the' TermQuery:
+      Query theQuery = new TermQuery(new Term("body", "the"));
+
+      // Persists connections
+      Map<Integer,Connection> connections = new HashMap<>();
+
+      while (stop.get() == false) {
+        NodeProcess node = nodes[random().nextInt(nodes.length)];
+        if (node == null || node.isOpen == false) {
+          continue;
+        }
+
+        if (node.lock.tryLock() == false) {
+          // Node is in the process of closing or crashing or something
+          continue;
+        }
+
+        try {
+
+          Thread.currentThread().setName("Searcher node=" + node);
+
+          //System.out.println("S: cycle; conns=" + connections);
+
+          Connection c = connections.get(node.id);
+
+          long version;
+          try {
+            if (c == null) {
+              //System.out.println("S: new connection " + node.id + " " + Thread.currentThread().getName());
+              c = new Connection(node.tcpPort);
+              connections.put(node.id, c);
+            } else {
+              //System.out.println("S: reuse connection " + node.id + " " + Thread.currentThread().getName());
+            }
+
+            c.out.writeByte(SimplePrimaryNode.CMD_SEARCH);
+            c.flush();
+
+            while (c.sockIn.available() == 0) {
+              if (stop.get()) {
+                break;
+              }
+              if (node.isOpen == false) {
+                throw new IOException("node closed");
+              }
+              Thread.sleep(1);
+            }
+            version = c.in.readVLong();
+
+            while (c.sockIn.available() == 0) {
+              if (stop.get()) {
+                break;
+              }
+              if (node.isOpen == false) {
+                throw new IOException("node closed");
+              }
+              Thread.sleep(1);
+            }
+            int hitCount = c.in.readVInt();
+
+            Integer oldHitCount = hitCounts.get(version);
+
+            // TODO: we never prune this map...
+            if (oldHitCount == null) {
+              hitCounts.put(version, hitCount);
+              message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
+            } else {
+              // Just ensure that all nodes show the same hit count for
+              // the same version, i.e. they really are replicas of one another:
+              if (oldHitCount.intValue() != hitCount) {
+                failed.set(true);
+                stop.set(true);
+                message("top: searcher: wrong version hitCount: version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+                fail("version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+              }
+            }
+          } catch (IOException ioe) {
+            //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+            //ioe.printStackTrace(System.out);
+            IOUtils.closeWhileHandlingException(c);
+            connections.remove(node.id);
+            continue;
+          }
+
+          // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
+          Integer expectedAtLeastHitCount = versionToMarker.get(version);
+
+          if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
+            try {
+              c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
+              c.flush();
+              while (c.sockIn.available() == 0) {
+                if (stop.get()) {
+                  break;
+                }
+                if (node.isOpen == false) {
+                  throw new IOException("node died");
+                }
+                Thread.sleep(1);
+              }
+
+              version = c.in.readVLong();
+
+              while (c.sockIn.available() == 0) {
+                if (stop.get()) {
+                  break;
+                }
+                if (node.isOpen == false) {
+                  throw new IOException("node died");
+                }
+                Thread.sleep(1);
+              }
+
+              int hitCount = c.in.readVInt();
+
+              // Look for data loss: make sure all marker docs are visible:
+            
+              if (hitCount < expectedAtLeastHitCount) {
+
+                String failMessage = "node=" + node + ": documents were lost version=" + version + " hitCount=" + hitCount + " vs expectedAtLeastHitCount=" + expectedAtLeastHitCount;
+                message(failMessage);
+                failed.set(true);
+                stop.set(true);
+                fail(failMessage);
+              }
+            } catch (IOException ioe) {
+              //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+              //throw new RuntimeException(ioe);
+              //ioe.printStackTrace(System.out);
+              IOUtils.closeWhileHandlingException(c);
+              connections.remove(node.id);
+              continue;
+            }
+          }
+
+          Thread.sleep(10);
+
+        } catch (Throwable t) {
+          failed.set(true);
+          stop.set(true);
+          throw new RuntimeException(t);
+        } finally {
+          node.lock.unlock();
+        }
+      }
+      System.out.println("Searcher: now stop");
+      IOUtils.closeWhileHandlingException(connections.values());
+    }
+  }
+
+  private class IndexThread extends Thread {
+
+    @Override
+    public void run() {
+
+      try {
+        LineFileDocs docs = new LineFileDocs(random());
+        int docCount = 0;
+
+        // How often we do an update/delete vs add:
+        double updatePct = random().nextDouble();
+
+        // Varies how many docs/sec we index:
+        int sleepChance = TestUtil.nextInt(random(), 4, 100);
+
+        message("top: indexer: updatePct=" + updatePct + " sleepChance=" + sleepChance);
+
+        long lastTransLogLoc = transLog.getNextLocation();
+        
+        NodeProcess curPrimary = null;
+        Connection c = null;
+
+        while (stop.get() == false) {
+
+          try {
+            while (stop.get() == false && curPrimary == null) {
+              Thread.sleep(10);
+              curPrimary = primary;
+              if (curPrimary != null) {
+                c = new Connection(curPrimary.tcpPort);
+                c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+                break;
+              }
+            }
+
+            if (stop.get()) {
+              break;
+            }
+
+            Thread.currentThread().setName("indexer p" + curPrimary.id);
+
+            if (random().nextInt(10) == 7) {
+              // We use the marker docs to check for data loss in search thread:
+              Document doc = new Document();
+              int id = markerUpto.getAndIncrement();
+              String idString = "m"+id;
+              doc.add(newStringField("docid", idString, Field.Store.YES));
+              doc.add(newStringField("marker", "marker", Field.Store.YES));
+              curPrimary.addOrUpdateDocument(c, doc, false);
+              transLog.addDocument(idString, doc);
+              message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
+            }
+
+            if (docCount > 0 && random().nextDouble() < updatePct) {
+              int randomID = random().nextInt(docCount);
+              String randomIDString = Integer.toString(randomID);
+              if (random().nextBoolean()) {
+                // Replace previous doc
+                Document doc = docs.nextDoc();
+                ((Field) doc.getField("docid")).setStringValue(randomIDString);
+                curPrimary.addOrUpdateDocument(c, doc, true);
+                transLog.updateDocument(randomIDString, doc);
+              } else {
+                // Delete previous doc
+                curPrimary.deleteDocument(c, randomIDString);
+                transLog.deleteDocuments(randomIDString);
+              }
+            } else {
+              // Add new doc:
+              Document doc = docs.nextDoc();
+              String idString = Integer.toString(docCount++);
+              ((Field) doc.getField("docid")).setStringValue(idString);
+              curPrimary.addOrUpdateDocument(c, doc, false);
+              transLog.addDocument(idString, doc);
+
+              if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
+                long curLoc = transLog.getNextLocation();
+                // randomly replay chunks of translog just to test replay:
+                message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
+                transLog.replay(curPrimary, lastTransLogLoc, curLoc);
+                lastTransLogLoc = curLoc;
+              }
+            }
+          } catch (IOException se) {
+            // Assume primary crashed
+            message("top: indexer lost connection to primary");
+            try {
+              c.close();
+            } catch (Throwable t) {
+            }
+            curPrimary = null;
+            c = null;
+          }
+
+          if (random().nextInt(sleepChance) == 0) {
+            Thread.sleep(1);
+          }
+
+          if (random().nextInt(100) == 17) {
+            System.out.println("Indexer: now pause for a bit...");
+            Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
+            System.out.println("Indexer: done pause for a bit...");
+          }
+        }
+        if (curPrimary != null) {
+          try {
+            c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
+            c.flush();
+            c.in.readByte();
+          } catch (IOException se) {
+            // Assume primary crashed
+            message("top: indexer lost connection to primary");
+            try {
+              c.close();
+            } catch (Throwable t) {
+            }
+            curPrimary = null;
+            c = null;
+          }
+        }
+        System.out.println("Indexer: now stop");
+      } catch (Throwable t) {
+        failed.set(true);
+        stop.set(true);
+        throw new RuntimeException(t);
+      }
+    }
+  }
+
+  static void message(String message) {
+    long now = System.nanoTime();
+    System.out.println(String.format(Locale.ROOT,
+                                     "%5.3fs       :     parent [%11s] %s",
+                                     (now-Node.globalStartNS)/1000000000.,
+                                     Thread.currentThread().getName(),
+                                     message));
+  }
+}


[02/15] lucene-solr git commit: current patch

Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
new file mode 100644
index 0000000..4e29508
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -0,0 +1,238 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.UnknownHostException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+
+/** Parent JVM hold this "wrapper" to refer to each child JVM.  This is roughly equivalent e.g. to a client-side "sugar" API. */
+class NodeProcess implements Closeable {
+  final Process p;
+
+  // Port sub-process is listening on
+  final int tcpPort;
+
+  final int id;
+
+  final Thread pumper;
+
+  // Acquired when searching or indexing wants to use this node:
+  final ReentrantLock lock;
+
+  final boolean isPrimary;
+
+  // Version in the commit point we opened on init:
+  final long initCommitVersion;
+
+  // SegmentInfos.version, which can be higher than the initCommitVersion
+  final long initInfosVersion;
+
+  volatile boolean isOpen = true;
+
+  public NodeProcess(Process p, int id, int tcpPort, Thread pumper, boolean isPrimary, long initCommitVersion, long initInfosVersion) {
+    this.p = p;
+    this.id = id;
+    this.tcpPort = tcpPort;
+    this.pumper = pumper;
+    this.isPrimary = isPrimary;
+    this.initCommitVersion = initCommitVersion;
+    this.initInfosVersion = initInfosVersion;
+    assert initInfosVersion >= initCommitVersion: "initInfosVersion=" + initInfosVersion + " initCommitVersion=" + initCommitVersion;
+    lock = new ReentrantLock();
+  }
+
+  @Override
+  public String toString() {
+    if (isPrimary) {
+      return "P" + id + " tcpPort=" + tcpPort;
+    } else {
+      return "R" + id + " tcpPort=" + tcpPort;
+    }
+  }
+
+  public synchronized void crash() {
+    if (isOpen) {
+      isOpen = false;
+      p.destroy();
+      try {
+        pumper.join();
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        throw new RuntimeException(ie);
+      }
+    }
+  }
+
+  public boolean commit() {
+    try (Connection c = new Connection(tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
+      c.flush();
+      c.s.shutdownOutput();
+      if (c.in.readByte() != 1) {
+        throw new RuntimeException("commit failed");
+      }
+      return true;
+    } catch (Throwable t) {
+      // Something wrong with this replica; skip it:
+      System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
+      return false;
+    }
+  }
+
+  public void commitAsync() {
+    try (Connection c = new Connection(tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
+      c.flush();
+    } catch (Throwable t) {
+      // Something wrong with this replica; skip it:
+      System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
+    }
+  }
+
+  public long getSearchingVersion() {
+    try (Connection c = new Connection(tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_GET_SEARCHING_VERSION);
+      c.flush();
+      c.s.shutdownOutput();
+      return c.in.readVLong();
+    } catch (Throwable t) {
+      // Something wrong with this replica; skip it:
+      System.out.println("PARENT: top: hit SocketException during getSearchingVersion with R" + id + "; skipping");
+      return -1L;
+    }
+  }
+
+  /** Ask the primary node process to flush.  We send it all currently up replicas so it can notify them about the new NRT point.  Returns the newly
+   *  flushed version, or a negative (current) version if there were no changes. */
+  public synchronized long flush() throws IOException {
+    assert isPrimary;
+    try (Connection c = new Connection(tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_FLUSH);
+      c.flush();
+      c.s.shutdownOutput();
+      return c.in.readLong();
+    }
+  }
+
+  @Override
+  public void close() {
+    shutdown();
+  }
+
+  public synchronized boolean shutdown() {
+    lock.lock();
+    try {
+      System.out.println("PARENT: now shutdown node=" + id + " isOpen=" + isOpen);
+      if (isOpen) {
+        // Ask the child process to shutdown gracefully:
+        isOpen = false;
+        System.out.println("PARENT: send CMD_CLOSE to node=" + id);
+        try (Connection c = new Connection(tcpPort)) {
+          c.out.writeByte(SimplePrimaryNode.CMD_CLOSE);
+          c.flush();
+          if (c.in.readByte() != 1) {
+            throw new RuntimeException("shutdown failed");
+          }
+        } catch (Throwable t) {
+          System.out.println("top: shutdown failed; skipping");
+          t.printStackTrace(System.out);
+          return false;
+        }
+        try {
+          p.waitFor();
+          pumper.join();
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          throw new RuntimeException(ie);
+        }
+      }
+
+      return true;
+    } finally {
+      lock.unlock();
+    }
+  }
+
+  public void addOrUpdateDocument(Connection c, Document doc, boolean isUpdate) throws IOException {
+    if (isPrimary == false) {
+      throw new IllegalStateException("only primary can index");
+    }
+    int fieldCount = 0;
+
+    String title = doc.get("title");
+    if (title != null) {
+      fieldCount++;
+    }
+
+    String docid = doc.get("docid");
+    assert docid != null;
+    fieldCount++;
+
+    String body = doc.get("body");
+    if (body != null) {
+      fieldCount++;
+    }
+
+    String marker = doc.get("marker");
+    if (marker != null) {
+      fieldCount++;
+    }
+
+    c.out.writeByte(isUpdate ? SimplePrimaryNode.CMD_UPDATE_DOC : SimplePrimaryNode.CMD_ADD_DOC);
+    c.out.writeVInt(fieldCount);
+    c.out.writeString("docid");
+    c.out.writeString(docid);
+    if (title != null) {
+      c.out.writeString("title");
+      c.out.writeString(title);
+    }
+    if (body != null) {
+      c.out.writeString("body");
+      c.out.writeString(body);
+    }
+    if (marker != null) {
+      c.out.writeString("marker");
+      c.out.writeString(marker);
+    }
+    c.flush();
+    c.in.readByte();
+  }
+
+  public void deleteDocument(Connection c, String docid) throws IOException {
+    if (isPrimary == false) {
+      throw new IllegalStateException("only primary can index");
+    }
+    c.out.writeByte(SimplePrimaryNode.CMD_DELETE_DOC);
+    c.out.writeString(docid);
+    c.flush();
+    c.in.readByte();
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
new file mode 100644
index 0000000..1180967
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
@@ -0,0 +1,294 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.OutputStreamIndexOutput;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Handles one set of files that need copying, either because we have a
+ *  new NRT point, or we are pre-copying merged files for merge warming. */
+class SimpleCopyJob extends CopyJob {
+  final Connection c;
+
+  final byte[] copyBuffer = new byte[65536];
+  final CopyState copyState;
+
+  private Iterator<Map.Entry<String,FileMetaData>> iter;
+
+  public SimpleCopyJob(String reason, Connection c, CopyState copyState, SimpleReplicaNode dest, Map<String,FileMetaData> files, boolean highPriority, OnceDone onceDone)
+    throws IOException {
+    super(reason, files, dest, highPriority, onceDone);
+    dest.message("create SimpleCopyJob o" + ord);
+    this.c = c;
+    this.copyState = copyState;
+  }
+
+  @Override
+  public synchronized void start() throws IOException {
+    if (iter == null) {
+      iter = toCopy.iterator();
+
+      // Send all file names / offsets up front to avoid ping-ping latency:
+      try {
+
+        // This means we resumed an already in-progress copy; we do this one first:
+        if (current != null) {
+          c.out.writeByte((byte) 0);
+          c.out.writeString(current.name);
+          c.out.writeVLong(current.getBytesCopied());
+          totBytes += current.metaData.length;
+        }
+
+        for (Map.Entry<String,FileMetaData> ent : toCopy) {
+          String fileName = ent.getKey();
+          FileMetaData metaData = ent.getValue();
+          totBytes += metaData.length;
+          c.out.writeByte((byte) 0);
+          c.out.writeString(fileName);
+          c.out.writeVLong(0);
+        }
+        c.out.writeByte((byte) 1);
+        c.flush();
+        c.s.shutdownOutput();
+
+        if (current != null) {
+          // Do this only at the end, after sending all requested files, so we don't deadlock due to socket buffering waiting for primary to
+          // send us this length:
+          long len = c.in.readVLong();
+          if (len != current.metaData.length) {
+            throw new IllegalStateException("file " + current.name + ": meta data says length=" + current.metaData.length + " but c.in says " + len);
+          }
+        }
+
+        dest.message("SimpleCopyJob.init: done start files count=" + toCopy.size() + " totBytes=" + totBytes);
+
+      } catch (Throwable t) {
+        cancel("exc during start", t);
+        throw new NodeCommunicationException("exc during start", t);
+      }
+    } else {
+      throw new IllegalStateException("already started");
+    }
+  }
+
+  @Override
+  public long getTotalBytesCopied() {
+    return totBytesCopied;
+  }
+
+  @Override
+  public Set<String> getFileNamesToCopy() {
+    Set<String> fileNames = new HashSet<>();
+    for(Map.Entry<String,FileMetaData> ent : toCopy) {
+      fileNames.add(ent.getKey());
+    }
+    return fileNames;
+  }
+
+  @Override
+  public Set<String> getFileNames() {
+    return files.keySet();
+  }
+
+  /** Higher priority and then "first come first serve" order. */
+  @Override
+  public int compareTo(CopyJob _other) {
+    SimpleCopyJob other = (SimpleCopyJob) _other;
+    if (highPriority != other.highPriority) {
+      return highPriority ? -1 : 1;
+    } else if (ord < other.ord) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+
+  @Override
+  public void finish() throws IOException {
+    dest.message(String.format(Locale.ROOT,
+                               "top: file copy done; took %.1f msec to copy %d bytes; now rename %d tmp files",
+                               (System.nanoTime() - startNS)/1000000.0,
+                               totBytesCopied,
+                               copiedFiles.size()));
+
+    // NOTE: if any of the files we copied overwrote a file in the current commit point, we (ReplicaNode) removed the commit point up
+    // front so that the commit is not corrupt.  This way if we hit exc here, or if we crash here, we won't leave a corrupt commit in
+    // the index:
+    for(Map.Entry<String,String> ent : copiedFiles.entrySet()) {
+      String tmpFileName = ent.getValue();
+      String fileName = ent.getKey();
+
+      // Tricky: if primary crashes while warming (pre-copying) a merged segment _X, the new primary can easily flush or merge to _X (since we don't
+      // have a distributed inflateGens for the new primary) and _X file names will be reused.  In this case, our local deleter will be
+      // thinking it must remove _X's files (from the warmed merge that never went live), but this is dangerous when virus checker is active
+      // since deleter may finally succeed in deleting the file after we have copied the new _X flushed files.  So at this point was ask the
+      // deleter to NOT delete the file anymore:
+      dest.deleter.clearPending(Collections.singleton(fileName));
+
+      if (Node.VERBOSE_FILES) {
+        dest.message("rename file " + tmpFileName + " to " + fileName);
+      }
+
+      // NOTE: if this throws exception, then some files have been moved to their true names, and others are leftover .tmp files.  I don't
+      // think heroic exception handling is necessary (no harm will come, except some leftover files),  nor warranted here (would make the
+      // code more complex, for the exceptional cases when something is wrong w/ your IO system):
+      dest.dir.renameFile(tmpFileName, fileName);
+    }
+
+    copiedFiles.clear();
+  }
+
+  /** Do an iota of work; returns true if all copying is done */
+  synchronized boolean visit() throws IOException {
+    if (exc != null) {
+      // We were externally cancelled:
+      return true;
+    }
+
+    if (current == null) {
+      if (iter.hasNext() == false) {
+        c.close();
+        return true;
+      }
+
+      Map.Entry<String,FileMetaData> next = iter.next();
+      FileMetaData metaData = next.getValue();
+      String fileName = next.getKey();
+      long len = c.in.readVLong();
+      if (len != metaData.length) {
+        throw new IllegalStateException("file " + fileName + ": meta data says length=" + metaData.length + " but c.in says " + len);
+      }
+      current = new CopyOneFile(c.in, dest, fileName, metaData, copyBuffer);
+    }
+
+    if (current.visit()) {
+      // This file is done copying
+      copiedFiles.put(current.name, current.tmpName);
+      totBytesCopied += current.getBytesCopied();
+      assert totBytesCopied <= totBytes: "totBytesCopied=" + totBytesCopied + " totBytes=" + totBytes;
+      current = null;
+      return false;
+    }
+
+    return false;
+  }
+
+  protected CopyOneFile newCopyOneFile(CopyOneFile prev) {
+    return new CopyOneFile(prev, c.in);
+  }
+
+  @Override
+  public synchronized void transferAndCancel(CopyJob prevJob) throws IOException {
+    try {
+      super.transferAndCancel(prevJob);
+    } finally {
+      IOUtils.closeWhileHandlingException(((SimpleCopyJob) prevJob).c);
+    }
+  }
+
+  public synchronized void cancel(String reason, Throwable exc) {
+    try {
+      super.cancel(reason, exc);
+    } finally {
+      IOUtils.closeWhileHandlingException(c);
+    }
+  }
+
+  @Override
+  public boolean getFailed() {
+    return exc != null;
+  }
+  
+  @Override
+  public String toString() {
+    return "SimpleCopyJob(ord=" + ord + " " + reason + " highPriority=" + highPriority + " files count=" + files.size() + " bytesCopied=" + totBytesCopied + " (of " + totBytes + ") filesCopied=" + copiedFiles.size() + ")";
+  }
+
+  @Override
+  public void runBlocking() throws IOException {
+    while (visit() == false);
+
+    if (getFailed()) {
+      throw new RuntimeException("copy failed: " + cancelReason, exc);
+    }
+  }
+
+  @Override
+  public CopyState getCopyState() {
+    return copyState;
+  }
+
+  @Override
+  public synchronized boolean conflicts(CopyJob _other) {
+    Set<String> filesToCopy = new HashSet<>();
+    for(Map.Entry<String,FileMetaData> ent : toCopy) {
+      filesToCopy.add(ent.getKey());
+    }
+
+    SimpleCopyJob other = (SimpleCopyJob) _other;
+    synchronized (other) {
+      for(Map.Entry<String,FileMetaData> ent : other.toCopy) {
+        if (filesToCopy.contains(ent.getKey())) {
+          return true;
+        }
+      }
+    }
+
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
new file mode 100644
index 0000000..18e77ef
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -0,0 +1,674 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** A primary node that uses simple TCP connections to send commands and copy files */
+
+class SimplePrimaryNode extends PrimaryNode {
+
+  final int tcpPort;
+
+  // These are updated by parent test process whenever replicas change:
+  int[] replicaTCPPorts;
+  int[] replicaIDs;
+
+  // So we only flip a bit once per file name:
+  final Set<String> bitFlipped = Collections.synchronizedSet(new HashSet<>());
+
+  static class MergePreCopy {
+    final List<Connection> connections = Collections.synchronizedList(new ArrayList<>());
+    final Map<String,FileMetaData> files;
+    private boolean finished;
+
+    public MergePreCopy(Map<String,FileMetaData> files) {
+      this.files = files;
+    }
+
+    public synchronized boolean tryAddConnection(Connection c) {
+      if (finished == false) {
+        connections.add(c);
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    public synchronized boolean finished() {
+      if (connections.isEmpty()) {
+        finished = true;
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+  final List<MergePreCopy> warmingSegments = Collections.synchronizedList(new ArrayList<>());
+
+  public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException {
+    super(initWriter(id, random, indexPath), id, primaryGen, forcePrimaryVersion, searcherFactory);
+    this.tcpPort = tcpPort;
+  }
+
+  /** Records currently alive replicas. */
+  public synchronized void setReplicas(int[] replicaIDs, int[] replicaTCPPorts) {
+    message("top: set replicasIDs=" + Arrays.toString(replicaIDs) + " tcpPorts=" + Arrays.toString(replicaTCPPorts));
+    this.replicaIDs = replicaIDs;
+    this.replicaTCPPorts = replicaTCPPorts;
+  }
+
+  private static IndexWriter initWriter(int id, Random random, Path indexPath) throws IOException {
+    Directory dir = SimpleReplicaNode.getDirectory(random, id, indexPath);
+
+    MockAnalyzer analyzer = new MockAnalyzer(random);
+    analyzer.setMaxTokenLength(TestUtil.nextInt(random, 1, IndexWriter.MAX_TERM_LENGTH));
+    IndexWriterConfig iwc = LuceneTestCase.newIndexWriterConfig(random, analyzer);
+
+    MergePolicy mp = iwc.getMergePolicy();
+    //iwc.setInfoStream(new PrintStreamInfoStream(System.out));
+
+    // Force more frequent merging so we stress merge warming:
+    if (mp instanceof TieredMergePolicy) {
+      TieredMergePolicy tmp = (TieredMergePolicy) mp;
+      tmp.setSegmentsPerTier(3);
+      tmp.setMaxMergeAtOnce(3);
+    } else if (mp instanceof LogMergePolicy) {
+      LogMergePolicy lmp = (LogMergePolicy) mp;
+      lmp.setMergeFactor(3);
+    }
+
+    IndexWriter writer = new IndexWriter(dir, iwc);
+
+    TestUtil.reduceOpenFiles(writer);
+    return writer;
+  }
+
+  @Override
+  protected void preCopyMergedSegmentFiles(SegmentCommitInfo info, Map<String,FileMetaData> files) throws IOException {
+    int[] replicaTCPPorts = this.replicaTCPPorts;
+    if (replicaTCPPorts == null) {
+      message("no replicas; skip warming " + info);
+      return;
+    }
+
+    message("top: warm merge " + info + " to " + replicaTCPPorts.length + " replicas: files=" + files.keySet());
+
+    MergePreCopy preCopy = new MergePreCopy(files);
+    warmingSegments.add(preCopy);
+
+    try {
+
+      Set<String> fileNames = files.keySet();
+
+      // Ask all currently known replicas to pre-copy this newly merged segment's files:
+      for (int replicaTCPPort : replicaTCPPorts) {
+        try {
+          Connection c = new Connection(replicaTCPPort);
+          c.out.writeByte(SimpleReplicaNode.CMD_PRE_COPY_MERGE);
+          c.out.writeVLong(primaryGen);
+          c.out.writeVInt(tcpPort);
+          SimpleServer.writeFilesMetaData(c.out, files);
+          c.flush();
+          c.s.shutdownOutput();
+          message("warm connection " + c.s);
+          preCopy.connections.add(c);
+        } catch (Throwable t) {
+          message("top: ignore exception trying to warm to replica port " + replicaTCPPort + ": " + t);
+          //t.printStackTrace(System.out);
+        }
+      }
+
+      long startNS = System.nanoTime();
+      long lastWarnNS = startNS;
+
+      // TODO: maybe ... place some sort of time limit on how long we are willing to wait for slow replicas to finish copying?
+      while (preCopy.finished() == false) {
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException ie) {
+          throw new ThreadInterruptedException(ie);
+        }
+
+        if (isClosed()) {
+          message("top: primary is closing: now cancel segment warming");
+          synchronized(preCopy.connections) {
+            IOUtils.closeWhileHandlingException(preCopy.connections);
+          }
+          return;
+        }
+
+        long ns = System.nanoTime();
+        if (ns - lastWarnNS > 1000000000L) {
+          message(String.format(Locale.ROOT, "top: warning: still warming merge " + info + " to " + preCopy.connections.size() + " replicas for %.1f sec...", (ns - startNS)/1000000000.0));
+          lastWarnNS = ns;
+        }
+
+        // Process keep-alives:
+        synchronized(preCopy.connections) {
+          Iterator<Connection> it = preCopy.connections.iterator();
+          while (it.hasNext()) {
+            Connection c = it.next();
+            try {
+              long nowNS = System.nanoTime();
+              boolean done = false;
+              while (c.sockIn.available() > 0) {
+                byte b = c.in.readByte();
+                if (b == 0) {
+                  // keep-alive
+                  c.lastKeepAliveNS = nowNS;
+                  message("keep-alive for socket=" + c.s + " merge files=" + files.keySet());
+                } else {
+                  // merge is done pre-copying to this node
+                  if (b != 1) {
+                    throw new IllegalArgumentException();
+                  }
+                  message("connection socket=" + c.s + " is done warming its merge " + info + " files=" + files.keySet());
+                  IOUtils.closeWhileHandlingException(c);
+                  it.remove();
+                  done = true;
+                  break;
+                }
+              }
+
+              // If > 2 sec since we saw a keep-alive, assume this replica is dead:
+              if (done == false && nowNS - c.lastKeepAliveNS > 2000000000L) {
+                message("top: warning: replica socket=" + c.s + " for segment=" + info + " seems to be dead; closing files=" + files.keySet());
+                IOUtils.closeWhileHandlingException(c);
+                it.remove();
+              }
+            } catch (Throwable t) {
+              message("top: ignore exception trying to read byte during warm for segment=" + info + " to replica socket=" + c.s + ": " + t + " files=" + files.keySet());
+              IOUtils.closeWhileHandlingException(c);
+              it.remove();
+            }
+          }
+        }
+      }
+    } finally {
+      warmingSegments.remove(preCopy);
+    }
+  }
+
+  /** Flushes all indexing ops to disk and notifies all replicas that they should now copy */
+  private void handleFlush(DataInput topIn, DataOutput topOut, BufferedOutputStream bos) throws IOException {
+    Thread.currentThread().setName("flush");
+
+    int[] replicaTCPPorts;
+    int[] replicaIDs;
+    synchronized (this) {
+      replicaTCPPorts = this.replicaTCPPorts;
+      replicaIDs = this.replicaIDs;
+    }
+
+    message("now flush; " + replicaIDs.length + " replicas");
+
+    if (flushAndRefresh()) {
+      // Something did get flushed (there were indexing ops since the last flush):
+
+      // Tell caller the version before pushing to replicas, so that even if we crash after this, caller will know what version we
+      // (possibly) pushed to some replicas.  Alternatively we could make this 2 separate ops?
+      long version = getCopyStateVersion();
+      message("send flushed version=" + version);
+      topOut.writeLong(version);
+      bos.flush();
+
+      // Notify current replicas:
+      for(int i=0;i<replicaIDs.length;i++) {
+        int replicaID = replicaIDs[i];
+        try (Connection c = new Connection(replicaTCPPorts[i])) {
+          c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
+          c.out.writeVLong(version);
+          c.out.writeInt(tcpPort);
+          c.flush();
+          // TODO: we should use multicast to broadcast files out to replicas
+          // TODO: ... replicas could copy from one another instead of just primary
+          // TODO: we could also prioritize one replica at a time?
+        } catch (Throwable t) {
+          message("top: failed to connect R" + replicaID + " for newNRTPoint; skipping: " + t.getMessage());
+        }
+      }
+    } else {
+      // No changes flushed:
+      topOut.writeLong(-getCopyStateVersion());
+    }
+  }
+
+  /** Pushes CopyState on the wire */
+  private static void writeCopyState(CopyState state, DataOutput out) throws IOException {
+    // TODO (opto): we could encode to byte[] once when we created the copyState, and then just send same byts to all replicas...
+    out.writeVInt(state.infosBytes.length);
+    out.writeBytes(state.infosBytes, 0, state.infosBytes.length);
+    out.writeVLong(state.gen);
+    out.writeVLong(state.version);
+    SimpleServer.writeFilesMetaData(out, state.files);
+
+    out.writeVInt(state.completedMergeFiles.size());
+    for(String fileName : state.completedMergeFiles) {
+      out.writeString(fileName);
+    }
+    out.writeVLong(state.primaryGen);
+  }
+
+  /** Called when another node (replica) wants to copy files from us */
+  private boolean handleFetchFiles(Random random, Socket socket, DataInput destIn, DataOutput destOut, BufferedOutputStream bos) throws IOException {
+    Thread.currentThread().setName("send");
+
+    int replicaID = destIn.readVInt();
+    message("top: start fetch for R" + replicaID + " socket=" + socket);
+    byte b = destIn.readByte();
+    CopyState copyState;
+    if (b == 0) {
+      // Caller already has CopyState
+      copyState = null;
+    } else if (b == 1) {
+      // Caller does not have CopyState; we pull the latest one:
+      copyState = getCopyState();
+      Thread.currentThread().setName("send-R" + replicaID + "-" + copyState.version);
+    } else {
+      // Protocol error:
+      throw new IllegalArgumentException("invalid CopyState byte=" + b);
+    }
+
+    try {
+      if (copyState != null) {
+        // Serialize CopyState on the wire to the client:
+        writeCopyState(copyState, destOut);
+        bos.flush();
+      }
+
+      byte[] buffer = new byte[16384];
+      int fileCount = 0;
+      long totBytesSent = 0;
+      while (true) {
+        byte done = destIn.readByte();
+        if (done == 1) {
+          break;
+        } else if (done != 0) {
+          throw new IllegalArgumentException("expected 0 or 1 byte but got " + done);
+        }
+
+        // Name of the file the replica wants us to send:
+        String fileName = destIn.readString();
+
+        // Starting offset in the file we should start sending bytes from:
+        long fpStart = destIn.readVLong();
+
+        try (IndexInput in = dir.openInput(fileName, IOContext.DEFAULT)) {
+          long len = in.length();
+          //message("fetch " + fileName + ": send len=" + len);
+          destOut.writeVLong(len);
+          in.seek(fpStart);
+          long upto = fpStart;
+          while (upto < len) {
+            int chunk = (int) Math.min(buffer.length, (len-upto));
+            in.readBytes(buffer, 0, chunk);
+            if (TestNRTReplication.DO_BIT_FLIPS_DURING_COPY) {
+              if (random.nextInt(3000) == 17 && bitFlipped.contains(fileName) == false) {
+                bitFlipped.add(fileName);
+                message("file " + fileName + " to R" + replicaID + ": now randomly flipping a bit at byte=" + upto);
+                int x = random.nextInt(chunk);
+                int bit = random.nextInt(8);
+                buffer[x] ^= 1 << bit;
+              }
+            }
+            destOut.writeBytes(buffer, 0, chunk);
+            upto += chunk;
+            totBytesSent += chunk;
+          }
+        }
+
+        fileCount++;
+      }
+
+      message("top: done fetch files for R" + replicaID + ": sent " + fileCount + " files; sent " + totBytesSent + " bytes");
+    } catch (Throwable t) {
+      message("top: exception during fetch: " + t.getMessage() + "; now close socket");
+      socket.close();
+      return false;
+    } finally {
+      if (copyState != null) {
+        message("top: fetch: now release CopyState");
+        releaseCopyState(copyState);
+      }
+    }
+
+    return true;
+  }
+
+  static final FieldType tokenizedWithTermVectors;
+
+  static {
+    tokenizedWithTermVectors = new FieldType(TextField.TYPE_STORED);
+    tokenizedWithTermVectors.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+    tokenizedWithTermVectors.setStoreTermVectors(true);
+    tokenizedWithTermVectors.setStoreTermVectorOffsets(true);
+    tokenizedWithTermVectors.setStoreTermVectorPositions(true);
+  }
+
+  private void handleIndexing(Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException {
+    Thread.currentThread().setName("indexing");
+    message("start handling indexing socket=" + socket);
+    while (true) {
+      byte cmd;
+      try {
+        cmd = in.readByte();
+      } catch (EOFException eofe) {
+        // done
+        return;
+      }
+      //message("INDEXING OP " + cmd);
+      if (cmd == CMD_ADD_DOC) {
+        handleAddDocument(in, out);
+        out.writeByte((byte) 1);
+        bos.flush();
+      } else if (cmd == CMD_UPDATE_DOC) {
+        handleUpdateDocument(in, out);
+        out.writeByte((byte) 1);
+        bos.flush();
+      } else if (cmd == CMD_DELETE_DOC) {
+        handleDeleteDocument(in, out);
+        out.writeByte((byte) 1);
+        bos.flush();
+      } else if (cmd == CMD_INDEXING_DONE) {
+        out.writeByte((byte) 1);
+        bos.flush();
+        break;
+      } else {
+        throw new IllegalArgumentException("cmd must be add, update or delete; got " + cmd);
+      }
+    }
+  }
+
+  private void handleAddDocument(DataInput in, DataOutput out) throws IOException {
+    int fieldCount = in.readVInt();
+    Document doc = new Document();
+    for(int i=0;i<fieldCount;i++) {
+      String name = in.readString();
+      String value = in.readString();
+      // NOTE: clearly NOT general!
+      if (name.equals("docid") || name.equals("marker")) {
+        doc.add(new StringField(name, value, Field.Store.YES));
+      } else if (name.equals("title")) {
+        doc.add(new StringField("title", value, Field.Store.YES));
+        doc.add(new Field("titleTokenized", value, tokenizedWithTermVectors));
+      } else if (name.equals("body")) {
+        doc.add(new Field("body", value, tokenizedWithTermVectors));
+      } else {
+        throw new IllegalArgumentException("unhandled field name " + name);
+      }
+    }
+
+    writer.addDocument(doc);
+  }
+
+  private void handleUpdateDocument(DataInput in, DataOutput out) throws IOException {
+    int fieldCount = in.readVInt();
+    Document doc = new Document();
+    String docid = null;
+    for(int i=0;i<fieldCount;i++) {
+      String name = in.readString();
+      String value = in.readString();
+      // NOTE: clearly NOT general!
+      if (name.equals("docid")) {
+        docid = value;
+        doc.add(new StringField("docid", value, Field.Store.YES));
+      } else if (name.equals("marker")) {
+        doc.add(new StringField("marker", value, Field.Store.YES));
+      } else if (name.equals("title")) {
+        doc.add(new StringField("title", value, Field.Store.YES));
+        doc.add(new Field("titleTokenized", value, tokenizedWithTermVectors));
+      } else if (name.equals("body")) {
+        doc.add(new Field("body", value, tokenizedWithTermVectors));
+      } else {
+        throw new IllegalArgumentException("unhandled field name " + name);
+      }
+    }
+
+    writer.updateDocument(new Term("docid", docid), doc);
+  }
+
+  private void handleDeleteDocument(DataInput in, DataOutput out) throws IOException {
+    String docid = in.readString();
+    writer.deleteDocuments(new Term("docid", docid));
+  }
+
+  // Sent to primary to cutover new SIS:
+  static final byte CMD_FLUSH = 10;
+
+  // Sent by replica to primary asking to copy a set of files over:
+  static final byte CMD_FETCH_FILES = 1;
+  static final byte CMD_GET_SEARCHING_VERSION = 12;
+  static final byte CMD_SEARCH = 2;
+  static final byte CMD_MARKER_SEARCH = 3;
+  static final byte CMD_COMMIT = 4;
+  static final byte CMD_CLOSE = 5;
+
+  // Send (to primary) the list of currently running replicas:
+  static final byte CMD_SET_REPLICAS = 16;
+
+  // Multiple indexing ops
+  static final byte CMD_INDEXING = 18;
+  static final byte CMD_ADD_DOC = 6;
+  static final byte CMD_UPDATE_DOC = 7;
+  static final byte CMD_DELETE_DOC = 8;
+  static final byte CMD_INDEXING_DONE = 19;
+
+  // Sent by replica to primary when replica first starts up, so primary can add it to any warming merges:
+  static final byte CMD_NEW_REPLICA = 20;
+
+  /** Handles incoming request to the naive TCP server wrapping this node */
+  void handleOneConnection(Random random, ServerSocket ss, AtomicBoolean stop, InputStream is, Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
+
+    outer:
+    while (true) {
+      byte cmd;
+      while (true) {
+        if (is.available() > 0) {
+          break;
+        }
+        if (stop.get()) {
+          return;
+        }
+        Thread.sleep(10);
+      }
+
+      try {
+        cmd = in.readByte();
+      } catch (EOFException eofe) {
+        break;
+      }
+
+      switch (cmd) {
+
+      case CMD_FLUSH:
+        handleFlush(in, out, bos);
+        break;
+
+      case CMD_FETCH_FILES:
+        // Replica (other node) is asking us (primary node) for files to copy
+        handleFetchFiles(random, socket, in, out, bos);
+        break;
+
+      case CMD_INDEXING:
+        handleIndexing(socket, in, out, bos);
+        break;
+
+      case CMD_GET_SEARCHING_VERSION:
+        out.writeVLong(getCurrentSearchingVersion());
+        break;
+
+      case CMD_SEARCH:
+        {
+          Thread.currentThread().setName("search");
+          IndexSearcher searcher = mgr.acquire();
+          try {
+            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+            int hitCount = searcher.search(new TermQuery(new Term("body", "the")), 1).totalHits;
+            //message("version=" + version + " searcher=" + searcher);
+            out.writeVLong(version);
+            out.writeVInt(hitCount);
+          } finally {
+            mgr.release(searcher);
+          }
+        }
+        continue outer;
+
+      case CMD_MARKER_SEARCH:
+        {
+          Thread.currentThread().setName("msearch");
+          IndexSearcher searcher = mgr.acquire();
+          try {
+            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+            int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+            out.writeVLong(version);
+            out.writeVInt(hitCount);
+          } finally {
+            mgr.release(searcher);
+          }
+        }
+        continue outer;
+
+      case CMD_COMMIT:
+        Thread.currentThread().setName("commit");
+        commit();
+        out.writeByte((byte) 1);
+        break;
+
+      case CMD_CLOSE:
+        Thread.currentThread().setName("close");
+        message("top close: now close server socket");
+        ss.close();
+        out.writeByte((byte) 1);
+        message("top close: done close server socket");
+        break;
+
+      case CMD_SET_REPLICAS:
+        Thread.currentThread().setName("set repls");
+        int count = in.readVInt();
+        int[] replicaIDs = new int[count];
+        int[] replicaTCPPorts = new int[count];
+        for(int i=0;i<count;i++) {
+          replicaIDs[i] = in.readVInt();
+          replicaTCPPorts[i] = in.readVInt();
+        }
+        out.writeByte((byte) 1);
+        setReplicas(replicaIDs, replicaTCPPorts);
+        break;
+
+      case CMD_NEW_REPLICA:
+        Thread.currentThread().setName("new repl");
+        int replicaTCPPort = in.readVInt();
+        message("new replica: " + warmingSegments.size() + " current warming merges");
+        // Step through all currently warming segments and try to add this replica if it isn't there already:
+        for(MergePreCopy preCopy : warmingSegments) {
+          message("warming segment " + preCopy.files.keySet());
+          boolean found = false;
+          synchronized (preCopy.connections) {
+            for(Connection c : preCopy.connections) {
+              if (c.destTCPPort == replicaTCPPort) {
+                found = true;
+                break;
+              }
+            }
+          }
+
+          if (found) {
+            message("this replica is already warming this segment; skipping");
+            // It's possible (maybe) that the replica started up, then a merge kicked off, and it warmed to this new replica, all before the
+            // replica sent us this command:
+            continue;
+          }
+
+          // OK, this new replica is not already warming this segment, so attempt (could fail) to start warming now:
+
+          Connection c = new Connection(replicaTCPPort);
+          if (preCopy.tryAddConnection(c) == false) {
+            // This can happen, if all other replicas just now finished warming this segment, and so we were just a bit too late.  In this
+            // case the segment will be copied over in the next nrt point sent to this replica
+            message("failed to add connection to segment warmer (too late); closing");
+            c.close();
+          }
+          c.out.writeByte(SimpleReplicaNode.CMD_PRE_COPY_MERGE);
+          c.out.writeVLong(primaryGen);
+          c.out.writeVInt(tcpPort);
+          SimpleServer.writeFilesMetaData(c.out, preCopy.files);
+          c.flush();
+          c.s.shutdownOutput();
+          message("successfully started warming");
+        }
+        break;
+
+      default:
+        throw new IllegalArgumentException("unrecognized cmd=" + cmd + " via socket=" + socket);
+      }
+      bos.flush();
+      break;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
new file mode 100644
index 0000000..8667df1
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -0,0 +1,316 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.RateLimitedIndexOutput;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.LuceneTestCase;
+
+class SimpleReplicaNode extends ReplicaNode {
+  final int tcpPort;
+  final Jobs jobs;
+
+  // Rate limits incoming bytes/sec when fetching files:
+  final RateLimiter fetchRateLimiter;
+  final AtomicLong bytesSinceLastRateLimiterCheck = new AtomicLong();
+  final Random random;
+
+  /** Changes over time, as primary node crashes and moves around */
+  int curPrimaryTCPPort;
+
+  public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory) throws IOException {
+    super(id, getDirectory(random, id, indexPath), searcherFactory);
+    this.tcpPort = tcpPort;
+    this.random = new Random(random.nextLong());
+
+    // Random IO throttling on file copies: 5 - 20 MB/sec:
+    double mbPerSec = 5 * (1.0 + 3*random.nextDouble());
+    message(String.format(Locale.ROOT, "top: will rate limit file fetch to %.2f MB/sec", mbPerSec));
+    fetchRateLimiter = new RateLimiter.SimpleRateLimiter(mbPerSec);
+    this.curPrimaryTCPPort = primaryTCPPort;
+    
+    start(curPrimaryGen);
+
+    // Handles fetching files from primary:
+    jobs = new Jobs(this);
+    jobs.setName("R" + id + ".copyJobs");
+    jobs.setDaemon(true);
+    jobs.start();
+  }
+
+  @Override
+  protected void launch(CopyJob job) {
+    jobs.launch(job);
+  }
+
+  @Override
+  public void close() throws IOException {
+    // Can't be sync'd when calling jobs since it can lead to deadlock:
+    jobs.close();
+    message("top: jobs closed");
+    synchronized(mergeCopyJobs) {
+      for (CopyJob job : mergeCopyJobs) {
+        message("top: cancel merge copy job " + job);
+        job.cancel("jobs closing", null);
+      }
+    }
+    super.close();
+  }
+
+  @Override
+  protected CopyJob newCopyJob(String reason, Map<String,FileMetaData> files, Map<String,FileMetaData> prevFiles,
+                               boolean highPriority, CopyJob.OnceDone onceDone) throws IOException {
+    Connection c;
+    CopyState copyState;
+
+    // Exceptions in here mean something went wrong talking over the socket, which are fine (e.g. primary node crashed):
+    try {
+      c = new Connection(curPrimaryTCPPort);
+      c.out.writeByte(SimplePrimaryNode.CMD_FETCH_FILES);
+      c.out.writeVInt(id);
+      if (files == null) {
+        // No incoming CopyState: ask primary for latest one now
+        c.out.writeByte((byte) 1);
+        c.flush();
+        copyState = SimpleServer.readCopyState(c.in);
+        files = copyState.files;
+      } else {
+        c.out.writeByte((byte) 0);
+        copyState = null;
+      }
+    } catch (Throwable t) {
+      throw new NodeCommunicationException("exc while reading files to copy", t);
+    }
+
+    return new SimpleCopyJob(reason, c, copyState, this, files, highPriority, onceDone);
+  }
+
+  static Directory getDirectory(Random random, int id, Path path) throws IOException {
+    MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path);
+    
+    dir.setAssertNoUnrefencedFilesOnClose(true);
+    // This is very costly (takes more time to check than it did to index); we do this ourselves in the end instead of each time a replica
+    // is restarted:
+    dir.setCheckIndexOnClose(false);
+
+    // Corrupt any index files not referenced by current commit point; this is important (increases test evilness) because we may have done
+    // a hard crash of the previous JVM writing to this directory and so MDW's corrupt-unknown-files-on-close never ran:
+    Node.nodeMessage(id, "top: corrupt unknown files");
+    dir.corruptUnknownFiles();
+
+    return dir;
+  }
+
+  static final byte CMD_NEW_NRT_POINT = 0;
+
+  // Sent by primary to replica to pre-copy merge files:
+  static final byte CMD_PRE_COPY_MERGE = 17;
+
+  /** Handles incoming request to the naive TCP server wrapping this node */
+  void handleOneConnection(ServerSocket ss, AtomicBoolean stop, InputStream is, Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
+    outer:
+    while (true) {
+      byte cmd;
+      while (true) {
+        if (is.available() > 0) {
+          break;
+        }
+        if (stop.get()) {
+          return;
+        }
+        Thread.sleep(10);
+      }
+
+      try {
+        cmd = in.readByte();
+      } catch (EOFException eofe) {
+        break;
+      }
+
+      switch(cmd) {
+      case CMD_NEW_NRT_POINT:
+        {
+          long version = in.readVLong();
+          Thread.currentThread().setName("recv-" + version);
+          curPrimaryTCPPort = in.readInt();
+          newNRTPoint(version);
+        }
+        break;
+
+      case SimplePrimaryNode.CMD_GET_SEARCHING_VERSION:
+        // nocommit this is hacky:
+
+        // Tricky: if a sync is just finishing up, i.e. managed to finish copying all files just before we crashed primary, and is now
+        // in the process of opening a new reader, we need to wait for it, to be sure we really pick the most current replica:
+        if (isCopying()) {
+          message("top: getSearchingVersion: now wait for finish sync");
+          // TODO: use immediate concurrency instead of polling:
+          while (isCopying() && stop.get() == false) {
+            Thread.sleep(50);
+            message("top: curNRTCopy=" + curNRTCopy);
+          }
+          message("top: getSearchingVersion: done wait for finish sync");
+        }
+        if (stop.get() == false) {
+          out.writeVLong(getCurrentSearchingVersion());
+        } else {
+          message("top: getSearchingVersion: stop waiting for finish sync: stop is set");
+        }
+        break;
+
+      case SimplePrimaryNode.CMD_SEARCH:
+        {
+          Thread.currentThread().setName("search");
+          IndexSearcher searcher = mgr.acquire();
+          try {
+            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+            int hitCount = searcher.search(new TermQuery(new Term("body", "the")), 1).totalHits;
+            //node.message("version=" + version + " searcher=" + searcher);
+            out.writeVLong(version);
+            out.writeVInt(hitCount);
+          } finally {
+            mgr.release(searcher);
+          }
+        }
+        continue outer;
+
+      case SimplePrimaryNode.CMD_MARKER_SEARCH:
+        {
+          Thread.currentThread().setName("msearch");
+          IndexSearcher searcher = mgr.acquire();
+          try {
+            long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+            int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+            out.writeVLong(version);
+            out.writeVInt(hitCount);
+          } finally {
+            mgr.release(searcher);
+          }
+        }
+        continue outer;
+
+      case SimplePrimaryNode.CMD_COMMIT:
+        Thread.currentThread().setName("commit");
+        commit();
+        out.writeByte((byte) 1);
+        break;
+
+      case SimplePrimaryNode.CMD_CLOSE:
+        Thread.currentThread().setName("close");
+        ss.close();
+        out.writeByte((byte) 1);
+        break outer;
+
+      case CMD_PRE_COPY_MERGE:
+        Thread.currentThread().setName("merge copy");
+
+        long newPrimaryGen = in.readVLong();
+        curPrimaryTCPPort = in.readVInt();
+        Map<String,FileMetaData> files = SimpleServer.readFilesMetaData(in);
+        message("done reading files to copy files=" + files.keySet());
+        AtomicBoolean finished = new AtomicBoolean();
+        CopyJob job = launchPreCopyMerge(finished, newPrimaryGen, files);
+        message("done launching copy job files=" + files.keySet());
+
+        // Silly keep alive mechanism, else if e.g. we (replica node) crash, the primary
+        // won't notice for a very long time:
+        boolean success = false;
+        try {
+          int count = 0;
+          while (true) {
+            if (finished.get() || stop.get()) {
+              break;
+            }
+            Thread.sleep(10);
+            count++;
+            if (count == 100) {
+              // Once per second or so, we send a keep alive
+              message("send merge pre copy keep alive... files=" + files.keySet());
+
+              // To be evil, we sometimes fail to keep-alive, e.g. simulating a long GC pausing us:
+              if (random.nextBoolean()) {
+                out.writeByte((byte) 0);
+                count = 0;
+              }
+            }
+          }
+
+          out.writeByte((byte) 1);
+          bos.flush();
+          success = true;
+        } finally {
+          message("done merge copy files=" + files.keySet() + " success=" + success);
+        }
+        break;
+
+      default:
+        throw new IllegalArgumentException("unrecognized cmd=" + cmd);
+      }
+      bos.flush();
+
+      break;
+    }
+  }
+
+  @Override
+  protected void sendNewReplica() throws IOException {
+    message("send new_replica to primary tcpPort=" + curPrimaryTCPPort);
+    try (Connection c = new Connection(curPrimaryTCPPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_NEW_REPLICA);
+      c.out.writeVInt(tcpPort);
+      c.flush();
+      c.s.shutdownOutput();
+    } catch (Throwable t) {
+      message("ignoring exc " + t + " sending new_replica to primary tcpPort=" + curPrimaryTCPPort);
+    }
+  }
+
+  @Override
+  public IndexOutput createTempOutput(String prefix, String suffix, IOContext ioContext) throws IOException {
+    return new RateLimitedIndexOutput(fetchRateLimiter, super.createTempOutput(prefix, suffix, ioContext));
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
new file mode 100644
index 0000000..f03a5c3
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -0,0 +1,390 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.file.Path;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Child process with silly naive TCP socket server to handle
+ *  between-node commands, launched for each node  by TestNRTReplication. */
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class SimpleServer extends LuceneTestCase {
+
+  final static Set<Thread> clientThreads = Collections.synchronizedSet(new HashSet<>());
+  final static AtomicBoolean stop = new AtomicBoolean();
+
+  /** Handles one client connection */
+  private static class ClientHandler extends Thread {
+
+    // We hold this just so we can close it to exit the process:
+    private final ServerSocket ss;
+    private final Socket socket;
+    private final Node node;
+    private final int bufferSize;
+
+    public ClientHandler(ServerSocket ss, Node node, Socket socket) {
+      this.ss = ss;
+      this.node = node;
+      this.socket = socket;
+      this.bufferSize = TestUtil.nextInt(random(), 128, 65536);
+      if (Node.VERBOSE_CONNECTIONS) {
+        node.message("new connection socket=" + socket);
+      }
+    }
+
+    @Override
+    public void run() {
+      boolean success = false;
+      try {
+        //node.message("using stream buffer size=" + bufferSize);
+        InputStream is = new BufferedInputStream(socket.getInputStream(), bufferSize);
+        DataInput in = new InputStreamDataInput(is);
+        BufferedOutputStream bos = new BufferedOutputStream(socket.getOutputStream(), bufferSize);
+        DataOutput out = new OutputStreamDataOutput(bos);
+
+        if (node instanceof SimplePrimaryNode) {
+          ((SimplePrimaryNode) node).handleOneConnection(random(), ss, stop, is, socket, in, out, bos);
+        } else {
+          ((SimpleReplicaNode) node).handleOneConnection(ss, stop, is, socket, in, out, bos);
+        }
+
+        bos.flush();
+        if (Node.VERBOSE_CONNECTIONS) {
+          node.message("bos.flush done");
+        }
+
+        success = true;
+      } catch (Throwable t) {
+        if (t instanceof SocketException == false) {
+          node.message("unexpected exception handling client connection:");
+          t.printStackTrace(System.out);
+          // Test should fail with this:
+          throw new RuntimeException(t);
+        } else {
+          node.message("SocketException " + t + " handling client connection; ignoring");
+        }
+      } finally {
+        if (success) {
+          try {
+            IOUtils.close(socket);
+          } catch (IOException ioe) {
+            throw new RuntimeException(ioe);
+          }
+        } else {
+          IOUtils.closeWhileHandlingException(socket);
+        }
+      }
+      if (Node.VERBOSE_CONNECTIONS) {
+        node.message("socket.close done");
+      }
+    }
+  }
+
+  /**
+   * currently, this only works/tested on Sun and IBM.
+   */
+
+  // poached from TestIndexWriterOnJRECrash ... should we factor out to TestUtil?  seems dangerous to give it such "publicity"?
+  private static void crashJRE() {
+    final String vendor = Constants.JAVA_VENDOR;
+    final boolean supportsUnsafeNpeDereference = 
+        vendor.startsWith("Oracle") || 
+        vendor.startsWith("Sun") || 
+        vendor.startsWith("Apple");
+
+    try {
+      if (supportsUnsafeNpeDereference) {
+        try {
+          Class<?> clazz = Class.forName("sun.misc.Unsafe");
+          java.lang.reflect.Field field = clazz.getDeclaredField("theUnsafe");
+          field.setAccessible(true);
+          Object o = field.get(null);
+          Method m = clazz.getMethod("putAddress", long.class, long.class);
+          m.invoke(o, 0L, 0L);
+        } catch (Throwable e) {
+          System.out.println("Couldn't kill the JVM via Unsafe.");
+          e.printStackTrace(System.out); 
+        }
+      }
+
+      // Fallback attempt to Runtime.halt();
+      Runtime.getRuntime().halt(-1);
+    } catch (Exception e) {
+      System.out.println("Couldn't kill the JVM.");
+      e.printStackTrace(System.out); 
+    }
+
+    // We couldn't get the JVM to crash for some reason.
+    throw new RuntimeException("JVM refuses to die!");
+  }
+
+  static void writeFilesMetaData(DataOutput out, Map<String,FileMetaData> files) throws IOException {
+    out.writeVInt(files.size());
+    for(Map.Entry<String,FileMetaData> ent : files.entrySet()) {
+      out.writeString(ent.getKey());
+
+      FileMetaData fmd = ent.getValue();
+      out.writeVLong(fmd.length);
+      out.writeVLong(fmd.checksum);
+      out.writeVInt(fmd.header.length);
+      out.writeBytes(fmd.header, 0, fmd.header.length);
+      out.writeVInt(fmd.footer.length);
+      out.writeBytes(fmd.footer, 0, fmd.footer.length);
+    }
+  }
+
+  static Map<String,FileMetaData> readFilesMetaData(DataInput in) throws IOException {
+    int fileCount = in.readVInt();
+    //System.out.println("readFilesMetaData: fileCount=" + fileCount);
+    Map<String,FileMetaData> files = new HashMap<>();
+    for(int i=0;i<fileCount;i++) {
+      String fileName = in.readString();
+      //System.out.println("readFilesMetaData: fileName=" + fileName);
+      long length = in.readVLong();
+      long checksum = in.readVLong();
+      byte[] header = new byte[in.readVInt()];
+      in.readBytes(header, 0, header.length);
+      byte[] footer = new byte[in.readVInt()];
+      in.readBytes(footer, 0, footer.length);
+      files.put(fileName, new FileMetaData(header, footer, length, checksum));
+    }
+    return files;
+  }
+
+  /** Pulls CopyState off the wire */
+  static CopyState readCopyState(DataInput in) throws IOException {
+
+    // Decode a new CopyState
+    byte[] infosBytes = new byte[in.readVInt()];
+    in.readBytes(infosBytes, 0, infosBytes.length);
+
+    long gen = in.readVLong();
+    long version = in.readVLong();
+    Map<String,FileMetaData> files = readFilesMetaData(in);
+
+    int count = in.readVInt();
+    Set<String> completedMergeFiles = new HashSet<>();
+    for(int i=0;i<count;i++) {
+      completedMergeFiles.add(in.readString());
+    }
+    long primaryGen = in.readVLong();
+
+    return new CopyState(files, version, gen, infosBytes, completedMergeFiles, primaryGen, null);
+  }
+
+  public void test() throws Exception {
+
+    int id = Integer.parseInt(System.getProperty("tests.nrtreplication.nodeid"));
+    Thread.currentThread().setName("init child " + id);
+    Path indexPath = Paths.get(System.getProperty("tests.nrtreplication.indexpath"));
+    boolean isPrimary = System.getProperty("tests.nrtreplication.isPrimary") != null;
+    int primaryTCPPort;
+    long forcePrimaryVersion;
+    if (isPrimary == false) {
+      forcePrimaryVersion = -1;
+      primaryTCPPort = Integer.parseInt(System.getProperty("tests.nrtreplication.primaryTCPPort"));
+    } else {
+      primaryTCPPort = -1;
+      forcePrimaryVersion = Long.parseLong(System.getProperty("tests.nrtreplication.forcePrimaryVersion"));
+    }
+    long primaryGen = Long.parseLong(System.getProperty("tests.nrtreplication.primaryGen"));
+    Node.globalStartNS = Long.parseLong(System.getProperty("tests.nrtreplication.startNS"));
+
+    boolean doRandomCrash = isPrimary ? TestNRTReplication.DO_CRASH_PRIMARY : TestNRTReplication.DO_CRASH_REPLICA;
+    boolean doRandomClose = isPrimary ? false : TestNRTReplication.DO_CLOSE_REPLICA;
+
+    // Create server socket that we listen for incoming requests on:
+    try (final ServerSocket ss = new ServerSocket(0)) {
+
+      int tcpPort = ((InetSocketAddress) ss.getLocalSocketAddress()).getPort();
+      System.out.println("\nPORT: " + tcpPort);
+      final Node node;
+      if (isPrimary) {
+        node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null);
+        System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
+      } else {
+        node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+      }
+      System.out.println("\nINFOS VERSION: " + node.getCurrentSearchingVersion());
+
+      if (doRandomClose || doRandomCrash) {
+        final int waitForMS;
+        if (isPrimary) {
+          waitForMS = TestUtil.nextInt(random(), 20000, 60000);
+        } else {
+          waitForMS = TestUtil.nextInt(random(), 5000, 60000);
+        }
+
+        boolean doClose;
+        if (doRandomCrash == false) {
+          doClose = true;
+        } else if (doRandomClose) {
+          doClose = random().nextBoolean();
+        } else {
+          doClose = false;
+        }
+
+        if (doClose) {
+          node.message("top: will close after " + (waitForMS/1000.0) + " seconds");
+        } else {
+          node.message("top: will crash after " + (waitForMS/1000.0) + " seconds");
+        }
+
+        Thread t = new Thread() {
+            @Override
+            public void run() {
+              long endTime = System.nanoTime() + waitForMS*1000000L;
+              while (System.nanoTime() < endTime) {
+                try {
+                  Thread.sleep(10);
+                } catch (InterruptedException e) {
+                }
+                if (stop.get()) {
+                  break;
+                }
+              }
+
+              if (stop.get() == false) {
+                if (doClose) {
+                  try {
+                    node.message("top: now force close server socket after " + (waitForMS/1000.0) + " seconds");
+                    node.state = "top-closing";
+                    ss.close();
+                  } catch (IOException ioe) {     
+                    throw new RuntimeException(ioe);
+                  }
+                } else {        
+                  node.message("top: now crash JVM after " + (waitForMS/1000.0) + " seconds");
+                  crashJRE();
+                }
+              }
+            }
+          };
+
+        if (isPrimary) {
+          t.setName("crasher P" + id);
+        } else {
+          t.setName("crasher R" + id);
+        }
+
+        // So that if node exits naturally, this thread won't prevent process exit:
+        t.setDaemon(true);
+        t.start();
+      }
+      System.out.println("\nNODE STARTED");
+
+      //List<Thread> clientThreads = new ArrayList<>();
+
+      // Naive thread-per-connection server:
+      while (true) {
+        Socket socket;
+        try {
+          socket = ss.accept();
+        } catch (SocketException se) {
+          // when ClientHandler closes our ss we will hit this
+          node.message("top: server socket exc; now exit");
+          break;
+        }
+        Thread thread = new ClientHandler(ss, node, socket);
+        thread.setDaemon(true);
+        thread.start();
+
+        clientThreads.add(thread);
+
+        // Prune finished client threads:
+        Iterator<Thread> it = clientThreads.iterator();
+        while (it.hasNext()) {
+          Thread t = it.next();
+          if (t.isAlive() == false) {
+            it.remove();
+          }
+        }
+        //node.message(clientThreads.size() + " client threads are still alive");
+      }
+
+      stop.set(true);
+
+      // Make sure all client threads are done, else we get annoying (yet ultimately "harmless") messages about threads still running /
+      // lingering for them to finish from the child processes:
+      for(Thread clientThread : clientThreads) {
+        node.message("top: join clientThread=" + clientThread);
+        clientThread.join();
+        node.message("top: done join clientThread=" + clientThread);
+      }
+      node.message("done join all client threads; now close node");
+      node.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
new file mode 100644
index 0000000..d409ffc
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
@@ -0,0 +1,250 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.RAMOutputStream;
+
+/** This is a stupid yet functional transaction log: it never fsync's, never prunes, it's over-synchronized, it hard-wires id field name to "docid", can
+ *  only handle specific docs/fields used by this test, etc.  It's just barely enough to show how a translog could work on top of NRT
+ *  replication to guarantee no data loss when nodes crash */
+
+class SimpleTransLog implements Closeable {
+
+  final FileChannel channel;
+  final RAMOutputStream buffer = new RAMOutputStream();
+  final byte[] intBuffer = new byte[4];
+  final ByteBuffer intByteBuffer = ByteBuffer.wrap(intBuffer);
+
+  private final static byte OP_ADD_DOCUMENT = (byte) 0;
+  private final static byte OP_UPDATE_DOCUMENT = (byte) 1;
+  private final static byte OP_DELETE_DOCUMENTS = (byte) 2;
+
+  public SimpleTransLog(Path path) throws IOException {
+    channel = FileChannel.open(path, StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
+  }
+
+  public synchronized long getNextLocation() throws IOException {
+    return channel.position();
+  }
+
+  /** Appends an addDocument op */
+  public synchronized long addDocument(String id, Document doc) throws IOException {
+    assert buffer.getFilePointer() == 0;
+    buffer.writeByte(OP_ADD_DOCUMENT);
+    encode(id, doc);
+    return flushBuffer();
+  }
+
+  /** Appends an updateDocument op */
+  public synchronized long updateDocument(String id, Document doc) throws IOException {
+    assert buffer.getFilePointer() == 0;
+    buffer.writeByte(OP_UPDATE_DOCUMENT);
+    encode(id, doc);
+    return flushBuffer();
+  }
+
+  /** Appends a deleteDocuments op */
+  public synchronized long deleteDocuments(String id) throws IOException {
+    assert buffer.getFilePointer() == 0;
+    buffer.writeByte(OP_DELETE_DOCUMENTS);
+    buffer.writeString(id);
+    return flushBuffer();
+  }
+
+  /** Writes buffer to the file and returns the start position. */
+  private synchronized long flushBuffer() throws IOException {
+    long pos = channel.position();
+    int len = (int) buffer.getFilePointer();
+    byte[] bytes = new byte[len];
+    buffer.writeTo(bytes, 0);
+    buffer.reset();
+
+    intBuffer[0] = (byte) (len >> 24);
+    intBuffer[1] = (byte) (len >> 16);
+    intBuffer[2] = (byte) (len >> 8);
+    intBuffer[3] = (byte) len;
+    intByteBuffer.limit(4);
+    intByteBuffer.position(0);
+
+    writeBytesToChannel(intByteBuffer);
+    writeBytesToChannel(ByteBuffer.wrap(bytes));
+
+    return pos;
+  }
+
+  private void writeBytesToChannel(ByteBuffer src) throws IOException {
+    int left = src.limit();
+    while (left != 0) {
+      left -= channel.write(src);
+    }
+  }
+
+  private void readBytesFromChannel(long pos, ByteBuffer dest) throws IOException {
+    int left = dest.limit() - dest.position();
+    long end = pos + left;
+    while (pos < end) {
+      int inc = channel.read(dest, pos);
+      if (inc < 0) {
+        throw new EOFException();
+      }
+      pos += inc;
+    }
+  }
+
+  /** Replays ops between start and end location against the provided writer.  Can run concurrently with ongoing operations. */
+  public void replay(NodeProcess primary, long start, long end) throws IOException {
+    try (Connection c = new Connection(primary.tcpPort)) {
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+      byte[] intBuffer = new byte[4];
+      ByteBuffer intByteBuffer = ByteBuffer.wrap(intBuffer);
+      ByteArrayDataInput in = new ByteArrayDataInput();
+
+      long pos = start;
+      while (pos < end) {
+        intByteBuffer.position(0);
+        intByteBuffer.limit(4);
+        readBytesFromChannel(pos, intByteBuffer);
+        pos += 4;
+        int len = ((intBuffer[0] & 0xff) << 24) |
+          (intBuffer[1] & 0xff) << 16 |
+          (intBuffer[2] & 0xff) << 8 |
+          (intBuffer[3] & 0xff);
+
+        byte[] bytes = new byte[len];
+        readBytesFromChannel(pos, ByteBuffer.wrap(bytes));
+        pos += len;
+
+        in.reset(bytes);
+        
+        byte op = in.readByte();
+        //System.out.println("xlog: replay op=" + op);
+        switch (op) {
+        case 0:
+          // We replay add as update:
+          replayAddDocument(c, primary, in);
+          break;
+
+        case 1:
+          // We replay add as update:
+          replayAddDocument(c, primary, in);
+          break;
+
+        case 2:
+          replayDeleteDocuments(c, primary, in);
+          break;
+
+        default:
+          throw new CorruptIndexException("invalid operation " + op, in);
+        }
+      }
+      assert pos == end;
+      //System.out.println("xlog: done replay");
+      c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
+      c.flush();
+      //System.out.println("xlog: done flush");
+      c.in.readByte();
+      //System.out.println("xlog: done readByte");
+    }
+  }
+
+  private void replayAddDocument(Connection c, NodeProcess primary, DataInput in) throws IOException {
+    String id = in.readString();
+
+    Document doc = new Document();
+    doc.add(new StringField("docid", id, Field.Store.YES));
+
+    String title = readNullableString(in);
+    if (title != null) {
+      doc.add(new StringField("title", title, Field.Store.NO));
+      doc.add(new TextField("titleTokenized", title, Field.Store.NO));
+    }
+    String body = readNullableString(in);
+    if (body != null) {
+      doc.add(new TextField("body", body, Field.Store.NO));
+    }
+    String marker = readNullableString(in);
+    if (marker != null) {
+      //System.out.println("xlog: replay marker=" + id);
+      doc.add(new StringField("marker", marker, Field.Store.YES));
+    }
+
+    // For both add and update originally, we use updateDocument to replay,
+    // because the doc could in fact already be in the index:
+    // nocomit what if this fails?
+    primary.addOrUpdateDocument(c, doc, false);
+  }
+
+
+  private void replayDeleteDocuments(Connection c, NodeProcess primary, DataInput in) throws IOException {
+    String id = in.readString();
+    // nocomit what if this fails?
+    primary.deleteDocument(c, id);
+  }
+
+  /** Encodes doc into buffer.  NOTE: this is NOT general purpose!  It only handles the fields used in this test! */
+  private synchronized void encode(String id, Document doc) throws IOException {
+    assert id.equals(doc.get("docid")): "id=" + id + " vs docid=" + doc.get("docid");
+    buffer.writeString(id);
+    writeNullableString(doc.get("title"));
+    writeNullableString(doc.get("body"));
+    writeNullableString(doc.get("marker"));
+  }
+
+  private synchronized void writeNullableString(String s) throws IOException {
+    if (s == null) {
+      buffer.writeByte((byte) 0);
+    } else {
+      buffer.writeByte((byte) 1);
+      buffer.writeString(s);
+    }
+  }
+
+  private String readNullableString(DataInput in) throws IOException {
+    byte b = in.readByte();
+    if (b == 0) {
+      return null;
+    } else if (b == 1) {
+      return in.readString();
+    } else {
+      throw new CorruptIndexException("invalid string lead byte " + b, in);
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    channel.close();
+  }
+}