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/11 17:42:35 UTC

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

Repository: lucene-solr
Updated Branches:
  refs/heads/master 35337e8cf -> 12b8721a4


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) {


[27/31] lucene-solr git commit: cleanup some precommit failures

Posted by mi...@apache.org.
cleanup some precommit failures


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

Branch: refs/heads/master
Commit: 564047059333ba391595f0dfb080ef2c3157d3c0
Parents: 88fea23
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 9 10:43:36 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 9 10:43:36 2016 -0500

----------------------------------------------------------------------
 .../java/org/apache/lucene/index/DirectoryReader.java    |  2 +-
 .../src/java/org/apache/lucene/index/IndexWriter.java    | 11 +++++++++--
 .../src/java/org/apache/lucene/index/SegmentInfos.java   |  2 +-
 .../org/apache/lucene/replicator/nrt/SimpleServer.java   |  2 ++
 .../apache/lucene/replicator/nrt/TestNRTReplication.java |  2 ++
 .../lucene/replicator/nrt/TestStressNRTReplication.java  |  4 +++-
 6 files changed, 18 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/56404705/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 c60e6c3..7f65d15 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
@@ -93,7 +93,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
    * 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
+   * reader directly in heap
    *
    * @see #open(IndexWriter)
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/56404705/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 bde5686..eee17fc 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -438,7 +438,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
             anyChanges |= maybeApplyDeletes(applyAllDeletes);
             if (writeAllDeletes) {
               // Must move the deletes to disk:
-              System.out.println("IW: now readerPool.commit");
               readerPool.commit(segmentInfos);
             }
 
@@ -1174,7 +1173,9 @@ 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. */
+  /** If {@link SegmentInfos#getVersion} is below {@code newVersion} then update it to this value.
+   *
+   * @lucene.internal */
   public synchronized void advanceSegmentInfosVersion(long newVersion) {
     ensureOpen();
     if (segmentInfos.getVersion() < newVersion) {
@@ -2897,6 +2898,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     setCommitData(commitUserData, true);
   }
 
+  /**
+   * Sets the commit user data map, controlling whether to advance the {@link SegmentInfos#getVersion}.
+   *
+   * @see #setCommitData(Map)
+   *
+   * @lucene.internal */
   public final synchronized void setCommitData(Map<String,String> commitUserData, boolean doIncrementVersion) {
     segmentInfos.setUserData(new HashMap<>(commitUserData), doIncrementVersion);
     changeCount.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/56404705/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 3b95f25..324f8dd 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -887,7 +887,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     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);
+    //System.out.println(Thread.currentThread().getName() + ": SIS.setVersion change from " + version + " to " + newVersion);
     version = newVersion;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/56404705/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 3bad39b..49d2ce2 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
@@ -46,12 +46,14 @@ 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.SuppressForbidden;
 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")
+@SuppressForbidden(reason = "We need Unsafe to actually crush :-)")
 public class SimpleServer extends LuceneTestCase {
 
   final static Set<Thread> clientThreads = Collections.synchronizedSet(new HashSet<>());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/56404705/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 262e68e..773390e 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
@@ -35,6 +35,7 @@ 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.SuppressForbidden;
 import org.apache.lucene.util.TestUtil;
 
 import com.carrotsearch.randomizedtesting.SeedUtils;
@@ -53,6 +54,7 @@ public class TestNRTReplication extends LuceneTestCase {
   LineFileDocs docs;
 
   /** Launches a child "server" (separate JVM), which is either primary or replica node */
+  @SuppressForbidden(reason = "ProcessBuilder requires java.io.File for CWD")
   private NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, long forcePrimaryVersion, boolean willCrash) throws IOException {
     List<String> cmd = new ArrayList<>();
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/56404705/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 d9222c0..6300589 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
@@ -50,9 +50,10 @@ 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.LuceneTestCase;
+import org.apache.lucene.util.SuppressForbidden;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.ThreadInterruptedException;
 
@@ -502,6 +503,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
   }
 
   /** Launches a child "server" (separate JVM), which is either primary or replica node */
+  @SuppressForbidden(reason = "ProcessBuilder requires java.io.File for CWD")
   NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
     nodeTimeStamps[id] = System.nanoTime();
     List<String> cmd = new ArrayList<>();


[26/31] lucene-solr git commit: turn off debug prints; add lucene.experimental to javadocs

Posted by mi...@apache.org.
turn off debug prints; add lucene.experimental to javadocs


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

Branch: refs/heads/master
Commit: 88fea2317629d545a1c8ccee18384d38b305215e
Parents: c03bb2e
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 9 05:22:16 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 9 05:22:16 2016 -0500

----------------------------------------------------------------------
 .../core/src/java/org/apache/lucene/index/SegmentInfos.java  | 2 +-
 .../src/java/org/apache/lucene/replicator/nrt/CopyJob.java   | 5 +++--
 .../src/java/org/apache/lucene/replicator/nrt/CopyState.java | 4 +++-
 .../java/org/apache/lucene/replicator/nrt/FileMetaData.java  | 4 +++-
 .../src/java/org/apache/lucene/replicator/nrt/Node.java      | 4 +++-
 .../lucene/replicator/nrt/NodeCommunicationException.java    | 6 ++++++
 .../java/org/apache/lucene/replicator/nrt/PrimaryNode.java   | 5 ++++-
 .../org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java | 4 +---
 .../java/org/apache/lucene/replicator/nrt/ReplicaNode.java   | 4 +++-
 .../lucene/replicator/nrt/SegmentInfosSearcherManager.java   | 4 +++-
 .../lucene/replicator/nrt/TestStressNRTReplication.java      | 8 +++-----
 .../java/org/apache/lucene/store/MockDirectoryWrapper.java   | 2 +-
 .../org/apache/solr/core/StandardIndexReaderFactory.java     | 1 -
 13 files changed, 34 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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 40f03e8..3b95f25 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -485,7 +485,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     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);
+    //System.out.println(Thread.currentThread().getName() + ": now write " + out.getName() + " with version=" + version);
 
     out.writeLong(version); 
     out.writeInt(counter); // write counter

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
index 217c2c0..1e63d1f 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
@@ -30,8 +30,9 @@ 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. */
-
+ *  This notifies the caller via OnceDone when the job finishes or failed.
+ *
+ * @lucene.experimental */
 public abstract class CopyJob implements Comparable<CopyJob> {
   private final static AtomicLong counter = new AtomicLong();
   protected final ReplicaNode dest;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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
index c19fabc..630c3a9 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
@@ -24,7 +24,9 @@ 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. */
+/** Holds incRef'd file level details for one point-in-time segment infos on the primary node.
+ *
+ * @lucene.experimental */
 public class CopyState {
 
   public final Map<String,FileMetaData> files;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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
index aca408c..427b1d1 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
@@ -17,7 +17,9 @@ package org.apache.lucene.replicator.nrt;
  * 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". */
+/** Holds metadata details about a single file that we use to confirm two files (one remote, one local) are in fact "identical".
+ *
+ * @lucene.experimental */
 
 class FileMetaData {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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..3417572 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
@@ -44,7 +44,9 @@ 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}. */
+/** Common base class for {@link PrimaryNode} and {@link ReplicaNode}.
+ *
+ * @lucene.experimental */
 
 abstract class Node implements Closeable {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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
index 67a9d0a..d286f3d 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
@@ -17,6 +17,12 @@ package org.apache.lucene.replicator.nrt;
  * limitations under the License.
  */
 
+/**
+ * Should be thrown by subclasses of {@link PrimaryNode} and {@link ReplicaNode} if a non-fatal exception
+ * occurred while communicating between nodes.
+ *
+ * @lucene.experimental
+ */
 public class NodeCommunicationException extends RuntimeException {
   public NodeCommunicationException(String when, Throwable cause) {
     super(when);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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 3cff95e..7343b69 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
@@ -46,7 +46,10 @@ import org.apache.lucene.util.ThreadInterruptedException;
  * 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. */
+/** Node that holds an IndexWriter, indexing documents into its local index.
+ *
+ * @lucene.experimental */
+
 public abstract class PrimaryNode extends Node {
 
   // Current NRT segment infos, incRef'd with IndexWriter.deleter:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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
index 652b3af..41e0a6f 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
@@ -34,8 +34,6 @@ 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 Directory dir;
@@ -50,7 +48,7 @@ class ReplicaFileDeleter {
    *  (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 {
+  private static boolean slowFileExists(Directory dir, String fileName) throws IOException {
     try {
       dir.openInput(fileName, IOContext.DEFAULT).close();
       return true;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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 62827e8..4caf5cf 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
@@ -50,7 +50,9 @@ 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 */
+/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files.
+ * 
+ *  @lucene.experimental */
 
 abstract class ReplicaNode extends Node {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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
index 72ed921..bae2606 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
@@ -36,7 +36,9 @@ 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. */
+ *  nrt replication to another index.
+ *
+ * @lucene.experimental */
 class SegmentInfosSearcherManager extends ReferenceManager<IndexSearcher> {
   private volatile SegmentInfos currentInfos;
   private final Directory dir;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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 fff59fa..d9222c0 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,6 @@ import org.apache.lucene.util.ThreadInterruptedException;
 
 import com.carrotsearch.randomizedtesting.SeedUtils;
 
-// nocommit why so many "hit SocketException during commit with R0"?
-
-// nocommit why all these NodeCommunicationExcs?
-
 /*
   TODO
     - fangs
@@ -1145,7 +1141,9 @@ public class TestStressNRTReplication extends LuceneTestCase {
             }
           } catch (IOException se) {
             // Assume primary crashed
-            message("top: indexer lost connection to primary");
+            if (c != null) {
+              message("top: indexer lost connection to primary");
+            }
             try {
               c.close();
             } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/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 aa89209..ef4397f 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
@@ -432,7 +432,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
         throw new AssertionError();
       }
 
-      if (true || LuceneTestCase.VERBOSE) {
+      if (LuceneTestCase.VERBOSE) {
         System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name);
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/88fea231/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java b/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
index 71cb012..9ad4003 100644
--- a/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
+++ b/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
@@ -26,7 +26,6 @@ import org.apache.lucene.store.Directory;
  * {@link DirectoryReader}.
  * 
  * @see DirectoryReader#open(Directory)
- * @see DirectoryReader#open(IndexWriter, boolean)
  */
 public class StandardIndexReaderFactory extends IndexReaderFactory {
   


[09/31] 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/master
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);


[21/31] lucene-solr git commit: merged

Posted by mi...@apache.org.
merged


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

Branch: refs/heads/master
Commit: 30613c74039931cf5ca2ef3f8e8c2cd6b75fee1b
Parents: c14fa12 9afa560
Author: Michael McCandless <ma...@mikemccandless.com>
Authored: Mon Feb 8 14:07:56 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 14:07:56 2016 -0500

----------------------------------------------------------------------
 .../lucene/replicator/nrt/ReplicaNode.java      | 21 ++++++--
 .../org/apache/lucene/replicator/nrt/Jobs.java  |  7 ++-
 .../lucene/replicator/nrt/NodeProcess.java      | 24 +++------
 .../replicator/nrt/SimplePrimaryNode.java       |  1 +
 .../replicator/nrt/SimpleReplicaNode.java       |  5 +-
 .../replicator/nrt/TestNRTReplication.java      |  6 +--
 .../nrt/TestStressNRTReplication.java           | 53 +++++++++++++-------
 .../lucene/replicator/nrt/ThreadPumper.java     |  4 +-
 lucene/replicator/test.cmd                      |  4 +-
 .../lucene/store/MockDirectoryWrapper.java      |  1 +
 10 files changed, 74 insertions(+), 52 deletions(-)
----------------------------------------------------------------------



[11/31] 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/master
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")


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

Posted by Michael McCandless <lu...@mikemccandless.com>.
Hmm, the Jira issue is mentioned in the merge commit message:

commit 12b8721a44dbd1fbc7878fa37186c16cf6045401
Merge: 35337e8 20c38e7
Author: Mike McCandless <mi...@apache.org>
Date:   Thu Feb 11 11:20:18 2016 -0500

    Merge branch 'jira/lucene-5438-nrt-replication'

But you're right, I should have inserted the LUCENE-5438 in there so
the commit would be added to the Jira ... hrmph.  Maybe we can improve
the "add comments to jira on commit" to recognize how we name
branches?

For this issue I'll go add a comment on the issue.

Mike McCandless

http://blog.mikemccandless.com


On Thu, Feb 11, 2016 at 12:20 PM, Steve Rowe <sa...@gmail.com> wrote:
> Mike,
>
> It looks like you committed this work to master without the JIRA number in the log?  Seems like a mistake?
>
> Or maybe I’m misinterpreting this push.  The switch to git has increased the commit list volume many fold...
>
> --
> Steve
> www.lucidworks.com
>
>> On Feb 11, 2016, at 8:42 AM, mikemccand@apache.org wrote:
>>
>> Repository: lucene-solr
>> Updated Branches:
>>  refs/heads/master 35337e8cf -> 12b8721a4
>>
>>
>> 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) {
>>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: dev-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


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

Posted by Steve Rowe <sa...@gmail.com>.
Mike,

It looks like you committed this work to master without the JIRA number in the log?  Seems like a mistake?

Or maybe I’m misinterpreting this push.  The switch to git has increased the commit list volume many fold...

--
Steve
www.lucidworks.com

> On Feb 11, 2016, at 8:42 AM, mikemccand@apache.org wrote:
> 
> Repository: lucene-solr
> Updated Branches:
>  refs/heads/master 35337e8cf -> 12b8721a4
> 
> 
> 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) {
> 


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


[30/31] lucene-solr git commit: mark tests as Nightly

Posted by mi...@apache.org.
mark tests as Nightly


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

Branch: refs/heads/master
Commit: 20c38e7b875afd51f6f567e6bb90c6b16daa044c
Parents: 3a47dd2
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Feb 11 04:25:38 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Feb 11 04:25:38 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/replicator/nrt/TestNRTReplication.java | 11 +++++++++++
 1 file changed, 11 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/20c38e7b/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 773390e..955d84e 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
@@ -32,6 +32,7 @@ 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.Nightly;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.lucene.util.LuceneTestCase;
@@ -202,6 +203,7 @@ public class TestNRTReplication extends LuceneTestCase {
     docs.close();
   }
 
+  @Nightly
   public void testReplicateDeleteAllDocuments() throws Exception {
 
     Path primaryPath = createTempDir("primary");
@@ -272,6 +274,7 @@ public class TestNRTReplication extends LuceneTestCase {
     primary.close();
   }
 
+  @Nightly
   public void testReplicateForceMerge() throws Exception {
 
     Path primaryPath = createTempDir("primary");
@@ -321,6 +324,7 @@ public class TestNRTReplication extends LuceneTestCase {
   }
 
   // Start up, index 10 docs, replicate, but crash and restart the replica without committing it:
+  @Nightly
   public void testReplicaCrashNoCommit() throws Exception {
 
     Path primaryPath = createTempDir("primary");
@@ -366,6 +370,7 @@ public class TestNRTReplication extends LuceneTestCase {
   }
 
   // Start up, index 10 docs, replicate, commit, crash and restart the replica
+  @Nightly
   public void testReplicaCrashWithCommit() throws Exception {
 
     Path primaryPath = createTempDir("primary");
@@ -408,6 +413,7 @@ public class TestNRTReplication extends LuceneTestCase {
   }
 
   // Start up, index 10 docs, replicate, commit, crash, index more docs, replicate, then restart the replica
+  @Nightly
   public void testIndexingWhileReplicaIsDown() throws Exception {
 
     Path primaryPath = createTempDir("primary");
@@ -472,6 +478,7 @@ public class TestNRTReplication extends LuceneTestCase {
   }
  
   // Crash primary and promote a replica
+  @Nightly
   public void testCrashPrimary1() throws Exception {
 
     Path path1 = createTempDir("1");
@@ -515,6 +522,7 @@ public class TestNRTReplication extends LuceneTestCase {
   }
 
   // Crash primary and then restart it
+  @Nightly
   public void testCrashPrimary2() throws Exception {
 
     Path path1 = createTempDir("1");
@@ -581,6 +589,7 @@ public class TestNRTReplication extends LuceneTestCase {
   }
 
   // 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
+  @Nightly
   public void testCrashPrimary3() throws Exception {
 
     Path path1 = createTempDir("1");
@@ -639,6 +648,7 @@ public class TestNRTReplication extends LuceneTestCase {
     replica.close();
   }
 
+  @Nightly
   public void testCrashPrimaryWhileCopying() throws Exception {
 
     Path path1 = createTempDir("1");
@@ -747,6 +757,7 @@ public class TestNRTReplication extends LuceneTestCase {
     primary.close();
   }
 
+  @Nightly
   public void testFullClusterCrash() throws Exception {
 
     Path path1 = createTempDir("1");


[03/31] 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);
+    }
+  }
+}


[13/31] 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
----------------------------------------------------------------------


[05/31] 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/master
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) {


[12/31] 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/master
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;
       }
     }
 


[19/31] lucene-solr git commit: fix some nocommits; fix one stress test failure

Posted by mi...@apache.org.
fix some nocommits; fix one stress test failure


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

Branch: refs/heads/master
Commit: 9afa560054aad72b8cc623b130745bc8778d1d5c
Parents: f0f4278
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Feb 8 14:07:04 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 14:07:04 2016 -0500

----------------------------------------------------------------------
 .../lucene/replicator/nrt/ReplicaNode.java      | 21 ++++++--
 .../org/apache/lucene/replicator/nrt/Jobs.java  | 18 +++++--
 .../lucene/replicator/nrt/NodeProcess.java      | 24 +++------
 .../replicator/nrt/SimplePrimaryNode.java       |  1 +
 .../replicator/nrt/SimpleReplicaNode.java       |  5 +-
 .../replicator/nrt/TestNRTReplication.java      |  6 +--
 .../nrt/TestStressNRTReplication.java           | 53 +++++++++++++-------
 .../lucene/replicator/nrt/ThreadPumper.java     |  4 +-
 lucene/replicator/test.cmd                      |  4 +-
 .../lucene/store/MockDirectoryWrapper.java      |  1 +
 10 files changed, 84 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 e191caf..62827e8 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
@@ -462,12 +462,19 @@ abstract class ReplicaNode extends Node {
 
   /** 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 {
+  public synchronized CopyJob newNRTPoint(long newPrimaryGen, long version) throws IOException {
 
     if (isClosed()) {
       throw new AlreadyClosedException("this replica is closed: state=" + state);
     }
 
+    // Cutover (possibly) to new primary first, so we discard any pre-copied merged segments up front, before checking for which files need
+    // copying.  While it's possible the pre-copied merged segments could still be useful to us, in the case that the new primary is either
+    // the same primary (just e.g. rebooted), or a promoted replica that had a newer NRT point than we did that included the pre-copied
+    // merged segments, it's still a bit risky to rely solely on checksum/file length to catch the difference, so we defensively discard
+    // here and re-copy in that case:
+    maybeNewPrimary(newPrimaryGen);
+
     // Caller should not "publish" us until we have finished .start():
     assert mgr != null;
 
@@ -520,9 +527,9 @@ abstract class ReplicaNode extends Node {
       return null;
     }
 
+    assert newPrimaryGen == job.getCopyState().primaryGen;
+
     Collection<String> newNRTFiles = job.getFileNames();
-    long newPrimaryGen = job.getCopyState().primaryGen;
-    maybeNewPrimary(newPrimaryGen);
 
     message("top: newNRTPoint: job files=" + newNRTFiles);
 
@@ -608,9 +615,15 @@ abstract class ReplicaNode extends Node {
   }
 
   /** Called when the primary changed */
-  protected synchronized void maybeNewPrimary(long newPrimaryGen) {
+  protected synchronized void maybeNewPrimary(long newPrimaryGen) throws IOException {
     if (newPrimaryGen != lastPrimaryGen) {
       message("top: now change lastPrimaryGen from " + lastPrimaryGen + " to " + newPrimaryGen + " pendingMergeFiles=" + pendingMergeFiles);
+
+      message("top: delete if no ref pendingMergeFiles=" + pendingMergeFiles);
+      for(String fileName : pendingMergeFiles) {
+        deleter.deleteIfNoRef(fileName);
+      }
+
       assert newPrimaryGen > lastPrimaryGen: "newPrimaryGen=" + newPrimaryGen + " vs lastPrimaryGen=" + lastPrimaryGen;
       lastPrimaryGen = newPrimaryGen;
       pendingMergeFiles.clear();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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
index a0b9535..f75a027 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
@@ -80,7 +80,12 @@ class Jobs extends Thread implements Closeable {
         } else {
           node.message("AlreadyClosedException during job.visit job=" + topJob + "; now cancel");
         }
-        topJob.cancel("unexpected exception in visit", t);
+        try {
+          topJob.cancel("unexpected exception in visit", t);
+        } catch (Throwable t2) {
+          node.message("ignore exception calling cancel: " + t2);
+          t2.printStackTrace(System.out);
+        }
         try {
           topJob.onceDone.run(topJob);
         } catch (Throwable t2) {
@@ -101,6 +106,7 @@ class Jobs extends Thread implements Closeable {
           topJob.onceDone.run(topJob);
         } catch (Throwable t) {
           node.message("ignore exception calling OnceDone: " + t);
+          t.printStackTrace(System.out);
         }
       }
     }
@@ -112,11 +118,17 @@ class Jobs extends Thread implements Closeable {
       while (queue.isEmpty() == false) {
         SimpleCopyJob job = (SimpleCopyJob) queue.poll();
         node.message("top: Jobs: now cancel job=" + job);
-        job.cancel("jobs closing", null);
+        try {
+          job.cancel("jobs closing", null);
+        } catch (Throwable t) {
+          node.message("ignore exception calling cancel");
+          t.printStackTrace(System.out);
+        }
         try {
           job.onceDone.run(job);
         } catch (Throwable t) {
-          node.message("ignore exception calling OnceDone: " + t);
+          node.message("ignore exception calling OnceDone");
+          t.printStackTrace(System.out);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 9d8b764..a0bfb78 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
@@ -86,7 +86,7 @@ class NodeProcess implements Closeable {
     }
   }
 
-  public boolean commit() {
+  public boolean commit() throws IOException {
     try (Connection c = new Connection(tcpPort)) {
       c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
       c.flush();
@@ -95,36 +95,22 @@ class NodeProcess implements Closeable {
         throw new RuntimeException("commit failed");
       }
       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;
     }
   }
 
-  public void commitAsync() {
+  public void commitAsync() throws IOException {
     try (Connection c = new Connection(tcpPort)) {
       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");
     }
   }
 
-  public long getSearchingVersion() {
+  public long getSearchingVersion() throws IOException {
     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) {
-      // 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;
     }
   }
 
@@ -162,6 +148,7 @@ class NodeProcess implements Closeable {
           }
         } catch (Throwable t) {
           System.out.println("top: shutdown failed; ignoring");
+          t.printStackTrace(System.out);
         }
         try {
           p.waitFor();
@@ -178,10 +165,11 @@ class NodeProcess implements Closeable {
     }
   }
 
-  public void newNRTPoint(long version, int primaryTCPPort) throws IOException {
+  public void newNRTPoint(long version, long primaryGen, int primaryTCPPort) throws IOException {
     try (Connection c = new Connection(tcpPort)) {
       c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
       c.out.writeVLong(version);
+      c.out.writeVLong(primaryGen);
       c.out.writeInt(primaryTCPPort);
       c.flush();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 7f5634c..fe14234 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
@@ -304,6 +304,7 @@ class SimplePrimaryNode extends PrimaryNode {
           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.writeVLong(primaryGen);
           c.out.writeInt(tcpPort);
           c.flush();
           // TODO: we should use multicast to broadcast files out to replicas

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 4868338..9658ad1 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
@@ -176,10 +176,11 @@ class SimpleReplicaNode extends ReplicaNode {
       case CMD_NEW_NRT_POINT:
         {
           long version = in.readVLong();
+          long newPrimaryGen = in.readVLong();
           Thread.currentThread().setName("recv-" + version);
           curPrimaryTCPPort = in.readInt();
-          message("newNRTPoint primaryTCPPort=" + curPrimaryTCPPort);
-          newNRTPoint(version);
+          message("newNRTPoint primaryTCPPort=" + curPrimaryTCPPort + " version=" + version + " newPrimaryGen=" + newPrimaryGen);
+          newNRTPoint(newPrimaryGen, version);
         }
         break;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 2c66994..262e68e 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
@@ -356,7 +356,7 @@ public class TestNRTReplication extends LuceneTestCase {
     assertVersionAndHits(replica, 0, 0);
 
     // Ask replica to sync:
-    replica.newNRTPoint(primaryVersion1, primary.tcpPort);
+    replica.newNRTPoint(primaryVersion1, 0, primary.tcpPort);
     waitForVersionAndHits(replica, primaryVersion1, 10);
 
     replica.close();
@@ -461,7 +461,7 @@ public class TestNRTReplication extends LuceneTestCase {
     assertVersionAndHits(replica, primaryVersion1, 10);
 
     // Now ask replica to sync:
-    replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+    replica.newNRTPoint(primaryVersion2, 0, primary.tcpPort);
 
     waitForVersionAndHits(replica, primaryVersion2, 20);
 
@@ -736,7 +736,7 @@ public class TestNRTReplication extends LuceneTestCase {
     sendReplicasToPrimary(primary, replica);
 
     // Now ask replica to sync:
-    replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+    replica.newNRTPoint(primaryVersion2, 0, primary.tcpPort);
 
     // Make sure it sees all docs that were indexed while it was down:
     assertVersionAndHits(primary, primaryVersion2, 110);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 a765f11..28b15f8 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
@@ -60,14 +60,8 @@ 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
@@ -161,8 +155,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
   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. */
-  // nocommit
-  static final Integer NUM_NODES = 2;
+  static final Integer NUM_NODES = null;
 
   final AtomicBoolean failed = new AtomicBoolean();
 
@@ -214,9 +207,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
     // Silly bootstrapping:
     versionToTransLogLocation.put(0L, 0L);
 
-    // nocommit why also 1?
-    //versionToTransLogLocation.put(1L, 0L);
-
     versionToMarker.put(0L, 0);
 
     int numNodes;
@@ -334,10 +324,15 @@ public class TestStressNRTReplication extends LuceneTestCase {
       {
         NodeProcess node = nodes[random().nextInt(nodes.length)];
         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.
+          // TODO: if this node is primary, it means we committed an unpublished 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();
+          try {
+            node.commitAsync();
+          } catch (Throwable t) {
+            message("top: hit exception during commit with R" + node.id + "; skipping");
+            t.printStackTrace(System.out);
+          }
         }
       }
     }
@@ -400,7 +395,14 @@ public class TestStressNRTReplication extends LuceneTestCase {
     for (NodeProcess node : nodes) {
       if (node != null) {
         message("ask " + node + " for its current searching version");
-        long searchingVersion = node.getSearchingVersion();
+        long searchingVersion;
+        try {
+          searchingVersion = node.getSearchingVersion();
+        } catch (Throwable t) {
+          message("top: hit SocketException during getSearchingVersion with R" + node.id + "; skipping");
+          t.printStackTrace(System.out);
+          continue;
+        }
         message(node + " has searchingVersion=" + searchingVersion);
         if (searchingVersion > maxSearchingVersion) {
           maxSearchingVersion = searchingVersion;
@@ -415,8 +417,12 @@ public class TestStressNRTReplication extends LuceneTestCase {
     }
 
     message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
-    if (replicaToPromote.commit() == false) {
-      message("top: commit failed; skipping primary promotion");
+    try {
+      replicaToPromote.commit();
+    } catch (Throwable t) {
+      // Something wrong with this replica; skip it:
+      message("top: hit exception during commit with R" + replicaToPromote.id + "; skipping");
+      t.printStackTrace(System.out);
       return;
     }
 
@@ -478,8 +484,9 @@ public class TestStressNRTReplication extends LuceneTestCase {
     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");
+      message("top: replay xlog failed; shutdown new primary");
+      ioe.printStackTrace(System.out);
+      newPrimary.shutdown();
       return;
     }
 
@@ -1182,4 +1189,14 @@ public class TestStressNRTReplication extends LuceneTestCase {
                                      Thread.currentThread().getName(),
                                      message));
   }
+
+  static void message(String message, long localStartNS) {
+    long now = System.nanoTime();
+    System.out.println(String.format(Locale.ROOT,
+                                     "%5.3fs %5.1fs:     parent [%11s] %s",
+                                     (now-Node.globalStartNS)/1000000000.,
+                                     (now-localStartNS)/1000000000.,
+                                     Thread.currentThread().getName(),
+                                     message));
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 d74e170..73f3908 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
@@ -31,6 +31,7 @@ class ThreadPumper {
         @Override
         public void run() {
           try {
+            long startTimeNS = System.nanoTime();
             Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
             String line;
             while ((line = from.readLine()) != null) {
@@ -42,7 +43,7 @@ class ThreadPumper {
                 // Already a well-formed log output:
                 System.out.println(line);
               } else {
-                TestNRTReplication.message(line);
+                TestStressNRTReplication.message(line, startTimeNS);
               }
               if (line.contains("now force close server socket after")) {
                 nodeClosing.set(true);
@@ -60,4 +61,3 @@ class ThreadPumper {
     return t;
   }
 }
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/lucene/replicator/test.cmd
----------------------------------------------------------------------
diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd
index f636a61..18045ce 100644
--- a/lucene/replicator/test.cmd
+++ b/lucene/replicator/test.cmd
@@ -1,3 +1 @@
-python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 3
-
-#  -mult 4 -nightly
+python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 3 -verbose -mult 4 -nightly

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9afa5600/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 a36d6d4..aa89209 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
@@ -852,6 +852,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
           
         // TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles
         if (assertNoUnreferencedFilesOnClose) {
+          System.out.println("MDW: now assert no unref'd files at close");
 
           // now look for unreferenced files: discount ones that we tried to delete but could not
           Set<String> allFiles = new HashSet<>(Arrays.asList(listAll()));


[16/31] lucene-solr git commit: fix a few nocommits

Posted by mi...@apache.org.
fix a few nocommits


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

Branch: refs/heads/master
Commit: 6369012d332431848971c1ba6f8012ae021aa74c
Parents: bd6804b
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Feb 7 13:58:01 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Feb 7 13:58:01 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/index/IndexFileDeleter.java   | 25 ++++++++++++++++++++
 .../replicator/nrt/ReplicaFileDeleter.java      |  1 +
 .../lucene/replicator/nrt/ReplicaNode.java      |  2 --
 .../org/apache/lucene/replicator/nrt/Jobs.java  |  1 +
 .../replicator/nrt/SimpleReplicaNode.java       |  6 ++---
 .../replicator/nrt/TestNRTReplication.java      |  2 --
 .../nrt/TestStressNRTReplication.java           |  9 +++----
 .../lucene/store/MockDirectoryWrapper.java      | 19 +++++++++------
 8 files changed, 47 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
index 6886055..84e070a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
@@ -699,7 +699,32 @@ final class IndexFileDeleter implements Closeable {
       infoStream.message("IFD", "delete \"" + names + "\"");
     }
 
+    // We make two passes, first deleting any segments_N files, second deleting all the rest.  We do this so that if we throw exc or JVM
+    // crashes during deletions, we don't leave the index in an "apparently corrupt" state:
     for(String name : names) {
+      if (name.startsWith(IndexFileNames.SEGMENTS) == false) {
+        continue;
+      }
+      try {
+        directory.deleteFile(name);
+      } catch (NoSuchFileException | FileNotFoundException e) {
+        // IndexWriter should only ask us to delete files it knows it wrote, so if we hit this, something is wrong!
+
+        if (Constants.WINDOWS) {
+          // TODO: can we remove this OS-specific hacky logic?  If windows deleteFile is buggy, we should instead contain this workaround in
+          // a WindowsFSDirectory ...
+          // LUCENE-6684: we suppress this assert for Windows, since a file could be in a confusing "pending delete" state, and falsely
+          // return NSFE/FNFE
+        } else {
+          throw e;
+        }
+      }
+    }
+
+    for(String name : names) {
+      if (name.startsWith(IndexFileNames.SEGMENTS) == true) {
+        continue;
+      }
       try {
         directory.deleteFile(name);
       } catch (NoSuchFileException | FileNotFoundException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/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
index 005f938..b15fc05 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
@@ -122,6 +122,7 @@ class ReplicaFileDeleter {
       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) {
+      // nocommit remove this retry logic!  it's Directory's job now...
       if (Node.VERBOSE_FILES) {
         node.message("file " + fileName + ": delete failed: " + ioe + "; will retry later");
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/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 a7adbe2..54083b4 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
@@ -92,8 +92,6 @@ 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);
     } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/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
index 369414f..3cb2fbb 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
@@ -84,6 +84,7 @@ class Jobs extends Thread implements Closeable {
           topJob.onceDone.run(topJob);
         } catch (Throwable t2) {
           node.message("ignore exception calling OnceDone: " + t2);
+          t2.printStackTrace(System.out);
         }
         continue;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/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 2510c40..83ce6cb 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
@@ -135,9 +135,9 @@ class SimpleReplicaNode extends ReplicaNode {
     MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path);
     
     dir.setAssertNoUnrefencedFilesOnClose(true);
-    if (doCheckIndexOnClose) {
-      dir.setCheckIndexOnClose(false);
-    }
+    // nocommit
+    //dir.setCheckIndexOnClose(doCheckIndexOnClose);
+    dir.setCheckIndexOnClose(true);
 
     // 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/6369012d/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 cd98b48..2c66994 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
@@ -39,8 +39,6 @@ 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")

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/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 04bbdc1..a765f11 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
@@ -161,7 +161,8 @@ public class TestStressNRTReplication extends LuceneTestCase {
   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;
+  // nocommit
+  static final Integer NUM_NODES = 2;
 
   final AtomicBoolean failed = new AtomicBoolean();
 
@@ -980,10 +981,10 @@ 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):
+          // This can be null if primary is flushing, has already refreshed its searcher, but is e.g. still notifying replicas and hasn't
+          // yet returned the version to us, in which case this searcher thread can see the version before the main thread has added it to
+          // versionToMarker:
           Integer expectedAtLeastHitCount = versionToMarker.get(version);
-          assertNotNull("version=" + version, expectedAtLeastHitCount);
 
           if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
             try {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/6369012d/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 a5fc397..9e25889 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
@@ -230,10 +230,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
       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;
     try {
       in.renameFile(source, dest);
@@ -275,7 +271,14 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     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));
+        SegmentInfos infos;
+        try {
+          infos = SegmentInfos.readCommit(this, fileName);
+        } catch (IOException ioe) {
+          System.out.println("MDW: exception reading segment infos " + fileName + "; files: " + Arrays.toString(listAll()));
+          throw ioe;
+        }
+        knownFiles.addAll(infos.files(true));
       }
     }
 
@@ -438,7 +441,6 @@ 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<>();
@@ -451,6 +453,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
       } catch (Exception ignored) {}
     }
     corruptFiles(unSyncedFiles);
+    crashed = true;
     unSyncedFiles = new HashSet<>();
   }
 
@@ -569,6 +572,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
 
     unSyncedFiles.remove(name);
     in.deleteFile(name);
+    createdFiles.remove(name);
   }
 
   // sets the cause of the incoming ioe to be the stack
@@ -829,7 +833,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
         }
         throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks, cause);
       }
-      
+
       if (getCheckIndexOnClose()) {
         randomIOExceptionRate = 0.0;
         randomIOExceptionRateOnOpen = 0.0;
@@ -846,6 +850,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
           TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true);
           
           // TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles
+          // nocommit pull this outside of "getCheckIndexOnClose"
           if (assertNoUnreferencedFilesOnClose) {
 
             // now look for unreferenced files: discount ones that we tried to delete but could not


[22/31] lucene-solr git commit: indexer threads should stop on shutdown; don't suppress exceptions; decrease random commit frequency

Posted by mi...@apache.org.
indexer threads should stop on shutdown; don't suppress exceptions; decrease random commit frequency


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

Branch: refs/heads/master
Commit: ae2b58c03c82b6faedc35698542f4898892683b2
Parents: 30613c7
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Feb 8 19:15:17 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 19:15:17 2016 -0500

----------------------------------------------------------------------
 .../lucene/replicator/nrt/SimplePrimaryNode.java | 19 ++++++++++++++-----
 .../lucene/replicator/nrt/SimpleServer.java      |  3 ++-
 .../replicator/nrt/TestStressNRTReplication.java |  7 ++++---
 3 files changed, 20 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae2b58c0/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 fe14234..d3415ad 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
@@ -163,7 +163,7 @@ class SimplePrimaryNode extends PrimaryNode {
       return;
     }
 
-    message("top: warm merge " + info + " to " + replicaTCPPorts.length + " replicas: files=" + files.keySet());
+    message("top: warm merge " + info + " to " + replicaTCPPorts.length + " replicas; tcpPort=" + tcpPort + ": files=" + files.keySet());
 
     MergePreCopy preCopy = new MergePreCopy(files);
     warmingSegments.add(preCopy);
@@ -289,8 +289,8 @@ class SimplePrimaryNode extends PrimaryNode {
       // 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
+ 
+     // 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);
@@ -432,10 +432,19 @@ class SimplePrimaryNode extends PrimaryNode {
     tokenizedWithTermVectors.setStoreTermVectorPositions(true);
   }
 
-  private void handleIndexing(Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException {
+  private void handleIndexing(Socket socket, AtomicBoolean stop, InputStream is, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
     Thread.currentThread().setName("indexing");
     message("start handling indexing socket=" + socket);
     while (true) {
+      while (true) {
+        if (is.available() > 0) {
+          break;
+        }
+        if (stop.get()) {
+          return;
+        }
+        Thread.sleep(10);
+      }
       byte cmd;
       try {
         cmd = in.readByte();
@@ -587,7 +596,7 @@ class SimplePrimaryNode extends PrimaryNode {
         break;
 
       case CMD_INDEXING:
-        handleIndexing(socket, in, out, bos);
+        handleIndexing(socket, stop, is, in, out, bos);
         break;
 
       case CMD_GET_SEARCHING_VERSION:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae2b58c0/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 72e33d7..3bad39b 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
@@ -106,7 +106,8 @@ public class SimpleServer extends LuceneTestCase {
           // Test should fail with this:
           throw new RuntimeException(t);
         } else {
-          node.message("exception " + t + " handling client connection; ignoring");
+          node.message("exception handling client connection; ignoring:");
+          t.printStackTrace(System.out);
         }
       } finally {
         if (success) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/ae2b58c0/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 28b15f8..86550c5 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
@@ -155,7 +155,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
   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 Integer NUM_NODES;
 
   final AtomicBoolean failed = new AtomicBoolean();
 
@@ -321,7 +321,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
 
       // Commit a random node, primary or replica
 
-      {
+      if (random().nextInt(10) == 1) {
         NodeProcess node = nodes[random().nextInt(nodes.length)];
         if (node != null && node.nodeIsClosing.get() == false) {
           // TODO: if this node is primary, it means we committed an unpublished version (not exposed as an NRT point)... not sure it matters.
@@ -751,7 +751,8 @@ public class TestStressNRTReplication extends LuceneTestCase {
         c.flush();
         c.in.readByte();
       } catch (Throwable t) {
-        message("top: ignore exc sending replicas to primary: " + t);
+        message("top: ignore exc sending replicas to primary P" + curPrimary.id + " at tcpPort=" + curPrimary.tcpPort);
+        t.printStackTrace(System.out);
       }
     }
   }


[07/31] 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/master
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) {


[02/31] 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();
+  }
+}


[10/31] 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/master
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")


[25/31] lucene-solr git commit: Merge branch 'jira/lucene-5438-nrt-replication' of https://git-wip-us.apache.org/repos/asf/lucene-solr into jira/lucene-5438-nrt-replication

Posted by mi...@apache.org.
Merge branch 'jira/lucene-5438-nrt-replication' of https://git-wip-us.apache.org/repos/asf/lucene-solr into jira/lucene-5438-nrt-replication


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

Branch: refs/heads/master
Commit: c03bb2ecf91a8302d947e423ae770f9d5e30b9be
Parents: e7351d3 f7a56ed
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Feb 8 19:18:25 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 19:18:25 2016 -0500

----------------------------------------------------------------------
 .../lucene/replicator/nrt/SimplePrimaryNode.java | 19 ++++++++++++++-----
 .../lucene/replicator/nrt/SimpleServer.java      |  3 ++-
 .../replicator/nrt/TestStressNRTReplication.java |  5 +++--
 3 files changed, 19 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c03bb2ec/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------


[08/31] 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/master
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


[17/31] lucene-solr git commit: Merge branch 'master' into jira/lucene-5438-nrt-replication

Posted by mi...@apache.org.
Merge branch 'master' into jira/lucene-5438-nrt-replication

Conflicts:
	lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
	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/9ba62e5e
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/9ba62e5e
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/9ba62e5e

Branch: refs/heads/master
Commit: 9ba62e5e3833b99fbec4ba1775a5d1c12254556d
Parents: 6369012 1d4d9c5
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Feb 7 14:03:13 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Feb 7 14:03:13 2016 -0500

----------------------------------------------------------------------
 .../icu/segmentation/ScriptIterator.java        |   2 +-
 .../apache/lucene/index/IndexFileDeleter.java   |  36 +------
 .../org/apache/lucene/index/IndexWriter.java    |   2 +-
 .../org/apache/lucene/store/FSDirectory.java    |  15 ++-
 .../lucene/store/FileSwitchDirectory.java       |   5 +-
 .../lucene/store/NRTCachingDirectory.java       |   4 +-
 .../lucene/TestMergeSchedulerExternal.java      |  39 ++++---
 .../lucene/index/TestIndexFileDeleter.java      |  76 ++++++++++++++
 .../apache/lucene/index/TestIndexWriter.java    |  11 +-
 .../jaspell/JaspellTernarySearchTrie.java       |   2 +-
 .../lucene/store/BaseDirectoryTestCase.java     |  26 +++++
 .../lucene/store/BaseLockFactoryTestCase.java   |   9 +-
 .../lucene/store/MockDirectoryWrapper.java      | 103 +++++++++----------
 solr/CHANGES.txt                                |   6 ++
 .../solr/update/StreamingSolrClients.java       |   4 +-
 .../solr/client/solrj/io/SolrClientCache.java   |  13 ++-
 .../client/solrj/io/sql/ConnectionImpl.java     |  14 +--
 .../solrj/io/sql/DatabaseMetaDataImpl.java      |  95 ++++++++++++++---
 .../solr/client/solrj/io/sql/StatementImpl.java |  16 ++-
 .../solr/client/solrj/io/sql/JdbcTest.java      |  28 ++++-
 20 files changed, 360 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


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

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

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9ba62e5e/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 9e25889,515f012..a36d6d4
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@@ -833,12 -721,13 +833,12 @@@ public class MockDirectoryWrapper exten
          }
          throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks, cause);
        }
 -      
+       randomIOExceptionRate = 0.0;
+       randomIOExceptionRateOnOpen = 0.0;
  
-       if (getCheckIndexOnClose()) {
-         randomIOExceptionRate = 0.0;
-         randomIOExceptionRateOnOpen = 0.0;
+       if ((getCheckIndexOnClose() || assertNoUnreferencedFilesOnClose) && DirectoryReader.indexExists(this)) {
+         if (getCheckIndexOnClose()) {
  
-         if (DirectoryReader.indexExists(this)) {
            if (LuceneTestCase.VERBOSE) {
              System.out.println("\nNOTE: MockDirectoryWrapper: now crush");
            }


[24/31] lucene-solr git commit: fix test failure when primary is flushing just before crashing

Posted by mi...@apache.org.
fix test failure when primary is flushing just before crashing


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

Branch: refs/heads/master
Commit: e7351d34050b4b5cdaf4b85ec9260ddfd3af9dda
Parents: 9afa560
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Feb 8 19:18:22 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 19:18:22 2016 -0500

----------------------------------------------------------------------
 .../lucene/replicator/nrt/TestStressNRTReplication.java   | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e7351d34/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 28b15f8..f259580 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
@@ -785,7 +785,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
       try {
         while (stop.get() == false) {
           Thread.sleep(TestUtil.nextInt(random(), 50, 500));
-          message("top: restarter cycle");
+          //message("top: restarter cycle");
 
           // Randomly crash full cluster:
           if (DO_FULL_CLUSTER_CRASH && random().nextInt(500) == 17) {
@@ -921,6 +921,8 @@ public class TestStressNRTReplication extends LuceneTestCase {
           continue;
         }
 
+        boolean nodeIsPrimary = node == primary;
+
         try {
 
           Thread.currentThread().setName("Searcher node=" + node);
@@ -970,6 +972,12 @@ public class TestStressNRTReplication extends LuceneTestCase {
             if (oldHitCount == null) {
               hitCounts.put(version, hitCount);
               message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
+              if (nodeIsPrimary && version > lastPrimaryVersion) {
+                // It's possible a search request sees a new primary version because it's in the process of flushing, but then the primary
+                // crashes.  In this case we need to ensure new primary forces its version beyond this:
+                message("top: searcher: set lastPrimaryVersion=" + lastPrimaryVersion + " vs " + version);
+                lastPrimaryVersion = version;
+              }
             } else {
               // Just ensure that all nodes show the same hit count for
               // the same version, i.e. they really are replicas of one another:


[29/31] lucene-solr git commit: remove some core changes; add missing sync that caused stress test failure

Posted by mi...@apache.org.
remove some core changes; add missing sync that caused stress test failure


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

Branch: refs/heads/master
Commit: 3a47dd29bcf5d3a277a16ced687cf296c85e8996
Parents: 027bc0e
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 10 09:09:49 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 10 09:09:49 2016 -0500

----------------------------------------------------------------------
 .../lucene/store/NRTCachingDirectory.java       |  6 +-
 .../java/org/apache/lucene/util/IOUtils.java    |  3 -
 .../replicator/nrt/SimplePrimaryNode.java       | 60 ++++++++++----------
 3 files changed, 33 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a47dd29/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 908722f..22a9571 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
@@ -175,10 +175,8 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
   @Override
   public void renameFile(String source, String dest) throws IOException {
     unCache(source);
-    try {
-      cache.deleteFile(dest);
-    } catch (FileNotFoundException fnfe) {
-      // OK -- it may not exist
+    if (cache.fileNameExists(dest)) {
+      throw new IllegalArgumentException("target file " + dest + " already exists");
     }
     in.renameFile(source, dest);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/3a47dd29/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 1a26542..ce8884c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -107,9 +107,6 @@ 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/3a47dd29/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 93d20f7..3d41b32 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
@@ -679,41 +679,43 @@ class SimplePrimaryNode extends PrimaryNode {
         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;
+        synchronized(warmingSegments) {
+          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;
-          }
+            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:
+            // 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();
+            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");
           }
-          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;
 


[23/31] lucene-solr git commit: fix compilation

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


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

Branch: refs/heads/master
Commit: f7a56ed6bca6d47de1b14e1aa6c13941076eca2d
Parents: ae2b58c
Author: Mike McCandless <mi...@apache.org>
Authored: Mon Feb 8 19:16:33 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 19:16:33 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/replicator/nrt/TestStressNRTReplication.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f7a56ed6/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 86550c5..f31376b 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
@@ -155,7 +155,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
   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;
+  static final Integer NUM_NODES = null;
 
   final AtomicBoolean failed = new AtomicBoolean();
 


[28/31] lucene-solr git commit: cleanup stale TODOs; fix precommit

Posted by mi...@apache.org.
cleanup stale TODOs; fix precommit


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

Branch: refs/heads/master
Commit: 027bc0e4d65fb5089478134ae3b208a08cf5f90f
Parents: 5640470
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 9 13:59:05 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 9 13:59:05 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/index/IndexWriter.java    |  9 ++-
 .../org/apache/lucene/index/SegmentInfos.java   |  8 ++-
 .../lucene/index/StandardDirectoryReader.java   |  5 +-
 .../index/TestNumericDocValuesUpdates.java      |  1 -
 .../lucene/replicator/nrt/CopyOneFile.java      |  2 +-
 .../lucene/replicator/nrt/FileMetaData.java     |  2 +-
 .../org/apache/lucene/replicator/nrt/Node.java  | 63 +++++++++++---------
 .../lucene/replicator/nrt/PrimaryNode.java      |  8 ++-
 .../lucene/replicator/nrt/ReplicaNode.java      | 15 ++---
 .../nrt/SegmentInfosSearcherManager.java        |  1 +
 .../lucene/replicator/nrt/package-info.java     | 21 +++++++
 .../replicator/nrt/SimplePrimaryNode.java       |  2 +-
 .../replicator/nrt/SimpleReplicaNode.java       |  7 ++-
 .../nrt/TestStressNRTReplication.java           | 28 +++------
 14 files changed, 105 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 eee17fc..8886ab1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -4738,7 +4738,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     }
   }
   
-  /** @lucene.internal */
+  /** Record that the files referenced by this {@link SegmentInfos} are still in use.
+   *
+   * @lucene.internal */
   public synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
     ensureOpen();
     deleter.incRef(segmentInfos, false);
@@ -4747,7 +4749,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     }
   }
 
-  /** @lucene.internal */
+  /** Record that the files referenced by this {@link SegmentInfos} are no longer in use.  Only call this if you are sure you previously
+   *  called {@link #incRefDeleter}.
+   *
+  * @lucene.internal */
   public synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
     ensureOpen();
     deleter.decRef(segmentInfos);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 324f8dd..642b320 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -285,6 +285,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     }
   }
 
+  /** Read the commit from the provided {@link ChecksumIndexInput}. */
   public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput input, long generation) throws IOException {
 
     // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
@@ -479,6 +480,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     }
   }
 
+  /** Write ourselves to the provided {@link IndexOutput} */
   public void write(Directory directory, IndexOutput out) throws IOException {
     CodecUtil.writeIndexHeader(out, "segments", VERSION_CURRENT, 
                                StringHelper.randomId(), Long.toString(generation, Character.MAX_RADIX));
@@ -725,8 +727,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     this.counter = other.counter;
   }
 
+  /** Set the generation to be used for the next commit */
   public void setNextWriteGeneration(long generation) {
-    assert generation >= this.generation;
+    if (generation < this.generation) {
+      throw new IllegalStateException("cannot decrease generation to " + generation + " from current generation " + this.generation);
+    }
     this.generation = generation;
   }
 
@@ -843,6 +848,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
     return userData;
   }
 
+  /** Sets the commit data. */
   public void setUserData(Map<String,String> data, boolean doIncrementVersion) {
     if (data == null) {
       userData = Collections.<String,String>emptyMap();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 1261f9d..c0425c6 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
@@ -30,6 +30,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
 
+/** Default implementation of {@link DirectoryReader}. */
 public final class StandardDirectoryReader extends DirectoryReader {
 
   final IndexWriter writer;
@@ -336,7 +337,9 @@ public final class StandardDirectoryReader extends DirectoryReader {
     return segmentInfos.getVersion();
   }
 
-  /** @lucene.internal */
+  /** Return the {@link SegmentInfos} for this reader.
+   *
+   * @lucene.internal */
   public SegmentInfos getSegmentInfos() {
     return segmentInfos;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
index 26f1cbb..3366646 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java
@@ -54,7 +54,6 @@ import org.junit.Test;
 
 import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 
-
 @SuppressWarnings("resource")
 public class TestNumericDocValuesUpdates extends LuceneTestCase {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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
index e3f0f7d..faec778 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
@@ -29,7 +29,7 @@ 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 {
+public class CopyOneFile implements Closeable {
   private final DataInput in;
   private final IndexOutput out;
   private final ReplicaNode dest;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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
index 427b1d1..897d5ca 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
@@ -21,7 +21,7 @@ package org.apache.lucene.replicator.nrt;
  *
  * @lucene.experimental */
 
-class FileMetaData {
+public class FileMetaData {
 
   // Header and footer of the file must be identical between primary and replica to consider the files equal:
   public final byte[] header;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 3417572..f5853ce 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
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.Path;
 import java.util.ArrayList;
@@ -78,6 +79,9 @@ abstract class Node implements Closeable {
   /** When this node was started */
   public static final long localStartNS = System.nanoTime();
 
+  /** For debug logging */
+  protected final PrintStream printStream;
+
   // public static final long globalStartNS;
 
   // For debugging:
@@ -86,10 +90,11 @@ abstract class Node implements Closeable {
   /** 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) {
+  public Node(int id, Directory dir, SearcherFactory searcherFactory, PrintStream printStream) {
     this.id = id;
     this.dir = dir;
     this.searcherFactory = searcherFactory;
+    this.printStream = printStream;
   }
 
   @Override
@@ -99,37 +104,41 @@ abstract class Node implements Closeable {
 
   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(PrintStream printStream, String message) {
+    if (printStream != null) {
+      long now = System.nanoTime();
+      printStream.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));
-
+  public static void nodeMessage(PrintStream printStream, int id, String message) {
+    if (printStream != null) {
+      long now = System.nanoTime();
+      printStream.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));
+    if (printStream != null) {
+      long now = System.nanoTime();
+      printStream.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() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 7343b69..a045cde 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
@@ -18,6 +18,7 @@ package org.apache.lucene.replicator.nrt;
  */
 
 import java.io.IOException;
+import java.io.PrintStream;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -67,8 +68,9 @@ public abstract class PrimaryNode extends Node {
 
   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);
+  public PrimaryNode(IndexWriter writer, int id, long primaryGen, long forcePrimaryVersion,
+                     SearcherFactory searcherFactory, PrintStream printStream) throws IOException {
+    super(id, writer.getDirectory(), searcherFactory, printStream);
     message("top: now init primary");
     this.writer = writer;
     this.primaryGen = primaryGen;
@@ -115,7 +117,7 @@ public abstract class PrimaryNode extends Node {
 
     } catch (Throwable t) {
       message("init: exception");
-      t.printStackTrace(System.out);
+      t.printStackTrace(printStream);
       throw new RuntimeException(t);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 4caf5cf..e7d40ac 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
@@ -18,6 +18,7 @@ package org.apache.lucene.replicator.nrt;
  */
 
 import java.io.IOException;
+import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -54,7 +55,7 @@ import org.apache.lucene.util.IOUtils;
  * 
  *  @lucene.experimental */
 
-abstract class ReplicaNode extends Node {
+public abstract class ReplicaNode extends Node {
 
   ReplicaFileDeleter deleter;
 
@@ -79,8 +80,8 @@ abstract class ReplicaNode extends Node {
   /** Primary gen last time we successfully replicated: */
   protected long lastPrimaryGen;
 
-  public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory) throws IOException {
-    super(id, dir, searcherFactory);
+  public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory, PrintStream printStream) throws IOException {
+    super(id, dir, searcherFactory, printStream);
 
     if (dir instanceof FSDirectory && ((FSDirectory) dir).checkPendingDeletions()) {
       throw new IllegalArgumentException("Directory " + dir + " still has pending deleted files; cannot initialize IndexWriter");
@@ -98,7 +99,7 @@ abstract class ReplicaNode extends Node {
       deleter = new ReplicaFileDeleter(this, dir);
     } catch (Throwable t) {
       message("exc on init:");
-      t.printStackTrace(System.out);
+      t.printStackTrace(printStream);
       throw t;
     } finally {
       if (success == false) {
@@ -307,7 +308,7 @@ abstract class ReplicaNode extends Node {
     } catch (Throwable t) {
       if (t.getMessage().startsWith("replica cannot start") == false) {
         message("exc on start:");
-        t.printStackTrace(System.out);
+        t.printStackTrace(printStream);
       } else {
         dir.close();
       }
@@ -522,7 +523,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(printStream);
       if (state.equals("syncing")) {
         state = "idle";
       }
@@ -560,7 +561,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 exception starting CopyJob: " + nce);
-      nce.printStackTrace(System.out);
+      nce.printStackTrace(printStream);
       if (state.equals("syncing")) {
         state = "idle";
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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
index bae2606..bdb9510 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SegmentInfos;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java
new file mode 100644
index 0000000..bbbae55
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * <h1>Near-real-time replication framework</h1>
+ */
+package org.apache.lucene.replicator.nrt;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 d3415ad..93d20f7 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
@@ -116,7 +116,7 @@ class SimplePrimaryNode extends PrimaryNode {
 
   public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory,
                            boolean doFlipBitsDuringCopy, boolean doCheckIndexOnClose) throws IOException {
-    super(initWriter(id, random, indexPath, doCheckIndexOnClose), id, primaryGen, forcePrimaryVersion, searcherFactory);
+    super(initWriter(id, random, indexPath, doCheckIndexOnClose), id, primaryGen, forcePrimaryVersion, searcherFactory, System.out);
     this.tcpPort = tcpPort;
     this.random = new Random(random.nextLong());
     this.doFlipBitsDuringCopy = doFlipBitsDuringCopy;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 9658ad1..6948b78 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,9 @@ 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, boolean doCheckIndexOnClose) throws IOException {
-    super(id, getDirectory(random, id, indexPath, doCheckIndexOnClose), 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, System.out);
     this.tcpPort = tcpPort;
     this.random = new Random(random.nextLong());
 
@@ -139,7 +140,7 @@ class SimpleReplicaNode extends ReplicaNode {
 
     // 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");
+    Node.nodeMessage(System.out, id, "top: corrupt unknown files");
     dir.corruptUnknownFiles();
 
     return dir;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/027bc0e4/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 6300589..f2d0bf8 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
@@ -50,6 +50,7 @@ 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.Nightly;
 import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
 import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
 import org.apache.lucene.util.LuceneTestCase;
@@ -63,29 +64,17 @@ import com.carrotsearch.randomizedtesting.SeedUtils;
   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!
-    - 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
+      - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
+      - 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
+      - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
       - 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
@@ -190,6 +179,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
 
   final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
 
+  @Nightly
   public void test() throws Exception {
 
     Node.globalStartNS = System.nanoTime();


[06/31] 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));
+  }
+}


[14/31] 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));
 +  }
 +}


[15/31] 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/master
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(-)
----------------------------------------------------------------------



[31/31] lucene-solr git commit: Merge branch 'jira/lucene-5438-nrt-replication'

Posted by mi...@apache.org.
Merge branch 'jira/lucene-5438-nrt-replication'


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

Branch: refs/heads/master
Commit: 12b8721a44dbd1fbc7878fa37186c16cf6045401
Parents: 35337e8 20c38e7
Author: Mike McCandless <mi...@apache.org>
Authored: Thu Feb 11 11:20:18 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu Feb 11 11:20:18 2016 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/codecs/CodecUtil.java     |   30 +
 .../apache/lucene/index/DirectoryReader.java    |    9 +-
 .../org/apache/lucene/index/IndexWriter.java    |   55 +-
 .../org/apache/lucene/index/ReaderManager.java  |    8 +-
 .../org/apache/lucene/index/SegmentInfos.java   |  377 +++---
 .../lucene/index/StandardDirectoryReader.java   |   34 +-
 .../apache/lucene/search/SearcherManager.java   |    8 +-
 .../lucene/store/ByteArrayIndexInput.java       |  163 +++
 .../lucene/store/NRTCachingDirectory.java       |    4 +-
 .../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 +-
 .../index/TestNumericDocValuesUpdates.java      |    1 -
 .../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   |  238 ++++
 .../lucene/replicator/nrt/CopyOneFile.java      |  132 ++
 .../apache/lucene/replicator/nrt/CopyState.java |   58 +
 .../lucene/replicator/nrt/FileMetaData.java     |   42 +
 .../org/apache/lucene/replicator/nrt/Node.java  |  224 ++++
 .../nrt/NodeCommunicationException.java         |   32 +
 .../nrt/PreCopyMergedSegmentWarmer.java         |   62 +
 .../lucene/replicator/nrt/PrimaryNode.java      |  306 +++++
 .../replicator/nrt/ReplicaFileDeleter.java      |  169 +++
 .../lucene/replicator/nrt/ReplicaNode.java      |  771 +++++++++++
 .../nrt/SegmentInfosSearcherManager.java        |  132 ++
 .../lucene/replicator/nrt/package-info.java     |   21 +
 .../lucene/replicator/nrt/Connection.java       |   63 +
 .../org/apache/lucene/replicator/nrt/Jobs.java  |  166 +++
 .../lucene/replicator/nrt/NodeProcess.java      |  251 ++++
 .../lucene/replicator/nrt/SimpleCopyJob.java    |  287 +++++
 .../replicator/nrt/SimplePrimaryNode.java       |  760 +++++++++++
 .../replicator/nrt/SimpleReplicaNode.java       |  355 ++++++
 .../lucene/replicator/nrt/SimpleServer.java     |  376 ++++++
 .../lucene/replicator/nrt/SimpleTransLog.java   |  250 ++++
 .../replicator/nrt/TestNRTReplication.java      |  887 +++++++++++++
 .../nrt/TestStressNRTReplication.java           | 1201 ++++++++++++++++++
 .../lucene/replicator/nrt/ThreadPumper.java     |   63 +
 lucene/replicator/test.cmd                      |    1 +
 .../spatial/util/BaseGeoPointTestCase.java      |    2 +-
 .../index/BaseIndexFileFormatTestCase.java      |    2 +-
 .../apache/lucene/index/RandomIndexWriter.java  |    8 +-
 .../lucene/store/MockDirectoryWrapper.java      |  239 +++-
 .../org/apache/lucene/util/LuceneTestCase.java  |   13 +-
 .../java/org/apache/lucene/util/TestUtil.java   |    3 +-
 .../solr/core/StandardIndexReaderFactory.java   |    1 -
 53 files changed, 7560 insertions(+), 320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12b8721a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------


[20/31] 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/c14fa123
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/c14fa123
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/c14fa123

Branch: refs/heads/master
Commit: c14fa123da435a34c4f40e6685c3184ecd9a60e0
Parents: f0f4278
Author: Michael McCandless <ma...@mikemccandless.com>
Authored: Mon Feb 8 14:07:14 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Mon Feb 8 14:07:14 2016 -0500

----------------------------------------------------------------------
 .../test/org/apache/lucene/replicator/nrt/Jobs.java    | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c14fa123/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
index a0b9535..de8c0c3 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
@@ -80,7 +80,12 @@ class Jobs extends Thread implements Closeable {
         } else {
           node.message("AlreadyClosedException during job.visit job=" + topJob + "; now cancel");
         }
-        topJob.cancel("unexpected exception in visit", t);
+        try {
+          topJob.cancel("unexpected exception in visit", t);
+        } catch (Throwable t2) {
+          node.message("ignore exception calling cancel: " + t2);
+          t2.printStackTrace(System.out);
+        }
         try {
           topJob.onceDone.run(topJob);
         } catch (Throwable t2) {
@@ -112,7 +117,11 @@ class Jobs extends Thread implements Closeable {
       while (queue.isEmpty() == false) {
         SimpleCopyJob job = (SimpleCopyJob) queue.poll();
         node.message("top: Jobs: now cancel job=" + job);
-        job.cancel("jobs closing", null);
+        try {
+          job.cancel("jobs closing", null);
+        } catch (Throwable t) {
+          node.message("ignore exception calling cancel: " + t);
+        }
         try {
           job.onceDone.run(job);
         } catch (Throwable t) {


[04/31] 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/master
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;
+  }
+}
+


[18/31] lucene-solr git commit: cutover to 'Directory owns retrying deletes on buggy filesystems, cleaning up a lot of crazy retry logic

Posted by mi...@apache.org.
cutover to 'Directory owns retrying deletes on buggy filesystems, cleaning up a lot of crazy retry logic


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

Branch: refs/heads/master
Commit: f0f42780a91852ed05b3793922f73d85d8b3092a
Parents: 9ba62e5
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Feb 7 14:46:08 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Feb 7 14:46:08 2016 -0500

----------------------------------------------------------------------
 .../apache/lucene/replicator/nrt/CopyJob.java   |   2 +-
 .../org/apache/lucene/replicator/nrt/Node.java  |   3 +-
 .../replicator/nrt/ReplicaFileDeleter.java      | 102 +++++--------------
 .../lucene/replicator/nrt/ReplicaNode.java      |  21 +---
 .../org/apache/lucene/replicator/nrt/Jobs.java  |   3 +-
 .../lucene/replicator/nrt/SimpleCopyJob.java    |   9 +-
 .../replicator/nrt/SimpleReplicaNode.java       |   4 +-
 lucene/replicator/test.cmd                      |   4 +-
 8 files changed, 41 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
index 5baf1d3..217c2c0 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
@@ -186,7 +186,7 @@ public abstract class CopyJob implements Comparable<CopyJob> {
   /** 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) {
+  public void cancel(String reason, Throwable exc) throws IOException {
     if (this.exc != null) {
       // Already cancelled
       return;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/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 e54c01e..742b19f 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,8 +49,7 @@ import org.apache.lucene.util.StringHelper;
 abstract class Node implements Closeable {
 
   static boolean VERBOSE_FILES = true;
-  // nocommit
-  static boolean VERBOSE_CONNECTIONS = true;
+  static boolean VERBOSE_CONNECTIONS = false;
 
   // Keys we store into IndexWriter's commit user data:
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/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
index b15fc05..652b3af 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
@@ -38,7 +38,6 @@ import org.apache.lucene.store.IOContext;
 
 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;
 
@@ -63,10 +62,6 @@ class ReplicaFileDeleter {
   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);
@@ -78,24 +73,23 @@ class ReplicaFileDeleter {
     }
   }
 
-  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:
+  public synchronized void decRef(Collection<String> fileNames) throws IOException {
+    Set<String> toDelete = new HashSet<>();
     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);
+        toDelete.add(fileName);
       } else {
         refCounts.put(fileName, curCount.intValue() - 1);
       }
     }
 
-    deletePending();
+    delete(toDelete);
 
-    // TODO: this local IR could incRef files here, like we do now with IW ... then we can assert this again:
+    // TODO: this local IR could incRef files here, like we do now with IW's NRT readers ... 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:
@@ -109,111 +103,69 @@ class ReplicaFileDeleter {
     */
   }
 
-  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) {
-      // nocommit remove this retry logic!  it's Directory's job now...
-      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() {
+  private synchronized void delete(Collection<String> toDelete) throws IOException {
     if (Node.VERBOSE_FILES) {
-      node.message("now deletePending: " + pending.size() + " files to try: " + pending);
+      node.message("now delete " + toDelete.size() + " files: " + toDelete);
     }
 
-    // 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:
+    // before we remove any files they reference, in case we crash right now:
     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;
-        }
+      assert refCounts.containsKey(fileName) == false;
+      if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+        delete(fileName);
       }
     }
 
     // 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) {
+      assert refCounts.containsKey(fileName) == false;
       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");
-      }
+  private synchronized void delete(String fileName) throws IOException {
+    if (Node.VERBOSE_FILES) {
+      node.message("file " + fileName + ": now delete");
     }
+    dir.deleteFile(fileName);
+  }
+
+  public synchronized Integer getRefCount(String fileName) {
+    return refCounts.get(fileName);
   }
 
-  public synchronized void deleteIfNoRef(String fileName) {
+  public synchronized void deleteIfNoRef(String fileName) throws IOException {
     if (refCounts.containsKey(fileName) == false) {
       deleteNewFile(fileName);
     }
   }
 
-  public synchronized void deleteNewFile(String fileName) {
+  public synchronized void deleteNewFile(String fileName) throws IOException {
     delete(fileName);
   }
 
+  /*
   public synchronized Set<String> getPending() {
     return new HashSet<String>(pending);
   }
+  */
 
   public synchronized void deleteUnknownFiles(String segmentsFileName) throws IOException {
+    Set<String> toDelete = new HashSet<>();
     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);
+        toDelete.add(fileName);
       }
     }
 
-    deletePending();
+    delete(toDelete);
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/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 54083b4..e191caf 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
@@ -195,10 +195,13 @@ abstract class ReplicaNode extends Node {
         // 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:
+
+        if (dir instanceof FSDirectory && ((FSDirectory) dir).checkPendingDeletions()) {
+          // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else there is a definite window during
+          // which if we carsh, we 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;
@@ -427,9 +430,6 @@ abstract class ReplicaNode extends Node {
       }
 
       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;
@@ -720,17 +720,6 @@ abstract class ReplicaNode extends Node {
    * (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.):

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/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
index 3cb2fbb..a0b9535 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
@@ -18,6 +18,7 @@ package org.apache.lucene.replicator.nrt;
  */
 
 import java.io.Closeable;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.PriorityQueue;
@@ -131,7 +132,7 @@ class Jobs extends Thread implements Closeable {
   }
 
   /** Cancels any existing jobs that are copying the same file names as this one */
-  public synchronized void cancelConflictingJobs(CopyJob newJob) {
+  public synchronized void cancelConflictingJobs(CopyJob newJob) throws IOException {
     for (CopyJob job : queue) {
       if (job.conflicts(newJob)) {
         node.message("top: now cancel existing conflicting job=" + job + " due to newJob=" + newJob);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/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
index 1180967..c8663e2 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
@@ -173,13 +173,6 @@ class SimpleCopyJob extends CopyJob {
       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);
       }
@@ -241,7 +234,7 @@ class SimpleCopyJob extends CopyJob {
     }
   }
 
-  public synchronized void cancel(String reason, Throwable exc) {
+  public synchronized void cancel(String reason, Throwable exc) throws IOException {
     try {
       super.cancel(reason, exc);
     } finally {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f0f42780/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 83ce6cb..4868338 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
@@ -135,9 +135,7 @@ class SimpleReplicaNode extends ReplicaNode {
     MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path);
     
     dir.setAssertNoUnrefencedFilesOnClose(true);
-    // nocommit
-    //dir.setCheckIndexOnClose(doCheckIndexOnClose);
-    dir.setCheckIndexOnClose(true);
+    dir.setCheckIndexOnClose(doCheckIndexOnClose);
 
     // 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/f0f42780/lucene/replicator/test.cmd
----------------------------------------------------------------------
diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd
index c38fc96..f636a61 100644
--- a/lucene/replicator/test.cmd
+++ b/lucene/replicator/test.cmd
@@ -1 +1,3 @@
-python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 1 -mult 4 -nightly
+python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 3
+
+#  -mult 4 -nightly