You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2016/02/06 17:23:55 UTC
[01/15] lucene-solr git commit: current patch
Repository: lucene-solr
Updated Branches:
refs/heads/jira/lucene-5438-nrt-replication [created] bd6804bc6
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
new file mode 100644
index 0000000..5a073ff
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -0,0 +1,1175 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.charset.MalformedInputException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.ConcurrentMergeScheduler;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
+// nocommit sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
+
+// nocommit randomly p.destroy() one replica?
+
+/*
+ TODO
+ - why do we do the "rename temp to actual" all at the end...? what really does that buy us?
+ - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
+ - test should not print scary exceptions and then succeed!
+ - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
+ - are the pre-copied-completed-merged files not being cleared in primary?
+ - hmm the logic isn't right today? a replica may skip pulling a given copy state, that recorded the finished merged segments?
+ - beast & fix bugs
+ - graceful cluster restart
+ - better translog integration
+ - get "graceful primary shutdown" working
+ - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion
+ - clean up how version is persisted in commit data
+ - why am i not using hashes here? how does ES use them?
+ - get all other "single shard" functions working too: this cluster should "act like" a single shard
+ - SLM
+ - controlled nrt reopen thread / returning long gen on write
+ - live field values
+ - add indexes
+ - make cluster level APIs to search, index, that deal w/ primary failover, etc.
+ - must prune xlog
+ - refuse to start primary unless we have quorum
+ - later
+ - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback?
+ - back pressure on indexing if replicas can't keep up?
+ - get xlog working on top? needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit
+ quorum
+ - maybe fix IW to return "gen" or "seq id" or "segment name" or something?
+ - replica can copy files from other replicas too / use multicast / rsync / something
+ - each replica could also pre-open a SegmentReader after pre-copy when warming a merge
+ - we can pre-copy newly flushed files too, for cases where reopen rate is low vs IW's flushing because RAM buffer is full
+ - opto: pre-copy files as they are written; if they will become CFS, we can build CFS on the replica?
+ - what about multiple commit points?
+ - fix primary to init directly from an open replica, instead of having to commit/close the replica first
+*/
+
+// Tricky cases:
+// - we are pre-copying a merge, then replica starts up part way through, so it misses that pre-copy and must do it on next nrt point
+// - a down replica starts up, but it's "from the future" vs the current primary, and must copy over file names with different contents
+// but referenced by its latest commit point, so it must fully remove that commit ... which is a hazardous window
+// - replica comes up just as the primary is crashing / moving
+// - electing a new primary when a replica is just finishing its nrt sync: we need to wait for it so we are sure to get the "most up to
+// date" replica
+// - replica comes up after merged segment finished so it doesn't copy over the merged segment "promptly" (i.e. only sees it on NRT refresh)
+
+/**
+ * Test case showing how to implement NRT replication. This test spawns a sub-process per-node, running TestNRTReplicationChild.
+ *
+ * One node is primary, and segments are periodically flushed there, then concurrently the N replica nodes copy the new files over and open new readers, while
+ * primary also opens a new reader.
+ *
+ * Nodes randomly crash and are restarted. If the primary crashes, a replica is promoted.
+ *
+ * Merges are currently first finished on the primary and then pre-copied out to replicas with a merged segment warmer so they don't block
+ * ongoing NRT reopens. Probably replicas could do their own merging instead, but this is more complex and may not be better overall
+ * (merging takes a lot of IO resources).
+ *
+ * Slow network is simulated with a RateLimiter.
+ */
+
+// nocommit randomly delete all doc sometimes, 1) using IW.deleteAll and 2) doing it inefficiently (by query, by id)
+
+// MockRandom's .sd file has no index header/footer:
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class TestNRTReplication extends LuceneTestCase {
+
+ // Test evilness controls:
+
+ /** Randomly crash the current primary (losing data!) and promote the "next best" replica. */
+ static final boolean DO_CRASH_PRIMARY = true;
+
+ /** Randomly crash (JVM core dumps) a replica; it will later randomly be restarted and sync itself. */
+ static final boolean DO_CRASH_REPLICA = true;
+
+ /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
+ static final boolean DO_CLOSE_REPLICA = true;
+
+ /** If false, all child + parent output is interleaved into single stdout/err */
+ static final boolean SEPARATE_CHILD_OUTPUT = false;
+
+ // nocommit DO_CLOSE_PRIMARY?
+
+ /** Randomly crash whole cluster and then restart it */
+ static final boolean DO_FULL_CLUSTER_CRASH = true;
+
+ /** True if we randomly flip a bit while copying files out */
+ static final boolean DO_BIT_FLIPS_DURING_COPY = true;
+
+ /** Set to a non-null value to force exactly that many nodes; else, it's random. */
+ static final Integer NUM_NODES = null;
+
+ static final boolean DO_RANDOM_XLOG_REPLAY = false;
+
+ final AtomicBoolean failed = new AtomicBoolean();
+
+ final AtomicBoolean stop = new AtomicBoolean();
+
+ /** cwd where we start each child (server) node */
+ private Path childTempDir;
+
+ long primaryGen;
+
+ volatile long lastPrimaryVersion;
+
+ volatile NodeProcess primary;
+ volatile NodeProcess[] nodes;
+ volatile long[] nodeTimeStamps;
+ volatile boolean[] starting;
+
+ Path[] indexPaths;
+
+ Path transLogPath;
+ SimpleTransLog transLog;
+ final AtomicInteger markerUpto = new AtomicInteger();
+
+ /** Maps searcher version to how many hits the query body:the matched. */
+ final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
+
+ /** Maps searcher version to how many marker documents matched. This should only ever grow (we never delete marker documents). */
+ final Map<Long,Integer> versionToMarker = new ConcurrentHashMap<>();
+
+ /** Maps searcher version to xlog location when refresh of this version started. */
+ final Map<Long,Long> versionToTransLogLocation = new ConcurrentHashMap<>();
+
+ public void test() throws Exception {
+
+ Node.globalStartNS = System.nanoTime();
+
+ message("change thread name from " + Thread.currentThread().getName());
+ Thread.currentThread().setName("main");
+
+ childTempDir = createTempDir("child");
+
+ // We are parent process:
+
+ // Silly bootstrapping:
+ versionToTransLogLocation.put(0L, 0L);
+ versionToTransLogLocation.put(1L, 0L);
+
+ int numNodes;
+
+ if (NUM_NODES == null) {
+ numNodes = TestUtil.nextInt(random(), 2, 10);
+ } else {
+ numNodes = NUM_NODES.intValue();
+ }
+
+ System.out.println("TEST: using " + numNodes + " nodes");
+
+ transLogPath = createTempDir("NRTReplication").resolve("translog");
+ transLog = new SimpleTransLog(transLogPath);
+
+ //state.rateLimiters = new RateLimiter[numNodes];
+ indexPaths = new Path[numNodes];
+ nodes = new NodeProcess[numNodes];
+ nodeTimeStamps = new long[numNodes];
+ Arrays.fill(nodeTimeStamps, Node.globalStartNS);
+ starting = new boolean[numNodes];
+
+ for(int i=0;i<numNodes;i++) {
+ indexPaths[i] = createTempDir("index" + i);
+ }
+
+ Thread[] indexers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+ System.out.println("TEST: launch " + indexers.length + " indexer threads");
+ for(int i=0;i<indexers.length;i++) {
+ indexers[i] = new IndexThread();
+ indexers[i].setName("indexer" + i);
+ indexers[i].setDaemon(true);
+ indexers[i].start();
+ }
+
+ Thread[] searchers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+ System.out.println("TEST: launch " + searchers.length + " searcher threads");
+ for(int i=0;i<searchers.length;i++) {
+ searchers[i] = new SearchThread();
+ searchers[i].setName("searcher" + i);
+ searchers[i].setDaemon(true);
+ searchers[i].start();
+ }
+
+ Thread restarter = new RestartThread();
+ restarter.setName("restarter");
+ restarter.setDaemon(true);
+ restarter.start();
+
+ int runTimeSec;
+ if (TEST_NIGHTLY) {
+ runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 120, 240);
+ } else {
+ runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 45, 120);
+ }
+
+ System.out.println("TEST: will run for " + runTimeSec + " sec");
+
+ long endTime = System.nanoTime() + runTimeSec*1000000000L;
+
+ sendReplicasToPrimary();
+
+ while (failed.get() == false && System.nanoTime() < endTime) {
+
+ // Wait a bit:
+ Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
+ if (primary != null && random().nextBoolean()) {
+ message("top: now flush primary");
+ NodeProcess curPrimary = primary;
+ if (curPrimary != null) {
+
+ // Save these before we start flush:
+ long nextTransLogLoc = transLog.getNextLocation();
+ int markerUptoSav = markerUpto.get();
+
+ long result;
+ try {
+ result = primary.flush();
+ } catch (Throwable t) {
+ message("top: flush failed; skipping: " + t.getMessage());
+ result = -1;
+ }
+ if (result > 0) {
+ // There were changes
+ lastPrimaryVersion = result;
+ addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
+ addVersionMarker(lastPrimaryVersion, markerUptoSav);
+ }
+ }
+ }
+
+ StringBuilder sb = new StringBuilder();
+ int liveCount = 0;
+ for(int i=0;i<nodes.length;i++) {
+ NodeProcess node = nodes[i];
+ if (node != null) {
+ if (sb.length() != 0) {
+ sb.append(" ");
+ }
+ liveCount++;
+ if (node.isPrimary) {
+ sb.append('P');
+ } else {
+ sb.append('R');
+ }
+ sb.append(i);
+ }
+ }
+
+ message("PG=" + (primary == null ? "X" : primaryGen) + " " + liveCount + " (of " + nodes.length + ") nodes running: " + sb);
+
+ // Commit a random node, primary or replica
+
+ {
+ NodeProcess node = nodes[random().nextInt(nodes.length)];
+ if (node != null) {
+ // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
+ // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
+ message("top: now commit node=" + node);
+ node.commitAsync();
+ }
+ }
+ }
+
+ message("TEST: top: test done, now close");
+ stop.set(true);
+ for(Thread thread : indexers) {
+ thread.join();
+ }
+ for(Thread thread : searchers) {
+ thread.join();
+ }
+ restarter.join();
+
+ // Close replicas before primary so we cancel any in-progres replications:
+ System.out.println("TEST: top: now close replicas");
+ List<Closeable> toClose = new ArrayList<>();
+ for(NodeProcess node : nodes) {
+ if (node != primary && node != null) {
+ toClose.add(node);
+ }
+ }
+ IOUtils.close(toClose);
+ IOUtils.close(primary);
+ IOUtils.close(transLog);
+
+ if (failed.get() == false) {
+ message("TEST: top: now checkIndex");
+ for(Path path : indexPaths) {
+ message("TEST: check " + path);
+ MockDirectoryWrapper dir = newMockFSDirectory(path);
+ // Just too slow otherwise
+ dir.setCrossCheckTermVectorsOnClose(false);
+ dir.close();
+ }
+ } else {
+ message("TEST: failed; skip checkIndex");
+ }
+ }
+
+ private boolean anyNodesStarting() {
+ for(int id=0;id<nodes.length;id++) {
+ if (starting[id]) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ /** Picks a replica and promotes it as new primary. */
+ private void promoteReplica() throws IOException {
+ message("top: primary crashed; now pick replica to promote");
+ long maxSearchingVersion = -1;
+ NodeProcess replicaToPromote = null;
+
+ // We must promote the most current replica, because otherwise file name reuse can cause a replication to fail when it needs to copy
+ // over a file currently held open for searching. This also minimizes recovery work since the most current replica means less xlog
+ // replay to catch up:
+ for (NodeProcess node : nodes) {
+ if (node != null) {
+ message("ask " + node + " for its current searching version");
+ long searchingVersion = node.getSearchingVersion();
+ message(node + " has searchingVersion=" + searchingVersion);
+ if (searchingVersion > maxSearchingVersion) {
+ maxSearchingVersion = searchingVersion;
+ replicaToPromote = node;
+ }
+ }
+ }
+
+ if (replicaToPromote == null) {
+ message("top: no replicas running; skipping primary promotion");
+ return;
+ }
+
+ message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
+ if (replicaToPromote.commit() == false) {
+ message("top: commit failed; skipping primary promotion");
+ return;
+ }
+
+ message("top: now shutdown " + replicaToPromote);
+ if (replicaToPromote.shutdown() == false) {
+ message("top: shutdown failed for R" + replicaToPromote.id + "; skipping primary promotion");
+ return;
+ }
+
+ int id = replicaToPromote.id;
+ message("top: now startPrimary " + replicaToPromote);
+ startPrimary(replicaToPromote.id);
+ }
+
+ void startPrimary(int id) throws IOException {
+ message(id + ": top: startPrimary lastPrimaryVersion=" + lastPrimaryVersion);
+ assert nodes[id] == null;
+
+ // Force version of new primary to advance beyond where old primary was, so we never re-use versions. It may have
+ // already advanced beyond newVersion, e.g. if it flushed new segments while during xlog replay:
+
+ // First start node as primary (it opens an IndexWriter) but do not publish it for searching until we replay xlog:
+ NodeProcess newPrimary = startNode(id, indexPaths[id], true, lastPrimaryVersion+1);
+ if (newPrimary == null) {
+ message("top: newPrimary failed to start; abort");
+ return;
+ }
+
+ // Get xlog location that this node was guaranteed to already have indexed through; this may replay some ops already indexed but it's OK
+ // because the ops are idempotent: we updateDocument (by docid) on replay even for original addDocument:
+ Long startTransLogLoc;
+ Integer markerCount;
+ if (newPrimary.initCommitVersion == 0) {
+ startTransLogLoc = 0L;
+ markerCount = 0;
+ } else {
+ startTransLogLoc = versionToTransLogLocation.get(newPrimary.initCommitVersion);
+ markerCount = versionToMarker.get(newPrimary.initCommitVersion);
+ }
+ assert startTransLogLoc != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToTransLogLocation: keys=" + versionToTransLogLocation.keySet();
+ assert markerCount != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToMarker: keys=" + versionToMarker.keySet();
+
+ // When the primary starts, the userData in its latest commit point tells us which version it had indexed up to, so we know where to
+ // replay from in the xlog. However, we forcefuly advance the version, and then IW on init (or maybe getReader) also adds 1 to it.
+ // Since we publish the primary in this state (before xlog replay is done), a replica can start up at this point and pull this version,
+ // and possibly later be chosen as a primary, causing problems if the version is known recorded in the translog map. So we record it
+ // here:
+
+ addTransLogLoc(newPrimary.initInfosVersion, startTransLogLoc);
+ addVersionMarker(newPrimary.initInfosVersion, markerCount);
+
+ assert newPrimary.initInfosVersion >= lastPrimaryVersion;
+ message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
+ lastPrimaryVersion = newPrimary.initInfosVersion;
+
+ // Publish new primary, before replaying xlog. This means other indexing ops can come in at the same time as we catch up indexing
+ // previous ops. Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
+ // translog) concurrently with new incoming ops.
+ nodes[id] = newPrimary;
+ primary = newPrimary;
+
+ sendReplicasToPrimary();
+
+ long nextTransLogLoc = transLog.getNextLocation();
+ int nextMarkerUpto = markerUpto.get();
+ message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
+ try {
+ transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
+ } catch (IOException ioe) {
+ message("top: replay xlog failed; abort");
+ return;
+ }
+ message("top: done replay trans log");
+ }
+
+ final AtomicLong nodeStartCounter = new AtomicLong();
+
+ final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
+
+ /** Launches a child "server" (separate JVM), which is either primary or replica node */
+ NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
+ nodeTimeStamps[id] = System.nanoTime();
+ List<String> cmd = new ArrayList<>();
+
+ NodeProcess curPrimary = primary;
+
+ cmd.add(System.getProperty("java.home")
+ + System.getProperty("file.separator")
+ + "bin"
+ + System.getProperty("file.separator")
+ + "java");
+ cmd.add("-Xmx512m");
+
+ if (curPrimary != null) {
+ cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + curPrimary.tcpPort);
+ } else if (isPrimary == false) {
+ // We cannot start a replica when there is no primary:
+ return null;
+ }
+
+ cmd.add("-Dtests.nrtreplication.node=true");
+ cmd.add("-Dtests.nrtreplication.nodeid=" + id);
+ cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
+ cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
+ if (isPrimary) {
+ cmd.add("-Dtests.nrtreplication.isPrimary=true");
+ cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
+ }
+
+ long myPrimaryGen = primaryGen;
+ cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
+
+ // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
+ // new node:
+ long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
+
+ cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(seed));
+ cmd.add("-ea");
+ cmd.add("-cp");
+ cmd.add(System.getProperty("java.class.path"));
+ cmd.add("org.junit.runner.JUnitCore");
+ cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
+
+ Writer childLog;
+
+ if (SEPARATE_CHILD_OUTPUT) {
+ Path childOut = childTempDir.resolve(id + ".log");
+ message("logging to " + childOut);
+ childLog = Files.newBufferedWriter(childOut, StandardCharsets.UTF_8, StandardOpenOption.APPEND, StandardOpenOption.CREATE);
+ childLog.write("\n\nSTART NEW CHILD:\n");
+ } else {
+ childLog = null;
+ }
+
+ message("child process command: " + cmd);
+ ProcessBuilder pb = new ProcessBuilder(cmd);
+ pb.redirectErrorStream(true);
+
+ // Important, so that the scary looking hs_err_<pid>.log appear under our test temp dir:
+ pb.directory(childTempDir.toFile());
+
+ Process p = pb.start();
+
+ BufferedReader r;
+ try {
+ r = new BufferedReader(new InputStreamReader(p.getInputStream(), IOUtils.UTF_8));
+ } catch (UnsupportedEncodingException uee) {
+ throw new RuntimeException(uee);
+ }
+
+ int tcpPort = -1;
+ long initCommitVersion = -1;
+ long initInfosVersion = -1;
+ Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+ boolean willCrash = false;
+ boolean sawExistingSegmentsFile = false;
+
+ while (true) {
+ String l = r.readLine();
+ if (l == null) {
+ message("top: node=" + id + " failed to start");
+ try {
+ p.waitFor();
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ message("exit value=" + p.exitValue());
+
+ // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
+ if (isPrimary == false) {
+ if (sawExistingSegmentsFile) {
+ // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
+ // and retry again later:
+ message("failed to remove segments_N; skipping");
+ return null;
+ }
+ for(int i=0;i<10;i++) {
+ if (primaryGen != myPrimaryGen || primary == null) {
+ // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
+ message("primary crashed/closed while replica R" + id + " tried to start; skipping");
+ return null;
+ } else {
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
+ }
+ }
+ }
+ }
+
+ // Should fail the test:
+ message("top: now fail test replica R" + id + " failed to start");
+ failed.set(true);
+ throw new RuntimeException("replica R" + id + " failed to start");
+ }
+
+ if (childLog != null) {
+ childLog.write(l);
+ childLog.write("\n");
+ childLog.flush();
+ } else if (logTimeStart.matcher(l).matches()) {
+ // Already a well-formed log output:
+ System.out.println(l);
+ } else {
+ message(l);
+ }
+
+ if (l.startsWith("PORT: ")) {
+ tcpPort = Integer.parseInt(l.substring(6).trim());
+ } else if (l.startsWith("COMMIT VERSION: ")) {
+ initCommitVersion = Integer.parseInt(l.substring(16).trim());
+ } else if (l.startsWith("INFOS VERSION: ")) {
+ initInfosVersion = Integer.parseInt(l.substring(15).trim());
+ } else if (l.contains("will crash after")) {
+ willCrash = true;
+ } else if (l.startsWith("NODE STARTED")) {
+ break;
+ } else if (l.contains("replica cannot start: existing segments file=")) {
+ sawExistingSegmentsFile = true;
+ }
+ }
+
+ final boolean finalWillCrash = willCrash;
+
+ // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
+ Thread pumper = ThreadPumper.start(
+ new Runnable() {
+ @Override
+ public void run() {
+ message("now wait for process " + p);
+ try {
+ p.waitFor();
+ } catch (Throwable t) {
+ throw new RuntimeException(t);
+ }
+
+ message("done wait for process " + p);
+ int exitValue = p.exitValue();
+ message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
+ if (childLog != null) {
+ try {
+ childLog.write("process done; exitValue=" + exitValue + "\n");
+ childLog.close();
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ }
+ if (exitValue != 0 && finalWillCrash == false && crashingNodes.remove(id) == false) {
+ // should fail test
+ failed.set(true);
+ if (childLog != null) {
+ throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue + "; see " + childLog + " for details");
+ } else {
+ throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
+ }
+ }
+ nodeClosed(id);
+ }
+ }, r, System.out, childLog);
+ pumper.setName("pump" + id);
+
+ message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
+ return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+ }
+
+ private void nodeClosed(int id) {
+ NodeProcess oldNode = nodes[id];
+ if (primary != null && oldNode == primary) {
+ message("top: " + primary + ": primary process finished");
+ primary = null;
+ primaryGen++;
+ } else {
+ message("top: " + oldNode + ": replica process finished");
+ }
+ if (oldNode != null) {
+ oldNode.isOpen = false;
+ }
+ nodes[id] = null;
+ nodeTimeStamps[id] = System.nanoTime();
+
+ sendReplicasToPrimary();
+ }
+
+ /** Sends currently alive replicas to primary, which uses this to know who to notify when it does a refresh */
+ private void sendReplicasToPrimary() {
+ NodeProcess curPrimary = primary;
+ if (curPrimary != null) {
+ List<NodeProcess> replicas = new ArrayList<>();
+ for (NodeProcess node : nodes) {
+ if (node != null && node.isPrimary == false) {
+ replicas.add(node);
+ }
+ }
+
+ message("top: send " + replicas.size() + " replicas to primary");
+
+ try (Connection c = new Connection(curPrimary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+ c.out.writeVInt(replicas.size());
+ for(NodeProcess replica : replicas) {
+ c.out.writeVInt(replica.id);
+ c.out.writeVInt(replica.tcpPort);
+ }
+ c.flush();
+ c.in.readByte();
+ } catch (Throwable t) {
+ message("top: ignore exc sending replicas to primary: " + t);
+ }
+ }
+ }
+
+ void addVersionMarker(long version, int count) {
+ //System.out.println("ADD VERSION MARKER version=" + version + " count=" + count);
+ if (versionToMarker.containsKey(version)) {
+ int curCount = versionToMarker.get(version);
+ if (curCount != count) {
+ message("top: wrong marker count version=" + version + " count=" + count + " curCount=" + curCount);
+ throw new IllegalStateException("version=" + version + " count=" + count + " curCount=" + curCount);
+ }
+ } else {
+ message("top: record marker count: version=" + version + " count=" + count);
+ versionToMarker.put(version, count);
+ }
+ }
+
+ void addTransLogLoc(long version, long loc) {
+ message("top: record transLogLoc: version=" + version + " loc=" + loc);
+ versionToTransLogLocation.put(version, loc);
+ }
+
+ // Periodically wakes up and starts up any down nodes:
+ private class RestartThread extends Thread {
+ @Override
+ public void run() {
+
+ List<Thread> startupThreads = Collections.synchronizedList(new ArrayList<>());
+
+ try {
+ while (stop.get() == false) {
+ Thread.sleep(TestUtil.nextInt(random(), 50, 500));
+ message("top: restarter cycle");
+
+ // Randomly crash full cluster:
+ if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
+ message("top: full cluster crash");
+ for(int i=0;i<nodes.length;i++) {
+ if (starting[i]) {
+ message("N" + i + ": top: wait for startup so we can crash...");
+ while (starting[i]) {
+ Thread.sleep(10);
+ }
+ message("N" + i + ": top: done wait for startup");
+ }
+ NodeProcess node = nodes[i];
+ if (node != null) {
+ crashingNodes.add(i);
+ message("top: N" + node.id + ": top: now crash node");
+ node.crash();
+ message("top: N" + node.id + ": top: done crash node");
+ }
+ }
+ }
+
+ List<Integer> downNodes = new ArrayList<>();
+ StringBuilder b = new StringBuilder();
+ long nowNS = System.nanoTime();
+ for(int i=0;i<nodes.length;i++) {
+ b.append(' ');
+ double sec = (nowNS - nodeTimeStamps[i])/1000000000.0;
+ String prefix;
+ if (nodes[i] == null) {
+ downNodes.add(i);
+ if (starting[i]) {
+ prefix = "s";
+ } else {
+ prefix = "x";
+ }
+ } else {
+ prefix = "";
+ }
+ if (primary != null && nodes[i] == primary) {
+ prefix += "p";
+ }
+ b.append(String.format(Locale.ROOT, "%s%d(%.1fs)", prefix, i, sec));
+ }
+ message("node status" + b.toString());
+ message("downNodes=" + downNodes);
+
+ // If primary is down, promote a replica:
+ if (primary == null) {
+ if (anyNodesStarting()) {
+ message("top: skip promote replica: nodes are still starting");
+ continue;
+ }
+ promoteReplica();
+ }
+
+ // Randomly start up a down a replica:
+
+ // Stop or start a replica
+ if (downNodes.isEmpty() == false) {
+ int idx = downNodes.get(random().nextInt(downNodes.size()));
+ if (starting[idx] == false) {
+ if (primary == null) {
+ if (downNodes.size() == nodes.length) {
+ // Cold start: entire cluster is down, start this node up as the new primary
+ message("N" + idx + ": top: cold start as primary");
+ startPrimary(idx);
+ }
+ } else if (random().nextDouble() < ((double) downNodes.size())/nodes.length) {
+ // Start up replica:
+ starting[idx] = true;
+ message("N" + idx + ": top: start up: launch thread");
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ try {
+ message("N" + idx + ": top: start up thread");
+ nodes[idx] = startNode(idx, indexPaths[idx], false, -1);
+ sendReplicasToPrimary();
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ } finally {
+ starting[idx] = false;
+ startupThreads.remove(Thread.currentThread());
+ }
+ }
+ };
+ t.setName("start R" + idx);
+ t.start();
+ startupThreads.add(t);
+ }
+ } else {
+ message("node " + idx + " still starting");
+ }
+ }
+ }
+
+ System.out.println("Restarter: now stop: join " + startupThreads.size() + " startup threads");
+
+ while (startupThreads.size() > 0) {
+ Thread.sleep(10);
+ }
+
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ }
+ }
+ }
+
+ /** Randomly picks a node and runs a search against it */
+ private class SearchThread extends Thread {
+
+ @Override
+ public void run() {
+ // Maps version to number of hits for silly 'the' TermQuery:
+ Query theQuery = new TermQuery(new Term("body", "the"));
+
+ // Persists connections
+ Map<Integer,Connection> connections = new HashMap<>();
+
+ while (stop.get() == false) {
+ NodeProcess node = nodes[random().nextInt(nodes.length)];
+ if (node == null || node.isOpen == false) {
+ continue;
+ }
+
+ if (node.lock.tryLock() == false) {
+ // Node is in the process of closing or crashing or something
+ continue;
+ }
+
+ try {
+
+ Thread.currentThread().setName("Searcher node=" + node);
+
+ //System.out.println("S: cycle; conns=" + connections);
+
+ Connection c = connections.get(node.id);
+
+ long version;
+ try {
+ if (c == null) {
+ //System.out.println("S: new connection " + node.id + " " + Thread.currentThread().getName());
+ c = new Connection(node.tcpPort);
+ connections.put(node.id, c);
+ } else {
+ //System.out.println("S: reuse connection " + node.id + " " + Thread.currentThread().getName());
+ }
+
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH);
+ c.flush();
+
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node closed");
+ }
+ Thread.sleep(1);
+ }
+ version = c.in.readVLong();
+
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node closed");
+ }
+ Thread.sleep(1);
+ }
+ int hitCount = c.in.readVInt();
+
+ Integer oldHitCount = hitCounts.get(version);
+
+ // TODO: we never prune this map...
+ if (oldHitCount == null) {
+ hitCounts.put(version, hitCount);
+ message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
+ } else {
+ // Just ensure that all nodes show the same hit count for
+ // the same version, i.e. they really are replicas of one another:
+ if (oldHitCount.intValue() != hitCount) {
+ failed.set(true);
+ stop.set(true);
+ message("top: searcher: wrong version hitCount: version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+ fail("version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+ }
+ }
+ } catch (IOException ioe) {
+ //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+ //ioe.printStackTrace(System.out);
+ IOUtils.closeWhileHandlingException(c);
+ connections.remove(node.id);
+ continue;
+ }
+
+ // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
+ Integer expectedAtLeastHitCount = versionToMarker.get(version);
+
+ if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
+ try {
+ c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
+ c.flush();
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node died");
+ }
+ Thread.sleep(1);
+ }
+
+ version = c.in.readVLong();
+
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node died");
+ }
+ Thread.sleep(1);
+ }
+
+ int hitCount = c.in.readVInt();
+
+ // Look for data loss: make sure all marker docs are visible:
+
+ if (hitCount < expectedAtLeastHitCount) {
+
+ String failMessage = "node=" + node + ": documents were lost version=" + version + " hitCount=" + hitCount + " vs expectedAtLeastHitCount=" + expectedAtLeastHitCount;
+ message(failMessage);
+ failed.set(true);
+ stop.set(true);
+ fail(failMessage);
+ }
+ } catch (IOException ioe) {
+ //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+ //throw new RuntimeException(ioe);
+ //ioe.printStackTrace(System.out);
+ IOUtils.closeWhileHandlingException(c);
+ connections.remove(node.id);
+ continue;
+ }
+ }
+
+ Thread.sleep(10);
+
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ } finally {
+ node.lock.unlock();
+ }
+ }
+ System.out.println("Searcher: now stop");
+ IOUtils.closeWhileHandlingException(connections.values());
+ }
+ }
+
+ private class IndexThread extends Thread {
+
+ @Override
+ public void run() {
+
+ try {
+ LineFileDocs docs = new LineFileDocs(random());
+ int docCount = 0;
+
+ // How often we do an update/delete vs add:
+ double updatePct = random().nextDouble();
+
+ // Varies how many docs/sec we index:
+ int sleepChance = TestUtil.nextInt(random(), 4, 100);
+
+ message("top: indexer: updatePct=" + updatePct + " sleepChance=" + sleepChance);
+
+ long lastTransLogLoc = transLog.getNextLocation();
+
+ NodeProcess curPrimary = null;
+ Connection c = null;
+
+ while (stop.get() == false) {
+
+ try {
+ while (stop.get() == false && curPrimary == null) {
+ Thread.sleep(10);
+ curPrimary = primary;
+ if (curPrimary != null) {
+ c = new Connection(curPrimary.tcpPort);
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ break;
+ }
+ }
+
+ if (stop.get()) {
+ break;
+ }
+
+ Thread.currentThread().setName("indexer p" + curPrimary.id);
+
+ if (random().nextInt(10) == 7) {
+ // We use the marker docs to check for data loss in search thread:
+ Document doc = new Document();
+ int id = markerUpto.getAndIncrement();
+ String idString = "m"+id;
+ doc.add(newStringField("docid", idString, Field.Store.YES));
+ doc.add(newStringField("marker", "marker", Field.Store.YES));
+ curPrimary.addOrUpdateDocument(c, doc, false);
+ transLog.addDocument(idString, doc);
+ message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
+ }
+
+ if (docCount > 0 && random().nextDouble() < updatePct) {
+ int randomID = random().nextInt(docCount);
+ String randomIDString = Integer.toString(randomID);
+ if (random().nextBoolean()) {
+ // Replace previous doc
+ Document doc = docs.nextDoc();
+ ((Field) doc.getField("docid")).setStringValue(randomIDString);
+ curPrimary.addOrUpdateDocument(c, doc, true);
+ transLog.updateDocument(randomIDString, doc);
+ } else {
+ // Delete previous doc
+ curPrimary.deleteDocument(c, randomIDString);
+ transLog.deleteDocuments(randomIDString);
+ }
+ } else {
+ // Add new doc:
+ Document doc = docs.nextDoc();
+ String idString = Integer.toString(docCount++);
+ ((Field) doc.getField("docid")).setStringValue(idString);
+ curPrimary.addOrUpdateDocument(c, doc, false);
+ transLog.addDocument(idString, doc);
+
+ if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
+ long curLoc = transLog.getNextLocation();
+ // randomly replay chunks of translog just to test replay:
+ message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
+ transLog.replay(curPrimary, lastTransLogLoc, curLoc);
+ lastTransLogLoc = curLoc;
+ }
+ }
+ } catch (IOException se) {
+ // Assume primary crashed
+ message("top: indexer lost connection to primary");
+ try {
+ c.close();
+ } catch (Throwable t) {
+ }
+ curPrimary = null;
+ c = null;
+ }
+
+ if (random().nextInt(sleepChance) == 0) {
+ Thread.sleep(1);
+ }
+
+ if (random().nextInt(100) == 17) {
+ System.out.println("Indexer: now pause for a bit...");
+ Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
+ System.out.println("Indexer: done pause for a bit...");
+ }
+ }
+ if (curPrimary != null) {
+ try {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
+ c.flush();
+ c.in.readByte();
+ } catch (IOException se) {
+ // Assume primary crashed
+ message("top: indexer lost connection to primary");
+ try {
+ c.close();
+ } catch (Throwable t) {
+ }
+ curPrimary = null;
+ c = null;
+ }
+ }
+ System.out.println("Indexer: now stop");
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ }
+ }
+ }
+
+ static void message(String message) {
+ long now = System.nanoTime();
+ System.out.println(String.format(Locale.ROOT,
+ "%5.3fs : parent [%11s] %s",
+ (now-Node.globalStartNS)/1000000000.,
+ Thread.currentThread().getName(),
+ message));
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
new file mode 100644
index 0000000..6ddb777
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
@@ -0,0 +1,59 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.io.Writer;
+import java.util.regex.Pattern;
+
+/** A pipe thread. It'd be nice to reuse guava's implementation for this... */
+class ThreadPumper {
+ public static Thread start(final Runnable onExit, final BufferedReader from, final PrintStream to, final Writer toFile) {
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ try {
+ Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+ String line;
+ while ((line = from.readLine()) != null) {
+ if (toFile != null) {
+ toFile.write(line);
+ toFile.write("\n");
+ toFile.flush();
+ } else if (logTimeStart.matcher(line).matches()) {
+ // Already a well-formed log output:
+ System.out.println(line);
+ } else {
+ TestNRTReplication.message(line);
+ }
+ }
+ // Sub-process finished
+ } catch (IOException e) {
+ System.err.println("ignore IOExc reading from forked process pipe: " + e);
+ } finally {
+ onExit.run();
+ }
+ }
+ };
+ t.start();
+ return t;
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/test.cmd
----------------------------------------------------------------------
diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd
new file mode 100644
index 0000000..14e3bd2
--- /dev/null
+++ b/lucene/replicator/test.cmd
@@ -0,0 +1 @@
+python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestNRTReplication -jvms 1 -mult 4 -nightly
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java b/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
index 4236e88..9f876ef 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/util/BaseGeoPointTestCase.java
@@ -637,7 +637,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
for (int iter=0;iter<iters && failed.get() == false;iter++) {
if (VERBOSE) {
- System.out.println("\nTEST: iter=" + iter + " s=" + s);
+ System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " s=" + s);
}
Query query;
VerifyHits verifyHits;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
index b4b6f7d..68eed39 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
@@ -457,7 +457,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
if (random().nextBoolean()) {
DirectoryReader ir = null;
try {
- ir = DirectoryReader.open(iw, random().nextBoolean());
+ ir = DirectoryReader.open(iw, random().nextBoolean(), false);
dir.setRandomIOExceptionRateOnOpen(0.0); // disable exceptions on openInput until next iteration
TestUtil.checkReader(ir);
} finally {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
index 047ef4b..22fee48 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
@@ -303,7 +303,7 @@ public class RandomIndexWriter implements Closeable {
public DirectoryReader getReader() throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
- return getReader(true);
+ return getReader(true, false);
}
private boolean doRandomForceMerge = true;
@@ -353,7 +353,7 @@ public class RandomIndexWriter implements Closeable {
}
}
- public DirectoryReader getReader(boolean applyDeletions) throws IOException {
+ public DirectoryReader getReader(boolean applyDeletions, boolean writeAllDeletes) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
getReaderCalled = true;
if (r.nextInt(20) == 2) {
@@ -366,7 +366,7 @@ public class RandomIndexWriter implements Closeable {
if (r.nextInt(5) == 1) {
w.commit();
}
- return w.getReader(applyDeletions);
+ return w.getReader(applyDeletions, writeAllDeletes);
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("RIW.getReader: open new reader");
@@ -375,7 +375,7 @@ public class RandomIndexWriter implements Closeable {
if (r.nextBoolean()) {
return DirectoryReader.open(w.getDirectory());
} else {
- return w.getReader(applyDeletions);
+ return w.getReader(applyDeletions, writeAllDeletes);
}
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index ec99c7e..b19045b 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@ -38,8 +38,10 @@ import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.regex.Matcher;
import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.NoDeletionPolicy;
@@ -239,12 +241,24 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (openFiles.containsKey(source)) {
if (assertNoDeleteOpenFile) {
- throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
+ throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
} else if (noDeleteOpenFile) {
- throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
+ throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
}
}
+ if (openFiles.containsKey(dest)) {
+ if (assertNoDeleteOpenFile) {
+ throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
+ } else if (noDeleteOpenFile) {
+ throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
+ }
+ }
+
+ if (createdFiles.contains(dest)) {
+ throw new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" already exists: cannot rename");
+ }
+
boolean success = false;
try {
in.renameFile(source, dest);
@@ -257,6 +271,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
unSyncedFiles.add(dest);
}
openFilesDeleted.remove(source);
+ triedToDelete.remove(dest);
+ createdFiles.add(dest);
}
}
}
@@ -278,89 +294,215 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
}
}
- /** Simulates a crash of OS or machine by overwriting
- * unsynced files. */
- public synchronized void crash() throws IOException {
- crashed = true;
- openFiles = new HashMap<>();
- openFilesForWrite = new HashSet<>();
- openFilesDeleted = new HashSet<>();
- Iterator<String> it = unSyncedFiles.iterator();
- unSyncedFiles = new HashSet<>();
- // first force-close all files, so we can corrupt on windows etc.
- // clone the file map, as these guys want to remove themselves on close.
- Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
- for (Closeable f : m.keySet()) {
- try {
- f.close();
- } catch (Exception ignored) {}
+ public synchronized void corruptUnknownFiles() throws IOException {
+
+ System.out.println("MDW: corrupt unknown files");
+ Set<String> knownFiles = new HashSet<>();
+ for(String fileName : listAll()) {
+ if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+ System.out.println("MDW: read " + fileName + " to gather files it references");
+ knownFiles.addAll(SegmentInfos.readCommit(this, fileName).files(true));
+ }
}
-
- while(it.hasNext()) {
- String name = it.next();
- int damage = randomState.nextInt(5);
+
+ Set<String> toCorrupt = new HashSet<>();
+ Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
+ for(String fileName : listAll()) {
+ m.reset(fileName);
+ if (knownFiles.contains(fileName) == false &&
+ fileName.endsWith("write.lock") == false &&
+ (m.matches() || fileName.startsWith(IndexFileNames.PENDING_SEGMENTS))) {
+ toCorrupt.add(fileName);
+ }
+ }
+
+ corruptFiles(toCorrupt);
+ }
+
+ public synchronized void corruptFiles(Collection<String> files) {
+ // Must make a copy because we change the incoming unsyncedFiles
+ // when we create temp files, delete, etc., below:
+ for(String name : new ArrayList<>(files)) {
+ int damage = randomState.nextInt(6);
String action = null;
- if (damage == 0) {
+ switch(damage) {
+
+ case 0:
action = "deleted";
- deleteFile(name, true);
- } else if (damage == 1) {
+ try {
+ deleteFile(name, true);
+ } catch (IOException ioe) {
+ // ignore
+ }
+ break;
+
+ case 1:
action = "zeroed";
// Zero out file entirely
- long length = fileLength(name);
+ long length;
+ try {
+ length = fileLength(name);
+ } catch (IOException ioe) {
+ // Ignore
+ continue;
+ }
byte[] zeroes = new byte[256];
long upto = 0;
- IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
- while(upto < length) {
- final int limit = (int) Math.min(length-upto, zeroes.length);
- out.writeBytes(zeroes, 0, limit);
- upto += limit;
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
+ while(upto < length) {
+ final int limit = (int) Math.min(length-upto, zeroes.length);
+ out.writeBytes(zeroes, 0, limit);
+ upto += limit;
+ }
+ } catch (IOException ioe) {
+ // ignore
}
- out.close();
- } else if (damage == 2) {
- action = "partially truncated";
- // Partially Truncate the file:
-
- // First, make temp file and copy only half this
- // file over:
- String tempFileName;
- while (true) {
- tempFileName = ""+randomState.nextInt();
- if (!LuceneTestCase.slowFileExists(in, tempFileName)) {
- break;
+ break;
+
+ case 2:
+ {
+ action = "partially truncated";
+ // Partially Truncate the file:
+
+ // First, make temp file and copy only half this
+ // file over:
+ String tempFileName = null;
+ try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
+ tempFileName = tempOut.getName();
+ tempOut.copyBytes(ii, ii.length()/2);
+ } catch (IOException ioe) {
+ // ignore
+ }
+
+ try {
+ // Delete original and copy bytes back:
+ deleteFile(name, true);
+ } catch (IOException ioe) {
+ // ignore
+ }
+
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
+ out.copyBytes(ii, ii.length());
+ } catch (IOException ioe) {
+ // ignore
+ }
+ try {
+ deleteFile(tempFileName, true);
+ } catch (IOException ioe) {
+ // ignore
}
}
- final IndexOutput tempOut = in.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
- IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState));
- tempOut.copyBytes(ii, ii.length()/2);
- tempOut.close();
- ii.close();
-
- // Delete original and copy bytes back:
- deleteFile(name, true);
-
- final IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
- ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
- out.copyBytes(ii, ii.length());
- out.close();
- ii.close();
- deleteFile(tempFileName, true);
- } else if (damage == 3) {
+ break;
+
+ case 3:
// The file survived intact:
action = "didn't change";
- } else {
+ break;
+
+ case 4:
+ // Corrupt one bit randomly in the file:
+
+ {
+
+ String tempFileName = null;
+ try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
+ tempFileName = tempOut.getName();
+ if (ii.length() > 0) {
+ // Copy first part unchanged:
+ long byteToCorrupt = (long) (randomState.nextDouble() * ii.length());
+ if (byteToCorrupt > 0) {
+ tempOut.copyBytes(ii, byteToCorrupt);
+ }
+
+ // Randomly flip one bit from this byte:
+ byte b = ii.readByte();
+ int bitToFlip = randomState.nextInt(8);
+ b = (byte) (b ^ (1 << bitToFlip));
+ tempOut.writeByte(b);
+
+ action = "flip bit " + bitToFlip + " of byte " + byteToCorrupt + " out of " + ii.length() + " bytes";
+
+ // Copy last part unchanged:
+ long bytesLeft = ii.length() - byteToCorrupt - 1;
+ if (bytesLeft > 0) {
+ tempOut.copyBytes(ii, bytesLeft);
+ }
+ } else {
+ action = "didn't change";
+ }
+ } catch (IOException ioe) {
+ // ignore
+ }
+
+ try {
+ // Delete original and copy bytes back:
+ deleteFile(name, true);
+ } catch (IOException ioe) {
+ // ignore
+ }
+
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
+ out.copyBytes(ii, ii.length());
+ } catch (IOException ioe) {
+ // ignore
+ }
+ try {
+ deleteFile(tempFileName, true);
+ } catch (IOException ioe) {
+ // ignore
+ }
+ }
+ break;
+
+ case 5:
action = "fully truncated";
// Totally truncate the file to zero bytes
- deleteFile(name, true);
- IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
- out.close();
+ try {
+ deleteFile(name, true);
+ } catch (IOException ioe) {
+ // ignore
+ }
+
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
+ } catch (IOException ioe) {
+ // ignore
+ }
+ break;
+
+ default:
+ throw new AssertionError();
}
- if (LuceneTestCase.VERBOSE) {
+
+ if (true || LuceneTestCase.VERBOSE) {
System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name);
}
}
}
+ /** Simulates a crash of OS or machine by overwriting
+ * unsynced files. */
+ public synchronized void crash() {
+ crashed = true;
+ openFiles = new HashMap<>();
+ openFilesForWrite = new HashSet<>();
+ openFilesDeleted = new HashSet<>();
+ // first force-close all files, so we can corrupt on windows etc.
+ // clone the file map, as these guys want to remove themselves on close.
+ Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
+ for (Closeable f : m.keySet()) {
+ try {
+ f.close();
+ } catch (Exception ignored) {}
+ }
+ corruptFiles(unSyncedFiles);
+ unSyncedFiles = new HashSet<>();
+ }
+
public synchronized void clearCrash() {
crashed = false;
openLocks.clear();
@@ -520,9 +662,9 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (!forced && enableVirusScanner && (randomState.nextInt(4) == 0)) {
triedToDelete.add(name);
if (LuceneTestCase.VERBOSE) {
- System.out.println("MDW: now refuse to delete file: " + name);
+ System.out.println(Thread.currentThread().getName() + ": MDW: now refuse to delete file: " + name + " this=" + this);
}
- throw new IOException("cannot delete file: " + name + ", a virus scanner has it open");
+ throw new IOException("cannot delete file: " + name + ", a virus scanner has it open (exists?=" + LuceneTestCase.slowFileExists(in, name));
}
triedToDelete.remove(name);
in.deleteFile(name);
@@ -571,6 +713,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
unSyncedFiles.add(name);
createdFiles.add(name);
+ triedToDelete.remove(name);
if (in instanceof RAMDirectory) {
RAMDirectory ramdir = (RAMDirectory) in;
@@ -801,7 +944,11 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
IndexWriterConfig iwc = new IndexWriterConfig(null);
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
new IndexWriter(in, iwc).rollback();
- String[] endFiles = in.listAll();
+
+ Set<String> files = new HashSet<>(Arrays.asList(listAll()));
+ // Disregard what happens with the pendingDeletions files:
+ files.removeAll(pendingDeletions);
+ String[] endFiles = files.toArray(new String[0]);
Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
@@ -839,7 +986,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
assert pendingDeletions.contains(s);
if (LuceneTestCase.VERBOSE) {
System.out.println("MDW: Unreferenced check: Ignoring referenced file: " + s + " " +
- "from " + file + " that we could not delete.");
+ "from " + file + " that we could not delete.");
}
startSet.add(s);
}
@@ -884,7 +1031,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
}
- throw new RuntimeException("unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
+ throw new RuntimeException(this + ": unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
}
DirectoryReader ir1 = DirectoryReader.open(this);
@@ -1036,7 +1183,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
}
}
-
// don't override optional methods like copyFrom: we need the default impl for things like disk
// full checks. we randomly exercise "raw" directories anyway. We ensure default impls are used:
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index aaab030..e6536f3 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -1155,11 +1155,14 @@ public abstract class LuceneTestCase extends Assert {
}
if (rarely(r)) {
- // change warmer parameters
- if (r.nextBoolean()) {
- c.setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(c.getInfoStream()));
- } else {
- c.setMergedSegmentWarmer(null);
+ IndexWriter.IndexReaderWarmer curWarmer = c.getMergedSegmentWarmer();
+ if (curWarmer == null || curWarmer instanceof SimpleMergedSegmentWarmer) {
+ // change warmer parameters
+ if (r.nextBoolean()) {
+ c.setMergedSegmentWarmer(new SimpleMergedSegmentWarmer(c.getInfoStream()));
+ } else {
+ c.setMergedSegmentWarmer(null);
+ }
}
didChange = true;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
index 99d4be3..de2cf57 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
@@ -976,15 +976,14 @@ public final class TestUtil {
public static void reduceOpenFiles(IndexWriter w) {
// keep number of open files lowish
MergePolicy mp = w.getConfig().getMergePolicy();
+ mp.setNoCFSRatio(1.0);
if (mp instanceof LogMergePolicy) {
LogMergePolicy lmp = (LogMergePolicy) mp;
lmp.setMergeFactor(Math.min(5, lmp.getMergeFactor()));
- lmp.setNoCFSRatio(1.0);
} else if (mp instanceof TieredMergePolicy) {
TieredMergePolicy tmp = (TieredMergePolicy) mp;
tmp.setMaxMergeAtOnce(Math.min(5, tmp.getMaxMergeAtOnce()));
tmp.setSegmentsPerTier(Math.min(5, tmp.getSegmentsPerTier()));
- tmp.setNoCFSRatio(1.0);
}
MergeScheduler ms = w.getConfig().getMergeScheduler();
if (ms instanceof ConcurrentMergeScheduler) {
[08/15] lucene-solr git commit: reduce some verbosity;
fix a silly vInt/int disagreement in the test protocol;
reduce indexing rate so translog replay can keep up vs primary crashing itself
Posted by mi...@apache.org.
reduce some verbosity; fix a silly vInt/int disagreement in the test protocol; reduce indexing rate so translog replay can keep up vs primary crashing itself
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8889469b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8889469b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8889469b
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 8889469b8bb1445133c39069adfdb5db028dbad1
Parents: 022540e
Author: Mike McCandless <mi...@apache.org>
Authored: Tue Feb 2 06:09:57 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 2 06:09:57 2016 -0500
----------------------------------------------------------------------
.../org/apache/lucene/index/SegmentInfos.java | 2 +-
.../lucene/replicator/nrt/PrimaryNode.java | 7 +-
.../lucene/replicator/nrt/ReplicaNode.java | 26 +++--
.../lucene/replicator/nrt/NodeProcess.java | 22 +++-
.../replicator/nrt/SimplePrimaryNode.java | 55 ++++++++--
.../replicator/nrt/SimpleReplicaNode.java | 27 ++++-
.../lucene/replicator/nrt/SimpleServer.java | 20 +++-
.../lucene/replicator/nrt/SimpleTransLog.java | 2 +-
.../replicator/nrt/TestNRTReplication.java | 86 ++++++++++++++-
.../nrt/TestStressNRTReplication.java | 107 ++++++++++++-------
.../lucene/replicator/nrt/ThreadPumper.java | 6 +-
lucene/replicator/test.cmd | 2 +-
12 files changed, 285 insertions(+), 77 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index 2f8d914..046f3ea 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -879,7 +879,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
* segments. */
public void changed() {
version++;
- System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
+ //System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
//new Throwable().printStackTrace(System.out);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index ccd8848..3cff95e 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -104,7 +104,8 @@ public abstract class PrimaryNode extends Node {
IndexSearcher s = mgr.acquire();
try {
- message("init: marker hit count: " + s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits);
+ // TODO: this is test code specific!!
+ message("init: marker count: " + s.count(new TermQuery(new Term("marker", "marker"))));
} finally {
mgr.release(s);
}
@@ -135,7 +136,7 @@ public abstract class PrimaryNode extends Node {
if (result) {
message("top: opened NRT reader version=" + curInfos.getVersion());
finishedMergedFiles.removeAll(completedMergeFiles);
- message("flushAndRefresh: version=" + curInfos.getVersion() + " completedMergeFiles=" + completedMergeFiles + " finishedMergedFiles=" + finishedMergedFiles);
+ message("flushAndRefresh: version=" + curInfos.getVersion() + " completedMergeFiles=" + completedMergeFiles + " finishedMergedFiles=" + finishedMergedFiles);
} else {
message("top: no changes in flushAndRefresh; still version=" + curInfos.getVersion());
}
@@ -208,6 +209,8 @@ public abstract class PrimaryNode extends Node {
try {
searcher = mgr.acquire();
infos = ((StandardDirectoryReader) searcher.getIndexReader()).getSegmentInfos();
+ // TODO: this is test code specific!!
+ message("setCurrentInfos: marker count: " + searcher.count(new TermQuery(new Term("marker", "marker"))) + " version=" + infos.getVersion() + " searcher=" + searcher);
} finally {
if (searcher != null) {
mgr.release(searcher);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index 713c6f1..c7af429 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -32,6 +32,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SegmentInfos;
@@ -152,8 +153,6 @@ abstract class ReplicaNode extends Node {
deleter.deleteUnknownFiles(segmentsFileName);
message("top: done delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
- // nocommit make test where index has all docs deleted (all segments dropped, so 0 segments) and is then replicated
-
String s = infos.getUserData().get(PRIMARY_GEN_KEY);
long myPrimaryGen;
if (s == null) {
@@ -277,6 +276,15 @@ abstract class ReplicaNode extends Node {
// Finally, we are open for business, since our index now "agrees" with the primary:
mgr = new SegmentInfosSearcherManager(dir, this, infos, searcherFactory);
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ // TODO: this is test specific:
+ int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+ message("top: marker count=" + hitCount + " version=" + ((DirectoryReader) searcher.getIndexReader()).getVersion());
+ } finally {
+ mgr.release(searcher);
+ }
+
// Must commit after init mgr:
if (doCommit) {
// Very important to commit what we just sync'd over, because we removed the pre-existing commit point above if we had to
@@ -287,9 +295,13 @@ abstract class ReplicaNode extends Node {
message("top: done start");
state = "idle";
} catch (Throwable t) {
- message("exc on start:");
- t.printStackTrace(System.out);
- throw new RuntimeException(t);
+ if (t.getMessage().startsWith("replica cannot start") == false) {
+ message("exc on start:");
+ t.printStackTrace(System.out);
+ } else {
+ dir.close();
+ }
+ IOUtils.reThrow(t);
}
}
@@ -418,7 +430,7 @@ abstract class ReplicaNode extends Node {
int markerCount;
IndexSearcher s = mgr.acquire();
try {
- markerCount = s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+ markerCount = s.count(new TermQuery(new Term("marker", "marker")));
} finally {
mgr.release(s);
}
@@ -496,7 +508,7 @@ abstract class ReplicaNode extends Node {
} catch (NodeCommunicationException nce) {
// E.g. primary could crash/close when we are asking it for the copy state:
message("top: ignoring communication exception creating CopyJob: " + nce);
- nce.printStackTrace(System.out);
+ //nce.printStackTrace(System.out);
if (state.equals("syncing")) {
state = "idle";
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index 9de2c04..dcbef87 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -19,6 +19,7 @@ package org.apache.lucene.replicator.nrt;
import java.io.Closeable;
import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.document.Document;
@@ -47,7 +48,9 @@ class NodeProcess implements Closeable {
volatile boolean isOpen = true;
- public NodeProcess(Process p, int id, int tcpPort, Thread pumper, boolean isPrimary, long initCommitVersion, long initInfosVersion) {
+ final AtomicBoolean nodeIsClosing;
+
+ public NodeProcess(Process p, int id, int tcpPort, Thread pumper, boolean isPrimary, long initCommitVersion, long initInfosVersion, AtomicBoolean nodeIsClosing) {
this.p = p;
this.id = id;
this.tcpPort = tcpPort;
@@ -55,6 +58,7 @@ class NodeProcess implements Closeable {
this.isPrimary = isPrimary;
this.initCommitVersion = initCommitVersion;
this.initInfosVersion = initInfosVersion;
+ this.nodeIsClosing = nodeIsClosing;
assert initInfosVersion >= initCommitVersion: "initInfosVersion=" + initInfosVersion + " initCommitVersion=" + initCommitVersion;
lock = new ReentrantLock();
}
@@ -122,10 +126,11 @@ class NodeProcess implements Closeable {
/** Ask the primary node process to flush. We send it all currently up replicas so it can notify them about the new NRT point. Returns the newly
* flushed version, or a negative (current) version if there were no changes. */
- public synchronized long flush() throws IOException {
+ public synchronized long flush(int atLeastMarkerCount) throws IOException {
assert isPrimary;
try (Connection c = new Connection(tcpPort)) {
c.out.writeByte(SimplePrimaryNode.CMD_FLUSH);
+ c.out.writeVInt(atLeastMarkerCount);
c.flush();
c.s.shutdownOutput();
return c.in.readLong();
@@ -218,7 +223,7 @@ class NodeProcess implements Closeable {
public void deleteDocument(Connection c, String docid) throws IOException {
if (isPrimary == false) {
- throw new IllegalStateException("only primary can index");
+ throw new IllegalStateException("only primary can delete documents");
}
c.out.writeByte(SimplePrimaryNode.CMD_DELETE_DOC);
c.out.writeString(docid);
@@ -228,11 +233,20 @@ class NodeProcess implements Closeable {
public void deleteAllDocuments(Connection c) throws IOException {
if (isPrimary == false) {
- throw new IllegalStateException("only primary can index");
+ throw new IllegalStateException("only primary can delete documents");
}
c.out.writeByte(SimplePrimaryNode.CMD_DELETE_ALL_DOCS);
c.flush();
c.in.readByte();
}
+
+ public void forceMerge(Connection c) throws IOException {
+ if (isPrimary == false) {
+ throw new IllegalStateException("only primary can force merge");
+ }
+ c.out.writeByte(SimplePrimaryNode.CMD_FORCE_MERGE);
+ c.flush();
+ c.in.readByte();
+ }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
index 0afd1b4..b9ecced 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -53,8 +53,10 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
@@ -272,6 +274,8 @@ class SimplePrimaryNode extends PrimaryNode {
private void handleFlush(DataInput topIn, DataOutput topOut, BufferedOutputStream bos) throws IOException {
Thread.currentThread().setName("flush");
+ int atLeastMarkerCount = topIn.readVInt();
+
int[] replicaTCPPorts;
int[] replicaIDs;
synchronized (this) {
@@ -284,6 +288,8 @@ class SimplePrimaryNode extends PrimaryNode {
if (flushAndRefresh()) {
// Something did get flushed (there were indexing ops since the last flush):
+ verifyAtLeastMarkerCount(atLeastMarkerCount, null);
+
// Tell caller the version before pushing to replicas, so that even if we crash after this, caller will know what version we
// (possibly) pushed to some replicas. Alternatively we could make this 2 separate ops?
long version = getCopyStateVersion();
@@ -295,6 +301,7 @@ class SimplePrimaryNode extends PrimaryNode {
for(int i=0;i<replicaIDs.length;i++) {
int replicaID = replicaIDs[i];
try (Connection c = new Connection(replicaTCPPorts[i])) {
+ message("send NEW_NRT_POINT to R" + replicaID + " at tcpPort=" + replicaTCPPorts[i]);
c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
c.out.writeVLong(version);
c.out.writeInt(tcpPort);
@@ -452,6 +459,10 @@ class SimplePrimaryNode extends PrimaryNode {
writer.deleteAll();
out.writeByte((byte) 1);
bos.flush();
+ } else if (cmd == CMD_FORCE_MERGE) {
+ writer.forceMerge(1);
+ out.writeByte((byte) 1);
+ bos.flush();
} else if (cmd == CMD_INDEXING_DONE) {
out.writeByte((byte) 1);
bos.flush();
@@ -480,7 +491,6 @@ class SimplePrimaryNode extends PrimaryNode {
throw new IllegalArgumentException("unhandled field name " + name);
}
}
-
writer.addDocument(doc);
}
@@ -537,6 +547,7 @@ class SimplePrimaryNode extends PrimaryNode {
static final byte CMD_DELETE_DOC = 8;
static final byte CMD_INDEXING_DONE = 19;
static final byte CMD_DELETE_ALL_DOCS = 22;
+ static final byte CMD_FORCE_MERGE = 23;
// Sent by replica to primary when replica first starts up, so primary can add it to any warming merges:
static final byte CMD_NEW_REPLICA = 20;
@@ -617,15 +628,8 @@ class SimplePrimaryNode extends PrimaryNode {
case CMD_MARKER_SEARCH:
{
Thread.currentThread().setName("msearch");
- IndexSearcher searcher = mgr.acquire();
- try {
- long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
- int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
- out.writeVLong(version);
- out.writeVInt(hitCount);
- } finally {
- mgr.release(searcher);
- }
+ int expectedAtLeastCount = in.readVInt();
+ verifyAtLeastMarkerCount(expectedAtLeastCount, out);
}
continue outer;
@@ -706,4 +710,35 @@ class SimplePrimaryNode extends PrimaryNode {
break;
}
}
+
+ private void verifyAtLeastMarkerCount(int expectedAtLeastCount, DataOutput out) throws IOException {
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+
+ if (hitCount < expectedAtLeastCount) {
+ message("marker search: expectedAtLeastCount=" + expectedAtLeastCount + " but hitCount=" + hitCount);
+ TopDocs hits = searcher.search(new TermQuery(new Term("marker", "marker")), expectedAtLeastCount);
+ List<Integer> seen = new ArrayList<>();
+ for(ScoreDoc hit : hits.scoreDocs) {
+ Document doc = searcher.doc(hit.doc);
+ seen.add(Integer.parseInt(doc.get("docid").substring(1)));
+ }
+ Collections.sort(seen);
+ message("saw markers:");
+ for(int marker : seen) {
+ message("saw m" + marker);
+ }
+ throw new IllegalStateException("at flush: marker count " + hitCount + " but expected at least " + expectedAtLeastCount + " version=" + version);
+ }
+
+ if (out != null) {
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ }
+ } finally {
+ mgr.release(searcher);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
index 27a5547..bc8bb03 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -24,18 +24,24 @@ import java.io.InputStream;
import java.net.ServerSocket;
import java.net.Socket;
import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
+import org.apache.lucene.document.Document;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
@@ -148,6 +154,7 @@ class SimpleReplicaNode extends ReplicaNode {
/** Handles incoming request to the naive TCP server wrapping this node */
void handleOneConnection(ServerSocket ss, AtomicBoolean stop, InputStream is, Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
+ //message("one connection: " + socket);
outer:
while (true) {
byte cmd;
@@ -173,6 +180,7 @@ class SimpleReplicaNode extends ReplicaNode {
long version = in.readVLong();
Thread.currentThread().setName("recv-" + version);
curPrimaryTCPPort = in.readInt();
+ message("newNRTPoint primaryTCPPort=" + curPrimaryTCPPort);
newNRTPoint(version);
}
break;
@@ -235,10 +243,26 @@ class SimpleReplicaNode extends ReplicaNode {
case SimplePrimaryNode.CMD_MARKER_SEARCH:
{
Thread.currentThread().setName("msearch");
+ int expectedAtLeastCount = in.readVInt();
IndexSearcher searcher = mgr.acquire();
try {
long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
- int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+ int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+ if (hitCount < expectedAtLeastCount) {
+ message("marker search: expectedAtLeastCount=" + expectedAtLeastCount + " but hitCount=" + hitCount);
+ TopDocs hits = searcher.search(new TermQuery(new Term("marker", "marker")), expectedAtLeastCount);
+ List<Integer> seen = new ArrayList<>();
+ for(ScoreDoc hit : hits.scoreDocs) {
+ Document doc = searcher.doc(hit.doc);
+ seen.add(Integer.parseInt(doc.get("docid").substring(1)));
+ }
+ Collections.sort(seen);
+ message("saw markers:");
+ for(int marker : seen) {
+ message("saw m" + marker);
+ }
+ }
+
out.writeVLong(version);
out.writeVInt(hitCount);
bos.flush();
@@ -305,7 +329,6 @@ class SimpleReplicaNode extends ReplicaNode {
default:
throw new IllegalArgumentException("unrecognized cmd=" + cmd);
}
- System.out.println("NOW FLUSH");
bos.flush();
break;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index 5b04721..3fdc45f 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -22,6 +22,7 @@ import java.io.BufferedOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.lang.reflect.Method;
+import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
@@ -42,9 +43,9 @@ import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/** Child process with silly naive TCP socket server to handle
@@ -99,8 +100,9 @@ public class SimpleServer extends LuceneTestCase {
success = true;
} catch (Throwable t) {
if (t instanceof SocketException == false && t instanceof NodeCommunicationException == false) {
- node.message("unexpected exception handling client connection:");
+ node.message("unexpected exception handling client connection; now failing test:");
t.printStackTrace(System.out);
+ IOUtils.closeWhileHandlingException(ss);
// Test should fail with this:
throw new RuntimeException(t);
} else {
@@ -218,7 +220,7 @@ public class SimpleServer extends LuceneTestCase {
public void test() throws Exception {
int id = Integer.parseInt(System.getProperty("tests.nrtreplication.nodeid"));
- Thread.currentThread().setName("init child " + id);
+ Thread.currentThread().setName("main child " + id);
Path indexPath = Paths.get(System.getProperty("tests.nrtreplication.indexpath"));
boolean isPrimary = System.getProperty("tests.nrtreplication.isPrimary") != null;
int primaryTCPPort;
@@ -238,7 +240,7 @@ public class SimpleServer extends LuceneTestCase {
boolean doFlipBitsDuringCopy = "true".equals(System.getProperty("tests.nrtreplication.doFlipBitsDuringCopy"));
// Create server socket that we listen for incoming requests on:
- try (final ServerSocket ss = new ServerSocket(0)) {
+ try (final ServerSocket ss = new ServerSocket(0, 0, InetAddress.getLoopbackAddress())) {
int tcpPort = ((InetSocketAddress) ss.getLocalSocketAddress()).getPort();
System.out.println("\nPORT: " + tcpPort);
@@ -247,7 +249,15 @@ public class SimpleServer extends LuceneTestCase {
node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy);
System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
} else {
- node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+ try {
+ node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+ } catch (RuntimeException re) {
+ if (re.getMessage().startsWith("replica cannot start")) {
+ // this is "OK": it means MDW's refusal to delete a segments_N commit point means we cannot start:
+ assumeTrue(re.getMessage(), false);
+ }
+ throw re;
+ }
}
System.out.println("\nINFOS VERSION: " + node.getCurrentSearchingVersion());
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
index d409ffc..4c11e52 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
@@ -197,7 +197,7 @@ class SimpleTransLog implements Closeable {
}
String marker = readNullableString(in);
if (marker != null) {
- //System.out.println("xlog: replay marker=" + id);
+ //TestStressNRTReplication.message("xlog: replay marker=" + id);
doc.add(new StringField("marker", marker, Field.Store.YES));
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 755b234..b2240eb 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -231,8 +231,8 @@ public class TestNRTReplication extends LuceneTestCase {
replicaC.flush();
version2 = replicaC.in.readVLong();
hitCount = replicaC.in.readVInt();
- if (hitCount == 10) {
- assertTrue(version2 > version1);
+ if (version2 == primaryVersion1) {
+ assertEquals(10, hitCount);
// good!
break;
}
@@ -269,8 +269,9 @@ public class TestNRTReplication extends LuceneTestCase {
replicaC.flush();
version4 = replicaC.in.readVLong();
hitCount = replicaC.in.readVInt();
- if (hitCount == 0) {
+ if (version4 == primaryVersion2) {
assertTrue(version4 > version3);
+ assertEquals(0, hitCount);
// good!
break;
}
@@ -293,7 +294,8 @@ public class TestNRTReplication extends LuceneTestCase {
replicaC.flush();
long version5 = replicaC.in.readVLong();
hitCount = replicaC.in.readVInt();
- if (hitCount == 10) {
+ if (version5 == primaryVersion3) {
+ assertEquals(10, hitCount);
assertTrue(version5 > version4);
// good!
break;
@@ -308,6 +310,82 @@ public class TestNRTReplication extends LuceneTestCase {
primary.close();
}
+ public void testReplicateForceMerge() throws Exception {
+
+ Node.globalStartNS = System.nanoTime();
+ childTempDir = createTempDir("child");
+
+ message("change thread name from " + Thread.currentThread().getName());
+ Thread.currentThread().setName("main");
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+
+ // Tell primary current replicas:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+ c.out.writeVInt(1);
+ c.out.writeVInt(replica.id);
+ c.out.writeVInt(replica.tcpPort);
+ c.flush();
+ c.in.readByte();
+ }
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ Connection primaryC = new Connection(primary.tcpPort);
+ primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush();
+ assertTrue(primaryVersion1 > 0);
+
+ // Index 10 more docs into primary:
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion2 = primary.flush();
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ primary.forceMerge(primaryC);
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion3 = primary.flush();
+ assertTrue(primaryVersion3 > primaryVersion2);
+
+ Connection replicaC = new Connection(replica.tcpPort);
+
+ // Wait for replica to show the change
+ while (true) {
+ replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ replicaC.flush();
+ long version = replicaC.in.readVLong();
+ int hitCount = replicaC.in.readVInt();
+ if (version == primaryVersion3) {
+ assertEquals(20, hitCount);
+ // good!
+ break;
+ }
+ Thread.sleep(10);
+ }
+
+ replicaC.close();
+ primaryC.close();
+
+ replica.close();
+ primary.close();
+ }
+
static void message(String message) {
long now = System.nanoTime();
System.out.println(String.format(Locale.ROOT,
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
index 271c5d2..63ff12a 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
@@ -58,10 +58,21 @@ import org.apache.lucene.util.ThreadInterruptedException;
import com.carrotsearch.randomizedtesting.SeedUtils;
+// nocommit why so many "hit SocketException during commit with R0"?
+
+// nocommit why so much time when so many nodes are down
+
+// nocommit indexing is too fast? (xlog replay fails to finish before primary crashes itself)
+
+// nocommit why all these NodeCommunicationExcs?
+
+// nocommit the sockets are a pita on jvm crashing ...
+
/*
TODO
- fangs
- sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
+ - graceful primary close
- why do we do the "rename temp to actual" all at the end...? what really does that buy us?
- replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
- test should not print scary exceptions and then succeed!
@@ -137,11 +148,12 @@ public class TestStressNRTReplication extends LuceneTestCase {
/** Randomly gracefully close a replica; it will later be restarted and sync itself. */
static final boolean DO_CLOSE_REPLICA = true;
+ /** Randomly gracefully close the primary; it will later be restarted and sync itself. */
+ static final boolean DO_CLOSE_PRIMARY = true;
+
/** If false, all child + parent output is interleaved into single stdout/err */
static final boolean SEPARATE_CHILD_OUTPUT = false;
- // nocommit DO_CLOSE_PRIMARY?
-
/** Randomly crash whole cluster and then restart it */
static final boolean DO_FULL_CLUSTER_CRASH = true;
@@ -151,8 +163,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
/** Set to a non-null value to force exactly that many nodes; else, it's random. */
static final Integer NUM_NODES = null;
- static final boolean DO_RANDOM_XLOG_REPLAY = false;
-
final AtomicBoolean failed = new AtomicBoolean();
final AtomicBoolean stop = new AtomicBoolean();
@@ -174,6 +184,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
Path transLogPath;
SimpleTransLog transLog;
final AtomicInteger markerUpto = new AtomicInteger();
+ final AtomicInteger markerID = new AtomicInteger();
/** Maps searcher version to how many hits the query body:the matched. */
final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
@@ -201,7 +212,11 @@ public class TestStressNRTReplication extends LuceneTestCase {
// Silly bootstrapping:
versionToTransLogLocation.put(0L, 0L);
- versionToTransLogLocation.put(1L, 0L);
+
+ // nocommit why also 1?
+ //versionToTransLogLocation.put(1L, 0L);
+
+ versionToMarker.put(0L, 0);
int numNodes;
@@ -268,23 +283,24 @@ public class TestStressNRTReplication extends LuceneTestCase {
// Wait a bit:
Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
if (primary != null && random().nextBoolean()) {
- message("top: now flush primary");
NodeProcess curPrimary = primary;
if (curPrimary != null) {
// Save these before we start flush:
long nextTransLogLoc = transLog.getNextLocation();
int markerUptoSav = markerUpto.get();
+ message("top: now flush primary; at least marker count=" + markerUptoSav);
long result;
try {
- result = primary.flush();
+ result = primary.flush(markerUptoSav);
} catch (Throwable t) {
message("top: flush failed; skipping: " + t.getMessage());
result = -1;
}
if (result > 0) {
// There were changes
+ message("top: flush finished with changed; new primary version=" + result);
lastPrimaryVersion = result;
addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
addVersionMarker(lastPrimaryVersion, markerUptoSav);
@@ -316,7 +332,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
{
NodeProcess node = nodes[random().nextInt(nodes.length)];
- if (node != null) {
+ if (node != null && node.nodeIsClosing.get() == false) {
// TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
// maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
message("top: now commit node=" + node);
@@ -452,27 +468,33 @@ public class TestStressNRTReplication extends LuceneTestCase {
addVersionMarker(newPrimary.initInfosVersion, markerCount);
assert newPrimary.initInfosVersion >= lastPrimaryVersion;
- message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
+ message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion + "; startup marker count " + markerCount);
lastPrimaryVersion = newPrimary.initInfosVersion;
- // Publish new primary, before replaying xlog. This means other indexing ops can come in at the same time as we catch up indexing
- // previous ops. Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
- // translog) concurrently with new incoming ops.
- nodes[id] = newPrimary;
- primary = newPrimary;
-
- sendReplicasToPrimary();
-
long nextTransLogLoc = transLog.getNextLocation();
- int nextMarkerUpto = markerUpto.get();
- message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
+ long t0 = System.nanoTime();
+ message("top: start translog replay " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
try {
transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
} catch (IOException ioe) {
+ // nocommit what if primary node is still running here, and we failed for some other reason?
message("top: replay xlog failed; abort");
return;
}
- message("top: done replay trans log");
+
+ long t1 = System.nanoTime();
+ message("top: done translog replay; took " + ((t1 - t0)/1000000.0) + " msec; now publish primary");
+
+ // Publish new primary only after translog has succeeded in replaying; this is important, for this test anyway, so we keep a "linear"
+ // history so enforcing marker counts is correct. E.g., if we publish first and replay translog concurrently with incoming ops, then
+ // a primary commit that happens while translog is still replaying will incorrectly record the translog loc into the commit user data
+ // when in fact that commit did NOT reflect all prior ops. So if we crash and start up again from that commit point, we are missing
+ // ops.
+ nodes[id] = newPrimary;
+ primary = newPrimary;
+
+ sendReplicasToPrimary();
+
}
/** Launches a child "server" (separate JVM), which is either primary or replica node */
@@ -506,6 +528,9 @@ public class TestStressNRTReplication extends LuceneTestCase {
if (DO_CRASH_PRIMARY) {
cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
}
+ if (DO_CLOSE_PRIMARY) {
+ cmd.add("-Dtests.nrtreplication.doRandomClose=true");
+ }
} else {
if (DO_CRASH_REPLICA) {
cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
@@ -544,7 +569,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
childLog = null;
}
- message("child process command: " + cmd);
+ //message("child process command: " + cmd);
ProcessBuilder pb = new ProcessBuilder(cmd);
pb.redirectErrorStream(true);
@@ -577,6 +602,10 @@ public class TestStressNRTReplication extends LuceneTestCase {
throw new RuntimeException(ie);
}
message("exit value=" + p.exitValue());
+ if (p.exitValue() == 0) {
+ message("zero exit status; assuming failed to remove segments_N; skipping");
+ return null;
+ }
// Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
if (isPrimary == false) {
@@ -586,8 +615,9 @@ public class TestStressNRTReplication extends LuceneTestCase {
message("failed to remove segments_N; skipping");
return null;
}
- for(int i=0;i<10;i++) {
- if (primaryGen != myPrimaryGen || primary == null) {
+ for(int i=0;i<100;i++) {
+ NodeProcess primary2 = primary;
+ if (primaryGen != myPrimaryGen || primary2 == null || primary2.nodeIsClosing.get()) {
// OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
message("primary crashed/closed while replica R" + id + " tried to start; skipping");
return null;
@@ -634,6 +664,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
}
final boolean finalWillCrash = willCrash;
+ final AtomicBoolean nodeIsClosing = new AtomicBoolean();
// Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
Thread pumper = ThreadPumper.start(
@@ -669,11 +700,11 @@ public class TestStressNRTReplication extends LuceneTestCase {
}
nodeClosed(id);
}
- }, r, System.out, childLog);
+ }, r, System.out, childLog, nodeIsClosing);
pumper.setName("pump" + id);
message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
- return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+ return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion, nodeIsClosing);
}
private void nodeClosed(int id) {
@@ -754,7 +785,7 @@ public class TestStressNRTReplication extends LuceneTestCase {
message("top: restarter cycle");
// Randomly crash full cluster:
- if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
+ if (DO_FULL_CLUSTER_CRASH && random().nextInt(500) == 17) {
message("top: full cluster crash");
for(int i=0;i<nodes.length;i++) {
if (starting[i]) {
@@ -954,12 +985,15 @@ public class TestStressNRTReplication extends LuceneTestCase {
continue;
}
+ // nocommit not anymore?
// This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
Integer expectedAtLeastHitCount = versionToMarker.get(version);
+ assertNotNull("version=" + version, expectedAtLeastHitCount);
if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
try {
c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
+ c.out.writeVInt(expectedAtLeastHitCount);
c.flush();
while (c.sockIn.available() == 0) {
if (stop.get()) {
@@ -1064,13 +1098,15 @@ public class TestStressNRTReplication extends LuceneTestCase {
if (random().nextInt(10) == 7) {
// We use the marker docs to check for data loss in search thread:
Document doc = new Document();
- int id = markerUpto.getAndIncrement();
+ int id = markerID.getAndIncrement();
String idString = "m"+id;
doc.add(newStringField("docid", idString, Field.Store.YES));
doc.add(newStringField("marker", "marker", Field.Store.YES));
curPrimary.addOrUpdateDocument(c, doc, false);
transLog.addDocument(idString, doc);
- message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
+ // Only increment after primary replies:
+ markerUpto.getAndIncrement();
+ //message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
}
if (docCount > 0 && random().nextDouble() < updatePct) {
@@ -1094,14 +1130,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
((Field) doc.getField("docid")).setStringValue(idString);
curPrimary.addOrUpdateDocument(c, doc, false);
transLog.addDocument(idString, doc);
-
- if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
- long curLoc = transLog.getNextLocation();
- // randomly replay chunks of translog just to test replay:
- message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
- transLog.replay(curPrimary, lastTransLogLoc, curLoc);
- lastTransLogLoc = curLoc;
- }
}
} catch (IOException se) {
// Assume primary crashed
@@ -1115,12 +1143,13 @@ public class TestStressNRTReplication extends LuceneTestCase {
}
if (random().nextInt(sleepChance) == 0) {
- Thread.sleep(1);
+ Thread.sleep(10);
}
if (random().nextInt(100) == 17) {
- System.out.println("Indexer: now pause for a bit...");
- Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
+ int pauseMS = TestUtil.nextInt(random(), 500, 2000);
+ System.out.println("Indexer: now pause for " + pauseMS + " msec...");
+ Thread.sleep(pauseMS);
System.out.println("Indexer: done pause for a bit...");
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
index 6ddb777..d74e170 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java
@@ -21,11 +21,12 @@ import java.io.BufferedReader;
import java.io.IOException;
import java.io.PrintStream;
import java.io.Writer;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.regex.Pattern;
/** A pipe thread. It'd be nice to reuse guava's implementation for this... */
class ThreadPumper {
- public static Thread start(final Runnable onExit, final BufferedReader from, final PrintStream to, final Writer toFile) {
+ public static Thread start(final Runnable onExit, final BufferedReader from, final PrintStream to, final Writer toFile, final AtomicBoolean nodeClosing) {
Thread t = new Thread() {
@Override
public void run() {
@@ -43,6 +44,9 @@ class ThreadPumper {
} else {
TestNRTReplication.message(line);
}
+ if (line.contains("now force close server socket after")) {
+ nodeClosing.set(true);
+ }
}
// Sub-process finished
} catch (IOException e) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8889469b/lucene/replicator/test.cmd
----------------------------------------------------------------------
diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd
index 14e3bd2..c38fc96 100644
--- a/lucene/replicator/test.cmd
+++ b/lucene/replicator/test.cmd
@@ -1 +1 @@
-python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestNRTReplication -jvms 1 -mult 4 -nightly
+python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 1 -mult 4 -nightly
[03/15] lucene-solr git commit: current patch
Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
new file mode 100644
index 0000000..742b19f
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
@@ -0,0 +1,213 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ReferenceManager;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.StringHelper;
+
+/** Common base class for {@link PrimaryNode} and {@link ReplicaNode}. */
+
+abstract class Node implements Closeable {
+
+ static boolean VERBOSE_FILES = true;
+ static boolean VERBOSE_CONNECTIONS = false;
+
+ // Keys we store into IndexWriter's commit user data:
+
+ /** Key to store the primary gen in the commit data, which increments every time we promote a new primary, so replicas can detect when the
+ * primary they were talking to is changed */
+ public static String PRIMARY_GEN_KEY = "__primaryGen";
+
+ /** Key to store the version in the commit data, which increments every time we open a new NRT reader */
+ public static String VERSION_KEY = "__version";
+
+ /** Compact ordinal for this node */
+ protected final int id;
+
+ protected final Directory dir;
+
+ protected final SearcherFactory searcherFactory;
+
+ // Tracks NRT readers, opened from IW (primary) or opened from replicated SegmentInfos pulled across the wire (replica):
+ protected ReferenceManager<IndexSearcher> mgr;
+
+ /** Startup time of original test, carefully propogated to all nodes to produce consistent "seconds since start time" in messages */
+ public static long globalStartNS;
+
+ /** When this node was started */
+ public static final long localStartNS = System.nanoTime();
+
+ // public static final long globalStartNS;
+
+ // For debugging:
+ volatile String state = "idle";
+
+ /** File metadata for last sync that succeeded; we use this as a cache */
+ protected volatile Map<String,FileMetaData> lastFileMetaData;
+
+ public Node(int id, Directory dir, SearcherFactory searcherFactory) {
+ this.id = id;
+ this.dir = dir;
+ this.searcherFactory = searcherFactory;
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(id=" + id + ")";
+ }
+
+ public abstract void commit() throws IOException;
+
+ public static void nodeMessage(String message) {
+ long now = System.nanoTime();
+ System.out.println(String.format(Locale.ROOT,
+ "%5.3fs %5.1fs: [%11s] %s",
+ (now-globalStartNS)/1000000000.,
+ (now-localStartNS)/1000000000.,
+ Thread.currentThread().getName(),
+ message));
+
+ }
+
+ public static void nodeMessage(int id, String message) {
+ long now = System.nanoTime();
+ System.out.println(String.format(Locale.ROOT,
+ "%5.3fs %5.1fs: N%d [%11s] %s",
+ (now-globalStartNS)/1000000000.,
+ (now-localStartNS)/1000000000.,
+ id,
+ Thread.currentThread().getName(),
+ message));
+
+ }
+
+ protected void message(String message) {
+ long now = System.nanoTime();
+ System.out.println(String.format(Locale.ROOT,
+ "%5.3fs %5.1fs: %7s %2s [%11s] %s",
+ (now-globalStartNS)/1000000000.,
+ (now-localStartNS)/1000000000.,
+ state, name(),
+ Thread.currentThread().getName(), message));
+ }
+
+ public String name() {
+ char mode = this instanceof PrimaryNode ? 'P' : 'R';
+ return mode + Integer.toString(id);
+ }
+
+ public abstract boolean isClosed();
+
+ public long getCurrentSearchingVersion() throws IOException {
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ return ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+
+ public static String bytesToString(long bytes) {
+ if (bytes < 1024) {
+ return bytes + " b";
+ } else if (bytes < 1024 * 1024) {
+ return String.format(Locale.ROOT, "%.1f KB", bytes/1024.);
+ } else if (bytes < 1024 * 1024 * 1024) {
+ return String.format(Locale.ROOT, "%.1f MB", bytes/1024./1024.);
+ } else {
+ return String.format(Locale.ROOT, "%.1f GB", bytes/1024./1024./1024.);
+ }
+ }
+
+ /** Opens the specified file, reads its identifying information, including file length, full index header (includes the unique segment
+ * ID) and the full footer (includes checksum), and returns the resulting {@link FileMetaData}.
+ *
+ * <p>This returns null, logging a message, if there are any problems (the file does not exist, is corrupt, truncated, etc.).</p> */
+ public FileMetaData readLocalFileMetaData(String fileName) throws IOException {
+
+ Map<String,FileMetaData> cache = lastFileMetaData;
+ FileMetaData result;
+ if (cache != null) {
+ // We may already have this file cached from the last NRT point:
+ result = cache.get(fileName);
+ } else {
+ result = null;
+ }
+
+ if (result == null) {
+ // Pull from the filesystem
+ long checksum;
+ long length;
+ byte[] header;
+ byte[] footer;
+ try (IndexInput in = dir.openInput(fileName, IOContext.DEFAULT)) {
+ try {
+ length = in.length();
+ header = CodecUtil.readIndexHeader(in);
+ footer = CodecUtil.readFooter(in);
+ checksum = CodecUtil.retrieveChecksum(in);
+ } catch (EOFException | CorruptIndexException cie) {
+ // File exists but is busted: we must copy it. This happens when node had crashed, corrupting an un-fsync'd file. On init we try
+ // to delete such unreferenced files, but virus checker can block that, leaving this bad file.
+ if (VERBOSE_FILES) {
+ message("file " + fileName + ": will copy [existing file is corrupt]");
+ }
+ return null;
+ }
+ if (VERBOSE_FILES) {
+ message("file " + fileName + " has length=" + bytesToString(length));
+ }
+ } catch (FileNotFoundException | NoSuchFileException e) {
+ if (VERBOSE_FILES) {
+ message("file " + fileName + ": will copy [file does not exist]");
+ }
+ return null;
+ }
+
+ // NOTE: checksum is redundant w/ footer, but we break it out separately because when the bits cross the wire we need direct access to
+ // checksum when copying to catch bit flips:
+ result = new FileMetaData(header, footer, length, checksum);
+ }
+
+ return result;
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
new file mode 100644
index 0000000..67a9d0a
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java
@@ -0,0 +1,26 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class NodeCommunicationException extends RuntimeException {
+ public NodeCommunicationException(String when, Throwable cause) {
+ super(when);
+ assert cause != null;
+ initCause(cause);
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
new file mode 100644
index 0000000..1918ede
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
@@ -0,0 +1,80 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** A merged segment warmer that pre-copies the merged segment out to
+ * replicas before primary cuts over to the merged segment. This
+ * ensures that NRT reopen time on replicas is only in proportion to
+ * flushed segment sizes, not merged segments. */
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+
+import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentReader;
+import org.apache.lucene.replicator.nrt.CopyJob.OnceDone;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+// TODO: or ... replica node can do merging locally? tricky to keep things in sync, when one node merges more slowly than others...
+
+class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
+
+ private final PrimaryNode primary;
+
+ public PreCopyMergedSegmentWarmer(PrimaryNode primary) {
+ this.primary = primary;
+ }
+
+ @Override
+ public void warm(LeafReader reader) throws IOException {
+ long startNS = System.nanoTime();
+ final SegmentCommitInfo info = ((SegmentReader) reader).getSegmentInfo();
+ //System.out.println("TEST: warm merged segment files " + info);
+ Map<String,FileMetaData> filesMetaData = new HashMap<>();
+ for(String fileName : info.files()) {
+ FileMetaData metaData = primary.readLocalFileMetaData(fileName);
+ assert metaData != null;
+ assert filesMetaData.containsKey(fileName) == false;
+ filesMetaData.put(fileName, metaData);
+ }
+
+ // nocommit if one replica is very slow then it dos's all other replicas?
+
+ primary.preCopyMergedSegmentFiles(info, filesMetaData);
+ primary.message(String.format(Locale.ROOT, "top: done warm merge " + info + ": took %.3f sec, %.1f MB", (System.nanoTime()-startNS)/1000000000., info.sizeInBytes()/1024/1024.));
+ primary.finishedMergedFiles.addAll(filesMetaData.keySet());
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
new file mode 100644
index 0000000..183f16f
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -0,0 +1,316 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.StandardDirectoryReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.RAMFile;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.PrintStreamInfoStream;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/*
+ * This just asks IndexWriter to open new NRT reader, in order to publish a new NRT point. This could be improved, if we separated out 1)
+ * nrt flush (and incRef the SIS) from 2) opening a new reader, but this is tricky with IW's concurrency, and it would also be hard-ish to share
+ * IW's reader pool with our searcher manager. So we do the simpler solution now, but that adds some unecessary latency to NRT refresh on
+ * replicas since step 2) could otherwise be done concurrently with replicas copying files over.
+ */
+
+/** Node that holds an IndexWriter, indexing documents into its local index. */
+public abstract class PrimaryNode extends Node {
+
+ // Current NRT segment infos, incRef'd with IndexWriter.deleter:
+ private SegmentInfos curInfos;
+
+ final IndexWriter writer;
+
+ // IncRef'd state of the last published NRT point; when a replica comes asking, we give it this as the current NRT point:
+ private CopyState copyState;
+
+ protected final long primaryGen;
+
+ /** Contains merged segments that have been copied to all running replicas (as of when that merge started warming). */
+ final Set<String> finishedMergedFiles = Collections.synchronizedSet(new HashSet<String>());
+
+ private final AtomicInteger copyingCount = new AtomicInteger();
+
+ public PrimaryNode(IndexWriter writer, int id, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException {
+ super(id, writer.getDirectory(), searcherFactory);
+ message("top: now init primary");
+ this.writer = writer;
+ this.primaryGen = primaryGen;
+
+ try {
+ // So that when primary node's IndexWriter finishes a merge, but before it cuts over to the merged segment,
+ // it copies it out to the replicas. This ensures the whole system's NRT latency remains low even when a
+ // large merge completes:
+ writer.getConfig().setMergedSegmentWarmer(new PreCopyMergedSegmentWarmer(this));
+
+ message("IWC:\n" + writer.getConfig());
+ message("dir:\n" + writer.getDirectory());
+ message("commitData: " + writer.getCommitData());
+
+ // Record our primaryGen in the userData, and set initial version to 0:
+ Map<String,String> commitData = new HashMap<>(writer.getCommitData());
+ commitData.put(PRIMARY_GEN_KEY, Long.toString(primaryGen));
+ if (commitData.get(VERSION_KEY) == null) {
+ commitData.put(VERSION_KEY, "0");
+ message("add initial commitData version=0");
+ } else {
+ message("keep current commitData version=" + commitData.get(VERSION_KEY));
+ }
+ writer.setCommitData(commitData, false);
+
+ // We forcefully advance the SIS version to an unused future version. This is necessary if the previous primary crashed and we are
+ // starting up on an "older" index, else versions can be illegally reused but show different results:
+ if (forcePrimaryVersion != -1) {
+ message("now forcePrimaryVersion to version=" + forcePrimaryVersion);
+ writer.advanceSegmentInfosVersion(forcePrimaryVersion);
+ }
+
+ mgr = new SearcherManager(writer, true, true, searcherFactory);
+ setCurrentInfos(Collections.<String>emptySet());
+ message("init: infos version=" + curInfos.getVersion());
+
+ IndexSearcher s = mgr.acquire();
+ try {
+ message("init: marker hit count: " + s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits);
+ } finally {
+ mgr.release(s);
+ }
+
+ } catch (Throwable t) {
+ message("init: exception");
+ t.printStackTrace(System.out);
+ throw new RuntimeException(t);
+ }
+ }
+
+ // TODO: in the future, we should separate "flush" (returns an incRef'd SegmentInfos) from "refresh" (open new NRT reader from
+ // IndexWriter) so that the latter can be done concurrently while copying files out to replicas, minimizing the refresh time from the
+ // replicas. But fixing this is tricky because e.g. IndexWriter may complete a big merge just after returning the incRef'd SegmentInfos
+ // and before we can open a new reader causing us to close the just-merged readers only to then open them again from the (now stale)
+ // SegmentInfos. To fix this "properly" I think IW.inc/decRefDeleter must also incread the ReaderPool entry
+
+ /** Flush all index operations to disk and opens a new near-real-time reader.
+ * new NRT point, to make the changes visible to searching. Returns true if there were changes. */
+ public boolean flushAndRefresh() throws IOException {
+ message("top: now flushAndRefresh");
+ Set<String> completedMergeFiles;
+ synchronized(finishedMergedFiles) {
+ completedMergeFiles = Collections.unmodifiableSet(new HashSet<>(finishedMergedFiles));
+ }
+ mgr.maybeRefreshBlocking();
+ boolean result = setCurrentInfos(completedMergeFiles);
+ if (result) {
+ message("top: opened NRT reader version=" + curInfos.getVersion());
+ finishedMergedFiles.removeAll(completedMergeFiles);
+ message("flushAndRefresh: version=" + curInfos.getVersion() + " completedMergeFiles=" + completedMergeFiles + " finishedMergedFiles=" + finishedMergedFiles);
+ } else {
+ message("top: no changes in flushAndRefresh; still version=" + curInfos.getVersion());
+ }
+ return result;
+ }
+
+ public long getCopyStateVersion() {
+ return copyState.version;
+ }
+
+ public synchronized long getLastCommitVersion() {
+ String s = curInfos.getUserData().get(VERSION_KEY);
+ // In ctor we always install an initial version:
+ assert s != null;
+ return Long.parseLong(s);
+ }
+
+ @Override
+ public void commit() throws IOException {
+ Map<String,String> commitData = new HashMap<>();
+ commitData.put(PRIMARY_GEN_KEY, Long.toString(primaryGen));
+ // TODO (opto): it's a bit wasteful that we put "last refresh" version here, not the actual version we are committing, because it means
+ // on xlog replay we are replaying more ops than necessary.
+ commitData.put(VERSION_KEY, Long.toString(copyState.version));
+ message("top: commit commitData=" + commitData);
+ // nocommit this is now an NRT-visible change! make test where nothing is indexing and confirm we don't do silly commit + refresh loop forever!
+ writer.setCommitData(commitData, false);
+ writer.commit();
+ }
+
+ /** IncRef the current CopyState and return it */
+ public synchronized CopyState getCopyState() throws IOException {
+ ensureOpen(false);
+ //message("top: getCopyState replicaID=" + replicaID + " replicaNodeID=" + replicaNodeID + " version=" + curInfos.getVersion() + " infos=" + curInfos.toString());
+ assert curInfos == copyState.infos;
+ writer.incRefDeleter(copyState.infos);
+ int count = copyingCount.incrementAndGet();
+ assert count > 0;
+ return copyState;
+ }
+
+ /** Called once replica is done (or failed) copying an NRT point */
+ public void releaseCopyState(CopyState copyState) throws IOException {
+ //message("top: releaseCopyState version=" + copyState.version);
+ assert copyState.infos != null;
+ writer.decRefDeleter(copyState.infos);
+ int count = copyingCount.decrementAndGet();
+ assert count >= 0;
+ }
+
+ @Override
+ public boolean isClosed() {
+ return isClosed(false);
+ }
+
+ boolean isClosed(boolean allowClosing) {
+ return "closed".equals(state) || (allowClosing == false && "closing".equals(state));
+ }
+
+ private void ensureOpen(boolean allowClosing) {
+ if (isClosed(allowClosing)) {
+ throw new AlreadyClosedException(state);
+ }
+ }
+
+ /** Steals incoming infos refCount; returns true if there were changes. */
+ private synchronized boolean setCurrentInfos(Set<String> completedMergeFiles) throws IOException {
+
+ IndexSearcher searcher = null;
+ SegmentInfos infos;
+ try {
+ searcher = mgr.acquire();
+ infos = ((StandardDirectoryReader) searcher.getIndexReader()).getSegmentInfos();
+ } finally {
+ if (searcher != null) {
+ mgr.release(searcher);
+ }
+ }
+ if (curInfos != null && infos.getVersion() == curInfos.getVersion()) {
+ // no change
+ message("top: skip switch to infos: version=" + infos.getVersion() + " is unchanged: " + infos.toString());
+ return false;
+ }
+
+ SegmentInfos oldInfos = curInfos;
+ writer.incRefDeleter(infos);
+ curInfos = infos;
+ if (oldInfos != null) {
+ writer.decRefDeleter(oldInfos);
+ }
+
+ message("top: switch to infos=" + infos.toString() + " version=" + infos.getVersion());
+
+ // Serialize the SegmentInfos:
+ RAMOutputStream out = new RAMOutputStream(new RAMFile(), true);
+ infos.write(dir, out);
+ byte[] infosBytes = new byte[(int) out.getFilePointer()];
+ out.writeTo(infosBytes, 0);
+
+ Map<String,FileMetaData> filesMetaData = new HashMap<String,FileMetaData>();
+ for(SegmentCommitInfo info : infos) {
+ for(String fileName : info.files()) {
+ FileMetaData metaData = readLocalFileMetaData(fileName);
+ // NOTE: we hold a refCount on this infos, so this file better exist:
+ assert metaData != null;
+ assert filesMetaData.containsKey(fileName) == false;
+ filesMetaData.put(fileName, metaData);
+ }
+ }
+
+ lastFileMetaData = Collections.unmodifiableMap(filesMetaData);
+
+ message("top: set copyState primaryGen=" + primaryGen + " version=" + infos.getVersion() + " files=" + filesMetaData.keySet());
+ copyState = new CopyState(lastFileMetaData,
+ infos.getVersion(), infos.getGeneration(), infosBytes, completedMergeFiles,
+ primaryGen, curInfos);
+ return true;
+ }
+
+ private synchronized void waitForAllRemotesToClose() throws IOException {
+
+ // Wait for replicas to finish or crash:
+ while (true) {
+ int count = copyingCount.get();
+ if (count == 0) {
+ return;
+ }
+ message("pendingCopies: " + count);
+
+ try {
+ wait(10);
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ state = "closing";
+ message("top: close primary");
+
+ synchronized (this) {
+ waitForAllRemotesToClose();
+ if (curInfos != null) {
+ writer.decRefDeleter(curInfos);
+ curInfos = null;
+ }
+ }
+
+ mgr.close();
+
+ writer.rollback();
+ dir.close();
+
+ state = "closed";
+ }
+
+ /** Called when a merge has finished, but before IW switches to the merged segment */
+ protected abstract void preCopyMergedSegmentFiles(SegmentCommitInfo info, Map<String,FileMetaData> files) throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
new file mode 100644
index 0000000..005f938
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java
@@ -0,0 +1,218 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.NoSuchFileException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+// TODO: can we factor/share with IFD: this is doing exactly the same thing, but on the replica side
+
+// TODO: once LUCENE-6835 is in, this class becomes a lot simpler?
+
+class ReplicaFileDeleter {
+ private final Map<String,Integer> refCounts = new HashMap<String,Integer>();
+ private final Set<String> pending = new HashSet<String>();
+ private final Directory dir;
+ private final Node node;
+
+ public ReplicaFileDeleter(Node node, Directory dir) throws IOException {
+ this.dir = dir;
+ this.node = node;
+ }
+
+ /** Used only by asserts: returns true if the file exists
+ * (can be opened), false if it cannot be opened, and
+ * (unlike Java's File.exists) throws IOException if
+ * there's some unexpected error. */
+ static boolean slowFileExists(Directory dir, String fileName) throws IOException {
+ try {
+ dir.openInput(fileName, IOContext.DEFAULT).close();
+ return true;
+ } catch (NoSuchFileException | FileNotFoundException e) {
+ return false;
+ }
+ }
+
+ public synchronized void incRef(Collection<String> fileNames) throws IOException {
+ for(String fileName : fileNames) {
+
+ if (pending.contains(fileName)) {
+ throw new IllegalStateException("cannot incRef file \"" + fileName + "\": it is pending delete");
+ }
+
+ assert slowFileExists(dir, fileName): "file " + fileName + " does not exist!";
+
+ Integer curCount = refCounts.get(fileName);
+ if (curCount == null) {
+ refCounts.put(fileName, 1);
+ } else {
+ refCounts.put(fileName, curCount.intValue() + 1);
+ }
+ }
+ }
+
+ public synchronized void decRef(Collection<String> fileNames) {
+ // We don't delete the files immediately when their RC drops to 0; instead, we add to the pending set, and then call deletePending in
+ // the end:
+ for(String fileName : fileNames) {
+ Integer curCount = refCounts.get(fileName);
+ assert curCount != null: "fileName=" + fileName;
+ assert curCount.intValue() > 0;
+ if (curCount.intValue() == 1) {
+ refCounts.remove(fileName);
+ pending.add(fileName);
+ } else {
+ refCounts.put(fileName, curCount.intValue() - 1);
+ }
+ }
+
+ deletePending();
+
+ // TODO: this local IR could incRef files here, like we do now with IW ... then we can assert this again:
+
+ // we can't assert this, e.g a search can be running when we switch to a new NRT point, holding a previous IndexReader still open for
+ // a bit:
+ /*
+ // We should never attempt deletion of a still-open file:
+ Set<String> delOpen = ((MockDirectoryWrapper) dir).getOpenDeletedFiles();
+ if (delOpen.isEmpty() == false) {
+ node.message("fail: we tried to delete these still-open files: " + delOpen);
+ throw new AssertionError("we tried to delete these still-open files: " + delOpen);
+ }
+ */
+ }
+
+ private synchronized boolean delete(String fileName) {
+ try {
+ if (Node.VERBOSE_FILES) {
+ node.message("file " + fileName + ": now delete");
+ }
+ dir.deleteFile(fileName);
+ pending.remove(fileName);
+ return true;
+ } catch (FileNotFoundException|NoSuchFileException missing) {
+ // This should never happen: we should only be asked to track files that do exist
+ node.message("file " + fileName + ": delete failed: " + missing);
+ throw new IllegalStateException("file " + fileName + ": we attempted delete but the file does not exist?", missing);
+ } catch (IOException ioe) {
+ if (Node.VERBOSE_FILES) {
+ node.message("file " + fileName + ": delete failed: " + ioe + "; will retry later");
+ }
+ pending.add(fileName);
+ return false;
+ }
+ }
+
+ public synchronized Integer getRefCount(String fileName) {
+ return refCounts.get(fileName);
+ }
+
+ public synchronized boolean isPending(String fileName) {
+ return pending.contains(fileName);
+ }
+
+ public synchronized void deletePending() {
+ if (Node.VERBOSE_FILES) {
+ node.message("now deletePending: " + pending.size() + " files to try: " + pending);
+ }
+
+ // Clone the set because it will change as we iterate:
+ List<String> toDelete = new ArrayList<>(pending);
+
+ // First pass: delete any segments_N files. We do these first to be certain stale commit points are removed
+ // before we remove any files they reference. If any delete of segments_N fails, we leave all other files
+ // undeleted so index is never in a corrupt state:
+ for (String fileName : toDelete) {
+ Integer rc = refCounts.get(fileName);
+ if (rc != null && rc > 0) {
+ // Should never happen! This means we are about to pending-delete a referenced index file
+ throw new IllegalStateException("file \"" + fileName + "\" is in pending delete set but has non-zero refCount=" + rc);
+ } else if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+ if (delete(fileName) == false) {
+ if (Node.VERBOSE_FILES) {
+ node.message("failed to remove commit point \"" + fileName + "\"; skipping deletion of all other pending files");
+ }
+ return;
+ }
+ }
+ }
+
+ // Only delete other files if we were able to remove the segments_N files; this way we never
+ // leave a corrupt commit in the index even in the presense of virus checkers:
+ for(String fileName : toDelete) {
+ if (fileName.startsWith(IndexFileNames.SEGMENTS) == false) {
+ delete(fileName);
+ }
+ }
+
+ Set<String> copy = new HashSet<String>(pending);
+ pending.clear();
+ for(String fileName : copy) {
+ delete(fileName);
+ }
+ }
+
+ /** Necessary in case we had tried to delete this fileName before, it failed, but then it was later overwritten (because primary changed
+ * and new primary didn't know this segment name had been previously attempted) and now has > 0 refCount */
+ public synchronized void clearPending(Collection<String> fileNames) {
+ for(String fileName : fileNames) {
+ if (pending.remove(fileName)) {
+ node.message("file " + fileName + ": deleter.clearPending now clear from pending");
+ }
+ }
+ }
+
+ public synchronized void deleteIfNoRef(String fileName) {
+ if (refCounts.containsKey(fileName) == false) {
+ deleteNewFile(fileName);
+ }
+ }
+
+ public synchronized void deleteNewFile(String fileName) {
+ delete(fileName);
+ }
+
+ public synchronized Set<String> getPending() {
+ return new HashSet<String>(pending);
+ }
+
+ public synchronized void deleteUnknownFiles(String segmentsFileName) throws IOException {
+ for(String fileName : dir.listAll()) {
+ if (refCounts.containsKey(fileName) == false &&
+ fileName.equals("write.lock") == false &&
+ fileName.equals(segmentsFileName) == false) {
+ node.message("will delete unknown file \"" + fileName + "\"");
+ pending.add(fileName);
+ }
+ }
+
+ deletePending();
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
new file mode 100644
index 0000000..af142d5
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -0,0 +1,772 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteArrayIndexInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.Lock;
+import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */
+
+abstract class ReplicaNode extends Node {
+
+ ReplicaFileDeleter deleter;
+
+ /** IncRef'd files in the current commit point: */
+ private final Collection<String> lastCommitFiles = new HashSet<>();
+
+ /** IncRef'd files in the current NRT point: */
+ protected final Collection<String> lastNRTFiles = new HashSet<>();
+
+ /** Currently running merge pre-copy jobs */
+ protected final Set<CopyJob> mergeCopyJobs = Collections.synchronizedSet(new HashSet<>());
+
+ /** Non-null when we are currently copying files from a new NRT point: */
+ protected CopyJob curNRTCopy;
+
+ /** We hold this to ensure an external IndexWriter cannot also open on our directory: */
+ private final Lock writeFileLock;
+
+ /** Merged segment files that we pre-copied, but have not yet made visible in a new NRT point. */
+ final Set<String> pendingMergeFiles = Collections.synchronizedSet(new HashSet<String>());
+
+ /** Primary gen last time we successfully replicated: */
+ protected long lastPrimaryGen;
+
+ public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory) throws IOException {
+ super(id, dir, searcherFactory);
+
+ boolean success = false;
+
+ try {
+ message("top: init replica dir=" + dir);
+
+ // Obtain a write lock on this index since we "act like" an IndexWriter, to prevent any other IndexWriter or ReplicaNode from using it:
+ writeFileLock = dir.obtainLock(IndexWriter.WRITE_LOCK_NAME);
+
+ state = "init";
+ deleter = new ReplicaFileDeleter(this, dir);
+ } catch (Throwable t) {
+ message("exc on init:");
+ t.printStackTrace(System.out);
+ throw t;
+ } finally {
+ if (success == false) {
+ IOUtils.closeWhileHandlingException(this);
+ }
+ }
+ }
+
+ /** Start up this replica, which possibly requires heavy copying of files from the primary node, if we were down for a long time */
+ protected synchronized void start(long curPrimaryGen) throws IOException {
+
+ if (state.equals("init") == false) {
+ throw new IllegalStateException("already started");
+ }
+
+ message("top: now start");
+ try {
+
+ // Figure out what state our local index is in now:
+ String segmentsFileName = SegmentInfos.getLastCommitSegmentsFileName(dir);
+
+ // Also look for any pending_segments_N, in case we crashed mid-commit. We must "inflate" our infos gen to at least this, since
+ // otherwise we may wind up re-using the pending_segments_N file name on commit, and then our deleter can get angry because it still
+ // wants to delete this file:
+ long maxPendingGen = -1;
+ for(String fileName : dir.listAll()) {
+ if (fileName.startsWith(IndexFileNames.PENDING_SEGMENTS)) {
+ long gen = Long.parseLong(fileName.substring(IndexFileNames.PENDING_SEGMENTS.length()+1), Character.MAX_RADIX);
+ if (gen > maxPendingGen) {
+ maxPendingGen = gen;
+ }
+ }
+ }
+
+ SegmentInfos infos;
+ if (segmentsFileName == null) {
+ // No index here yet:
+ infos = new SegmentInfos();
+ message("top: init: no segments in index");
+ } else {
+ message("top: init: read existing segments commit " + segmentsFileName);
+ infos = SegmentInfos.readCommit(dir, segmentsFileName);
+ message("top: init: segments: " + infos.toString() + " version=" + infos.getVersion());
+ Collection<String> indexFiles = infos.files(false);
+
+ lastCommitFiles.add(segmentsFileName);
+ lastCommitFiles.addAll(indexFiles);
+
+ // Always protect the last commit:
+ deleter.incRef(lastCommitFiles);
+
+ lastNRTFiles.addAll(indexFiles);
+ deleter.incRef(lastNRTFiles);
+ message("top: commitFiles=" + lastCommitFiles);
+ message("top: nrtFiles=" + lastNRTFiles);
+ }
+
+ message("top: delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
+ deleter.deleteUnknownFiles(segmentsFileName);
+ message("top: done delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
+
+ // nocommit make test where index has all docs deleted (all segments dropped, so 0 segments) and is then replicated
+
+ String s = infos.getUserData().get(PRIMARY_GEN_KEY);
+ long myPrimaryGen;
+ if (s == null) {
+ assert infos.size() == 0;
+ myPrimaryGen = -1;
+ } else {
+ myPrimaryGen = Long.parseLong(s);
+ }
+ message("top: myPrimaryGen=" + myPrimaryGen);
+
+ boolean doCommit;
+
+ if (infos.size() > 0 && myPrimaryGen != -1 && myPrimaryGen != curPrimaryGen) {
+
+ assert myPrimaryGen < curPrimaryGen;
+
+ // Primary changed while we were down. In this case, we must sync from primary before opening a reader, because it's possible current
+ // files we have will need to be overwritten with different ones (if index rolled back and "forked"), and we can't overwrite open
+ // files on Windows:
+
+ final long initSyncStartNS = System.nanoTime();
+
+ message("top: init: primary changed while we were down myPrimaryGen=" + myPrimaryGen +
+ " vs curPrimaryGen=" + curPrimaryGen +
+ "; sync now before mgr init");
+
+ // Try until we succeed in copying over the latest NRT point:
+ CopyJob job = null;
+
+ // We may need to overwrite files referenced by our latest commit, either right now on initial sync, or on a later sync. To make
+ // sure the index is never even in an "apparently" corrupt state (where an old segments_N references invalid files) we forcefully
+ // remove the commit now, and refuse to start the replica if this delete fails:
+ message("top: now delete starting commit point " + segmentsFileName);
+
+ // If this throws exc (e.g. due to virus checker), we cannot start this replica:
+ assert deleter.getRefCount(segmentsFileName) == 1;
+ deleter.decRef(Collections.singleton(segmentsFileName));
+ if (deleter.isPending(segmentsFileName)) {
+ // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else we can cause corruption:
+ throw new RuntimeException("replica cannot start: existing segments file=" + segmentsFileName + " must be removed in order to start, but the file delete failed");
+ }
+ // So we don't later try to decRef it (illegally) again:
+ boolean didRemove = lastCommitFiles.remove(segmentsFileName);
+ assert didRemove;
+
+ while (true) {
+ job = newCopyJob("sync on startup replica=" + name() + " myVersion=" + infos.getVersion(),
+ null,
+ null,
+ true,
+ null);
+ job.start();
+
+ message("top: init: sync sis.version=" + job.getCopyState().version);
+
+ Collection<String> fileNamesToCopy = job.getFileNamesToCopy();
+
+ // Force this copy job to finish while we wait, now. Note that this can be very time consuming!
+ // NOTE: newNRTPoint detects we are still in init (mgr is null) and does not cancel our copy if a flush happens
+ try {
+ job.runBlocking();
+ job.finish();
+
+ // Success!
+ break;
+ } catch (IOException ioe) {
+ job.cancel("startup failed", ioe);
+ if (ioe.getMessage().contains("checksum mismatch after file copy")) {
+ // OK-ish
+ message("top: failed to copy: " + ioe + "; retrying");
+ } else {
+ throw ioe;
+ }
+ }
+ }
+
+ lastPrimaryGen = job.getCopyState().primaryGen;
+ byte[] infosBytes = job.getCopyState().infosBytes;
+
+ SegmentInfos syncInfos = SegmentInfos.readCommit(dir,
+ new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", job.getCopyState().infosBytes)),
+ job.getCopyState().gen);
+
+ // Must always commit to a larger generation than what's currently in the index:
+ syncInfos.updateGeneration(infos);
+ infos = syncInfos;
+
+ assert infos.getVersion() == job.getCopyState().version;
+ message(" version=" + infos.getVersion() + " segments=" + infos.toString());
+ message("top: init: incRef nrtFiles=" + job.getFileNames());
+ deleter.incRef(job.getFileNames());
+ message("top: init: decRef lastNRTFiles=" + lastNRTFiles);
+ deleter.decRef(lastNRTFiles);
+
+ lastNRTFiles.clear();
+ lastNRTFiles.addAll(job.getFileNames());
+
+ message("top: init: set lastNRTFiles=" + lastNRTFiles);
+ lastFileMetaData = job.getCopyState().files;
+ message(String.format(Locale.ROOT, "top: %d: start: done sync: took %.3fs for %s, opened NRT reader version=%d",
+ id,
+ (System.nanoTime()-initSyncStartNS)/1000000000.0,
+ bytesToString(job.getTotalBytesCopied()),
+ job.getCopyState().version));
+
+ doCommit = true;
+ } else {
+ doCommit = false;
+ lastPrimaryGen = curPrimaryGen;
+ message("top: same primary as before");
+ }
+
+ if (infos.getGeneration() < maxPendingGen) {
+ message("top: move infos generation from " + infos.getGeneration() + " to " + maxPendingGen);
+ infos.setNextWriteGeneration(maxPendingGen);
+ }
+
+ // Notify primary we started, to give it a chance to send any warming merges our way to reduce NRT latency of first sync:
+ sendNewReplica();
+
+ // Finally, we are open for business, since our index now "agrees" with the primary:
+ mgr = new SegmentInfosSearcherManager(dir, this, infos, searcherFactory);
+
+ // Must commit after init mgr:
+ if (doCommit) {
+ // Very important to commit what we just sync'd over, because we removed the pre-existing commit point above if we had to
+ // overwrite any files it referenced:
+ commit();
+ }
+
+ message("top: done start");
+ state = "idle";
+ } catch (Throwable t) {
+ message("exc on start:");
+ t.printStackTrace(System.out);
+ throw new RuntimeException(t);
+ }
+ }
+
+ final Object commitLock = new Object();
+
+ @Override
+ public void commit() throws IOException {
+
+ synchronized(commitLock) {
+
+ SegmentInfos infos;
+ Collection<String> indexFiles;
+
+ synchronized (this) {
+ infos = ((SegmentInfosSearcherManager) mgr).getCurrentInfos();
+ indexFiles = infos.files(false);
+ deleter.incRef(indexFiles);
+ }
+
+ message("top: commit primaryGen=" + lastPrimaryGen + " infos=" + infos.toString() + " files=" + indexFiles);
+
+ // fsync all index files we are now referencing
+ dir.sync(indexFiles);
+
+ Map<String,String> commitData = new HashMap<>();
+ commitData.put(PRIMARY_GEN_KEY, Long.toString(lastPrimaryGen));
+ commitData.put(VERSION_KEY, Long.toString(getCurrentSearchingVersion()));
+ infos.setUserData(commitData, false);
+
+ // write and fsync a new segments_N
+ infos.commit(dir);
+
+ // Notify current infos (which may have changed while we were doing dir.sync above) what generation we are up to; this way future
+ // commits are guaranteed to go to the next (unwritten) generations:
+ if (mgr != null) {
+ ((SegmentInfosSearcherManager) mgr).getCurrentInfos().updateGeneration(infos);
+ }
+ String segmentsFileName = infos.getSegmentsFileName();
+ message("top: commit wrote segments file " + segmentsFileName + " version=" + infos.getVersion() + " sis=" + infos.toString() + " commitData=" + commitData);
+ deleter.incRef(Collections.singletonList(segmentsFileName));
+ message("top: commit decRef lastCommitFiles=" + lastCommitFiles);
+ deleter.decRef(lastCommitFiles);
+ lastCommitFiles.clear();
+ lastCommitFiles.addAll(indexFiles);
+ lastCommitFiles.add(segmentsFileName);
+ message("top: commit version=" + infos.getVersion() + " files now " + lastCommitFiles);
+ }
+ }
+
+ void finishNRTCopy(CopyJob job, long startNS) throws IOException {
+ CopyState copyState = job.getCopyState();
+ message("top: finishNRTCopy: version=" + copyState.version + (job.getFailed() ? " FAILED" : "") + " job=" + job);
+
+ // NOTE: if primary crashed while we were still copying then the job will hit an exc trying to read bytes for the files from the primary node,
+ // and the job will be marked as failed here:
+
+ synchronized (this) {
+
+ if ("syncing".equals(state)) {
+ state = "idle";
+ }
+
+ if (curNRTCopy == job) {
+ message("top: now clear curNRTCopy; job=" + job);
+ curNRTCopy = null;
+ } else {
+ assert job.getFailed();
+ message("top: skip clear curNRTCopy: we were cancelled; job=" + job);
+ }
+
+ if (job.getFailed()) {
+ return;
+ }
+
+ // Does final file renames:
+ job.finish();
+
+ // Turn byte[] back to SegmentInfos:
+ byte[] infosBytes = copyState.infosBytes;
+ SegmentInfos infos = SegmentInfos.readCommit(dir,
+ new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", copyState.infosBytes)),
+ copyState.gen);
+ assert infos.getVersion() == copyState.version;
+
+ message(" version=" + infos.getVersion() + " segments=" + infos.toString());
+
+ // Cutover to new searcher:
+ if (mgr != null) {
+ ((SegmentInfosSearcherManager) mgr).setCurrentInfos(infos);
+ }
+
+ // Must first incRef new NRT files, then decRef old ones, to make sure we don't remove an NRT file that's in common to both:
+ Collection<String> newFiles = copyState.files.keySet();
+ message("top: incRef newNRTFiles=" + newFiles);
+ deleter.incRef(newFiles);
+
+ // If any of our new files were previously copied merges, we clear them now, so we don't try to later delete a non-existent file:
+ pendingMergeFiles.removeAll(newFiles);
+ message("top: after remove from pending merges pendingMergeFiles=" + pendingMergeFiles);
+
+ message("top: decRef lastNRTFiles=" + lastNRTFiles);
+ deleter.decRef(lastNRTFiles);
+ lastNRTFiles.clear();
+ lastNRTFiles.addAll(newFiles);
+ message("top: set lastNRTFiles=" + lastNRTFiles);
+
+ // At this point we can remove any completed merge segment files that we still do not reference. This can happen when a merge
+ // finishes, copies its files out to us, but is then merged away (or dropped due to 100% deletions) before we ever cutover to it
+ // in an NRT point:
+ if (copyState.completedMergeFiles.isEmpty() == false) {
+ message("now remove-if-not-ref'd completed merge files: " + copyState.completedMergeFiles);
+ for(String fileName : copyState.completedMergeFiles) {
+ if (pendingMergeFiles.contains(fileName)) {
+ pendingMergeFiles.remove(fileName);
+ deleter.deleteIfNoRef(fileName);
+ }
+ }
+ }
+
+ lastFileMetaData = copyState.files;
+
+ // It's a good time to delete pending files, since we just refreshed and some previously open files are now closed:
+ deleter.deletePending();
+ }
+
+ int markerCount;
+ IndexSearcher s = mgr.acquire();
+ try {
+ markerCount = s.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+ } finally {
+ mgr.release(s);
+ }
+
+ message(String.format(Locale.ROOT, "top: done sync: took %.3fs for %s, opened NRT reader version=%d markerCount=%d",
+ (System.nanoTime()-startNS)/1000000000.0,
+ bytesToString(job.getTotalBytesCopied()),
+ copyState.version,
+ markerCount));
+ }
+
+ /** Start a background copying job, to copy the specified files from the current primary node. If files is null then the latest copy
+ * state should be copied. If prevJob is not null, then the new copy job is replacing it and should 1) cancel the previous one, and
+ * 2) optionally salvage e.g. partially copied and, shared with the new copy job, files. */
+ protected abstract CopyJob newCopyJob(String reason, Map<String,FileMetaData> files, Map<String,FileMetaData> prevFiles,
+ boolean highPriority, CopyJob.OnceDone onceDone) throws IOException;
+
+ /** Runs this job async'd */
+ protected abstract void launch(CopyJob job);
+
+ /** Tell primary we (replica) just started, so primary can tell us to warm any already warming merges. This lets us keep low nrt refresh
+ * time for the first nrt sync after we started. */
+ protected abstract void sendNewReplica() throws IOException;
+
+ /** Call this to notify this replica node that a new NRT infos is available on the primary.
+ * We kick off a job (runs in the background) to copy files across, and open a new reader once that's done. */
+ public synchronized CopyJob newNRTPoint(long version) throws IOException {
+
+ if (isClosed()) {
+ throw new AlreadyClosedException("this replica is closed: state=" + state);
+ }
+
+ // Caller should not "publish" us until we have finished .start():
+ assert mgr != null;
+
+ if ("idle".equals(state)) {
+ state = "syncing";
+ }
+
+ long curVersion = getCurrentSearchingVersion();
+
+ message("top: start sync sis.version=" + version);
+
+ if (version == curVersion) {
+ // Caller releases the CopyState:
+ message("top: new NRT point has same version as current; skipping");
+ return null;
+ }
+
+ if (version < curVersion) {
+ // This can happen, if two syncs happen close together, and due to thread scheduling, the incoming older version runs after the newer version
+ message("top: new NRT point (version=" + version + ") is older than current (version=" + version + "); skipping");
+ return null;
+ }
+
+ final long startNS = System.nanoTime();
+
+ message("top: newNRTPoint");
+ CopyJob job = null;
+ try {
+ job = newCopyJob("NRT point sync version=" + version,
+ null,
+ lastFileMetaData,
+ true,
+ new CopyJob.OnceDone() {
+ @Override
+ public void run(CopyJob job) {
+ try {
+ finishNRTCopy(job, startNS);
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ }
+ });
+ } catch (NodeCommunicationException nce) {
+ // E.g. primary could crash/close when we are asking it for the copy state:
+ message("top: ignoring communication exception creating CopyJob: " + nce);
+ nce.printStackTrace(System.out);
+ if (state.equals("syncing")) {
+ state = "idle";
+ }
+ return null;
+ }
+
+ Collection<String> newNRTFiles = job.getFileNames();
+ long newPrimaryGen = job.getCopyState().primaryGen;
+ maybeNewPrimary(newPrimaryGen);
+
+ message("top: newNRTPoint: job files=" + newNRTFiles);
+
+ if (curNRTCopy != null) {
+ job.transferAndCancel(curNRTCopy);
+ assert curNRTCopy.getFailed();
+ }
+
+ curNRTCopy = job;
+
+ for(String fileName : curNRTCopy.getFileNamesToCopy()) {
+ assert lastCommitFiles.contains(fileName) == false: "fileName=" + fileName + " is in lastCommitFiles and is being copied?";
+ synchronized (mergeCopyJobs) {
+ for (CopyJob mergeJob : mergeCopyJobs) {
+ if (mergeJob.getFileNames().contains(fileName)) {
+ // nocommit can't we .transferAndCancel?
+ message("top: now cancel merge copy job=" + mergeJob + ": file " + fileName + " is now being copied via NRT point");
+ mergeJob.cancel("newNRTPoint is copying over the same file", null);
+ }
+ }
+ }
+ }
+
+ try {
+ job.start();
+ } catch (NodeCommunicationException nce) {
+ // E.g. primary could crash/close when we are asking it for the copy state:
+ message("top: ignoring exception starting CopyJob: " + nce);
+ nce.printStackTrace(System.out);
+ if (state.equals("syncing")) {
+ state = "idle";
+ }
+ return null;
+ }
+
+ // Runs in the background jobs thread, maybe slowly/throttled, and calls finishSync once it's done:
+ launch(curNRTCopy);
+ return curNRTCopy;
+ }
+
+ public synchronized boolean isCopying() {
+ return curNRTCopy != null;
+ }
+
+ @Override
+ public boolean isClosed() {
+ return "closed".equals(state) || "closing".equals(state) || "crashing".equals(state) || "crashed".equals(state);
+ }
+
+ @Override
+ public void close() throws IOException {
+ message("top: now close");
+
+ synchronized (this) {
+ state = "closing";
+ if (curNRTCopy != null) {
+ curNRTCopy.cancel("closing", null);
+ }
+ }
+
+ synchronized (this) {
+ message("top: close mgr");
+ mgr.close();
+
+ message("top: decRef lastNRTFiles=" + lastNRTFiles);
+ deleter.decRef(lastNRTFiles);
+ lastNRTFiles.clear();
+
+ // NOTE: do not decRef these!
+ lastCommitFiles.clear();
+
+ message("top: delete if no ref pendingMergeFiles=" + pendingMergeFiles);
+ for(String fileName : pendingMergeFiles) {
+ deleter.deleteIfNoRef(fileName);
+ }
+ pendingMergeFiles.clear();
+
+ message("top: close dir");
+ IOUtils.close(writeFileLock, dir);
+ }
+ message("top: done close");
+ state = "closed";
+ }
+
+ /** Called when the primary changed */
+ protected synchronized void maybeNewPrimary(long newPrimaryGen) {
+ if (newPrimaryGen != lastPrimaryGen) {
+ message("top: now change lastPrimaryGen from " + lastPrimaryGen + " to " + newPrimaryGen + " pendingMergeFiles=" + pendingMergeFiles);
+ assert newPrimaryGen > lastPrimaryGen: "newPrimaryGen=" + newPrimaryGen + " vs lastPrimaryGen=" + lastPrimaryGen;
+ lastPrimaryGen = newPrimaryGen;
+ pendingMergeFiles.clear();
+ } else {
+ message("top: keep current lastPrimaryGen=" + lastPrimaryGen);
+ }
+ }
+
+ protected synchronized CopyJob launchPreCopyMerge(AtomicBoolean finished, long newPrimaryGen, Map<String,FileMetaData> files) throws IOException {
+
+ CopyJob job;
+
+ maybeNewPrimary(newPrimaryGen);
+ final long primaryGenStart = lastPrimaryGen;
+ Set<String> fileNames = files.keySet();
+ message("now pre-copy warm merge files=" + fileNames + " primaryGen=" + newPrimaryGen);
+
+ for(String fileName : fileNames) {
+ assert pendingMergeFiles.contains(fileName) == false: "file \"" + fileName + "\" is already being warmed!";
+ assert lastNRTFiles.contains(fileName) == false: "file \"" + fileName + "\" is already NRT visible!";
+ }
+
+ job = newCopyJob("warm merge on " + name() + " filesNames=" + fileNames,
+ files, null, false,
+ new CopyJob.OnceDone() {
+
+ @Override
+ public void run(CopyJob job) throws IOException {
+ // Signals that this replica has finished
+ mergeCopyJobs.remove(job);
+ message("done warming merge " + fileNames + " failed?=" + job.getFailed());
+ synchronized(this) {
+ if (job.getFailed() == false) {
+ if (lastPrimaryGen != primaryGenStart) {
+ message("merge pre copy finished but primary has changed; cancelling job files=" + fileNames);
+ job.cancel("primary changed during merge copy", null);
+ } else {
+ boolean abort = false;
+ for (String fileName : fileNames) {
+ if (lastNRTFiles.contains(fileName)) {
+ message("abort merge finish: file " + fileName + " is referenced by last NRT point");
+ abort = true;
+ }
+ if (lastCommitFiles.contains(fileName)) {
+ message("abort merge finish: file " + fileName + " is referenced by last commit point");
+ abort = true;
+ }
+ }
+ if (abort) {
+ // Even though in newNRTPoint we have similar logic, which cancels any merge copy jobs if an NRT point
+ // shows up referencing the files we are warming (because primary got impatient and gave up on us), we also
+ // need it here in case replica is way far behind and fails to even receive the merge pre-copy request
+ // until after the newNRTPoint referenced those files:
+ job.cancel("merged segment was separately copied via NRT point", null);
+ } else {
+ job.finish();
+ message("merge pre copy finished files=" + fileNames);
+ for(String fileName : fileNames) {
+ assert pendingMergeFiles.contains(fileName) == false : "file \"" + fileName + "\" is already in pendingMergeFiles";
+ message("add file " + fileName + " to pendingMergeFiles");
+ pendingMergeFiles.add(fileName);
+ }
+ }
+ }
+ } else {
+ message("merge copy finished with failure");
+ }
+ }
+ finished.set(true);
+ }
+ });
+
+ job.start();
+
+ // When warming a merge we better not already have any of these files copied!
+ assert job.getFileNamesToCopy().size() == files.size();
+
+ mergeCopyJobs.add(job);
+ launch(job);
+
+ return job;
+ }
+
+ public IndexOutput createTempOutput(String prefix, String suffix, IOContext ioContext) throws IOException {
+ return dir.createTempOutput(prefix, suffix, IOContext.DEFAULT);
+ }
+
+ /** Compares incoming per-file identity (id, checksum, header, footer) versus what we have locally and returns the subset of the incoming
+ * files that need copying */
+ public List<Map.Entry<String,FileMetaData>> getFilesToCopy(Map<String,FileMetaData> files) throws IOException {
+
+ boolean doCopyCommitFiles = false;
+ List<Map.Entry<String,FileMetaData>> toCopy = new ArrayList<>();
+ for (Map.Entry<String,FileMetaData> ent : files.entrySet()) {
+ String fileName = ent.getKey();
+ FileMetaData fileMetaData = ent.getValue();
+ if (fileIsIdentical(fileName, fileMetaData) == false) {
+ toCopy.add(ent);
+ }
+ }
+
+ return toCopy;
+ }
+
+ /** Carefully determine if the file on the primary, identified by its {@code String fileName} along with the {@link FileMetaData}
+ * "summarizing" its contents, is precisely the same file that we have locally. If the file does not exist locally, or if its its header
+ * (inclues the segment id), length, footer (including checksum) differ, then this returns false, else true. */
+ private boolean fileIsIdentical(String fileName, FileMetaData srcMetaData) throws IOException {
+
+ if (deleter.isPending(fileName)) {
+ // This was a file we had wanted to delete yet a virus checker prevented us, and now we need to overwrite it.
+ // Such files are in an unknown state, and even if their header and footer and length all
+ // match, since they may not have been fsync'd by the previous node instance on this directory,
+ // they could in theory have corruption internally. So we always force ourselves to copy them here:
+ if (Node.VERBOSE_FILES) {
+ message("file " + fileName + ": will copy [we had wanted to delete this file on init, but failed]");
+ }
+ return false;
+ }
+
+ FileMetaData destMetaData = readLocalFileMetaData(fileName);
+ if (destMetaData == null) {
+ // Something went wrong in reading the file (it's corrupt, truncated, does not exist, etc.):
+ return false;
+ }
+
+ if (Arrays.equals(destMetaData.header, srcMetaData.header) == false ||
+ Arrays.equals(destMetaData.footer, srcMetaData.footer) == false) {
+ // Segment name was reused! This is rare but possible and otherwise devastating:
+ if (Node.VERBOSE_FILES) {
+ message("file " + fileName + ": will copy [header/footer is different]");
+ }
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ private ConcurrentMap<String,Boolean> copying = new ConcurrentHashMap<>();
+
+ // Used only to catch bugs, ensuring a given file name is only ever being copied bye one job:
+ public void startCopyFile(String name) {
+ if (copying.putIfAbsent(name, Boolean.TRUE) != null) {
+ throw new IllegalStateException("file " + name + " is being copied in two places!");
+ }
+ }
+
+ public void finishCopyFile(String name) {
+ if (copying.remove(name) == null) {
+ throw new IllegalStateException("file " + name + " was not actually being copied?");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
new file mode 100644
index 0000000..72ed921
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java
@@ -0,0 +1,129 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.StandardDirectoryReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ReferenceManager;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** A SearcherManager that refreshes via an externally provided (NRT) SegmentInfos, either from {@link IndexWriter} or via
+ * nrt replication to another index. */
+class SegmentInfosSearcherManager extends ReferenceManager<IndexSearcher> {
+ private volatile SegmentInfos currentInfos;
+ private final Directory dir;
+ private final Node node;
+ private final AtomicInteger openReaderCount = new AtomicInteger();
+ private final SearcherFactory searcherFactory;
+
+ public SegmentInfosSearcherManager(Directory dir, Node node, SegmentInfos infosIn, SearcherFactory searcherFactory) throws IOException {
+ this.dir = dir;
+ this.node = node;
+ if (searcherFactory == null) {
+ searcherFactory = new SearcherFactory();
+ }
+ this.searcherFactory = searcherFactory;
+ currentInfos = infosIn;
+ node.message("SegmentInfosSearcherManager.init: use incoming infos=" + infosIn.toString());
+ current = SearcherManager.getSearcher(searcherFactory, StandardDirectoryReader.open(dir, currentInfos, null), null);
+ addReaderClosedListener(current.getIndexReader());
+ }
+
+ @Override
+ protected int getRefCount(IndexSearcher s) {
+ return s.getIndexReader().getRefCount();
+ }
+
+ @Override
+ protected boolean tryIncRef(IndexSearcher s) {
+ return s.getIndexReader().tryIncRef();
+ }
+
+ @Override
+ protected void decRef(IndexSearcher s) throws IOException {
+ s.getIndexReader().decRef();
+ }
+
+ public SegmentInfos getCurrentInfos() {
+ return currentInfos;
+ }
+
+ /** Switch to new segments, refreshing if necessary. Note that it's the caller job to ensure there's a held refCount for the
+ * incoming infos, so all files exist. */
+ public void setCurrentInfos(SegmentInfos infos) throws IOException {
+ if (currentInfos != null) {
+ // So that if we commit, we will go to the next
+ // (unwritten so far) generation:
+ infos.updateGeneration(currentInfos);
+ node.message("mgr.setCurrentInfos: carry over infos gen=" + infos.getSegmentsFileName());
+ }
+ currentInfos = infos;
+ maybeRefresh();
+ }
+
+ @Override
+ protected IndexSearcher refreshIfNeeded(IndexSearcher old) throws IOException {
+ List<LeafReader> subs;
+ if (old == null) {
+ subs = null;
+ } else {
+ subs = new ArrayList<>();
+ for(LeafReaderContext ctx : old.getIndexReader().leaves()) {
+ subs.add(ctx.reader());
+ }
+ }
+
+ // Open a new reader, sharing any common segment readers with the old one:
+ DirectoryReader r = StandardDirectoryReader.open(dir, currentInfos, subs);
+ addReaderClosedListener(r);
+ node.message("refreshed to version=" + currentInfos.getVersion() + " r=" + r);
+ return SearcherManager.getSearcher(searcherFactory, r, (DirectoryReader) old.getIndexReader());
+ }
+
+ private void addReaderClosedListener(IndexReader r) {
+ openReaderCount.incrementAndGet();
+ r.addReaderClosedListener(new IndexReader.ReaderClosedListener() {
+ @Override
+ public void onClose(IndexReader reader) {
+ onReaderClosed();
+ }
+ });
+ }
+
+ /** Tracks how many readers are still open, so that when we are closed,
+ * we can additionally wait until all in-flight searchers are
+ * closed. */
+ synchronized void onReaderClosed() {
+ if (openReaderCount.decrementAndGet() == 0) {
+ notifyAll();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java
new file mode 100644
index 0000000..7db7bc1
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java
@@ -0,0 +1,63 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+
+/** Simple point-to-point TCP connection */
+class Connection implements Closeable {
+ public final DataInput in;
+ public final DataOutput out;
+ public final InputStream sockIn;
+ public final BufferedOutputStream bos;
+ public final Socket s;
+ public final int destTCPPort;
+ public long lastKeepAliveNS = System.nanoTime();
+
+ public Connection(int tcpPort) throws IOException {
+ this.destTCPPort = tcpPort;
+ this.s = new Socket(InetAddress.getLoopbackAddress(), tcpPort);
+ this.sockIn = s.getInputStream();
+ this.in = new InputStreamDataInput(sockIn);
+ this.bos = new BufferedOutputStream(s.getOutputStream());
+ this.out = new OutputStreamDataOutput(bos);
+ if (Node.VERBOSE_CONNECTIONS) {
+ System.out.println("make new client Connection socket=" + this.s + " destPort=" + tcpPort);
+ }
+ }
+
+ public void flush() throws IOException {
+ bos.flush();
+ }
+
+ @Override
+ public void close() throws IOException {
+ s.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
new file mode 100644
index 0000000..369414f
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java
@@ -0,0 +1,152 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.lucene.store.AlreadyClosedException;
+
+/** Runs CopyJob(s) in background thread; each ReplicaNode has an instance of this
+ * running. At a given there could be one NRT copy job running, and multiple
+ * pre-warm merged segments jobs. */
+class Jobs extends Thread implements Closeable {
+
+ private final PriorityQueue<CopyJob> queue = new PriorityQueue<>();
+
+ private final Node node;
+
+ public Jobs(Node node) {
+ this.node = node;
+ }
+
+ private boolean finish;
+
+ /** Returns null if we are closing, else, returns the top job or waits for one to arrive if the queue is empty. */
+ private synchronized SimpleCopyJob getNextJob() {
+ while (true) {
+ if (finish) {
+ return null;
+ } else if (queue.isEmpty()) {
+ try {
+ wait();
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ } else {
+ return (SimpleCopyJob) queue.poll();
+ }
+ }
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ SimpleCopyJob topJob = getNextJob();
+ if (topJob == null) {
+ assert finish;
+ break;
+ }
+
+ this.setName("jobs o" + topJob.ord);
+
+ assert topJob != null;
+
+ boolean result;
+ try {
+ result = topJob.visit();
+ } catch (Throwable t) {
+ if ((t instanceof AlreadyClosedException) == false) {
+ node.message("exception during job.visit job=" + topJob + "; now cancel");
+ t.printStackTrace(System.out);
+ } else {
+ node.message("AlreadyClosedException during job.visit job=" + topJob + "; now cancel");
+ }
+ topJob.cancel("unexpected exception in visit", t);
+ try {
+ topJob.onceDone.run(topJob);
+ } catch (Throwable t2) {
+ node.message("ignore exception calling OnceDone: " + t2);
+ }
+ continue;
+ }
+
+ if (result == false) {
+ // Job isn't done yet; put it back:
+ synchronized (this) {
+ queue.offer(topJob);
+ }
+ } else {
+ // Job finished, now notify caller:
+ try {
+ topJob.onceDone.run(topJob);
+ } catch (Throwable t) {
+ node.message("ignore exception calling OnceDone: " + t);
+ }
+ }
+ }
+
+ node.message("top: jobs now exit run thread");
+
+ synchronized(this) {
+ // Gracefully cancel any jobs we didn't finish:
+ while (queue.isEmpty() == false) {
+ SimpleCopyJob job = (SimpleCopyJob) queue.poll();
+ node.message("top: Jobs: now cancel job=" + job);
+ job.cancel("jobs closing", null);
+ try {
+ job.onceDone.run(job);
+ } catch (Throwable t) {
+ node.message("ignore exception calling OnceDone: " + t);
+ }
+ }
+ }
+ }
+
+ public synchronized void launch(CopyJob job) {
+ if (finish == false) {
+ queue.offer(job);
+ notify();
+ } else {
+ throw new AlreadyClosedException("closed");
+ }
+ }
+
+ /** Cancels any existing jobs that are copying the same file names as this one */
+ public synchronized void cancelConflictingJobs(CopyJob newJob) {
+ for (CopyJob job : queue) {
+ if (job.conflicts(newJob)) {
+ node.message("top: now cancel existing conflicting job=" + job + " due to newJob=" + newJob);
+ job.cancel("conflicts with new job", null);
+ }
+ }
+ }
+
+ @Override
+ public synchronized void close() {
+ finish = true;
+ notify();
+ try {
+ join();
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ }
+}
[05/15] lucene-solr git commit: exempt NodeCommunicationException as
well
Posted by mi...@apache.org.
exempt NodeCommunicationException as well
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/33890681
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/33890681
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/33890681
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 33890681a0efd840874c5ec79c8862a7e94e59ab
Parents: 1ae7291
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 24 19:50:59 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 24 19:50:59 2016 -0500
----------------------------------------------------------------------
.../src/test/org/apache/lucene/replicator/nrt/SimpleServer.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/33890681/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index f03a5c3..7a257de 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -127,13 +127,13 @@ public class SimpleServer extends LuceneTestCase {
success = true;
} catch (Throwable t) {
- if (t instanceof SocketException == false) {
+ if (t instanceof SocketException == false && t instanceof NodeCommunicationException == false) {
node.message("unexpected exception handling client connection:");
t.printStackTrace(System.out);
// Test should fail with this:
throw new RuntimeException(t);
} else {
- node.message("SocketException " + t + " handling client connection; ignoring");
+ node.message("exception " + t + " handling client connection; ignoring");
}
} finally {
if (success) {
[11/15] lucene-solr git commit: Merge branch 'nrt_replicas' of
github.com:mikemccand/lucene-solr into nrt_replicas
Posted by mi...@apache.org.
Merge branch 'nrt_replicas' of github.com:mikemccand/lucene-solr into nrt_replicas
Conflicts:
lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/82ecccf8
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/82ecccf8
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/82ecccf8
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 82ecccf8e9efbafd13c8a66362e8bebf5dca1c29
Parents: 7af83c4 d9f20f5
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 3 16:46:27 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 3 16:46:27 2016 -0500
----------------------------------------------------------------------
.../replicator/nrt/TestNRTReplication.java | 36 +++++++++++---------
1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/82ecccf8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --cc lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 510a075,e2824af..7ba3bc2
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@@ -28,17 -38,6 +38,8 @@@ import java.util.concurrent.atomic.Atom
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
- import org.apache.lucene.document.Document;
- import org.apache.lucene.util.IOUtils;
- import org.apache.lucene.util.LineFileDocs;
- import org.apache.lucene.util.LuceneTestCase;
- import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
- import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-
- import com.carrotsearch.randomizedtesting.SeedUtils;
-
+// nocommit make some explicit failure tests
+
// MockRandom's .sd file has no index header/footer:
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
[04/15] lucene-solr git commit: current patch
Posted by mi...@apache.org.
current patch
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/1ae72914
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/1ae72914
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/1ae72914
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 1ae7291429bad742715344f86cfa5200229b3698
Parents: b62c671
Author: Mike McCandless <mi...@apache.org>
Authored: Sun Jan 24 18:17:20 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sun Jan 24 18:17:20 2016 -0500
----------------------------------------------------------------------
.../org/apache/lucene/codecs/CodecUtil.java | 30 +
.../apache/lucene/index/DirectoryReader.java | 9 +-
.../org/apache/lucene/index/IndexWriter.java | 43 +-
.../org/apache/lucene/index/ReaderManager.java | 8 +-
.../org/apache/lucene/index/SegmentInfos.java | 369 +++---
.../lucene/index/StandardDirectoryReader.java | 31 +-
.../apache/lucene/search/SearcherManager.java | 8 +-
.../lucene/store/ByteArrayIndexInput.java | 163 +++
.../lucene/store/NRTCachingDirectory.java | 6 +-
.../java/org/apache/lucene/util/IOUtils.java | 3 +
.../apache/lucene/index/TestIndexWriter.java | 2 +-
.../lucene/index/TestIndexWriterDelete.java | 2 +-
.../index/TestIndexWriterExceptions2.java | 2 +-
.../lucene/index/TestIndexWriterOnJRECrash.java | 42 +-
.../lucene/index/TestIndexWriterOnVMError.java | 2 +-
.../apache/lucene/index/TestRollingUpdates.java | 2 +-
.../apache/lucene/index/TestStressDeletes.java | 2 +-
.../TestControlledRealTimeReopenThread.java | 6 +-
.../apache/lucene/search/TestLRUQueryCache.java | 2 +-
.../lucene/search/TestSearcherManager.java | 10 +-
.../facet/taxonomy/SearcherTaxonomyManager.java | 2 +-
.../directory/DirectoryTaxonomyWriter.java | 2 +-
.../apache/lucene/replicator/nrt/CopyJob.java | 237 ++++
.../lucene/replicator/nrt/CopyOneFile.java | 132 ++
.../apache/lucene/replicator/nrt/CopyState.java | 56 +
.../lucene/replicator/nrt/FileMetaData.java | 40 +
.../org/apache/lucene/replicator/nrt/Node.java | 213 ++++
.../nrt/NodeCommunicationException.java | 26 +
.../nrt/PreCopyMergedSegmentWarmer.java | 80 ++
.../lucene/replicator/nrt/PrimaryNode.java | 316 +++++
.../replicator/nrt/ReplicaFileDeleter.java | 218 ++++
.../lucene/replicator/nrt/ReplicaNode.java | 772 ++++++++++++
.../nrt/SegmentInfosSearcherManager.java | 129 ++
.../lucene/replicator/nrt/Connection.java | 63 +
.../org/apache/lucene/replicator/nrt/Jobs.java | 152 +++
.../lucene/replicator/nrt/NodeProcess.java | 238 ++++
.../lucene/replicator/nrt/SimpleCopyJob.java | 294 +++++
.../replicator/nrt/SimplePrimaryNode.java | 674 ++++++++++
.../replicator/nrt/SimpleReplicaNode.java | 316 +++++
.../lucene/replicator/nrt/SimpleServer.java | 390 ++++++
.../lucene/replicator/nrt/SimpleTransLog.java | 250 ++++
.../replicator/nrt/TestNRTReplication.java | 1175 ++++++++++++++++++
.../lucene/replicator/nrt/ThreadPumper.java | 59 +
lucene/replicator/test.cmd | 1 +
.../lucene/util/BaseGeoPointTestCase.java | 2 +-
.../index/BaseIndexFileFormatTestCase.java | 2 +-
.../apache/lucene/index/RandomIndexWriter.java | 8 +-
.../lucene/store/MockDirectoryWrapper.java | 286 +++--
.../org/apache/lucene/util/LuceneTestCase.java | 13 +-
.../java/org/apache/lucene/util/TestUtil.java | 3 +-
50 files changed, 6568 insertions(+), 323 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
index 1bc2f40..4ddad22 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
@@ -258,6 +258,36 @@ public final class CodecUtil {
checkIndexHeaderSuffix(in, expectedSuffix);
return version;
}
+
+ /** Retrieves the full index header from the provided {@link IndexInput}.
+ * This throws {@link CorruptIndexException} if this file does
+ * not appear to be an index file. */
+ public static byte[] readIndexHeader(IndexInput in) throws IOException {
+ in.seek(0);
+ final int actualHeader = in.readInt();
+ if (actualHeader != CODEC_MAGIC) {
+ throw new CorruptIndexException("codec header mismatch: actual header=" + actualHeader + " vs expected header=" + CODEC_MAGIC, in);
+ }
+ String codec = in.readString();
+ in.readInt();
+ in.seek(in.getFilePointer() + StringHelper.ID_LENGTH);
+ int suffixLength = in.readByte() & 0xFF;
+ byte[] bytes = new byte[headerLength(codec) + StringHelper.ID_LENGTH + 1 + suffixLength];
+ in.seek(0);
+ in.readBytes(bytes, 0, bytes.length);
+ return bytes;
+ }
+
+ /** Retrieves the full footer from the provided {@link IndexInput}. This throws
+ * {@link CorruptIndexException} if this file does not have a valid footer. */
+ public static byte[] readFooter(IndexInput in) throws IOException {
+ in.seek(in.length() - footerLength());
+ validateFooter(in);
+ in.seek(in.length() - footerLength());
+ byte[] bytes = new byte[footerLength()];
+ in.readBytes(bytes, 0, bytes.length);
+ return bytes;
+ }
/** Expert: just reads and verifies the object ID of an index header */
public static byte[] checkIndexHeaderID(DataInput in, byte[] expectedID) throws IOException {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
index 3df0b70..a926726 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
@@ -76,7 +76,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
* @lucene.experimental
*/
public static DirectoryReader open(final IndexWriter writer) throws IOException {
- return open(writer, true);
+ return open(writer, true, false);
}
/**
@@ -91,13 +91,16 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
* future. Applying deletes can be costly, so if your app
* can tolerate deleted documents being returned you might
* gain some performance by passing false.
+ * @param writeAllDeletes If true, new deletes will be written
+ * down to index files instead of carried over from writer to
+ * reader in heap
*
* @see #open(IndexWriter)
*
* @lucene.experimental
*/
- public static DirectoryReader open(final IndexWriter writer, boolean applyAllDeletes) throws IOException {
- return writer.getReader(applyAllDeletes);
+ public static DirectoryReader open(final IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
+ return writer.getReader(applyAllDeletes, writeAllDeletes);
}
/** Expert: returns an IndexReader reading the index in the given
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index b05e15a..96dfa0b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -332,7 +332,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
final CloseableThreadLocal<MergeRateLimiter> rateLimiters = new CloseableThreadLocal<>();
DirectoryReader getReader() throws IOException {
- return getReader(true);
+ return getReader(true, false);
}
/**
@@ -393,9 +393,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
*
* @throws IOException If there is a low-level I/O error
*/
- DirectoryReader getReader(boolean applyAllDeletes) throws IOException {
+ DirectoryReader getReader(boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
ensureOpen();
+ if (writeAllDeletes && applyAllDeletes == false) {
+ throw new IllegalArgumentException("applyAllDeletes must be true when writeAllDeletes=true");
+ }
+
final long tStart = System.currentTimeMillis();
if (infoStream.isEnabled("IW")) {
@@ -431,7 +435,13 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
// just like we do when loading segments_N
synchronized(this) {
anyChanges |= maybeApplyDeletes(applyAllDeletes);
- r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes);
+ if (writeAllDeletes) {
+ // Must move the deletes to disk:
+ System.out.println("IW: now readerPool.commit");
+ readerPool.commit(segmentInfos);
+ }
+
+ r = StandardDirectoryReader.open(this, segmentInfos, applyAllDeletes, writeAllDeletes);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "return reader version=" + r.getVersion() + " reader=" + r);
}
@@ -1159,6 +1169,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
return docWriter.getNumDocs() + segmentInfos.totalMaxDoc();
}
+ /** If {@link SegmentInfos#getVersion} is below {@code newVersion} then update it to this value. */
+ public synchronized void advanceSegmentInfosVersion(long newVersion) {
+ ensureOpen();
+ if (segmentInfos.getVersion() < newVersion) {
+ segmentInfos.setVersion(newVersion);
+ }
+ changed();
+ }
+
/** Returns total number of docs in this index, including
* docs not yet flushed (still in the RAM buffer), and
* including deletions. <b>NOTE:</b> buffered deletions
@@ -2870,7 +2889,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
* contents after calling this method has no effect.
*/
public final synchronized void setCommitData(Map<String,String> commitUserData) {
- segmentInfos.setUserData(new HashMap<>(commitUserData));
+ setCommitData(commitUserData, true);
+ }
+
+ public final synchronized void setCommitData(Map<String,String> commitUserData, boolean doIncrementVersion) {
+ segmentInfos.setUserData(new HashMap<>(commitUserData), doIncrementVersion);
changeCount.incrementAndGet();
}
@@ -4576,10 +4599,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
synchronized boolean nrtIsCurrent(SegmentInfos infos) {
//System.out.println("IW.nrtIsCurrent " + (infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedDeletesStream.any()));
ensureOpen();
- boolean isCurrent = infos.version == segmentInfos.version && !docWriter.anyChanges() && !bufferedUpdatesStream.any();
+ boolean isCurrent = infos.getVersion() == segmentInfos.getVersion() && !docWriter.anyChanges() && !bufferedUpdatesStream.any();
if (infoStream.isEnabled("IW")) {
if (isCurrent == false) {
- infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.version == segmentInfos.version) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any());
+ infoStream.message("IW", "nrtIsCurrent: infoVersion matches: " + (infos.getVersion() == segmentInfos.getVersion()) + "; DW changes: " + docWriter.anyChanges() + "; BD changes: "+ bufferedUpdatesStream.any());
}
}
return isCurrent;
@@ -4708,15 +4731,17 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
}
}
- synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
+ /** @lucene.internal */
+ public synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException {
ensureOpen();
deleter.incRef(segmentInfos, false);
if (infoStream.isEnabled("IW")) {
infoStream.message("IW", "incRefDeleter for NRT reader version=" + segmentInfos.getVersion() + " segments=" + segString(segmentInfos));
}
}
-
- synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
+
+ /** @lucene.internal */
+ public synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException {
ensureOpen();
deleter.decRef(segmentInfos);
if (infoStream.isEnabled("IW")) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java b/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
index 79d9a94..8044dc1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
@@ -45,7 +45,7 @@ public final class ReaderManager extends ReferenceManager<DirectoryReader> {
* @throws IOException If there is a low-level I/O error
*/
public ReaderManager(IndexWriter writer) throws IOException {
- this(writer, true);
+ this(writer, true, false);
}
/**
@@ -63,11 +63,13 @@ public final class ReaderManager extends ReferenceManager<DirectoryReader> {
* tolerate deleted documents being returned you might gain some
* performance by passing <code>false</code>. See
* {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}.
+ * @param writeAllDeletes
+ * If <code>true</code>, new deletes will be forcefully written to index files.
*
* @throws IOException If there is a low-level I/O error
*/
- public ReaderManager(IndexWriter writer, boolean applyAllDeletes) throws IOException {
- current = DirectoryReader.open(writer, applyAllDeletes);
+ public ReaderManager(IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
+ current = DirectoryReader.open(writer, applyAllDeletes, writeAllDeletes);
}
/**
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index de5dbff..2f8d914 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -279,124 +279,131 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
public static final SegmentInfos readCommit(Directory directory, String segmentFileName) throws IOException {
long generation = generationFromSegmentsFileName(segmentFileName);
+ //System.out.println(Thread.currentThread() + ": SegmentInfos.readCommit " + segmentFileName);
try (ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ)) {
- // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
- // to read the magic ourselves.
- int magic = input.readInt();
- if (magic != CodecUtil.CODEC_MAGIC) {
- throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
- }
- int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
- byte id[] = new byte[StringHelper.ID_LENGTH];
- input.readBytes(id, 0, id.length);
- CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
-
- SegmentInfos infos = new SegmentInfos();
- infos.id = id;
- infos.generation = generation;
- infos.lastGeneration = generation;
- if (format >= VERSION_53) {
- // TODO: in the future (7.0? sigh) we can use this to throw IndexFormatTooOldException ... or just rely on the
- // minSegmentLuceneVersion check instead:
- infos.luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
- } else {
- // else compute the min version down below in the for loop
- }
+ return readCommit(directory, input, generation);
+ }
+ }
- infos.version = input.readLong();
- infos.counter = input.readInt();
- int numSegments = input.readInt();
- if (numSegments < 0) {
- throw new CorruptIndexException("invalid segment count: " + numSegments, input);
- }
+ public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput input, long generation) throws IOException {
- if (format >= VERSION_53) {
- if (numSegments > 0) {
- infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
- if (infos.minSegmentLuceneVersion.onOrAfter(Version.LUCENE_5_0_0) == false) {
- throw new IndexFormatTooOldException(input, "this index contains a too-old segment (version: " + infos.minSegmentLuceneVersion + ")");
- }
- } else {
- // else leave as null: no segments
+ // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
+ // to read the magic ourselves.
+ int magic = input.readInt();
+ if (magic != CodecUtil.CODEC_MAGIC) {
+ throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
+ }
+ int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
+ byte id[] = new byte[StringHelper.ID_LENGTH];
+ input.readBytes(id, 0, id.length);
+ CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
+
+ SegmentInfos infos = new SegmentInfos();
+ infos.id = id;
+ infos.generation = generation;
+ infos.lastGeneration = generation;
+ if (format >= VERSION_53) {
+ // TODO: in the future (7.0? sigh) we can use this to throw IndexFormatTooOldException ... or just rely on the
+ // minSegmentLuceneVersion check instead:
+ infos.luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+ } else {
+ // else compute the min version down below in the for loop
+ }
+
+ infos.version = input.readLong();
+ //System.out.println("READ sis version=" + infos.version);
+ infos.counter = input.readInt();
+ int numSegments = input.readInt();
+ if (numSegments < 0) {
+ throw new CorruptIndexException("invalid segment count: " + numSegments, input);
+ }
+
+ if (format >= VERSION_53) {
+ if (numSegments > 0) {
+ infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+ if (infos.minSegmentLuceneVersion.onOrAfter(Version.LUCENE_5_0_0) == false) {
+ throw new IndexFormatTooOldException(input, "this index contains a too-old segment (version: " + infos.minSegmentLuceneVersion + ")");
}
} else {
- // else we recompute it below as we visit segments; it can't be used for throwing IndexFormatTooOldExc, but consumers of
- // SegmentInfos can maybe still use it for other reasons
+ // else leave as null: no segments
}
+ } else {
+ // else we recompute it below as we visit segments; it can't be used for throwing IndexFormatTooOldExc, but consumers of
+ // SegmentInfos can maybe still use it for other reasons
+ }
- long totalDocs = 0;
- for (int seg = 0; seg < numSegments; seg++) {
- String segName = input.readString();
- final byte segmentID[];
- byte hasID = input.readByte();
- if (hasID == 1) {
- segmentID = new byte[StringHelper.ID_LENGTH];
- input.readBytes(segmentID, 0, segmentID.length);
- } else if (hasID == 0) {
- throw new IndexFormatTooOldException(input, "Segment is from Lucene 4.x");
- } else {
- throw new CorruptIndexException("invalid hasID byte, got: " + hasID, input);
- }
- Codec codec = readCodec(input, format < VERSION_53);
- SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
- info.setCodec(codec);
- totalDocs += info.maxDoc();
- long delGen = input.readLong();
- int delCount = input.readInt();
- if (delCount < 0 || delCount > info.maxDoc()) {
- throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
- }
- long fieldInfosGen = input.readLong();
- long dvGen = input.readLong();
- SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
- if (format >= VERSION_51) {
- siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
- } else {
- siPerCommit.setFieldInfosFiles(Collections.unmodifiableSet(input.readStringSet()));
- }
- final Map<Integer,Set<String>> dvUpdateFiles;
- final int numDVFields = input.readInt();
- if (numDVFields == 0) {
- dvUpdateFiles = Collections.emptyMap();
- } else {
- Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
- for (int i = 0; i < numDVFields; i++) {
- if (format >= VERSION_51) {
- map.put(input.readInt(), input.readSetOfStrings());
- } else {
- map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
- }
+ long totalDocs = 0;
+ for (int seg = 0; seg < numSegments; seg++) {
+ String segName = input.readString();
+ final byte segmentID[];
+ byte hasID = input.readByte();
+ if (hasID == 1) {
+ segmentID = new byte[StringHelper.ID_LENGTH];
+ input.readBytes(segmentID, 0, segmentID.length);
+ } else if (hasID == 0) {
+ throw new IndexFormatTooOldException(input, "Segment is from Lucene 4.x");
+ } else {
+ throw new CorruptIndexException("invalid hasID byte, got: " + hasID, input);
+ }
+ Codec codec = readCodec(input, format < VERSION_53);
+ SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
+ info.setCodec(codec);
+ totalDocs += info.maxDoc();
+ long delGen = input.readLong();
+ int delCount = input.readInt();
+ if (delCount < 0 || delCount > info.maxDoc()) {
+ throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
+ }
+ long fieldInfosGen = input.readLong();
+ long dvGen = input.readLong();
+ SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, delGen, fieldInfosGen, dvGen);
+ if (format >= VERSION_51) {
+ siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
+ } else {
+ siPerCommit.setFieldInfosFiles(Collections.unmodifiableSet(input.readStringSet()));
+ }
+ final Map<Integer,Set<String>> dvUpdateFiles;
+ final int numDVFields = input.readInt();
+ if (numDVFields == 0) {
+ dvUpdateFiles = Collections.emptyMap();
+ } else {
+ Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
+ for (int i = 0; i < numDVFields; i++) {
+ if (format >= VERSION_51) {
+ map.put(input.readInt(), input.readSetOfStrings());
+ } else {
+ map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
}
- dvUpdateFiles = Collections.unmodifiableMap(map);
}
- siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
- infos.add(siPerCommit);
+ dvUpdateFiles = Collections.unmodifiableMap(map);
+ }
+ siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
+ infos.add(siPerCommit);
- Version segmentVersion = info.getVersion();
- if (format < VERSION_53) {
- if (infos.minSegmentLuceneVersion == null || segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
- infos.minSegmentLuceneVersion = segmentVersion;
- }
- } else if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
- throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
+ Version segmentVersion = info.getVersion();
+ if (format < VERSION_53) {
+ if (infos.minSegmentLuceneVersion == null || segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
+ infos.minSegmentLuceneVersion = segmentVersion;
}
+ } else if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
+ throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
}
+ }
- if (format >= VERSION_51) {
- infos.userData = input.readMapOfStrings();
- } else {
- infos.userData = Collections.unmodifiableMap(input.readStringStringMap());
- }
+ if (format >= VERSION_51) {
+ infos.userData = input.readMapOfStrings();
+ } else {
+ infos.userData = Collections.unmodifiableMap(input.readStringStringMap());
+ }
- CodecUtil.checkFooter(input);
+ CodecUtil.checkFooter(input);
- // LUCENE-6299: check we are in bounds
- if (totalDocs > IndexWriter.getActualMaxDocs()) {
- throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
- }
-
- return infos;
+ // LUCENE-6299: check we are in bounds
+ if (totalDocs > IndexWriter.getActualMaxDocs()) {
+ throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
}
+
+ return infos;
}
private static final List<String> unsupportedCodecs = Arrays.asList(
@@ -454,68 +461,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
try {
segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
- CodecUtil.writeIndexHeader(segnOutput, "segments", VERSION_CURRENT,
- StringHelper.randomId(), Long.toString(nextGeneration, Character.MAX_RADIX));
- segnOutput.writeVInt(Version.LATEST.major);
- segnOutput.writeVInt(Version.LATEST.minor);
- segnOutput.writeVInt(Version.LATEST.bugfix);
-
- segnOutput.writeLong(version);
- segnOutput.writeInt(counter); // write counter
- segnOutput.writeInt(size());
-
- if (size() > 0) {
-
- Version minSegmentVersion = null;
-
- // We do a separate loop up front so we can write the minSegmentVersion before
- // any SegmentInfo; this makes it cleaner to throw IndexFormatTooOldExc at read time:
- for (SegmentCommitInfo siPerCommit : this) {
- Version segmentVersion = siPerCommit.info.getVersion();
- if (minSegmentVersion == null || segmentVersion.onOrAfter(minSegmentVersion) == false) {
- minSegmentVersion = segmentVersion;
- }
- }
-
- segnOutput.writeVInt(minSegmentVersion.major);
- segnOutput.writeVInt(minSegmentVersion.minor);
- segnOutput.writeVInt(minSegmentVersion.bugfix);
- }
-
- // write infos
- for (SegmentCommitInfo siPerCommit : this) {
- SegmentInfo si = siPerCommit.info;
- segnOutput.writeString(si.name);
- byte segmentID[] = si.getId();
- // TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
- if (segmentID == null) {
- segnOutput.writeByte((byte)0);
- } else {
- if (segmentID.length != StringHelper.ID_LENGTH) {
- throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
- }
- segnOutput.writeByte((byte)1);
- segnOutput.writeBytes(segmentID, segmentID.length);
- }
- segnOutput.writeString(si.getCodec().getName());
- segnOutput.writeLong(siPerCommit.getDelGen());
- int delCount = siPerCommit.getDelCount();
- if (delCount < 0 || delCount > si.maxDoc()) {
- throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
- }
- segnOutput.writeInt(delCount);
- segnOutput.writeLong(siPerCommit.getFieldInfosGen());
- segnOutput.writeLong(siPerCommit.getDocValuesGen());
- segnOutput.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
- final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
- segnOutput.writeInt(dvUpdatesFiles.size());
- for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
- segnOutput.writeInt(e.getKey());
- segnOutput.writeSetOfStrings(e.getValue());
- }
- }
- segnOutput.writeMapOfStrings(userData);
- CodecUtil.writeFooter(segnOutput);
+ write(directory, segnOutput);
segnOutput.close();
directory.sync(Collections.singleton(segmentFileName));
success = true;
@@ -533,6 +479,72 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
}
}
+ public void write(Directory directory, IndexOutput out) throws IOException {
+ CodecUtil.writeIndexHeader(out, "segments", VERSION_CURRENT,
+ StringHelper.randomId(), Long.toString(generation, Character.MAX_RADIX));
+ out.writeVInt(Version.LATEST.major);
+ out.writeVInt(Version.LATEST.minor);
+ out.writeVInt(Version.LATEST.bugfix);
+ System.out.println(Thread.currentThread().getName() + ": now write " + out.getName() + " with version=" + version);
+
+ out.writeLong(version);
+ out.writeInt(counter); // write counter
+ out.writeInt(size());
+
+ if (size() > 0) {
+
+ Version minSegmentVersion = null;
+
+ // We do a separate loop up front so we can write the minSegmentVersion before
+ // any SegmentInfo; this makes it cleaner to throw IndexFormatTooOldExc at read time:
+ for (SegmentCommitInfo siPerCommit : this) {
+ Version segmentVersion = siPerCommit.info.getVersion();
+ if (minSegmentVersion == null || segmentVersion.onOrAfter(minSegmentVersion) == false) {
+ minSegmentVersion = segmentVersion;
+ }
+ }
+
+ out.writeVInt(minSegmentVersion.major);
+ out.writeVInt(minSegmentVersion.minor);
+ out.writeVInt(minSegmentVersion.bugfix);
+ }
+
+ // write infos
+ for (SegmentCommitInfo siPerCommit : this) {
+ SegmentInfo si = siPerCommit.info;
+ out.writeString(si.name);
+ byte segmentID[] = si.getId();
+ // TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
+ if (segmentID == null) {
+ out.writeByte((byte)0);
+ } else {
+ if (segmentID.length != StringHelper.ID_LENGTH) {
+ throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
+ }
+ out.writeByte((byte)1);
+ out.writeBytes(segmentID, segmentID.length);
+ }
+ out.writeString(si.getCodec().getName());
+ out.writeLong(siPerCommit.getDelGen());
+ int delCount = siPerCommit.getDelCount();
+ if (delCount < 0 || delCount > si.maxDoc()) {
+ throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
+ }
+ out.writeInt(delCount);
+ out.writeLong(siPerCommit.getFieldInfosGen());
+ out.writeLong(siPerCommit.getDocValuesGen());
+ out.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
+ final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
+ out.writeInt(dvUpdatesFiles.size());
+ for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
+ out.writeInt(e.getKey());
+ out.writeSetOfStrings(e.getValue());
+ }
+ }
+ out.writeMapOfStrings(userData);
+ CodecUtil.writeFooter(out);
+ }
+
/**
* Returns a copy of this instance, also copying each
* SegmentInfo.
@@ -698,8 +710,10 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
protected abstract T doBody(String segmentFileName) throws IOException;
}
- // Carry over generation numbers from another SegmentInfos
- void updateGeneration(SegmentInfos other) {
+ /** Carry over generation numbers from another SegmentInfos
+ *
+ * @lucene.internal */
+ public void updateGeneration(SegmentInfos other) {
lastGeneration = other.lastGeneration;
generation = other.generation;
}
@@ -711,7 +725,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
this.counter = other.counter;
}
- void setNextWriteGeneration(long generation) {
+ public void setNextWriteGeneration(long generation) {
assert generation >= this.generation;
this.generation = generation;
}
@@ -777,7 +791,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
boolean success = false;
final String dest;
try {
- final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
+ final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", generation);
dir.renameFile(src, dest);
success = true;
@@ -800,7 +814,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
* method if changes have been made to this {@link SegmentInfos} instance
* </p>
**/
- final void commit(Directory dir) throws IOException {
+ public final void commit(Directory dir) throws IOException {
prepareCommit(dir);
finishCommit(dir);
}
@@ -829,14 +843,15 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
return userData;
}
- void setUserData(Map<String,String> data) {
+ public void setUserData(Map<String,String> data, boolean doIncrementVersion) {
if (data == null) {
userData = Collections.<String,String>emptyMap();
} else {
userData = data;
}
-
- changed();
+ if (doIncrementVersion) {
+ changed();
+ }
}
/** Replaces all segments in this instance, but keeps
@@ -864,8 +879,18 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
* segments. */
public void changed() {
version++;
+ System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
+ //new Throwable().printStackTrace(System.out);
}
+ void setVersion(long newVersion) {
+ if (newVersion < version) {
+ throw new IllegalArgumentException("newVersion (=" + newVersion + ") cannot be less than current version (=" + version + ")");
+ }
+ System.out.println(Thread.currentThread().getName() + ": SIS.setVersion change from " + version + " to " + newVersion);
+ version = newVersion;
+ }
+
/** applies all changes caused by committing a merge to this SegmentInfos */
void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) {
final Set<SegmentCommitInfo> mergedAway = new HashSet<>(merge.segments);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
index 8d51532..28dd55f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
@@ -30,19 +30,21 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
-final class StandardDirectoryReader extends DirectoryReader {
+public final class StandardDirectoryReader extends DirectoryReader {
final IndexWriter writer;
final SegmentInfos segmentInfos;
private final boolean applyAllDeletes;
+ private final boolean writeAllDeletes;
/** called only from static open() methods */
StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
- SegmentInfos sis, boolean applyAllDeletes) throws IOException {
+ SegmentInfos sis, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
super(directory, readers);
this.writer = writer;
this.segmentInfos = sis;
this.applyAllDeletes = applyAllDeletes;
+ this.writeAllDeletes = writeAllDeletes;
}
/** called from DirectoryReader.open(...) methods */
@@ -60,7 +62,7 @@ final class StandardDirectoryReader extends DirectoryReader {
// This may throw CorruptIndexException if there are too many docs, so
// it must be inside try clause so we close readers in that case:
- DirectoryReader reader = new StandardDirectoryReader(directory, readers, null, sis, false);
+ DirectoryReader reader = new StandardDirectoryReader(directory, readers, null, sis, false, false);
success = true;
return reader;
@@ -74,7 +76,7 @@ final class StandardDirectoryReader extends DirectoryReader {
}
/** Used by near real-time search */
- static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes) throws IOException {
+ static DirectoryReader open(IndexWriter writer, SegmentInfos infos, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
// IndexWriter synchronizes externally before calling
// us, which ensures infos will not change; so there's
// no need to process segments in reverse order
@@ -113,8 +115,8 @@ final class StandardDirectoryReader extends DirectoryReader {
writer.incRefDeleter(segmentInfos);
StandardDirectoryReader result = new StandardDirectoryReader(dir,
- readers.toArray(new SegmentReader[readers.size()]), writer,
- segmentInfos, applyAllDeletes);
+ readers.toArray(new SegmentReader[readers.size()]), writer,
+ segmentInfos, applyAllDeletes, writeAllDeletes);
success = true;
return result;
} finally {
@@ -131,8 +133,10 @@ final class StandardDirectoryReader extends DirectoryReader {
}
}
- /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)} */
- private static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends LeafReader> oldReaders) throws IOException {
+ /** This constructor is only used for {@link #doOpenIfChanged(SegmentInfos)}, as well as NRT replication.
+ *
+ * @lucene.internal */
+ public static DirectoryReader open(Directory directory, SegmentInfos infos, List<? extends LeafReader> oldReaders) throws IOException {
// we put the old SegmentReaders in a map, that allows us
// to lookup a reader using its segment name
@@ -210,7 +214,7 @@ final class StandardDirectoryReader extends DirectoryReader {
}
}
}
- return new StandardDirectoryReader(directory, newReaders, null, infos, false);
+ return new StandardDirectoryReader(directory, newReaders, null, infos, false, false);
}
// TODO: move somewhere shared if it's useful elsewhere
@@ -270,7 +274,7 @@ final class StandardDirectoryReader extends DirectoryReader {
if (writer == this.writer && applyAllDeletes == this.applyAllDeletes) {
return doOpenFromWriter(null);
} else {
- return writer.getReader(applyAllDeletes);
+ return writer.getReader(applyAllDeletes, writeAllDeletes);
}
}
@@ -283,7 +287,7 @@ final class StandardDirectoryReader extends DirectoryReader {
return null;
}
- DirectoryReader reader = writer.getReader(applyAllDeletes);
+ DirectoryReader reader = writer.getReader(applyAllDeletes, writeAllDeletes);
// If in fact no changes took place, return null:
if (reader.getVersion() == segmentInfos.getVersion()) {
@@ -332,6 +336,11 @@ final class StandardDirectoryReader extends DirectoryReader {
return segmentInfos.getVersion();
}
+ /** @lucene.internal */
+ public SegmentInfos getSegmentInfos() {
+ return segmentInfos;
+ }
+
@Override
public boolean isCurrent() throws IOException {
ensureOpen();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java b/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
index 3d3b064..a57f260 100644
--- a/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
+++ b/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
@@ -73,7 +73,7 @@ public final class SearcherManager extends ReferenceManager<IndexSearcher> {
* @throws IOException if there is a low-level I/O error
*/
public SearcherManager(IndexWriter writer, SearcherFactory searcherFactory) throws IOException {
- this(writer, true, searcherFactory);
+ this(writer, true, false, searcherFactory);
}
/**
@@ -91,6 +91,8 @@ public final class SearcherManager extends ReferenceManager<IndexSearcher> {
* tolerate deleted documents being returned you might gain some
* performance by passing <code>false</code>. See
* {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}.
+ * @param writeAllDeletes
+ * If <code>true</code>, new deletes will be forcefully written to index files.
* @param searcherFactory
* An optional {@link SearcherFactory}. Pass <code>null</code> if you
* don't require the searcher to be warmed before going live or other
@@ -98,12 +100,12 @@ public final class SearcherManager extends ReferenceManager<IndexSearcher> {
*
* @throws IOException if there is a low-level I/O error
*/
- public SearcherManager(IndexWriter writer, boolean applyAllDeletes, SearcherFactory searcherFactory) throws IOException {
+ public SearcherManager(IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes, SearcherFactory searcherFactory) throws IOException {
if (searcherFactory == null) {
searcherFactory = new SearcherFactory();
}
this.searcherFactory = searcherFactory;
- current = getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes), null);
+ current = getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes, writeAllDeletes), null);
}
/**
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
new file mode 100644
index 0000000..6363ed1
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java
@@ -0,0 +1,163 @@
+package org.apache.lucene.store;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * DataInput backed by a byte array.
+ * <b>WARNING:</b> This class omits all low-level checks.
+ * @lucene.experimental
+ */
+public final class ByteArrayIndexInput extends IndexInput {
+
+ private byte[] bytes;
+
+ private int pos;
+ private int limit;
+
+ public ByteArrayIndexInput(String description, byte[] bytes) {
+ super(description);
+ this.bytes = bytes;
+ this.limit = bytes.length;
+ }
+
+ public long getFilePointer() {
+ return pos;
+ }
+
+ public void seek(long pos) {
+ this.pos = (int) pos;
+ }
+
+ public void reset(byte[] bytes, int offset, int len) {
+ this.bytes = bytes;
+ pos = offset;
+ limit = offset + len;
+ }
+
+ @Override
+ public long length() {
+ return limit;
+ }
+
+ public boolean eof() {
+ return pos == limit;
+ }
+
+ @Override
+ public void skipBytes(long count) {
+ pos += count;
+ }
+
+ @Override
+ public short readShort() {
+ return (short) (((bytes[pos++] & 0xFF) << 8) | (bytes[pos++] & 0xFF));
+ }
+
+ @Override
+ public int readInt() {
+ return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
+ | ((bytes[pos++] & 0xFF) << 8) | (bytes[pos++] & 0xFF);
+ }
+
+ @Override
+ public long readLong() {
+ final int i1 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
+ ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
+ final int i2 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
+ ((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
+ return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
+ }
+
+ @Override
+ public int readVInt() {
+ byte b = bytes[pos++];
+ if (b >= 0) return b;
+ int i = b & 0x7F;
+ b = bytes[pos++];
+ i |= (b & 0x7F) << 7;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7F) << 14;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7F) << 21;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ // Warning: the next ands use 0x0F / 0xF0 - beware copy/paste errors:
+ i |= (b & 0x0F) << 28;
+ if ((b & 0xF0) == 0) return i;
+ throw new RuntimeException("Invalid vInt detected (too many bits)");
+ }
+
+ @Override
+ public long readVLong() {
+ byte b = bytes[pos++];
+ if (b >= 0) return b;
+ long i = b & 0x7FL;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 7;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 14;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 21;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 28;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 35;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 42;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 49;
+ if (b >= 0) return i;
+ b = bytes[pos++];
+ i |= (b & 0x7FL) << 56;
+ if (b >= 0) return i;
+ throw new RuntimeException("Invalid vLong detected (negative values disallowed)");
+ }
+
+ // NOTE: AIOOBE not EOF if you read too much
+ @Override
+ public byte readByte() {
+ return bytes[pos++];
+ }
+
+ // NOTE: AIOOBE not EOF if you read too much
+ @Override
+ public void readBytes(byte[] b, int offset, int len) {
+ System.arraycopy(bytes, pos, b, offset, len);
+ pos += len;
+ }
+
+ @Override
+ public void close() {
+ }
+
+ public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
index fd5e3d7..538e7c0 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
@@ -174,8 +174,12 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable
@Override
public void renameFile(String source, String dest) throws IOException {
- // NOTE: uncache is unnecessary for lucene's usage, as we always sync() before renaming.
unCache(source);
+ try {
+ cache.deleteFile(dest);
+ } catch (FileNotFoundException fnfe) {
+ // OK -- it may not exist
+ }
in.renameFile(source, dest);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
index 510545f..1c5aabd 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -107,6 +107,9 @@ public final class IOUtils {
* objects to call <tt>close()</tt> on
*/
public static void closeWhileHandlingException(Closeable... objects) {
+ if (objects.length == 0) {
+ throw new IllegalArgumentException("pass at least one Closeable");
+ }
closeWhileHandlingException(Arrays.asList(objects));
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
index baa2484..ab75fb8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
@@ -1036,7 +1036,7 @@ public class TestIndexWriter extends LuceneTestCase {
if (random.nextInt(3) == 0) {
IndexReader r = null;
try {
- r = DirectoryReader.open(w, random.nextBoolean());
+ r = DirectoryReader.open(w, random.nextBoolean(), false);
if (random.nextBoolean() && r.maxDoc() > 0) {
int docid = random.nextInt(r.maxDoc());
w.tryDeleteDocument(r, docid);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
index 9213c25..b970519 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
@@ -1249,7 +1249,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND);
w = new IndexWriter(d, iwc);
- IndexReader r = DirectoryReader.open(w, false);
+ IndexReader r = DirectoryReader.open(w, false, false);
assertTrue(w.tryDeleteDocument(r, 1));
assertTrue(w.tryDeleteDocument(r.leaves().get(0).reader(), 0));
r.close();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
index 58e3ac7..2013ab3 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
@@ -200,7 +200,7 @@ public class TestIndexWriterExceptions2 extends LuceneTestCase {
if (random().nextBoolean()) {
DirectoryReader ir = null;
try {
- ir = DirectoryReader.open(iw, random().nextBoolean());
+ ir = DirectoryReader.open(iw, random().nextBoolean(), false);
TestUtil.checkReader(ir);
} finally {
IOUtils.closeWhileHandlingException(ir);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
index 777ef46..3fecf71 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
@@ -196,29 +196,29 @@ public class TestIndexWriterOnJRECrash extends TestNRTThreads {
vendor.startsWith("Sun") ||
vendor.startsWith("Apple");
- try {
- if (supportsUnsafeNpeDereference) {
- try {
- Class<?> clazz = Class.forName("sun.misc.Unsafe");
- Field field = clazz.getDeclaredField("theUnsafe");
- field.setAccessible(true);
- Object o = field.get(null);
- Method m = clazz.getMethod("putAddress", long.class, long.class);
- m.invoke(o, 0L, 0L);
- } catch (Throwable e) {
- System.out.println("Couldn't kill the JVM via Unsafe.");
- e.printStackTrace(System.out);
- }
+ try {
+ if (supportsUnsafeNpeDereference) {
+ try {
+ Class<?> clazz = Class.forName("sun.misc.Unsafe");
+ Field field = clazz.getDeclaredField("theUnsafe");
+ field.setAccessible(true);
+ Object o = field.get(null);
+ Method m = clazz.getMethod("putAddress", long.class, long.class);
+ m.invoke(o, 0L, 0L);
+ } catch (Throwable e) {
+ System.out.println("Couldn't kill the JVM via Unsafe.");
+ e.printStackTrace(System.out);
}
-
- // Fallback attempt to Runtime.halt();
- Runtime.getRuntime().halt(-1);
- } catch (Exception e) {
- System.out.println("Couldn't kill the JVM.");
- e.printStackTrace(System.out);
}
- // We couldn't get the JVM to crash for some reason.
- fail();
+ // Fallback attempt to Runtime.halt();
+ Runtime.getRuntime().halt(-1);
+ } catch (Exception e) {
+ System.out.println("Couldn't kill the JVM.");
+ e.printStackTrace(System.out);
+ }
+
+ // We couldn't get the JVM to crash for some reason.
+ fail();
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
index 90371fe..1c8a43e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
@@ -169,7 +169,7 @@ public class TestIndexWriterOnVMError extends LuceneTestCase {
if (random().nextBoolean()) {
DirectoryReader ir = null;
try {
- ir = DirectoryReader.open(iw, random().nextBoolean());
+ ir = DirectoryReader.open(iw, random().nextBoolean(), false);
TestUtil.checkReader(ir);
} finally {
IOUtils.closeWhileHandlingException(ir);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
index f367024..51c0eb5 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
@@ -124,7 +124,7 @@ public class TestRollingUpdates extends LuceneTestCase {
System.out.println("TEST: reopen applyDeletions=" + applyDeletions);
}
- r = w.getReader(applyDeletions);
+ r = w.getReader(applyDeletions, false);
if (applyDeletions) {
s = newSearcher(r);
} else {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
index 1df019c..cc4b80c 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
@@ -83,7 +83,7 @@ public class TestStressDeletes extends LuceneTestCase {
}
}
if (random().nextInt(500) == 2) {
- DirectoryReader.open(w, random().nextBoolean()).close();
+ DirectoryReader.open(w, random().nextBoolean(), false).close();
}
if (random().nextInt(500) == 2) {
w.commit();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
index 2e1f385..d90eaba 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
@@ -231,7 +231,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
}
};
- nrtNoDeletes = new SearcherManager(writer, false, sf);
+ nrtNoDeletes = new SearcherManager(writer, false, false, sf);
nrtDeletes = new SearcherManager(writer, sf);
nrtDeletesThread = new ControlledRealTimeReopenThread<>(genWriter, nrtDeletes, maxReopenSec, minReopenSec);
@@ -313,7 +313,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
LatchedIndexWriter _writer = new LatchedIndexWriter(d, conf, latch, signal);
final TrackingIndexWriter writer = new TrackingIndexWriter(_writer);
- final SearcherManager manager = new SearcherManager(_writer, false, null);
+ final SearcherManager manager = new SearcherManager(_writer, false, false, null);
Document doc = new Document();
doc.add(newTextField("test", "test", Field.Store.YES));
writer.addDocument(doc);
@@ -423,7 +423,7 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc
};
try {
- new SearcherManager(w.w, false, theEvilOne);
+ new SearcherManager(w.w, false, false, theEvilOne);
fail("didn't hit expected exception");
} catch (IllegalStateException ise) {
// expected
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index db632de..8eb7b44 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -94,7 +94,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
}
};
final boolean applyDeletes = random().nextBoolean();
- final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, searcherFactory);
+ final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, false, searcherFactory);
final AtomicBoolean indexing = new AtomicBoolean(true);
final AtomicReference<Throwable> error = new AtomicReference<>();
final int numDocs = atLeast(10000);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
index 76b56a5..3218569 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
@@ -231,7 +231,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
};
final SearcherManager searcherManager = random().nextBoolean()
? new SearcherManager(dir, factory)
- : new SearcherManager(writer, random().nextBoolean(), factory);
+ : new SearcherManager(writer, random().nextBoolean(), false, factory);
if (VERBOSE) {
System.out.println("sm created");
}
@@ -311,7 +311,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(
new MockAnalyzer(random())).setMergeScheduler(new ConcurrentMergeScheduler()));
- SearcherManager sm = new SearcherManager(writer, false, new SearcherFactory());
+ SearcherManager sm = new SearcherManager(writer, false, false, new SearcherFactory());
writer.addDocument(new Document());
writer.commit();
sm.maybeRefreshBlocking();
@@ -368,7 +368,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
Directory dir = newDirectory();
IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(null));
final AtomicBoolean afterRefreshCalled = new AtomicBoolean(false);
- SearcherManager sm = new SearcherManager(iw, false, new SearcherFactory());
+ SearcherManager sm = new SearcherManager(iw, false, false, new SearcherFactory());
sm.addListener(new ReferenceManager.RefreshListener() {
@Override
public void beforeRefresh() {
@@ -411,7 +411,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
// expected
}
try {
- new SearcherManager(w.w, random.nextBoolean(), theEvilOne);
+ new SearcherManager(w.w, random.nextBoolean(), false, theEvilOne);
} catch (IllegalStateException ise) {
// expected
}
@@ -522,7 +522,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
}
MySearcherFactory factory = new MySearcherFactory();
- final SearcherManager sm = new SearcherManager(w, random().nextBoolean(), factory);
+ final SearcherManager sm = new SearcherManager(w, random().nextBoolean(), false, factory);
assertEquals(1, factory.called);
assertNull(factory.lastPreviousReader);
assertNotNull(factory.lastReader);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
index cdd0a15..5a3a551 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
@@ -76,7 +76,7 @@ public class SearcherTaxonomyManager extends ReferenceManager<SearcherTaxonomyMa
this.searcherFactory = searcherFactory;
this.taxoWriter = taxoWriter;
DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
- current = new SearcherAndTaxonomy(SearcherManager.getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes), null), taxoReader);
+ current = new SearcherAndTaxonomy(SearcherManager.getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes, false), null), taxoReader);
this.taxoEpoch = taxoWriter.getTaxonomyEpoch();
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
index e241007..6396b16 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
@@ -271,7 +271,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
// verify that the taxo-writer hasn't been closed on us.
ensureOpen();
if (!initializedReaderManager) {
- readerManager = new ReaderManager(indexWriter, false);
+ readerManager = new ReaderManager(indexWriter, false, false);
shouldRefreshReaderManager = false;
initializedReaderManager = true;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
new file mode 100644
index 0000000..5baf1d3
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
@@ -0,0 +1,237 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.util.IOUtils;
+
+/** Handles copying one set of files, e.g. all files for a new NRT point, or files for pre-copying a merged segment.
+ * This notifies the caller via OnceDone when the job finishes or failed. */
+
+public abstract class CopyJob implements Comparable<CopyJob> {
+ private final static AtomicLong counter = new AtomicLong();
+ protected final ReplicaNode dest;
+
+ protected final Map<String,FileMetaData> files;
+
+ public final long ord = counter.incrementAndGet();
+
+ /** True for an NRT sync, false for pre-copying a newly merged segment */
+ public final boolean highPriority;
+
+ public final OnceDone onceDone;
+
+ public final long startNS = System.nanoTime();
+
+ public final String reason;
+
+ protected final List<Map.Entry<String,FileMetaData>> toCopy;
+
+ protected long totBytes;
+
+ protected long totBytesCopied;
+
+ // The file we are currently copying:
+ protected CopyOneFile current;
+
+ // Set when we are cancelled
+ protected volatile Throwable exc;
+ protected volatile String cancelReason;
+
+ // toString may concurrently access this:
+ protected final Map<String,String> copiedFiles = new ConcurrentHashMap<>();
+
+ protected CopyJob(String reason, Map<String,FileMetaData> files, ReplicaNode dest, boolean highPriority, OnceDone onceDone) throws IOException {
+ this.reason = reason;
+ this.files = files;
+ this.dest = dest;
+ this.highPriority = highPriority;
+ this.onceDone = onceDone;
+
+ // Exceptions in here are bad:
+ try {
+ this.toCopy = dest.getFilesToCopy(this.files);
+ } catch (Throwable t) {
+ cancel("exc during init", t);
+ throw new CorruptIndexException("exception while checking local files", "n/a", t);
+ }
+ }
+
+ /** Callback invoked by CopyJob once all files have (finally) finished copying */
+ public interface OnceDone {
+ public void run(CopyJob job) throws IOException;
+ }
+
+ /** Transfers whatever tmp files were already copied in this previous job and cancels the previous job */
+ public synchronized void transferAndCancel(CopyJob prevJob) throws IOException {
+ synchronized(prevJob) {
+ dest.message("CopyJob: now transfer prevJob " + prevJob);
+ try {
+ _transferAndCancel(prevJob);
+ } catch (Throwable t) {
+ dest.message("xfer: exc during transferAndCancel");
+ cancel("exc during transferAndCancel", t);
+ IOUtils.reThrow(t);
+ }
+ }
+ }
+
+ private synchronized void _transferAndCancel(CopyJob prevJob) throws IOException {
+
+ // Caller must already be sync'd on prevJob:
+ assert Thread.holdsLock(prevJob);
+
+ if (prevJob.exc != null) {
+ // Already cancelled
+ dest.message("xfer: prevJob was already cancelled; skip transfer");
+ return;
+ }
+
+ // Cancel the previous job
+ prevJob.exc = new Throwable();
+
+ // Carry over already copied files that we also want to copy
+ Iterator<Map.Entry<String,FileMetaData>> it = toCopy.iterator();
+ long bytesAlreadyCopied = 0;
+
+ // Iterate over all files we think we need to copy:
+ while (it.hasNext()) {
+ Map.Entry<String,FileMetaData> ent = it.next();
+ String fileName = ent.getKey();
+ String prevTmpFileName = prevJob.copiedFiles.get(fileName);
+ if (prevTmpFileName != null) {
+ // This fileName is common to both jobs, and the old job already finished copying it (to a temp file), so we keep it:
+ long fileLength = ent.getValue().length;
+ bytesAlreadyCopied += fileLength;
+ dest.message("xfer: carry over already-copied file " + fileName + " (" + prevTmpFileName + ", " + fileLength + " bytes)");
+ copiedFiles.put(fileName, prevTmpFileName);
+
+ // So we don't try to delete it, below:
+ prevJob.copiedFiles.remove(fileName);
+
+ // So it's not in our copy list anymore:
+ it.remove();
+ } else if (prevJob.current != null && prevJob.current.name.equals(fileName)) {
+ // This fileName is common to both jobs, and it's the file that the previous job was in the process of copying. In this case
+ // we continue copying it from the prevoius job. This is important for cases where we are copying over a large file
+ // because otherwise we could keep failing the NRT copy and restarting this file from the beginning and never catch up:
+ dest.message("xfer: carry over in-progress file " + fileName + " (" + prevJob.current.tmpName + ") bytesCopied=" + prevJob.current.getBytesCopied() + " of " + prevJob.current.bytesToCopy);
+ bytesAlreadyCopied += prevJob.current.getBytesCopied();
+
+ assert current == null;
+
+ // must set current first, before writing/read to c.in/out in case that hits an exception, so that we then close the temp
+ // IndexOutput when cancelling ourselves:
+ current = newCopyOneFile(prevJob.current);
+
+ // Tell our new (primary) connection we'd like to copy this file first, but resuming from how many bytes we already copied last time:
+ // We do this even if bytesToCopy == bytesCopied, because we still need to readLong() the checksum from the primary connection:
+ assert prevJob.current.getBytesCopied() <= prevJob.current.bytesToCopy;
+
+ prevJob.current = null;
+
+ totBytes += current.metaData.length;
+
+ // So it's not in our copy list anymore:
+ it.remove();
+ } else {
+ dest.message("xfer: file " + fileName + " will be fully copied");
+ }
+ }
+ dest.message("xfer: " + bytesAlreadyCopied + " bytes already copied of " + totBytes);
+
+ // Delete all temp files the old job wrote but we don't need:
+ dest.message("xfer: now delete old temp files: " + prevJob.copiedFiles.values());
+ IOUtils.deleteFilesIgnoringExceptions(dest.dir, prevJob.copiedFiles.values());
+
+ if (prevJob.current != null) {
+ IOUtils.closeWhileHandlingException(prevJob.current);
+ if (Node.VERBOSE_FILES) {
+ dest.message("remove partial file " + prevJob.current.tmpName);
+ }
+ dest.deleter.deleteNewFile(prevJob.current.tmpName);
+ prevJob.current = null;
+ }
+ }
+
+ protected abstract CopyOneFile newCopyOneFile(CopyOneFile current);
+
+ /** Begin copying files */
+ public abstract void start() throws IOException;
+
+ /** Use current thread (blocking) to do all copying and then return once done, or throw exception on failure */
+ public abstract void runBlocking() throws Exception;
+
+ public void cancel(String reason, Throwable exc) {
+ if (this.exc != null) {
+ // Already cancelled
+ return;
+ }
+
+ dest.message(String.format(Locale.ROOT, "top: cancel after copying %s; exc=%s:\n files=%s\n copiedFiles=%s",
+ Node.bytesToString(totBytesCopied),
+ exc,
+ files == null ? "null" : files.keySet(), copiedFiles.keySet()));
+
+ if (exc == null) {
+ exc = new Throwable();
+ }
+
+ this.exc = exc;
+ this.cancelReason = reason;
+
+ // Delete all temp files we wrote:
+ IOUtils.deleteFilesIgnoringExceptions(dest.dir, copiedFiles.values());
+
+ if (current != null) {
+ IOUtils.closeWhileHandlingException(current);
+ if (Node.VERBOSE_FILES) {
+ dest.message("remove partial file " + current.tmpName);
+ }
+ dest.deleter.deleteNewFile(current.tmpName);
+ current = null;
+ }
+ }
+
+ /** Return true if this job is trying to copy any of the same files as the other job */
+ public abstract boolean conflicts(CopyJob other);
+
+ /** Renames all copied (tmp) files to their true file names */
+ public abstract void finish() throws IOException;
+
+ public abstract boolean getFailed();
+
+ /** Returns only those file names (a subset of {@link #getFileNames}) that need to be copied */
+ public abstract Set<String> getFileNamesToCopy();
+
+ /** Returns all file names referenced in this copy job */
+ public abstract Set<String> getFileNames();
+
+ public abstract CopyState getCopyState();
+
+ public abstract long getTotalBytesCopied();
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
new file mode 100644
index 0000000..e3f0f7d
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
@@ -0,0 +1,132 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Locale;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.IOUtils;
+
+/** Copies one file from an incoming DataInput to a dest filename in a local Directory */
+
+class CopyOneFile implements Closeable {
+ private final DataInput in;
+ private final IndexOutput out;
+ private final ReplicaNode dest;
+ public final String name;
+ public final String tmpName;
+ public final FileMetaData metaData;
+ public final long bytesToCopy;
+ private final long copyStartNS;
+ private final byte[] buffer;
+
+ private long bytesCopied;
+
+ public CopyOneFile(DataInput in, ReplicaNode dest, String name, FileMetaData metaData, byte[] buffer) throws IOException {
+ this.in = in;
+ this.name = name;
+ this.dest = dest;
+ this.buffer = buffer;
+ // TODO: pass correct IOCtx, e.g. seg total size
+ out = dest.createTempOutput(name, "copy", IOContext.DEFAULT);
+ tmpName = out.getName();
+
+ // last 8 bytes are checksum:
+ bytesToCopy = metaData.length - 8;
+
+ if (Node.VERBOSE_FILES) {
+ dest.message("file " + name + ": start copying to tmp file " + tmpName + " length=" + (8+bytesToCopy));
+ }
+
+ copyStartNS = System.nanoTime();
+ this.metaData = metaData;
+ dest.startCopyFile(name);
+ }
+
+ /** Transfers this file copy to another input, continuing where the first one left off */
+ public CopyOneFile(CopyOneFile other, DataInput in) {
+ this.in = in;
+ this.dest = other.dest;
+ this.name = other.name;
+ this.out = other.out;
+ this.tmpName = other.tmpName;
+ this.metaData = other.metaData;
+ this.bytesCopied = other.bytesCopied;
+ this.bytesToCopy = other.bytesToCopy;
+ this.copyStartNS = other.copyStartNS;
+ this.buffer = other.buffer;
+ }
+
+ public void close() throws IOException {
+ out.close();
+ dest.finishCopyFile(name);
+ }
+
+ /** Copy another chunk of bytes, returning true once the copy is done */
+ public boolean visit() throws IOException {
+ // Copy up to 640 KB per visit:
+ for(int i=0;i<10;i++) {
+ long bytesLeft = bytesToCopy - bytesCopied;
+ if (bytesLeft == 0) {
+ long checksum = out.getChecksum();
+ if (checksum != metaData.checksum) {
+ // Bits flipped during copy!
+ dest.message("file " + tmpName + ": checksum mismatch after copy (bits flipped during network copy?) after-copy checksum=" + checksum + " vs expected=" + metaData.checksum + "; cancel job");
+ throw new IOException("file " + name + ": checksum mismatch after file copy");
+ }
+
+ // Paranoia: make sure the primary node is not smoking crack, by somehow sending us an already corrupted file whose checksum (in its
+ // footer) disagrees with reality:
+ long actualChecksumIn = in.readLong();
+ if (actualChecksumIn != checksum) {
+ dest.message("file " + tmpName + ": checksum claimed by primary disagrees with the file's footer: claimed checksum=" + checksum + " vs actual=" + actualChecksumIn);
+ throw new IOException("file " + name + ": checksum mismatch after file copy");
+ }
+ out.writeLong(checksum);
+ close();
+
+ if (Node.VERBOSE_FILES) {
+ dest.message(String.format(Locale.ROOT, "file %s: done copying [%s, %.3fms]",
+ name,
+ Node.bytesToString(metaData.length),
+ (System.nanoTime() - copyStartNS)/1000000.0));
+ }
+
+ return true;
+ }
+
+ int toCopy = (int) Math.min(bytesLeft, buffer.length);
+ in.readBytes(buffer, 0, toCopy);
+ out.writeBytes(buffer, 0, toCopy);
+
+ // TODO: rsync will fsync a range of the file; maybe we should do that here for large files in case we crash/killed
+ bytesCopied += toCopy;
+ }
+
+ return false;
+ }
+
+ public long getBytesCopied() {
+ return bytesCopied;
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
new file mode 100644
index 0000000..c19fabc
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java
@@ -0,0 +1,56 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.store.Directory;
+
+/** Holds incRef'd file level details for one point-in-time segment infos on the primary node. */
+public class CopyState {
+
+ public final Map<String,FileMetaData> files;
+ public final long version;
+ public final long gen;
+ public final byte[] infosBytes;
+ public final Set<String> completedMergeFiles;
+ public final long primaryGen;
+
+ // only non-null on the primary node
+ public final SegmentInfos infos;
+
+ public CopyState(Map<String,FileMetaData> files, long version, long gen, byte[] infosBytes,
+ Set<String> completedMergeFiles, long primaryGen, SegmentInfos infos) {
+ assert completedMergeFiles != null;
+ this.files = Collections.unmodifiableMap(files);
+ this.version = version;
+ this.gen = gen;
+ this.infosBytes = infosBytes;
+ this.completedMergeFiles = Collections.unmodifiableSet(completedMergeFiles);
+ this.primaryGen = primaryGen;
+ this.infos = infos;
+ }
+
+ @Override
+ public String toString() {
+ return getClass().getSimpleName() + "(version=" + version + ")";
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
new file mode 100644
index 0000000..aca408c
--- /dev/null
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java
@@ -0,0 +1,40 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Holds metadata details about a single file that we use to confirm two files (one remote, one local) are in fact "identical". */
+
+class FileMetaData {
+
+ // Header and footer of the file must be identical between primary and replica to consider the files equal:
+ public final byte[] header;
+ public final byte[] footer;
+
+ public final long length;
+
+ // Used to ensure no bit flips when copying the file:
+ public final long checksum;
+
+ public FileMetaData(byte[] header, byte[] footer, long length, long checksum) {
+ this.header = header;
+ this.footer = footer;
+ this.length = length;
+ this.checksum = checksum;
+ }
+}
+
[07/15] lucene-solr git commit: fix more nocommits;
add separate test that deleteAll can replicate
Posted by mi...@apache.org.
fix more nocommits; add separate test that deleteAll can replicate
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/022540e8
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/022540e8
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/022540e8
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 022540e8c23e57046f1b54a9cd485dab5ff4b563
Parents: 3389068
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Jan 27 06:00:12 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Jan 27 06:00:12 2016 -0500
----------------------------------------------------------------------
.../nrt/PreCopyMergedSegmentWarmer.java | 18 -
.../lucene/replicator/nrt/PrimaryNode.java | 18 -
.../lucene/replicator/nrt/ReplicaNode.java | 25 +-
.../lucene/replicator/nrt/NodeProcess.java | 18 +-
.../replicator/nrt/SimplePrimaryNode.java | 53 +-
.../replicator/nrt/SimpleReplicaNode.java | 30 +-
.../lucene/replicator/nrt/SimpleServer.java | 38 +-
.../replicator/nrt/TestNRTReplication.java | 1090 ++--------------
.../nrt/TestStressNRTReplication.java | 1160 ++++++++++++++++++
9 files changed, 1359 insertions(+), 1091 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
index 1918ede..77f23ab 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java
@@ -23,30 +23,14 @@ package org.apache.lucene.replicator.nrt;
* flushed segment sizes, not merged segments. */
import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.util.ArrayList;
-import java.util.Collections;
import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
import java.util.Locale;
import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.replicator.nrt.CopyJob.OnceDone;
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.ThreadInterruptedException;
// TODO: or ... replica node can do merging locally? tricky to keep things in sync, when one node merges more slowly than others...
@@ -71,8 +55,6 @@ class PreCopyMergedSegmentWarmer extends IndexReaderWarmer {
filesMetaData.put(fileName, metaData);
}
- // nocommit if one replica is very slow then it dos's all other replicas?
-
primary.preCopyMergedSegmentFiles(info, filesMetaData);
primary.message(String.format(Locale.ROOT, "top: done warm merge " + info + ": took %.3f sec, %.1f MB", (System.nanoTime()-startNS)/1000000000., info.sizeInBytes()/1024/1024.));
primary.finishedMergedFiles.addAll(filesMetaData.keySet());
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index 183f16f..ccd8848 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -17,43 +17,26 @@ package org.apache.lucene.replicator.nrt;
* limitations under the License.
*/
-import java.io.Closeable;
import java.io.IOException;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.List;
import java.util.Map;
-import java.util.Random;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LogMergePolicy;
-import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.StandardDirectoryReader;
import org.apache.lucene.index.Term;
-import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RAMFile;
import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.PrintStreamInfoStream;
import org.apache.lucene.util.ThreadInterruptedException;
/*
@@ -178,7 +161,6 @@ public abstract class PrimaryNode extends Node {
// on xlog replay we are replaying more ops than necessary.
commitData.put(VERSION_KEY, Long.toString(copyState.version));
message("top: commit commitData=" + commitData);
- // nocommit this is now an NRT-visible change! make test where nothing is indexing and confirm we don't do silly commit + refresh loop forever!
writer.setCommitData(commitData, false);
writer.commit();
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index af142d5..713c6f1 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -17,14 +17,7 @@ package org.apache.lucene.replicator.nrt;
* limitations under the License.
*/
-import java.io.EOFException;
-import java.io.FileNotFoundException;
import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.nio.file.NoSuchFileException;
-import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -34,15 +27,11 @@ import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
-import java.util.Random;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SegmentInfos;
@@ -52,24 +41,12 @@ import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.BufferedChecksumIndexInput;
-import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayIndexInput;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.store.RAMOutputStream;
-import org.apache.lucene.store.RateLimiter;
import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.ThreadInterruptedException;
/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */
@@ -544,7 +521,7 @@ abstract class ReplicaNode extends Node {
synchronized (mergeCopyJobs) {
for (CopyJob mergeJob : mergeCopyJobs) {
if (mergeJob.getFileNames().contains(fileName)) {
- // nocommit can't we .transferAndCancel?
+ // TODO: we could maybe transferAndCancel here? except CopyJob can't transferAndCancel more than one currently
message("top: now cancel merge copy job=" + mergeJob + ": file " + fileName + " is now being copied via NRT point");
mergeJob.cancel("newNRTPoint is copying over the same file", null);
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index 4e29508..9de2c04 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -19,18 +19,9 @@ package org.apache.lucene.replicator.nrt;
import java.io.Closeable;
import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.lucene.document.Document;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
/** Parent JVM hold this "wrapper" to refer to each child JVM. This is roughly equivalent e.g. to a client-side "sugar" API. */
class NodeProcess implements Closeable {
@@ -234,5 +225,14 @@ class NodeProcess implements Closeable {
c.flush();
c.in.readByte();
}
+
+ public void deleteAllDocuments(Connection c) throws IOException {
+ if (isPrimary == false) {
+ throw new IllegalStateException("only primary can index");
+ }
+ c.out.writeByte(SimplePrimaryNode.CMD_DELETE_ALL_DOCS);
+ c.flush();
+ c.in.readByte();
+ }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
index 18e77ef..0afd1b4 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -43,17 +43,16 @@ import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentCommitInfo;
-import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TieredMergePolicy;
import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.DataInput;
@@ -61,8 +60,6 @@ import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@@ -74,6 +71,8 @@ class SimplePrimaryNode extends PrimaryNode {
final int tcpPort;
+ final Random random;
+
// These are updated by parent test process whenever replicas change:
int[] replicaTCPPorts;
int[] replicaIDs;
@@ -81,6 +80,10 @@ class SimplePrimaryNode extends PrimaryNode {
// So we only flip a bit once per file name:
final Set<String> bitFlipped = Collections.synchronizedSet(new HashSet<>());
+ final List<MergePreCopy> warmingSegments = Collections.synchronizedList(new ArrayList<>());
+
+ final boolean doFlipBitsDuringCopy;
+
static class MergePreCopy {
final List<Connection> connections = Collections.synchronizedList(new ArrayList<>());
final Map<String,FileMetaData> files;
@@ -109,11 +112,12 @@ class SimplePrimaryNode extends PrimaryNode {
}
}
- final List<MergePreCopy> warmingSegments = Collections.synchronizedList(new ArrayList<>());
-
- public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException {
+ public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory,
+ boolean doFlipBitsDuringCopy) throws IOException {
super(initWriter(id, random, indexPath), id, primaryGen, forcePrimaryVersion, searcherFactory);
this.tcpPort = tcpPort;
+ this.random = new Random(random.nextLong());
+ this.doFlipBitsDuringCopy = doFlipBitsDuringCopy;
}
/** Records currently alive replicas. */
@@ -187,7 +191,7 @@ class SimplePrimaryNode extends PrimaryNode {
long startNS = System.nanoTime();
long lastWarnNS = startNS;
- // TODO: maybe ... place some sort of time limit on how long we are willing to wait for slow replicas to finish copying?
+ // TODO: maybe ... place some sort of time limit on how long we are willing to wait for slow replica(s) to finish copying?
while (preCopy.finished() == false) {
try {
Thread.sleep(10);
@@ -241,7 +245,16 @@ class SimplePrimaryNode extends PrimaryNode {
message("top: warning: replica socket=" + c.s + " for segment=" + info + " seems to be dead; closing files=" + files.keySet());
IOUtils.closeWhileHandlingException(c);
it.remove();
+ done = true;
}
+
+ if (done == false && random.nextInt(1000) == 17) {
+ message("top: warning: now randomly dropping replica from merge warming; files=" + files.keySet());
+ IOUtils.closeWhileHandlingException(c);
+ it.remove();
+ done = true;
+ }
+
} catch (Throwable t) {
message("top: ignore exception trying to read byte during warm for segment=" + info + " to replica socket=" + c.s + ": " + t + " files=" + files.keySet());
IOUtils.closeWhileHandlingException(c);
@@ -368,7 +381,7 @@ class SimplePrimaryNode extends PrimaryNode {
while (upto < len) {
int chunk = (int) Math.min(buffer.length, (len-upto));
in.readBytes(buffer, 0, chunk);
- if (TestNRTReplication.DO_BIT_FLIPS_DURING_COPY) {
+ if (doFlipBitsDuringCopy) {
if (random.nextInt(3000) == 17 && bitFlipped.contains(fileName) == false) {
bitFlipped.add(fileName);
message("file " + fileName + " to R" + replicaID + ": now randomly flipping a bit at byte=" + upto);
@@ -435,6 +448,10 @@ class SimplePrimaryNode extends PrimaryNode {
handleDeleteDocument(in, out);
out.writeByte((byte) 1);
bos.flush();
+ } else if (cmd == CMD_DELETE_ALL_DOCS) {
+ writer.deleteAll();
+ out.writeByte((byte) 1);
+ bos.flush();
} else if (cmd == CMD_INDEXING_DONE) {
out.writeByte((byte) 1);
bos.flush();
@@ -508,6 +525,7 @@ class SimplePrimaryNode extends PrimaryNode {
static final byte CMD_MARKER_SEARCH = 3;
static final byte CMD_COMMIT = 4;
static final byte CMD_CLOSE = 5;
+ static final byte CMD_SEARCH_ALL = 21;
// Send (to primary) the list of currently running replicas:
static final byte CMD_SET_REPLICAS = 16;
@@ -518,6 +536,7 @@ class SimplePrimaryNode extends PrimaryNode {
static final byte CMD_UPDATE_DOC = 7;
static final byte CMD_DELETE_DOC = 8;
static final byte CMD_INDEXING_DONE = 19;
+ static final byte CMD_DELETE_ALL_DOCS = 22;
// Sent by replica to primary when replica first starts up, so primary can add it to any warming merges:
static final byte CMD_NEW_REPLICA = 20;
@@ -579,6 +598,22 @@ class SimplePrimaryNode extends PrimaryNode {
}
continue outer;
+ case CMD_SEARCH_ALL:
+ {
+ Thread.currentThread().setName("search all");
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.search(new MatchAllDocsQuery(), 1).totalHits;
+ //message("version=" + version + " searcher=" + searcher);
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+ continue outer;
+
case CMD_MARKER_SEARCH:
{
Thread.currentThread().setName("msearch");
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
index 8667df1..27a5547 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -24,19 +24,16 @@ import java.io.InputStream;
import java.net.ServerSocket;
import java.net.Socket;
import java.nio.file.Path;
-import java.util.Collections;
-import java.util.HashSet;
import java.util.Locale;
import java.util.Map;
import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.DataInput;
@@ -45,7 +42,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
import org.apache.lucene.store.RateLimitedIndexOutput;
import org.apache.lucene.store.RateLimiter;
import org.apache.lucene.util.LuceneTestCase;
@@ -182,7 +178,7 @@ class SimpleReplicaNode extends ReplicaNode {
break;
case SimplePrimaryNode.CMD_GET_SEARCHING_VERSION:
- // nocommit this is hacky:
+ // This is called when primary has crashed and we need to elect a new primary from all the still running replicas:
// Tricky: if a sync is just finishing up, i.e. managed to finish copying all files just before we crashed primary, and is now
// in the process of opening a new reader, we need to wait for it, to be sure we really pick the most current replica:
@@ -190,7 +186,7 @@ class SimpleReplicaNode extends ReplicaNode {
message("top: getSearchingVersion: now wait for finish sync");
// TODO: use immediate concurrency instead of polling:
while (isCopying() && stop.get() == false) {
- Thread.sleep(50);
+ Thread.sleep(10);
message("top: curNRTCopy=" + curNRTCopy);
}
message("top: getSearchingVersion: done wait for finish sync");
@@ -212,6 +208,24 @@ class SimpleReplicaNode extends ReplicaNode {
//node.message("version=" + version + " searcher=" + searcher);
out.writeVLong(version);
out.writeVInt(hitCount);
+ bos.flush();
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+ continue outer;
+
+ case SimplePrimaryNode.CMD_SEARCH_ALL:
+ {
+ Thread.currentThread().setName("search all");
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.search(new MatchAllDocsQuery(), 1).totalHits;
+ //node.message("version=" + version + " searcher=" + searcher);
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ bos.flush();
} finally {
mgr.release(searcher);
}
@@ -227,6 +241,7 @@ class SimpleReplicaNode extends ReplicaNode {
int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
out.writeVLong(version);
out.writeVInt(hitCount);
+ bos.flush();
} finally {
mgr.release(searcher);
}
@@ -290,6 +305,7 @@ class SimpleReplicaNode extends ReplicaNode {
default:
throw new IllegalArgumentException("unrecognized cmd=" + cmd);
}
+ System.out.println("NOW FLUSH");
bos.flush();
break;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index 7a257de..5b04721 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -19,61 +19,32 @@ package org.apache.lucene.replicator.nrt;
import java.io.BufferedInputStream;
import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
-import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.PrintWriter;
import java.lang.reflect.Method;
-import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.SocketException;
import java.nio.file.Path;
-import java.nio.file.Path;
-import java.nio.file.Paths;
import java.nio.file.Paths;
-import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
-import java.util.List;
import java.util.Map;
-import java.util.Random;
import java.util.Set;
-import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
/** Child process with silly naive TCP socket server to handle
@@ -262,8 +233,9 @@ public class SimpleServer extends LuceneTestCase {
long primaryGen = Long.parseLong(System.getProperty("tests.nrtreplication.primaryGen"));
Node.globalStartNS = Long.parseLong(System.getProperty("tests.nrtreplication.startNS"));
- boolean doRandomCrash = isPrimary ? TestNRTReplication.DO_CRASH_PRIMARY : TestNRTReplication.DO_CRASH_REPLICA;
- boolean doRandomClose = isPrimary ? false : TestNRTReplication.DO_CLOSE_REPLICA;
+ boolean doRandomCrash = "true".equals(System.getProperty("tests.nrtreplication.doRandomCrash"));
+ boolean doRandomClose = "true".equals(System.getProperty("tests.nrtreplication.doRandomClose"));
+ boolean doFlipBitsDuringCopy = "true".equals(System.getProperty("tests.nrtreplication.doFlipBitsDuringCopy"));
// Create server socket that we listen for incoming requests on:
try (final ServerSocket ss = new ServerSocket(0)) {
@@ -272,7 +244,7 @@ public class SimpleServer extends LuceneTestCase {
System.out.println("\nPORT: " + tcpPort);
final Node node;
if (isPrimary) {
- node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null);
+ node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy);
System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
} else {
node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 5a073ff..755b234 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -18,499 +18,39 @@ package org.apache.lucene.replicator.nrt;
*/
import java.io.BufferedReader;
-import java.io.Closeable;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.UnsupportedEncodingException;
-import java.io.Writer;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.net.SocketException;
-import java.nio.charset.MalformedInputException;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
import java.util.List;
import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
import java.util.regex.Pattern;
-import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.ConcurrentMergeScheduler;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.ScoreDoc;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.NIOFSDirectory;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.store.RateLimiter;
-import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.ThreadInterruptedException;
import com.carrotsearch.randomizedtesting.SeedUtils;
-// nocommit sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
-
-// nocommit randomly p.destroy() one replica?
-
-/*
- TODO
- - why do we do the "rename temp to actual" all at the end...? what really does that buy us?
- - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
- - test should not print scary exceptions and then succeed!
- - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
- - are the pre-copied-completed-merged files not being cleared in primary?
- - hmm the logic isn't right today? a replica may skip pulling a given copy state, that recorded the finished merged segments?
- - beast & fix bugs
- - graceful cluster restart
- - better translog integration
- - get "graceful primary shutdown" working
- - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion
- - clean up how version is persisted in commit data
- - why am i not using hashes here? how does ES use them?
- - get all other "single shard" functions working too: this cluster should "act like" a single shard
- - SLM
- - controlled nrt reopen thread / returning long gen on write
- - live field values
- - add indexes
- - make cluster level APIs to search, index, that deal w/ primary failover, etc.
- - must prune xlog
- - refuse to start primary unless we have quorum
- - later
- - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback?
- - back pressure on indexing if replicas can't keep up?
- - get xlog working on top? needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit
- quorum
- - maybe fix IW to return "gen" or "seq id" or "segment name" or something?
- - replica can copy files from other replicas too / use multicast / rsync / something
- - each replica could also pre-open a SegmentReader after pre-copy when warming a merge
- - we can pre-copy newly flushed files too, for cases where reopen rate is low vs IW's flushing because RAM buffer is full
- - opto: pre-copy files as they are written; if they will become CFS, we can build CFS on the replica?
- - what about multiple commit points?
- - fix primary to init directly from an open replica, instead of having to commit/close the replica first
-*/
-
-// Tricky cases:
-// - we are pre-copying a merge, then replica starts up part way through, so it misses that pre-copy and must do it on next nrt point
-// - a down replica starts up, but it's "from the future" vs the current primary, and must copy over file names with different contents
-// but referenced by its latest commit point, so it must fully remove that commit ... which is a hazardous window
-// - replica comes up just as the primary is crashing / moving
-// - electing a new primary when a replica is just finishing its nrt sync: we need to wait for it so we are sure to get the "most up to
-// date" replica
-// - replica comes up after merged segment finished so it doesn't copy over the merged segment "promptly" (i.e. only sees it on NRT refresh)
-
-/**
- * Test case showing how to implement NRT replication. This test spawns a sub-process per-node, running TestNRTReplicationChild.
- *
- * One node is primary, and segments are periodically flushed there, then concurrently the N replica nodes copy the new files over and open new readers, while
- * primary also opens a new reader.
- *
- * Nodes randomly crash and are restarted. If the primary crashes, a replica is promoted.
- *
- * Merges are currently first finished on the primary and then pre-copied out to replicas with a merged segment warmer so they don't block
- * ongoing NRT reopens. Probably replicas could do their own merging instead, but this is more complex and may not be better overall
- * (merging takes a lot of IO resources).
- *
- * Slow network is simulated with a RateLimiter.
- */
-
-// nocommit randomly delete all doc sometimes, 1) using IW.deleteAll and 2) doing it inefficiently (by query, by id)
-
// MockRandom's .sd file has no index header/footer:
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
public class TestNRTReplication extends LuceneTestCase {
- // Test evilness controls:
-
- /** Randomly crash the current primary (losing data!) and promote the "next best" replica. */
- static final boolean DO_CRASH_PRIMARY = true;
-
- /** Randomly crash (JVM core dumps) a replica; it will later randomly be restarted and sync itself. */
- static final boolean DO_CRASH_REPLICA = true;
-
- /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
- static final boolean DO_CLOSE_REPLICA = true;
-
- /** If false, all child + parent output is interleaved into single stdout/err */
- static final boolean SEPARATE_CHILD_OUTPUT = false;
-
- // nocommit DO_CLOSE_PRIMARY?
-
- /** Randomly crash whole cluster and then restart it */
- static final boolean DO_FULL_CLUSTER_CRASH = true;
-
- /** True if we randomly flip a bit while copying files out */
- static final boolean DO_BIT_FLIPS_DURING_COPY = true;
-
- /** Set to a non-null value to force exactly that many nodes; else, it's random. */
- static final Integer NUM_NODES = null;
-
- static final boolean DO_RANDOM_XLOG_REPLAY = false;
-
- final AtomicBoolean failed = new AtomicBoolean();
-
- final AtomicBoolean stop = new AtomicBoolean();
-
/** cwd where we start each child (server) node */
private Path childTempDir;
- long primaryGen;
-
- volatile long lastPrimaryVersion;
-
- volatile NodeProcess primary;
- volatile NodeProcess[] nodes;
- volatile long[] nodeTimeStamps;
- volatile boolean[] starting;
-
- Path[] indexPaths;
-
- Path transLogPath;
- SimpleTransLog transLog;
- final AtomicInteger markerUpto = new AtomicInteger();
-
- /** Maps searcher version to how many hits the query body:the matched. */
- final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
-
- /** Maps searcher version to how many marker documents matched. This should only ever grow (we never delete marker documents). */
- final Map<Long,Integer> versionToMarker = new ConcurrentHashMap<>();
-
- /** Maps searcher version to xlog location when refresh of this version started. */
- final Map<Long,Long> versionToTransLogLocation = new ConcurrentHashMap<>();
-
- public void test() throws Exception {
-
- Node.globalStartNS = System.nanoTime();
-
- message("change thread name from " + Thread.currentThread().getName());
- Thread.currentThread().setName("main");
-
- childTempDir = createTempDir("child");
-
- // We are parent process:
-
- // Silly bootstrapping:
- versionToTransLogLocation.put(0L, 0L);
- versionToTransLogLocation.put(1L, 0L);
-
- int numNodes;
-
- if (NUM_NODES == null) {
- numNodes = TestUtil.nextInt(random(), 2, 10);
- } else {
- numNodes = NUM_NODES.intValue();
- }
-
- System.out.println("TEST: using " + numNodes + " nodes");
-
- transLogPath = createTempDir("NRTReplication").resolve("translog");
- transLog = new SimpleTransLog(transLogPath);
-
- //state.rateLimiters = new RateLimiter[numNodes];
- indexPaths = new Path[numNodes];
- nodes = new NodeProcess[numNodes];
- nodeTimeStamps = new long[numNodes];
- Arrays.fill(nodeTimeStamps, Node.globalStartNS);
- starting = new boolean[numNodes];
-
- for(int i=0;i<numNodes;i++) {
- indexPaths[i] = createTempDir("index" + i);
- }
-
- Thread[] indexers = new Thread[TestUtil.nextInt(random(), 1, 3)];
- System.out.println("TEST: launch " + indexers.length + " indexer threads");
- for(int i=0;i<indexers.length;i++) {
- indexers[i] = new IndexThread();
- indexers[i].setName("indexer" + i);
- indexers[i].setDaemon(true);
- indexers[i].start();
- }
-
- Thread[] searchers = new Thread[TestUtil.nextInt(random(), 1, 3)];
- System.out.println("TEST: launch " + searchers.length + " searcher threads");
- for(int i=0;i<searchers.length;i++) {
- searchers[i] = new SearchThread();
- searchers[i].setName("searcher" + i);
- searchers[i].setDaemon(true);
- searchers[i].start();
- }
-
- Thread restarter = new RestartThread();
- restarter.setName("restarter");
- restarter.setDaemon(true);
- restarter.start();
-
- int runTimeSec;
- if (TEST_NIGHTLY) {
- runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 120, 240);
- } else {
- runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 45, 120);
- }
-
- System.out.println("TEST: will run for " + runTimeSec + " sec");
-
- long endTime = System.nanoTime() + runTimeSec*1000000000L;
-
- sendReplicasToPrimary();
-
- while (failed.get() == false && System.nanoTime() < endTime) {
-
- // Wait a bit:
- Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
- if (primary != null && random().nextBoolean()) {
- message("top: now flush primary");
- NodeProcess curPrimary = primary;
- if (curPrimary != null) {
-
- // Save these before we start flush:
- long nextTransLogLoc = transLog.getNextLocation();
- int markerUptoSav = markerUpto.get();
-
- long result;
- try {
- result = primary.flush();
- } catch (Throwable t) {
- message("top: flush failed; skipping: " + t.getMessage());
- result = -1;
- }
- if (result > 0) {
- // There were changes
- lastPrimaryVersion = result;
- addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
- addVersionMarker(lastPrimaryVersion, markerUptoSav);
- }
- }
- }
-
- StringBuilder sb = new StringBuilder();
- int liveCount = 0;
- for(int i=0;i<nodes.length;i++) {
- NodeProcess node = nodes[i];
- if (node != null) {
- if (sb.length() != 0) {
- sb.append(" ");
- }
- liveCount++;
- if (node.isPrimary) {
- sb.append('P');
- } else {
- sb.append('R');
- }
- sb.append(i);
- }
- }
-
- message("PG=" + (primary == null ? "X" : primaryGen) + " " + liveCount + " (of " + nodes.length + ") nodes running: " + sb);
-
- // Commit a random node, primary or replica
-
- {
- NodeProcess node = nodes[random().nextInt(nodes.length)];
- if (node != null) {
- // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
- // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
- message("top: now commit node=" + node);
- node.commitAsync();
- }
- }
- }
-
- message("TEST: top: test done, now close");
- stop.set(true);
- for(Thread thread : indexers) {
- thread.join();
- }
- for(Thread thread : searchers) {
- thread.join();
- }
- restarter.join();
-
- // Close replicas before primary so we cancel any in-progres replications:
- System.out.println("TEST: top: now close replicas");
- List<Closeable> toClose = new ArrayList<>();
- for(NodeProcess node : nodes) {
- if (node != primary && node != null) {
- toClose.add(node);
- }
- }
- IOUtils.close(toClose);
- IOUtils.close(primary);
- IOUtils.close(transLog);
-
- if (failed.get() == false) {
- message("TEST: top: now checkIndex");
- for(Path path : indexPaths) {
- message("TEST: check " + path);
- MockDirectoryWrapper dir = newMockFSDirectory(path);
- // Just too slow otherwise
- dir.setCrossCheckTermVectorsOnClose(false);
- dir.close();
- }
- } else {
- message("TEST: failed; skip checkIndex");
- }
- }
-
- private boolean anyNodesStarting() {
- for(int id=0;id<nodes.length;id++) {
- if (starting[id]) {
- return true;
- }
- }
-
- return false;
- }
-
- /** Picks a replica and promotes it as new primary. */
- private void promoteReplica() throws IOException {
- message("top: primary crashed; now pick replica to promote");
- long maxSearchingVersion = -1;
- NodeProcess replicaToPromote = null;
-
- // We must promote the most current replica, because otherwise file name reuse can cause a replication to fail when it needs to copy
- // over a file currently held open for searching. This also minimizes recovery work since the most current replica means less xlog
- // replay to catch up:
- for (NodeProcess node : nodes) {
- if (node != null) {
- message("ask " + node + " for its current searching version");
- long searchingVersion = node.getSearchingVersion();
- message(node + " has searchingVersion=" + searchingVersion);
- if (searchingVersion > maxSearchingVersion) {
- maxSearchingVersion = searchingVersion;
- replicaToPromote = node;
- }
- }
- }
-
- if (replicaToPromote == null) {
- message("top: no replicas running; skipping primary promotion");
- return;
- }
-
- message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
- if (replicaToPromote.commit() == false) {
- message("top: commit failed; skipping primary promotion");
- return;
- }
-
- message("top: now shutdown " + replicaToPromote);
- if (replicaToPromote.shutdown() == false) {
- message("top: shutdown failed for R" + replicaToPromote.id + "; skipping primary promotion");
- return;
- }
-
- int id = replicaToPromote.id;
- message("top: now startPrimary " + replicaToPromote);
- startPrimary(replicaToPromote.id);
- }
-
- void startPrimary(int id) throws IOException {
- message(id + ": top: startPrimary lastPrimaryVersion=" + lastPrimaryVersion);
- assert nodes[id] == null;
-
- // Force version of new primary to advance beyond where old primary was, so we never re-use versions. It may have
- // already advanced beyond newVersion, e.g. if it flushed new segments while during xlog replay:
-
- // First start node as primary (it opens an IndexWriter) but do not publish it for searching until we replay xlog:
- NodeProcess newPrimary = startNode(id, indexPaths[id], true, lastPrimaryVersion+1);
- if (newPrimary == null) {
- message("top: newPrimary failed to start; abort");
- return;
- }
-
- // Get xlog location that this node was guaranteed to already have indexed through; this may replay some ops already indexed but it's OK
- // because the ops are idempotent: we updateDocument (by docid) on replay even for original addDocument:
- Long startTransLogLoc;
- Integer markerCount;
- if (newPrimary.initCommitVersion == 0) {
- startTransLogLoc = 0L;
- markerCount = 0;
- } else {
- startTransLogLoc = versionToTransLogLocation.get(newPrimary.initCommitVersion);
- markerCount = versionToMarker.get(newPrimary.initCommitVersion);
- }
- assert startTransLogLoc != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToTransLogLocation: keys=" + versionToTransLogLocation.keySet();
- assert markerCount != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToMarker: keys=" + versionToMarker.keySet();
-
- // When the primary starts, the userData in its latest commit point tells us which version it had indexed up to, so we know where to
- // replay from in the xlog. However, we forcefuly advance the version, and then IW on init (or maybe getReader) also adds 1 to it.
- // Since we publish the primary in this state (before xlog replay is done), a replica can start up at this point and pull this version,
- // and possibly later be chosen as a primary, causing problems if the version is known recorded in the translog map. So we record it
- // here:
-
- addTransLogLoc(newPrimary.initInfosVersion, startTransLogLoc);
- addVersionMarker(newPrimary.initInfosVersion, markerCount);
-
- assert newPrimary.initInfosVersion >= lastPrimaryVersion;
- message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
- lastPrimaryVersion = newPrimary.initInfosVersion;
-
- // Publish new primary, before replaying xlog. This means other indexing ops can come in at the same time as we catch up indexing
- // previous ops. Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
- // translog) concurrently with new incoming ops.
- nodes[id] = newPrimary;
- primary = newPrimary;
-
- sendReplicasToPrimary();
-
- long nextTransLogLoc = transLog.getNextLocation();
- int nextMarkerUpto = markerUpto.get();
- message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
- try {
- transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
- } catch (IOException ioe) {
- message("top: replay xlog failed; abort");
- return;
- }
- message("top: done replay trans log");
- }
-
final AtomicLong nodeStartCounter = new AtomicLong();
- final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
-
/** Launches a child "server" (separate JVM), which is either primary or replica node */
- NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
- nodeTimeStamps[id] = System.nanoTime();
+ NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
List<String> cmd = new ArrayList<>();
- NodeProcess curPrimary = primary;
-
cmd.add(System.getProperty("java.home")
+ System.getProperty("file.separator")
+ "bin"
@@ -518,12 +58,13 @@ public class TestNRTReplication extends LuceneTestCase {
+ "java");
cmd.add("-Xmx512m");
- if (curPrimary != null) {
- cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + curPrimary.tcpPort);
+ if (primaryTCPPort != -1) {
+ cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + primaryTCPPort);
} else if (isPrimary == false) {
// We cannot start a replica when there is no primary:
return null;
}
+ cmd.add("-Dtests.nrtreplication.closeorcrash=false");
cmd.add("-Dtests.nrtreplication.node=true");
cmd.add("-Dtests.nrtreplication.nodeid=" + id);
@@ -534,7 +75,7 @@ public class TestNRTReplication extends LuceneTestCase {
cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
}
- long myPrimaryGen = primaryGen;
+ long myPrimaryGen = 0;
cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
// Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
@@ -548,17 +89,6 @@ public class TestNRTReplication extends LuceneTestCase {
cmd.add("org.junit.runner.JUnitCore");
cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
- Writer childLog;
-
- if (SEPARATE_CHILD_OUTPUT) {
- Path childOut = childTempDir.resolve(id + ".log");
- message("logging to " + childOut);
- childLog = Files.newBufferedWriter(childOut, StandardCharsets.UTF_8, StandardOpenOption.APPEND, StandardOpenOption.CREATE);
- childLog.write("\n\nSTART NEW CHILD:\n");
- } else {
- childLog = null;
- }
-
message("child process command: " + cmd);
ProcessBuilder pb = new ProcessBuilder(cmd);
pb.redirectErrorStream(true);
@@ -592,41 +122,11 @@ public class TestNRTReplication extends LuceneTestCase {
throw new RuntimeException(ie);
}
message("exit value=" + p.exitValue());
-
- // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
- if (isPrimary == false) {
- if (sawExistingSegmentsFile) {
- // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
- // and retry again later:
- message("failed to remove segments_N; skipping");
- return null;
- }
- for(int i=0;i<10;i++) {
- if (primaryGen != myPrimaryGen || primary == null) {
- // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
- message("primary crashed/closed while replica R" + id + " tried to start; skipping");
- return null;
- } else {
- try {
- Thread.sleep(10);
- } catch (InterruptedException ie) {
- throw new ThreadInterruptedException(ie);
- }
- }
- }
- }
-
- // Should fail the test:
message("top: now fail test replica R" + id + " failed to start");
- failed.set(true);
throw new RuntimeException("replica R" + id + " failed to start");
}
- if (childLog != null) {
- childLog.write(l);
- childLog.write("\n");
- childLog.flush();
- } else if (logTimeStart.matcher(l).matches()) {
+ if (logTimeStart.matcher(l).matches()) {
// Already a well-formed log output:
System.out.println(l);
} else {
@@ -650,7 +150,7 @@ public class TestNRTReplication extends LuceneTestCase {
final boolean finalWillCrash = willCrash;
- // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
+ // Baby sits the child process, pulling its stdout and printing to our stdout:
Thread pumper = ThreadPumper.start(
new Runnable() {
@Override
@@ -665,503 +165,147 @@ public class TestNRTReplication extends LuceneTestCase {
message("done wait for process " + p);
int exitValue = p.exitValue();
message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
- if (childLog != null) {
- try {
- childLog.write("process done; exitValue=" + exitValue + "\n");
- childLog.close();
- } catch (IOException ioe) {
- throw new RuntimeException(ioe);
- }
- }
- if (exitValue != 0 && finalWillCrash == false && crashingNodes.remove(id) == false) {
+ if (exitValue != 0) {
// should fail test
- failed.set(true);
- if (childLog != null) {
- throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue + "; see " + childLog + " for details");
- } else {
- throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
- }
+ throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
}
- nodeClosed(id);
}
- }, r, System.out, childLog);
+ }, r, System.out, null);
pumper.setName("pump" + id);
message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
}
- private void nodeClosed(int id) {
- NodeProcess oldNode = nodes[id];
- if (primary != null && oldNode == primary) {
- message("top: " + primary + ": primary process finished");
- primary = null;
- primaryGen++;
- } else {
- message("top: " + oldNode + ": replica process finished");
+ public void testReplicateDeleteAllDocuments() throws Exception {
+
+ Node.globalStartNS = System.nanoTime();
+ childTempDir = createTempDir("child");
+
+ message("change thread name from " + Thread.currentThread().getName());
+ Thread.currentThread().setName("main");
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+
+ // Tell primary current replicas:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+ c.out.writeVInt(1);
+ c.out.writeVInt(replica.id);
+ c.out.writeVInt(replica.tcpPort);
+ c.flush();
+ c.in.readByte();
}
- if (oldNode != null) {
- oldNode.isOpen = false;
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ Connection primaryC = new Connection(primary.tcpPort);
+ primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
}
- nodes[id] = null;
- nodeTimeStamps[id] = System.nanoTime();
- sendReplicasToPrimary();
- }
+ // Nothing in replica index yet
+ Connection replicaC = new Connection(replica.tcpPort);
+ replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ replicaC.flush();
+ long version1 = replicaC.in.readVLong();
+ assertEquals(0L, version1);
+ int hitCount = replicaC.in.readVInt();
+ assertEquals(0, hitCount);
- /** Sends currently alive replicas to primary, which uses this to know who to notify when it does a refresh */
- private void sendReplicasToPrimary() {
- NodeProcess curPrimary = primary;
- if (curPrimary != null) {
- List<NodeProcess> replicas = new ArrayList<>();
- for (NodeProcess node : nodes) {
- if (node != null && node.isPrimary == false) {
- replicas.add(node);
- }
- }
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush();
+ assertTrue(primaryVersion1 > 0);
- message("top: send " + replicas.size() + " replicas to primary");
+ long version2;
- try (Connection c = new Connection(curPrimary.tcpPort)) {
- c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
- c.out.writeVInt(replicas.size());
- for(NodeProcess replica : replicas) {
- c.out.writeVInt(replica.id);
- c.out.writeVInt(replica.tcpPort);
- }
- c.flush();
- c.in.readByte();
- } catch (Throwable t) {
- message("top: ignore exc sending replicas to primary: " + t);
+ // Wait for replica to show the change
+ while (true) {
+ replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ replicaC.flush();
+ version2 = replicaC.in.readVLong();
+ hitCount = replicaC.in.readVInt();
+ if (hitCount == 10) {
+ assertTrue(version2 > version1);
+ // good!
+ break;
}
+ Thread.sleep(10);
}
- }
- void addVersionMarker(long version, int count) {
- //System.out.println("ADD VERSION MARKER version=" + version + " count=" + count);
- if (versionToMarker.containsKey(version)) {
- int curCount = versionToMarker.get(version);
- if (curCount != count) {
- message("top: wrong marker count version=" + version + " count=" + count + " curCount=" + curCount);
- throw new IllegalStateException("version=" + version + " count=" + count + " curCount=" + curCount);
+ // Delete all docs from primary
+ if (random().nextBoolean()) {
+ // Inefficiently:
+ for(int id=0;id<10;id++) {
+ primary.deleteDocument(primaryC, Integer.toString(id));
}
} else {
- message("top: record marker count: version=" + version + " count=" + count);
- versionToMarker.put(version, count);
+ // Efficiently:
+ primary.deleteAllDocuments(primaryC);
}
- }
-
- void addTransLogLoc(long version, long loc) {
- message("top: record transLogLoc: version=" + version + " loc=" + loc);
- versionToTransLogLocation.put(version, loc);
- }
-
- // Periodically wakes up and starts up any down nodes:
- private class RestartThread extends Thread {
- @Override
- public void run() {
-
- List<Thread> startupThreads = Collections.synchronizedList(new ArrayList<>());
-
- try {
- while (stop.get() == false) {
- Thread.sleep(TestUtil.nextInt(random(), 50, 500));
- message("top: restarter cycle");
-
- // Randomly crash full cluster:
- if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
- message("top: full cluster crash");
- for(int i=0;i<nodes.length;i++) {
- if (starting[i]) {
- message("N" + i + ": top: wait for startup so we can crash...");
- while (starting[i]) {
- Thread.sleep(10);
- }
- message("N" + i + ": top: done wait for startup");
- }
- NodeProcess node = nodes[i];
- if (node != null) {
- crashingNodes.add(i);
- message("top: N" + node.id + ": top: now crash node");
- node.crash();
- message("top: N" + node.id + ": top: done crash node");
- }
- }
- }
-
- List<Integer> downNodes = new ArrayList<>();
- StringBuilder b = new StringBuilder();
- long nowNS = System.nanoTime();
- for(int i=0;i<nodes.length;i++) {
- b.append(' ');
- double sec = (nowNS - nodeTimeStamps[i])/1000000000.0;
- String prefix;
- if (nodes[i] == null) {
- downNodes.add(i);
- if (starting[i]) {
- prefix = "s";
- } else {
- prefix = "x";
- }
- } else {
- prefix = "";
- }
- if (primary != null && nodes[i] == primary) {
- prefix += "p";
- }
- b.append(String.format(Locale.ROOT, "%s%d(%.1fs)", prefix, i, sec));
- }
- message("node status" + b.toString());
- message("downNodes=" + downNodes);
-
- // If primary is down, promote a replica:
- if (primary == null) {
- if (anyNodesStarting()) {
- message("top: skip promote replica: nodes are still starting");
- continue;
- }
- promoteReplica();
- }
-
- // Randomly start up a down a replica:
-
- // Stop or start a replica
- if (downNodes.isEmpty() == false) {
- int idx = downNodes.get(random().nextInt(downNodes.size()));
- if (starting[idx] == false) {
- if (primary == null) {
- if (downNodes.size() == nodes.length) {
- // Cold start: entire cluster is down, start this node up as the new primary
- message("N" + idx + ": top: cold start as primary");
- startPrimary(idx);
- }
- } else if (random().nextDouble() < ((double) downNodes.size())/nodes.length) {
- // Start up replica:
- starting[idx] = true;
- message("N" + idx + ": top: start up: launch thread");
- Thread t = new Thread() {
- @Override
- public void run() {
- try {
- message("N" + idx + ": top: start up thread");
- nodes[idx] = startNode(idx, indexPaths[idx], false, -1);
- sendReplicasToPrimary();
- } catch (Throwable t) {
- failed.set(true);
- stop.set(true);
- throw new RuntimeException(t);
- } finally {
- starting[idx] = false;
- startupThreads.remove(Thread.currentThread());
- }
- }
- };
- t.setName("start R" + idx);
- t.start();
- startupThreads.add(t);
- }
- } else {
- message("node " + idx + " still starting");
- }
- }
- }
- System.out.println("Restarter: now stop: join " + startupThreads.size() + " startup threads");
-
- while (startupThreads.size() > 0) {
- Thread.sleep(10);
- }
+ // Replica still shows 10 docs:
+ replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ replicaC.flush();
+ long version3 = replicaC.in.readVLong();
+ assertEquals(version2, version3);
+ hitCount = replicaC.in.readVInt();
+ assertEquals(10, hitCount);
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion2 = primary.flush();
+ assertTrue(primaryVersion2 > primaryVersion1);
- } catch (Throwable t) {
- failed.set(true);
- stop.set(true);
- throw new RuntimeException(t);
+ // Wait for replica to show the change
+ long version4;
+ while (true) {
+ replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ replicaC.flush();
+ version4 = replicaC.in.readVLong();
+ hitCount = replicaC.in.readVInt();
+ if (hitCount == 0) {
+ assertTrue(version4 > version3);
+ // good!
+ break;
}
+ Thread.sleep(10);
}
- }
-
- /** Randomly picks a node and runs a search against it */
- private class SearchThread extends Thread {
-
- @Override
- public void run() {
- // Maps version to number of hits for silly 'the' TermQuery:
- Query theQuery = new TermQuery(new Term("body", "the"));
-
- // Persists connections
- Map<Integer,Connection> connections = new HashMap<>();
-
- while (stop.get() == false) {
- NodeProcess node = nodes[random().nextInt(nodes.length)];
- if (node == null || node.isOpen == false) {
- continue;
- }
-
- if (node.lock.tryLock() == false) {
- // Node is in the process of closing or crashing or something
- continue;
- }
-
- try {
-
- Thread.currentThread().setName("Searcher node=" + node);
-
- //System.out.println("S: cycle; conns=" + connections);
-
- Connection c = connections.get(node.id);
-
- long version;
- try {
- if (c == null) {
- //System.out.println("S: new connection " + node.id + " " + Thread.currentThread().getName());
- c = new Connection(node.tcpPort);
- connections.put(node.id, c);
- } else {
- //System.out.println("S: reuse connection " + node.id + " " + Thread.currentThread().getName());
- }
-
- c.out.writeByte(SimplePrimaryNode.CMD_SEARCH);
- c.flush();
-
- while (c.sockIn.available() == 0) {
- if (stop.get()) {
- break;
- }
- if (node.isOpen == false) {
- throw new IOException("node closed");
- }
- Thread.sleep(1);
- }
- version = c.in.readVLong();
-
- while (c.sockIn.available() == 0) {
- if (stop.get()) {
- break;
- }
- if (node.isOpen == false) {
- throw new IOException("node closed");
- }
- Thread.sleep(1);
- }
- int hitCount = c.in.readVInt();
-
- Integer oldHitCount = hitCounts.get(version);
-
- // TODO: we never prune this map...
- if (oldHitCount == null) {
- hitCounts.put(version, hitCount);
- message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
- } else {
- // Just ensure that all nodes show the same hit count for
- // the same version, i.e. they really are replicas of one another:
- if (oldHitCount.intValue() != hitCount) {
- failed.set(true);
- stop.set(true);
- message("top: searcher: wrong version hitCount: version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
- fail("version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
- }
- }
- } catch (IOException ioe) {
- //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
- //ioe.printStackTrace(System.out);
- IOUtils.closeWhileHandlingException(c);
- connections.remove(node.id);
- continue;
- }
-
- // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
- Integer expectedAtLeastHitCount = versionToMarker.get(version);
-
- if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
- try {
- c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
- c.flush();
- while (c.sockIn.available() == 0) {
- if (stop.get()) {
- break;
- }
- if (node.isOpen == false) {
- throw new IOException("node died");
- }
- Thread.sleep(1);
- }
-
- version = c.in.readVLong();
- while (c.sockIn.available() == 0) {
- if (stop.get()) {
- break;
- }
- if (node.isOpen == false) {
- throw new IOException("node died");
- }
- Thread.sleep(1);
- }
-
- int hitCount = c.in.readVInt();
-
- // Look for data loss: make sure all marker docs are visible:
-
- if (hitCount < expectedAtLeastHitCount) {
-
- String failMessage = "node=" + node + ": documents were lost version=" + version + " hitCount=" + hitCount + " vs expectedAtLeastHitCount=" + expectedAtLeastHitCount;
- message(failMessage);
- failed.set(true);
- stop.set(true);
- fail(failMessage);
- }
- } catch (IOException ioe) {
- //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
- //throw new RuntimeException(ioe);
- //ioe.printStackTrace(System.out);
- IOUtils.closeWhileHandlingException(c);
- connections.remove(node.id);
- continue;
- }
- }
+ // Index 10 docs again:
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
- Thread.sleep(10);
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion3 = primary.flush();
+ assertTrue(primaryVersion3 > primaryVersion2);
- } catch (Throwable t) {
- failed.set(true);
- stop.set(true);
- throw new RuntimeException(t);
- } finally {
- node.lock.unlock();
- }
+ // Wait for replica to show the change
+ while (true) {
+ replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ replicaC.flush();
+ long version5 = replicaC.in.readVLong();
+ hitCount = replicaC.in.readVInt();
+ if (hitCount == 10) {
+ assertTrue(version5 > version4);
+ // good!
+ break;
}
- System.out.println("Searcher: now stop");
- IOUtils.closeWhileHandlingException(connections.values());
+ Thread.sleep(10);
}
- }
-
- private class IndexThread extends Thread {
-
- @Override
- public void run() {
-
- try {
- LineFileDocs docs = new LineFileDocs(random());
- int docCount = 0;
-
- // How often we do an update/delete vs add:
- double updatePct = random().nextDouble();
-
- // Varies how many docs/sec we index:
- int sleepChance = TestUtil.nextInt(random(), 4, 100);
- message("top: indexer: updatePct=" + updatePct + " sleepChance=" + sleepChance);
+ replicaC.close();
+ primaryC.close();
- long lastTransLogLoc = transLog.getNextLocation();
-
- NodeProcess curPrimary = null;
- Connection c = null;
-
- while (stop.get() == false) {
-
- try {
- while (stop.get() == false && curPrimary == null) {
- Thread.sleep(10);
- curPrimary = primary;
- if (curPrimary != null) {
- c = new Connection(curPrimary.tcpPort);
- c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
- break;
- }
- }
-
- if (stop.get()) {
- break;
- }
-
- Thread.currentThread().setName("indexer p" + curPrimary.id);
-
- if (random().nextInt(10) == 7) {
- // We use the marker docs to check for data loss in search thread:
- Document doc = new Document();
- int id = markerUpto.getAndIncrement();
- String idString = "m"+id;
- doc.add(newStringField("docid", idString, Field.Store.YES));
- doc.add(newStringField("marker", "marker", Field.Store.YES));
- curPrimary.addOrUpdateDocument(c, doc, false);
- transLog.addDocument(idString, doc);
- message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
- }
-
- if (docCount > 0 && random().nextDouble() < updatePct) {
- int randomID = random().nextInt(docCount);
- String randomIDString = Integer.toString(randomID);
- if (random().nextBoolean()) {
- // Replace previous doc
- Document doc = docs.nextDoc();
- ((Field) doc.getField("docid")).setStringValue(randomIDString);
- curPrimary.addOrUpdateDocument(c, doc, true);
- transLog.updateDocument(randomIDString, doc);
- } else {
- // Delete previous doc
- curPrimary.deleteDocument(c, randomIDString);
- transLog.deleteDocuments(randomIDString);
- }
- } else {
- // Add new doc:
- Document doc = docs.nextDoc();
- String idString = Integer.toString(docCount++);
- ((Field) doc.getField("docid")).setStringValue(idString);
- curPrimary.addOrUpdateDocument(c, doc, false);
- transLog.addDocument(idString, doc);
-
- if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
- long curLoc = transLog.getNextLocation();
- // randomly replay chunks of translog just to test replay:
- message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
- transLog.replay(curPrimary, lastTransLogLoc, curLoc);
- lastTransLogLoc = curLoc;
- }
- }
- } catch (IOException se) {
- // Assume primary crashed
- message("top: indexer lost connection to primary");
- try {
- c.close();
- } catch (Throwable t) {
- }
- curPrimary = null;
- c = null;
- }
-
- if (random().nextInt(sleepChance) == 0) {
- Thread.sleep(1);
- }
-
- if (random().nextInt(100) == 17) {
- System.out.println("Indexer: now pause for a bit...");
- Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
- System.out.println("Indexer: done pause for a bit...");
- }
- }
- if (curPrimary != null) {
- try {
- c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
- c.flush();
- c.in.readByte();
- } catch (IOException se) {
- // Assume primary crashed
- message("top: indexer lost connection to primary");
- try {
- c.close();
- } catch (Throwable t) {
- }
- curPrimary = null;
- c = null;
- }
- }
- System.out.println("Indexer: now stop");
- } catch (Throwable t) {
- failed.set(true);
- stop.set(true);
- throw new RuntimeException(t);
- }
- }
+ replica.close();
+ primary.close();
}
static void message(String message) {
[10/15] lucene-solr git commit: add nocommit;
try to fix false test failure due to concurrent check index running
while primary is still closing
Posted by mi...@apache.org.
add nocommit; try to fix false test failure due to concurrent check index running while primary is still closing
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/7af83c48
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/7af83c48
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/7af83c48
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: 7af83c486b50ceb13d58e79dd65c158bc96130dc
Parents: 8889469
Author: Mike McCandless <mi...@apache.org>
Authored: Wed Feb 3 16:45:46 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Wed Feb 3 16:45:46 2016 -0500
----------------------------------------------------------------------
.../src/test/org/apache/lucene/replicator/nrt/NodeProcess.java | 4 +---
.../org/apache/lucene/replicator/nrt/TestNRTReplication.java | 2 ++
2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af83c48/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index dcbef87..be0b3df3 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -157,9 +157,7 @@ class NodeProcess implements Closeable {
throw new RuntimeException("shutdown failed");
}
} catch (Throwable t) {
- System.out.println("top: shutdown failed; skipping");
- t.printStackTrace(System.out);
- return false;
+ System.out.println("top: shutdown failed; ignoring");
}
try {
p.waitFor();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7af83c48/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index b2240eb..510a075 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -37,6 +37,8 @@ import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
import com.carrotsearch.randomizedtesting.SeedUtils;
+// nocommit make some explicit failure tests
+
// MockRandom's .sd file has no index header/footer:
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
[13/15] lucene-solr git commit: Merge branch 'master' into
nrt_replicas
Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
----------------------------------------------------------------------
diff --cc lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
index 0000000,aa1da81..f2f8de7
mode 000000,100644..100644
--- a/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
+++ b/lucene/spatial/src/test/org/apache/lucene/spatial/util/BaseGeoPointTestCase.java
@@@ -1,0 -1,760 +1,760 @@@
+ /*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ package org.apache.lucene.spatial.util;
+
+ import java.io.IOException;
+ import java.text.DecimalFormat;
+ import java.text.DecimalFormatSymbols;
+ import java.util.ArrayList;
+ import java.util.Arrays;
+ import java.util.HashSet;
+ import java.util.List;
+ import java.util.Locale;
+ import java.util.Set;
+ import java.util.concurrent.CountDownLatch;
+ import java.util.concurrent.atomic.AtomicBoolean;
+
+ import org.apache.lucene.document.Document;
+ import org.apache.lucene.document.Field;
+ import org.apache.lucene.document.NumericDocValuesField;
+ import org.apache.lucene.index.DirectoryReader;
+ import org.apache.lucene.index.IndexReader;
+ import org.apache.lucene.index.IndexWriter;
+ import org.apache.lucene.index.IndexWriterConfig;
+ import org.apache.lucene.index.LeafReaderContext;
+ import org.apache.lucene.index.MultiDocValues;
+ import org.apache.lucene.index.NumericDocValues;
+ import org.apache.lucene.index.RandomIndexWriter;
+ import org.apache.lucene.index.Term;
+ import org.apache.lucene.search.IndexSearcher;
+ import org.apache.lucene.search.Query;
+ import org.apache.lucene.search.SimpleCollector;
+ import org.apache.lucene.store.Directory;
+ import org.apache.lucene.store.MockDirectoryWrapper;
+ import org.apache.lucene.util.FixedBitSet;
+ import org.apache.lucene.util.IOUtils;
+ import org.apache.lucene.util.LuceneTestCase;
+ import org.apache.lucene.util.SloppyMath;
+ import org.apache.lucene.util.TestUtil;
+ import org.junit.BeforeClass;
+
+ // TODO: cutover TestGeoUtils too?
+
+ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
+
+ protected static final String FIELD_NAME = "point";
+
+ private static final double LON_SCALE = (0x1L<< GeoEncodingUtils.BITS)/360.0D;
+ private static final double LAT_SCALE = (0x1L<< GeoEncodingUtils.BITS)/180.0D;
+
+ private static double originLat;
+ private static double originLon;
+ private static double lonRange;
+ private static double latRange;
+
+ @BeforeClass
+ public static void beforeClassBase() throws Exception {
+ // Between 1.0 and 3.0:
+ lonRange = 2 * (random().nextDouble() + 0.5);
+ latRange = 2 * (random().nextDouble() + 0.5);
+
+ originLon = GeoUtils.normalizeLon(GeoUtils.MIN_LON_INCL + lonRange + (GeoUtils.MAX_LON_INCL - GeoUtils.MIN_LON_INCL - 2 * lonRange) * random().nextDouble());
+ originLat = GeoUtils.normalizeLat(GeoUtils.MIN_LAT_INCL + latRange + (GeoUtils.MAX_LAT_INCL - GeoUtils.MIN_LAT_INCL - 2 * latRange) * random().nextDouble());
+ }
+
+ /** Return true when testing on a non-small region may be too slow (GeoPoint*Query) */
+ protected boolean forceSmall() {
+ return false;
+ }
+
+ // A particularly tricky adversary for BKD tree:
+ public void testSamePointManyTimes() throws Exception {
+
+ // For GeoPointQuery, only run this test nightly:
+ assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
+ int numPoints = atLeast(1000);
+ boolean small = random().nextBoolean();
+
+ // Every doc has 2 points:
+ double theLat = randomLat(small);
+ double theLon = randomLon(small);
+
+ double[] lats = new double[numPoints];
+ Arrays.fill(lats, theLat);
+
+ double[] lons = new double[numPoints];
+ Arrays.fill(lons, theLon);
+
+ verify(small, lats, lons);
+ }
+
+ public void testAllLatEqual() throws Exception {
+
+ // For GeoPointQuery, only run this test nightly:
+ assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
+ int numPoints = atLeast(10000);
+ boolean small = forceSmall() || random().nextBoolean();
+ double lat = randomLat(small);
+ double[] lats = new double[numPoints];
+ double[] lons = new double[numPoints];
+
+ boolean haveRealDoc = false;
+
+ for(int docID=0;docID<numPoints;docID++) {
+ int x = random().nextInt(20);
+ if (x == 17) {
+ // Some docs don't have a point:
+ lats[docID] = Double.NaN;
+ if (VERBOSE) {
+ System.out.println(" doc=" + docID + " is missing");
+ }
+ continue;
+ }
+
+ if (docID > 0 && x == 14 && haveRealDoc) {
+ int oldDocID;
+ while (true) {
+ oldDocID = random().nextInt(docID);
+ if (Double.isNaN(lats[oldDocID]) == false) {
+ break;
+ }
+ }
+
+ // Fully identical point:
+ lons[docID] = lons[oldDocID];
+ if (VERBOSE) {
+ System.out.println(" doc=" + docID + " lat=" + lat + " lon=" + lons[docID] + " (same lat/lon as doc=" + oldDocID + ")");
+ }
+ } else {
+ lons[docID] = randomLon(small);
+ haveRealDoc = true;
+ if (VERBOSE) {
+ System.out.println(" doc=" + docID + " lat=" + lat + " lon=" + lons[docID]);
+ }
+ }
+ lats[docID] = lat;
+ }
+
+ verify(small, lats, lons);
+ }
+
+ public void testAllLonEqual() throws Exception {
+
+ // For GeoPointQuery, only run this test nightly:
+ assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
+ int numPoints = atLeast(10000);
+ boolean small = forceSmall() || random().nextBoolean();
+ double theLon = randomLon(small);
+ double[] lats = new double[numPoints];
+ double[] lons = new double[numPoints];
+
+ boolean haveRealDoc = false;
+
+ //System.out.println("theLon=" + theLon);
+
+ for(int docID=0;docID<numPoints;docID++) {
+ int x = random().nextInt(20);
+ if (x == 17) {
+ // Some docs don't have a point:
+ lats[docID] = Double.NaN;
+ if (VERBOSE) {
+ System.out.println(" doc=" + docID + " is missing");
+ }
+ continue;
+ }
+
+ if (docID > 0 && x == 14 && haveRealDoc) {
+ int oldDocID;
+ while (true) {
+ oldDocID = random().nextInt(docID);
+ if (Double.isNaN(lats[oldDocID]) == false) {
+ break;
+ }
+ }
+
+ // Fully identical point:
+ lats[docID] = lats[oldDocID];
+ if (VERBOSE) {
+ System.out.println(" doc=" + docID + " lat=" + lats[docID] + " lon=" + theLon + " (same lat/lon as doc=" + oldDocID + ")");
+ }
+ } else {
+ lats[docID] = randomLat(small);
+ haveRealDoc = true;
+ if (VERBOSE) {
+ System.out.println(" doc=" + docID + " lat=" + lats[docID] + " lon=" + theLon);
+ }
+ }
+ lons[docID] = theLon;
+ }
+
+ verify(small, lats, lons);
+ }
+
+ public void testMultiValued() throws Exception {
+
+ // For GeoPointQuery, only run this test nightly:
+ assumeTrue("GeoPoint*Query is too slow otherwise", TEST_NIGHTLY || forceSmall() == false);
+
+ int numPoints = atLeast(10000);
+ // Every doc has 2 points:
+ double[] lats = new double[2*numPoints];
+ double[] lons = new double[2*numPoints];
+ Directory dir = newDirectory();
+ IndexWriterConfig iwc = newIndexWriterConfig();
+ initIndexWriterConfig(FIELD_NAME, iwc);
+
+ // We rely on docID order:
+ iwc.setMergePolicy(newLogMergePolicy());
+ RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+
+ boolean small = random().nextBoolean();
+
+ for (int id=0;id<numPoints;id++) {
+ Document doc = new Document();
+ lats[2*id] = randomLat(small);
+ lons[2*id] = randomLon(small);
+ doc.add(newStringField("id", ""+id, Field.Store.YES));
+ addPointToDoc(FIELD_NAME, doc, lats[2*id], lons[2*id]);
+ lats[2*id+1] = randomLat(small);
+ lons[2*id+1] = randomLon(small);
+ addPointToDoc(FIELD_NAME, doc, lats[2*id+1], lons[2*id+1]);
+
+ if (VERBOSE) {
+ System.out.println("id=" + id);
+ System.out.println(" lat=" + lats[2*id] + " lon=" + lons[2*id]);
+ System.out.println(" lat=" + lats[2*id+1] + " lon=" + lons[2*id+1]);
+ }
+ w.addDocument(doc);
+ }
+
+ // TODO: share w/ verify; just need parallel array of the expected ids
+ if (random().nextBoolean()) {
+ w.forceMerge(1);
+ }
+ IndexReader r = w.getReader();
+ w.close();
+
+ // We can't wrap with "exotic" readers because the BKD query must see the BKDDVFormat:
+ IndexSearcher s = newSearcher(r, false);
+
+ int iters = atLeast(75);
+ for (int iter=0;iter<iters;iter++) {
+ GeoRect rect = randomRect(small, small == false);
+
+ if (VERBOSE) {
+ System.out.println("\nTEST: iter=" + iter + " rect=" + rect);
+ }
+
+ Query query = newRectQuery(FIELD_NAME, rect);
+
+ final FixedBitSet hits = new FixedBitSet(r.maxDoc());
+ s.search(query, new SimpleCollector() {
+
+ private int docBase;
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ docBase = context.docBase;
+ }
+
+ @Override
+ public void collect(int doc) {
+ hits.set(docBase+doc);
+ }
+ });
+
+ boolean fail = false;
+
+ for(int docID=0;docID<lats.length/2;docID++) {
+ double latDoc1 = lats[2*docID];
+ double lonDoc1 = lons[2*docID];
+ double latDoc2 = lats[2*docID+1];
+ double lonDoc2 = lons[2*docID+1];
+
+ Boolean result1 = rectContainsPoint(rect, latDoc1, lonDoc1);
+ if (result1 == null) {
+ // borderline case: cannot test
+ continue;
+ }
+
+ Boolean result2 = rectContainsPoint(rect, latDoc2, lonDoc2);
+ if (result2 == null) {
+ // borderline case: cannot test
+ continue;
+ }
+
+ boolean expected = result1 == Boolean.TRUE || result2 == Boolean.TRUE;
+
+ if (hits.get(docID) != expected) {
+ String id = s.doc(docID).get("id");
+ if (expected) {
+ System.out.println(Thread.currentThread().getName() + ": id=" + id + " docID=" + docID + " should match but did not");
+ } else {
+ System.out.println(Thread.currentThread().getName() + ": id=" + id + " docID=" + docID + " should not match but did");
+ }
+ System.out.println(" rect=" + rect);
+ System.out.println(" lat=" + latDoc1 + " lon=" + lonDoc1 + "\n lat=" + latDoc2 + " lon=" + lonDoc2);
+ System.out.println(" result1=" + result1 + " result2=" + result2);
+ fail = true;
+ }
+ }
+
+ if (fail) {
+ fail("some hits were wrong");
+ }
+ }
+ r.close();
+ dir.close();
+ }
+
+ public void testRandomTiny() throws Exception {
+ // Make sure single-leaf-node case is OK:
+ doTestRandom(10);
+ }
+
+ public void testRandomMedium() throws Exception {
+ doTestRandom(10000);
+ }
+
+ @Nightly
+ public void testRandomBig() throws Exception {
+ assumeFalse("Direct codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Direct"));
+ assumeFalse("Memory codec can OOME on this test", TestUtil.getDocValuesFormat(FIELD_NAME).equals("Memory"));
+ doTestRandom(200000);
+ }
+
+ private void doTestRandom(int count) throws Exception {
+
+ int numPoints = atLeast(count);
+
+ if (VERBOSE) {
+ System.out.println("TEST: numPoints=" + numPoints);
+ }
+
+ double[] lats = new double[numPoints];
+ double[] lons = new double[numPoints];
+
+ boolean small = random().nextBoolean();
+
+ boolean haveRealDoc = false;
+
+ for (int id=0;id<numPoints;id++) {
+ int x = random().nextInt(20);
+ if (x == 17) {
+ // Some docs don't have a point:
+ lats[id] = Double.NaN;
+ if (VERBOSE) {
+ System.out.println(" id=" + id + " is missing");
+ }
+ continue;
+ }
+
+ if (id > 0 && x < 3 && haveRealDoc) {
+ int oldID;
+ while (true) {
+ oldID = random().nextInt(id);
+ if (Double.isNaN(lats[oldID]) == false) {
+ break;
+ }
+ }
+
+ if (x == 0) {
+ // Identical lat to old point
+ lats[id] = lats[oldID];
+ lons[id] = randomLon(small);
+ if (VERBOSE) {
+ System.out.println(" id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lat as doc=" + oldID + ")");
+ }
+ } else if (x == 1) {
+ // Identical lon to old point
+ lats[id] = randomLat(small);
+ lons[id] = lons[oldID];
+ if (VERBOSE) {
+ System.out.println(" id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lon as doc=" + oldID + ")");
+ }
+ } else {
+ assert x == 2;
+ // Fully identical point:
+ lats[id] = lats[oldID];
+ lons[id] = lons[oldID];
+ if (VERBOSE) {
+ System.out.println(" id=" + id + " lat=" + lats[id] + " lon=" + lons[id] + " (same lat/lon as doc=" + oldID + ")");
+ }
+ }
+ } else {
+ lats[id] = randomLat(small);
+ lons[id] = randomLon(small);
+ haveRealDoc = true;
+ if (VERBOSE) {
+ System.out.println(" id=" + id + " lat=" + lats[id] + " lon=" + lons[id]);
+ }
+ }
+ }
+
+ verify(small, lats, lons);
+ }
+
+ public double randomLat(boolean small) {
+ double result;
+ if (small) {
+ result = GeoUtils.normalizeLat(originLat + latRange * (random().nextDouble() - 0.5));
+ } else {
+ result = -90 + 180.0 * random().nextDouble();
+ }
+ return result;
+ }
+
+ public double randomLon(boolean small) {
+ double result;
+ if (small) {
+ result = GeoUtils.normalizeLon(originLon + lonRange * (random().nextDouble() - 0.5));
+ } else {
+ result = -180 + 360.0 * random().nextDouble();
+ }
+ return result;
+ }
+
+ protected GeoRect randomRect(boolean small, boolean canCrossDateLine) {
+ double lat0 = randomLat(small);
+ double lat1 = randomLat(small);
+ double lon0 = randomLon(small);
+ double lon1 = randomLon(small);
+
+ if (lat1 < lat0) {
+ double x = lat0;
+ lat0 = lat1;
+ lat1 = x;
+ }
+
+ if (canCrossDateLine == false && lon1 < lon0) {
+ double x = lon0;
+ lon0 = lon1;
+ lon1 = x;
+ }
+
+ return new GeoRect(lon0, lon1, lat0, lat1);
+ }
+
+ protected void initIndexWriterConfig(String field, IndexWriterConfig iwc) {
+ }
+
+ protected abstract void addPointToDoc(String field, Document doc, double lat, double lon);
+
+ protected abstract Query newRectQuery(String field, GeoRect bbox);
+
+ protected abstract Query newDistanceQuery(String field, double centerLat, double centerLon, double radiusMeters);
+
+ protected abstract Query newDistanceRangeQuery(String field, double centerLat, double centerLon, double minRadiusMeters, double radiusMeters);
+
+ protected abstract Query newPolygonQuery(String field, double[] lats, double[] lons);
+
+ /** Returns null if it's borderline case */
+ protected abstract Boolean rectContainsPoint(GeoRect rect, double pointLat, double pointLon);
+
+ /** Returns null if it's borderline case */
+ protected abstract Boolean polyRectContainsPoint(GeoRect rect, double pointLat, double pointLon);
+
+ /** Returns null if it's borderline case */
+ protected abstract Boolean circleContainsPoint(double centerLat, double centerLon, double radiusMeters, double pointLat, double pointLon);
+
+ protected abstract Boolean distanceRangeContainsPoint(double centerLat, double centerLon, double minRadiusMeters, double radiusMeters, double pointLat, double pointLon);
+
+ private static abstract class VerifyHits {
+
+ public void test(AtomicBoolean failed, boolean small, IndexSearcher s, NumericDocValues docIDToID, Set<Integer> deleted, Query query, double[] lats, double[] lons) throws Exception {
+ int maxDoc = s.getIndexReader().maxDoc();
+ final FixedBitSet hits = new FixedBitSet(maxDoc);
+ s.search(query, new SimpleCollector() {
+
+ private int docBase;
+
+ @Override
+ public boolean needsScores() {
+ return false;
+ }
+
+ @Override
+ protected void doSetNextReader(LeafReaderContext context) throws IOException {
+ docBase = context.docBase;
+ }
+
+ @Override
+ public void collect(int doc) {
+ hits.set(docBase+doc);
+ }
+ });
+
+ boolean fail = false;
+
+ for(int docID=0;docID<maxDoc;docID++) {
+ int id = (int) docIDToID.get(docID);
+ Boolean expected;
+ if (deleted.contains(id)) {
+ expected = false;
+ } else if (Double.isNaN(lats[id])) {
+ expected = false;
+ } else {
+ expected = shouldMatch(lats[id], lons[id]);
+ }
+
+ // null means it's a borderline case which is allowed to be wrong:
+ if (expected != null && hits.get(docID) != expected) {
+ if (expected) {
+ System.out.println(Thread.currentThread().getName() + ": id=" + id + " should match but did not");
+ } else {
+ System.out.println(Thread.currentThread().getName() + ": id=" + id + " should not match but did");
+ }
+ System.out.println(" small=" + small + " query=" + query +
+ " docID=" + docID + "\n lat=" + lats[id] + " lon=" + lons[id] +
+ "\n deleted?=" + deleted.contains(id));
+ if (Double.isNaN(lats[id]) == false) {
+ describe(docID, lats[id], lons[id]);
+ }
+ fail = true;
+ }
+ }
+
+ if (fail) {
+ failed.set(true);
+ fail("some hits were wrong");
+ }
+ }
+
+ /** Return true if we definitely should match, false if we definitely
+ * should not match, and null if it's a borderline case which might
+ * go either way. */
+ protected abstract Boolean shouldMatch(double lat, double lon);
+
+ protected abstract void describe(int docID, double lat, double lon);
+ }
+
+ protected void verify(boolean small, double[] lats, double[] lons) throws Exception {
+ IndexWriterConfig iwc = newIndexWriterConfig();
+ // Else we can get O(N^2) merging:
+ int mbd = iwc.getMaxBufferedDocs();
+ if (mbd != -1 && mbd < lats.length/100) {
+ iwc.setMaxBufferedDocs(lats.length/100);
+ }
+ Directory dir;
+ if (lats.length > 100000) {
+ dir = newFSDirectory(createTempDir(getClass().getSimpleName()));
+ } else {
+ dir = newDirectory();
+ }
+
+ Set<Integer> deleted = new HashSet<>();
+ // RandomIndexWriter is too slow here:
+ IndexWriter w = new IndexWriter(dir, iwc);
+ for(int id=0;id<lats.length;id++) {
+ Document doc = new Document();
+ doc.add(newStringField("id", ""+id, Field.Store.NO));
+ doc.add(new NumericDocValuesField("id", id));
+ if (Double.isNaN(lats[id]) == false) {
+ addPointToDoc(FIELD_NAME, doc, lats[id], lons[id]);
+ }
+ w.addDocument(doc);
+ if (id > 0 && random().nextInt(100) == 42) {
+ int idToDelete = random().nextInt(id);
+ w.deleteDocuments(new Term("id", ""+idToDelete));
+ deleted.add(idToDelete);
+ if (VERBOSE) {
+ System.out.println(" delete id=" + idToDelete);
+ }
+ }
+ }
+
+ if (random().nextBoolean()) {
+ w.forceMerge(1);
+ }
+ final IndexReader r = DirectoryReader.open(w);
+ w.close();
+
+ // We can't wrap with "exotic" readers because the BKD query must see the BKDDVFormat:
+ IndexSearcher s = newSearcher(r, false);
+
+ // Make sure queries are thread safe:
+ int numThreads = TestUtil.nextInt(random(), 2, 5);
+
+ List<Thread> threads = new ArrayList<>();
+ final int iters = atLeast(75);
+
+ final CountDownLatch startingGun = new CountDownLatch(1);
+ final AtomicBoolean failed = new AtomicBoolean();
+
+ for(int i=0;i<numThreads;i++) {
+ Thread thread = new Thread() {
+ @Override
+ public void run() {
+ try {
+ _run();
+ } catch (Exception e) {
+ failed.set(true);
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void _run() throws Exception {
+ startingGun.await();
+
+ NumericDocValues docIDToID = MultiDocValues.getNumericValues(r, "id");
+
+ for (int iter=0;iter<iters && failed.get() == false;iter++) {
+
+ if (VERBOSE) {
- System.out.println("\nTEST: iter=" + iter + " s=" + s);
++ System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " s=" + s);
+ }
+ Query query;
+ VerifyHits verifyHits;
+
+ if (random().nextBoolean()) {
+ // Rect: don't allow dateline crossing when testing small:
+ final GeoRect rect = randomRect(small, small == false);
+
+ query = newRectQuery(FIELD_NAME, rect);
+
+ verifyHits = new VerifyHits() {
+ @Override
+ protected Boolean shouldMatch(double pointLat, double pointLon) {
+ return rectContainsPoint(rect, pointLat, pointLon);
+ }
+ @Override
+ protected void describe(int docID, double lat, double lon) {
+ }
+ };
+
+ } else if (random().nextBoolean()) {
+ // Distance
+ final boolean rangeQuery = random().nextBoolean();
+ final double centerLat = randomLat(small);
+ final double centerLon = randomLon(small);
+
+ double radiusMeters;
+ double minRadiusMeters;
+
+ if (small) {
+ // Approx 3 degrees lon at the equator:
+ radiusMeters = random().nextDouble() * 333000 + 1.0;
+ } else {
+ // So the query can cover at most 50% of the earth's surface:
+ radiusMeters = random().nextDouble() * GeoProjectionUtils.SEMIMAJOR_AXIS * Math.PI / 2.0 + 1.0;
+ }
+
+ // generate a random minimum radius between 1% and 95% the max radius
+ minRadiusMeters = (0.01 + 0.94 * random().nextDouble()) * radiusMeters;
+
+ if (VERBOSE) {
+ final DecimalFormat df = new DecimalFormat("#,###.00", DecimalFormatSymbols.getInstance(Locale.ENGLISH));
+ System.out.println(" radiusMeters = " + df.format(radiusMeters)
+ + ((rangeQuery == true) ? " minRadiusMeters = " + df.format(minRadiusMeters) : ""));
+ }
+
+ try {
+ if (rangeQuery == true) {
+ query = newDistanceRangeQuery(FIELD_NAME, centerLat, centerLon, minRadiusMeters, radiusMeters);
+ } else {
+ query = newDistanceQuery(FIELD_NAME, centerLat, centerLon, radiusMeters);
+ }
+ } catch (IllegalArgumentException e) {
+ if (e.getMessage().contains("exceeds maxRadius")) {
+ continue;
+ }
+ throw e;
+ }
+
+ verifyHits = new VerifyHits() {
+ @Override
+ protected Boolean shouldMatch(double pointLat, double pointLon) {
+ if (rangeQuery == false) {
+ return circleContainsPoint(centerLat, centerLon, radiusMeters, pointLat, pointLon);
+ } else {
+ return distanceRangeContainsPoint(centerLat, centerLon, minRadiusMeters, radiusMeters, pointLat, pointLon);
+ }
+ }
+
+ @Override
+ protected void describe(int docID, double pointLat, double pointLon) {
+ double distanceKM = SloppyMath.haversin(centerLat, centerLon, pointLat, pointLon);
+ System.out.println(" docID=" + docID + " centerLon=" + centerLon + " centerLat=" + centerLat
+ + " pointLon=" + pointLon + " pointLat=" + pointLat + " distanceMeters=" + (distanceKM * 1000)
+ + " vs" + ((rangeQuery == true) ? " minRadiusMeters=" + minRadiusMeters : "") + " radiusMeters=" + radiusMeters);
+ }
+ };
+
+ // TODO: get poly query working with dateline crossing too (how?)!
+ } else {
+
+ // TODO: poly query can't handle dateline crossing yet:
+ final GeoRect bbox = randomRect(small, false);
+
+ // Polygon
+ double[] lats = new double[5];
+ double[] lons = new double[5];
+ lats[0] = bbox.minLat;
+ lons[0] = bbox.minLon;
+ lats[1] = bbox.maxLat;
+ lons[1] = bbox.minLon;
+ lats[2] = bbox.maxLat;
+ lons[2] = bbox.maxLon;
+ lats[3] = bbox.minLat;
+ lons[3] = bbox.maxLon;
+ lats[4] = bbox.minLat;
+ lons[4] = bbox.minLon;
+ query = newPolygonQuery(FIELD_NAME, lats, lons);
+
+ verifyHits = new VerifyHits() {
+ @Override
+ protected Boolean shouldMatch(double pointLat, double pointLon) {
+ return polyRectContainsPoint(bbox, pointLat, pointLon);
+ }
+
+ @Override
+ protected void describe(int docID, double lat, double lon) {
+ }
+ };
+ }
+
+ if (query != null) {
+
+ if (VERBOSE) {
+ System.out.println(" query=" + query);
+ }
+
+ verifyHits.test(failed, small, s, docIDToID, deleted, query, lats, lons);
+ }
+ }
+ }
+ };
+ thread.setName("T" + i);
+ thread.start();
+ threads.add(thread);
+ }
+ startingGun.countDown();
+ for(Thread thread : threads) {
+ thread.join();
+ }
+ IOUtils.close(r, dir);
+ assertFalse(failed.get());
+ }
+ }
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/index/RandomIndexWriter.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
----------------------------------------------------------------------
diff --cc lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index b19045b,51dbffe..a5fc397
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@@ -45,7 -43,8 +45,9 @@@ import org.apache.lucene.index.IndexFil
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.NoDeletionPolicy;
+import org.apache.lucene.index.SegmentInfos;
+ import org.apache.lucene.mockfile.FilterFileSystem;
+ import org.apache.lucene.mockfile.VirusCheckingFS;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@@@ -239,24 -219,8 +222,16 @@@ public class MockDirectoryWrapper exten
throw new IOException("cannot rename after crash");
}
- if (openFiles.containsKey(source)) {
- if (assertNoDeleteOpenFile) {
- throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
- } else if (noDeleteOpenFile) {
- throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
- }
- if (assertNoDeleteOpenFile && openFiles.containsKey(source)) {
- throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
++ if (openFiles.containsKey(source) && assertNoDeleteOpenFile) {
++ throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
+ }
+
- if (openFiles.containsKey(dest)) {
- if (assertNoDeleteOpenFile) {
- throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
- } else if (noDeleteOpenFile) {
- throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
- }
++ if (openFiles.containsKey(dest) && assertNoDeleteOpenFile) {
++ throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
+ }
+
+ if (createdFiles.contains(dest)) {
+ throw new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" already exists: cannot rename");
}
boolean success = false;
@@@ -271,8 -235,6 +246,7 @@@
unSyncedFiles.add(dest);
}
openFilesDeleted.remove(source);
- triedToDelete.remove(dest);
+ createdFiles.add(dest);
}
}
}
@@@ -294,213 -256,94 +268,190 @@@
}
}
- /** Simulates a crash of OS or machine by overwriting
- * unsynced files. */
- public synchronized void crash() throws IOException {
- openFiles = new HashMap<>();
- openFilesForWrite = new HashSet<>();
- openFilesDeleted = new HashSet<>();
- Iterator<String> it = unSyncedFiles.iterator();
- unSyncedFiles = new HashSet<>();
- // first force-close all files, so we can corrupt on windows etc.
- // clone the file map, as these guys want to remove themselves on close.
- Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
- for (Closeable f : m.keySet()) {
- try {
- f.close();
- } catch (Exception ignored) {}
+ public synchronized void corruptUnknownFiles() throws IOException {
+
+ System.out.println("MDW: corrupt unknown files");
+ Set<String> knownFiles = new HashSet<>();
+ for(String fileName : listAll()) {
+ if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
+ System.out.println("MDW: read " + fileName + " to gather files it references");
+ knownFiles.addAll(SegmentInfos.readCommit(this, fileName).files(true));
+ }
}
- // Maybe disable virus checker so it doesn't interfere with our efforts to corrupt files below:
- boolean virusCheckerWasEnabled = TestUtil.disableVirusChecker(in);
+ Set<String> toCorrupt = new HashSet<>();
+ Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
+ for(String fileName : listAll()) {
+ m.reset(fileName);
+ if (knownFiles.contains(fileName) == false &&
+ fileName.endsWith("write.lock") == false &&
+ (m.matches() || fileName.startsWith(IndexFileNames.PENDING_SEGMENTS))) {
+ toCorrupt.add(fileName);
+ }
+ }
- while(it.hasNext()) {
- String name = it.next();
- int damage = randomState.nextInt(5);
+ corruptFiles(toCorrupt);
+ }
+
- public synchronized void corruptFiles(Collection<String> files) {
++ public synchronized void corruptFiles(Collection<String> files) throws IOException {
+ // Must make a copy because we change the incoming unsyncedFiles
+ // when we create temp files, delete, etc., below:
+ for(String name : new ArrayList<>(files)) {
+ int damage = randomState.nextInt(6);
String action = null;
- if (damage == 0) {
+ switch(damage) {
+
+ case 0:
action = "deleted";
- try {
- deleteFile(name, true);
- } catch (IOException ioe) {
- // ignore
- }
+ deleteFile(name);
- } else if (damage == 1) {
+ break;
+
+ case 1:
action = "zeroed";
// Zero out file entirely
- long length = fileLength(name);
+ long length;
+ try {
+ length = fileLength(name);
+ } catch (IOException ioe) {
+ // Ignore
+ continue;
+ }
byte[] zeroes = new byte[256];
long upto = 0;
- IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
- while(upto < length) {
- final int limit = (int) Math.min(length-upto, zeroes.length);
- out.writeBytes(zeroes, 0, limit);
- upto += limit;
- }
- out.close();
- } else if (damage == 2) {
- action = "partially truncated";
- // Partially Truncate the file:
-
- // First, make temp file and copy only half this
- // file over:
- String tempFileName;
- while (true) {
- tempFileName = ""+randomState.nextInt();
- if (!LuceneTestCase.slowFileExists(in, tempFileName)) {
- break;
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
+ while(upto < length) {
+ final int limit = (int) Math.min(length-upto, zeroes.length);
+ out.writeBytes(zeroes, 0, limit);
+ upto += limit;
}
+ } catch (IOException ioe) {
+ // ignore
}
- final IndexOutput tempOut = in.createOutput(tempFileName, LuceneTestCase.newIOContext(randomState));
- IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState));
- tempOut.copyBytes(ii, ii.length()/2);
- tempOut.close();
- ii.close();
+ break;
- // Delete original and copy bytes back:
- deleteFile(name);
-
- try(IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
- ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState));
- out.copyBytes(ii, ii.length());
- ii.close();
+ case 2:
+ {
+ action = "partially truncated";
+ // Partially Truncate the file:
+
+ // First, make temp file and copy only half this
+ // file over:
+ String tempFileName = null;
+ try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
+ tempFileName = tempOut.getName();
+ tempOut.copyBytes(ii, ii.length()/2);
+ } catch (IOException ioe) {
+ // ignore
+ }
+
- try {
- // Delete original and copy bytes back:
- deleteFile(name, true);
- } catch (IOException ioe) {
- // ignore
- }
++ // Delete original and copy bytes back:
++ deleteFile(name);
+
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
+ out.copyBytes(ii, ii.length());
+ } catch (IOException ioe) {
+ // ignore
+ }
- try {
- deleteFile(tempFileName, true);
- } catch (IOException ioe) {
- // ignore
- }
++ deleteFile(tempFileName);
}
- deleteFile(tempFileName);
- } else if (damage == 3) {
+ break;
+
+ case 3:
// The file survived intact:
action = "didn't change";
- } else {
+ break;
+
+ case 4:
+ // Corrupt one bit randomly in the file:
+
+ {
+
+ String tempFileName = null;
+ try (IndexOutput tempOut = in.createTempOutput("name", "mdw_corrupt", LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(name, LuceneTestCase.newIOContext(randomState))) {
+ tempFileName = tempOut.getName();
+ if (ii.length() > 0) {
+ // Copy first part unchanged:
+ long byteToCorrupt = (long) (randomState.nextDouble() * ii.length());
+ if (byteToCorrupt > 0) {
+ tempOut.copyBytes(ii, byteToCorrupt);
+ }
+
+ // Randomly flip one bit from this byte:
+ byte b = ii.readByte();
+ int bitToFlip = randomState.nextInt(8);
+ b = (byte) (b ^ (1 << bitToFlip));
+ tempOut.writeByte(b);
+
+ action = "flip bit " + bitToFlip + " of byte " + byteToCorrupt + " out of " + ii.length() + " bytes";
+
+ // Copy last part unchanged:
+ long bytesLeft = ii.length() - byteToCorrupt - 1;
+ if (bytesLeft > 0) {
+ tempOut.copyBytes(ii, bytesLeft);
+ }
+ } else {
+ action = "didn't change";
+ }
+ } catch (IOException ioe) {
+ // ignore
+ }
+
- try {
- // Delete original and copy bytes back:
- deleteFile(name, true);
- } catch (IOException ioe) {
- // ignore
- }
++ // Delete original and copy bytes back:
++ deleteFile(name);
+
+ try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState));
+ IndexInput ii = in.openInput(tempFileName, LuceneTestCase.newIOContext(randomState))) {
+ out.copyBytes(ii, ii.length());
+ } catch (IOException ioe) {
+ // ignore
+ }
- try {
- deleteFile(tempFileName, true);
- } catch (IOException ioe) {
- // ignore
- }
++
++ deleteFile(tempFileName);
+ }
+ break;
+
+ case 5:
action = "fully truncated";
// Totally truncate the file to zero bytes
- try {
- deleteFile(name, true);
- } catch (IOException ioe) {
- // ignore
- }
+ deleteFile(name);
+
try (IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState))) {
+ } catch (IOException ioe) {
+ // ignore
}
+ break;
+
+ default:
+ throw new AssertionError();
}
- // Re-enable
- if (virusCheckerWasEnabled) {
- TestUtil.enableVirusChecker(in);
- }
- if (LuceneTestCase.VERBOSE) {
+
+ if (true || LuceneTestCase.VERBOSE) {
System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name);
}
}
+ }
+
+ /** Simulates a crash of OS or machine by overwriting
+ * unsynced files. */
- public synchronized void crash() {
++ public synchronized void crash() throws IOException {
crashed = true;
+ openFiles = new HashMap<>();
+ openFilesForWrite = new HashSet<>();
+ openFilesDeleted = new HashSet<>();
+ // first force-close all files, so we can corrupt on windows etc.
+ // clone the file map, as these guys want to remove themselves on close.
+ Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
+ for (Closeable f : m.keySet()) {
+ try {
+ f.close();
+ } catch (Exception ignored) {}
+ }
+ corruptFiles(unSyncedFiles);
+ unSyncedFiles = new HashSet<>();
}
public synchronized void clearCrash() {
@@@ -942,13 -745,12 +853,14 @@@
String[] startFiles = allFiles.toArray(new String[0]);
IndexWriterConfig iwc = new IndexWriterConfig(null);
iwc.setIndexDeletionPolicy(NoDeletionPolicy.INSTANCE);
+
+ // We must do this before opening writer otherwise writer will be angry if there are pending deletions:
+ TestUtil.disableVirusChecker(in);
+
new IndexWriter(in, iwc).rollback();
- String[] endFiles = in.listAll();
+
+ Set<String> files = new HashSet<>(Arrays.asList(listAll()));
- // Disregard what happens with the pendingDeletions files:
- files.removeAll(pendingDeletions);
+ String[] endFiles = files.toArray(new String[0]);
Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
@@@ -1027,11 -784,7 +894,7 @@@
extras += "\n\nThese files were added (waaaaaaaaaat!): " + added;
}
- if (pendingDeletions.size() != 0) {
- extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
- }
-
- throw new RuntimeException("unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
+ throw new RuntimeException(this + ": unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
}
DirectoryReader ir1 = DirectoryReader.open(this);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/test-framework/src/java/org/apache/lucene/util/TestUtil.java
----------------------------------------------------------------------
[15/15] lucene-solr git commit: Merge branch 'master' into
nrt_replicas
Posted by mi...@apache.org.
Merge branch 'master' into nrt_replicas
Conflicts:
lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/bd6804bc
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/bd6804bc
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/bd6804bc
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: bd6804bc6312c1a1d18294b5a670cc5eba7c9f48
Parents: b1b2c79 8e78469
Author: Mike McCandless <mi...@apache.org>
Authored: Sat Feb 6 11:14:15 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Feb 6 11:14:15 2016 -0500
----------------------------------------------------------------------
.../dot.settings/org.eclipse.jdt.ui.prefs | 3 +-
dev-tools/git/HELP.txt | 14 +
.../idea/.idea/copyright/profiles_settings.xml | 2 +-
dev-tools/idea/lucene/sandbox/sandbox.iml | 1 +
lucene/CHANGES.txt | 39 +
.../lucene/analysis/ar/ArabicAnalyzer.java | 4 +-
.../analysis/ar/ArabicNormalizationFilter.java | 4 +-
.../ar/ArabicNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/ar/ArabicNormalizer.java | 4 +-
.../lucene/analysis/ar/ArabicStemFilter.java | 4 +-
.../analysis/ar/ArabicStemFilterFactory.java | 4 +-
.../lucene/analysis/ar/ArabicStemmer.java | 6 +-
.../lucene/analysis/bg/BulgarianAnalyzer.java | 4 +-
.../lucene/analysis/bg/BulgarianStemFilter.java | 4 +-
.../analysis/bg/BulgarianStemFilterFactory.java | 4 +-
.../lucene/analysis/bg/BulgarianStemmer.java | 4 +-
.../lucene/analysis/br/BrazilianAnalyzer.java | 4 +-
.../lucene/analysis/br/BrazilianStemFilter.java | 4 +-
.../analysis/br/BrazilianStemFilterFactory.java | 4 +-
.../lucene/analysis/br/BrazilianStemmer.java | 8 +-
.../lucene/analysis/ca/CatalanAnalyzer.java | 4 +-
.../analysis/charfilter/BaseCharFilter.java | 1 -
.../charfilter/HTMLStripCharFilter.java | 3 +-
.../charfilter/HTMLStripCharFilter.jflex | 3 +-
.../charfilter/HTMLStripCharFilterFactory.java | 4 +-
.../analysis/charfilter/MappingCharFilter.java | 1 -
.../charfilter/MappingCharFilterFactory.java | 4 +-
.../analysis/charfilter/NormalizeCharMap.java | 1 -
.../apache/lucene/analysis/cjk/CJKAnalyzer.java | 4 +-
.../lucene/analysis/cjk/CJKBigramFilter.java | 4 +-
.../analysis/cjk/CJKBigramFilterFactory.java | 4 +-
.../lucene/analysis/cjk/CJKWidthFilter.java | 4 +-
.../analysis/cjk/CJKWidthFilterFactory.java | 4 +-
.../lucene/analysis/ckb/SoraniAnalyzer.java | 4 +-
.../analysis/ckb/SoraniNormalizationFilter.java | 4 +-
.../ckb/SoraniNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/ckb/SoraniNormalizer.java | 4 +-
.../lucene/analysis/ckb/SoraniStemFilter.java | 4 +-
.../analysis/ckb/SoraniStemFilterFactory.java | 4 +-
.../lucene/analysis/ckb/SoraniStemmer.java | 4 +-
.../analysis/commongrams/CommonGramsFilter.java | 1 -
.../commongrams/CommonGramsFilterFactory.java | 4 +-
.../CommonGramsQueryFilterFactory.java | 4 +-
.../compound/CompoundWordTokenFilterBase.java | 4 +-
.../DictionaryCompoundWordTokenFilter.java | 4 +-
...ictionaryCompoundWordTokenFilterFactory.java | 4 +-
.../HyphenationCompoundWordTokenFilter.java | 4 +-
...phenationCompoundWordTokenFilterFactory.java | 4 +-
.../compound/hyphenation/ByteVector.java | 7 +-
.../compound/hyphenation/CharVector.java | 7 +-
.../analysis/compound/hyphenation/Hyphen.java | 7 +-
.../compound/hyphenation/Hyphenation.java | 7 +-
.../compound/hyphenation/HyphenationTree.java | 7 +-
.../compound/hyphenation/PatternConsumer.java | 7 +-
.../compound/hyphenation/PatternParser.java | 7 +-
.../compound/hyphenation/TernaryTree.java | 7 +-
.../analysis/core/DecimalDigitFilter.java | 4 +-
.../core/DecimalDigitFilterFactory.java | 4 +-
.../lucene/analysis/core/KeywordAnalyzer.java | 4 +-
.../lucene/analysis/core/KeywordTokenizer.java | 4 +-
.../analysis/core/KeywordTokenizerFactory.java | 4 +-
.../lucene/analysis/core/LetterTokenizer.java | 4 +-
.../analysis/core/LetterTokenizerFactory.java | 4 +-
.../lucene/analysis/core/LowerCaseFilter.java | 4 +-
.../analysis/core/LowerCaseFilterFactory.java | 4 +-
.../analysis/core/LowerCaseTokenizer.java | 4 +-
.../core/LowerCaseTokenizerFactory.java | 4 +-
.../lucene/analysis/core/SimpleAnalyzer.java | 4 +-
.../lucene/analysis/core/StopAnalyzer.java | 4 +-
.../apache/lucene/analysis/core/StopFilter.java | 4 +-
.../lucene/analysis/core/StopFilterFactory.java | 4 +-
.../lucene/analysis/core/TypeTokenFilter.java | 4 +-
.../analysis/core/TypeTokenFilterFactory.java | 4 +-
.../core/UnicodeWhitespaceAnalyzer.java | 4 +-
.../core/UnicodeWhitespaceTokenizer.java | 4 +-
.../lucene/analysis/core/UpperCaseFilter.java | 4 +-
.../analysis/core/UpperCaseFilterFactory.java | 4 +-
.../analysis/core/WhitespaceAnalyzer.java | 4 +-
.../analysis/core/WhitespaceTokenizer.java | 4 +-
.../core/WhitespaceTokenizerFactory.java | 4 +-
.../lucene/analysis/custom/CustomAnalyzer.java | 4 +-
.../lucene/analysis/cz/CzechAnalyzer.java | 4 +-
.../lucene/analysis/cz/CzechStemFilter.java | 20 +-
.../analysis/cz/CzechStemFilterFactory.java | 4 +-
.../apache/lucene/analysis/cz/CzechStemmer.java | 4 +-
.../lucene/analysis/da/DanishAnalyzer.java | 4 +-
.../lucene/analysis/de/GermanAnalyzer.java | 6 +-
.../analysis/de/GermanLightStemFilter.java | 4 +-
.../de/GermanLightStemFilterFactory.java | 4 +-
.../lucene/analysis/de/GermanLightStemmer.java | 4 +-
.../analysis/de/GermanMinimalStemFilter.java | 4 +-
.../de/GermanMinimalStemFilterFactory.java | 4 +-
.../analysis/de/GermanMinimalStemmer.java | 4 +-
.../analysis/de/GermanNormalizationFilter.java | 4 +-
.../de/GermanNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/de/GermanStemFilter.java | 4 +-
.../analysis/de/GermanStemFilterFactory.java | 4 +-
.../lucene/analysis/de/GermanStemmer.java | 12 +-
.../lucene/analysis/el/GreekAnalyzer.java | 14 +-
.../analysis/el/GreekLowerCaseFilter.java | 14 +-
.../el/GreekLowerCaseFilterFactory.java | 4 +-
.../lucene/analysis/el/GreekStemFilter.java | 4 +-
.../analysis/el/GreekStemFilterFactory.java | 4 +-
.../apache/lucene/analysis/el/GreekStemmer.java | 12 +-
.../lucene/analysis/en/EnglishAnalyzer.java | 4 +-
.../analysis/en/EnglishMinimalStemFilter.java | 4 +-
.../en/EnglishMinimalStemFilterFactory.java | 4 +-
.../analysis/en/EnglishMinimalStemmer.java | 4 +-
.../analysis/en/EnglishPossessiveFilter.java | 4 +-
.../en/EnglishPossessiveFilterFactory.java | 4 +-
.../apache/lucene/analysis/en/KStemFilter.java | 4 +-
.../lucene/analysis/en/KStemFilterFactory.java | 4 +-
.../org/apache/lucene/analysis/en/KStemmer.java | 1 -
.../lucene/analysis/en/PorterStemFilter.java | 4 +-
.../analysis/en/PorterStemFilterFactory.java | 4 +-
.../lucene/analysis/en/PorterStemmer.java | 4 +-
.../lucene/analysis/es/SpanishAnalyzer.java | 4 +-
.../analysis/es/SpanishLightStemFilter.java | 4 +-
.../es/SpanishLightStemFilterFactory.java | 4 +-
.../lucene/analysis/es/SpanishLightStemmer.java | 4 +-
.../lucene/analysis/eu/BasqueAnalyzer.java | 4 +-
.../lucene/analysis/fa/PersianAnalyzer.java | 4 +-
.../lucene/analysis/fa/PersianCharFilter.java | 4 +-
.../analysis/fa/PersianCharFilterFactory.java | 4 +-
.../analysis/fa/PersianNormalizationFilter.java | 4 +-
.../fa/PersianNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/fa/PersianNormalizer.java | 4 +-
.../lucene/analysis/fi/FinnishAnalyzer.java | 4 +-
.../analysis/fi/FinnishLightStemFilter.java | 4 +-
.../fi/FinnishLightStemFilterFactory.java | 4 +-
.../lucene/analysis/fi/FinnishLightStemmer.java | 4 +-
.../lucene/analysis/fr/FrenchAnalyzer.java | 4 +-
.../analysis/fr/FrenchLightStemFilter.java | 4 +-
.../fr/FrenchLightStemFilterFactory.java | 4 +-
.../lucene/analysis/fr/FrenchLightStemmer.java | 4 +-
.../analysis/fr/FrenchMinimalStemFilter.java | 4 +-
.../fr/FrenchMinimalStemFilterFactory.java | 4 +-
.../analysis/fr/FrenchMinimalStemmer.java | 4 +-
.../lucene/analysis/ga/IrishAnalyzer.java | 4 +-
.../analysis/ga/IrishLowerCaseFilter.java | 4 +-
.../ga/IrishLowerCaseFilterFactory.java | 4 +-
.../lucene/analysis/gl/GalicianAnalyzer.java | 4 +-
.../analysis/gl/GalicianMinimalStemFilter.java | 4 +-
.../gl/GalicianMinimalStemFilterFactory.java | 4 +-
.../analysis/gl/GalicianMinimalStemmer.java | 4 +-
.../lucene/analysis/gl/GalicianStemFilter.java | 4 +-
.../analysis/gl/GalicianStemFilterFactory.java | 4 +-
.../lucene/analysis/gl/GalicianStemmer.java | 4 +-
.../lucene/analysis/hi/HindiAnalyzer.java | 4 +-
.../analysis/hi/HindiNormalizationFilter.java | 4 +-
.../hi/HindiNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/hi/HindiNormalizer.java | 4 +-
.../lucene/analysis/hi/HindiStemFilter.java | 4 +-
.../analysis/hi/HindiStemFilterFactory.java | 4 +-
.../apache/lucene/analysis/hi/HindiStemmer.java | 4 +-
.../lucene/analysis/hu/HungarianAnalyzer.java | 4 +-
.../analysis/hu/HungarianLightStemFilter.java | 4 +-
.../hu/HungarianLightStemFilterFactory.java | 4 +-
.../analysis/hu/HungarianLightStemmer.java | 4 +-
.../lucene/analysis/hunspell/Dictionary.java | 4 +-
.../analysis/hunspell/HunspellStemFilter.java | 4 +-
.../hunspell/HunspellStemFilterFactory.java | 4 +-
.../analysis/hunspell/ISO8859_14Decoder.java | 4 +-
.../lucene/analysis/hunspell/Stemmer.java | 4 +-
.../lucene/analysis/hy/ArmenianAnalyzer.java | 4 +-
.../lucene/analysis/id/IndonesianAnalyzer.java | 4 +-
.../analysis/id/IndonesianStemFilter.java | 4 +-
.../id/IndonesianStemFilterFactory.java | 4 +-
.../lucene/analysis/id/IndonesianStemmer.java | 4 +-
.../analysis/in/IndicNormalizationFilter.java | 4 +-
.../in/IndicNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/in/IndicNormalizer.java | 4 +-
.../lucene/analysis/it/ItalianAnalyzer.java | 4 +-
.../analysis/it/ItalianLightStemFilter.java | 4 +-
.../it/ItalianLightStemFilterFactory.java | 4 +-
.../lucene/analysis/it/ItalianLightStemmer.java | 4 +-
.../lucene/analysis/lt/LithuanianAnalyzer.java | 4 +-
.../lucene/analysis/lv/LatvianAnalyzer.java | 4 +-
.../lucene/analysis/lv/LatvianStemFilter.java | 4 +-
.../analysis/lv/LatvianStemFilterFactory.java | 4 +-
.../lucene/analysis/lv/LatvianStemmer.java | 8 +-
.../miscellaneous/ASCIIFoldingFilter.java | 4 +-
.../ASCIIFoldingFilterFactory.java | 4 +-
.../miscellaneous/CapitalizationFilter.java | 4 +-
.../CapitalizationFilterFactory.java | 4 +-
.../miscellaneous/CodepointCountFilter.java | 4 +-
.../CodepointCountFilterFactory.java | 4 +-
.../miscellaneous/DateRecognizerFilter.java | 4 +-
.../DateRecognizerFilterFactory.java | 4 +-
.../miscellaneous/EmptyTokenStream.java | 4 +-
.../miscellaneous/FingerprintFilter.java | 4 +-
.../miscellaneous/FingerprintFilterFactory.java | 4 +-
.../miscellaneous/HyphenatedWordsFilter.java | 4 +-
.../HyphenatedWordsFilterFactory.java | 4 +-
.../analysis/miscellaneous/KeepWordFilter.java | 1 -
.../miscellaneous/KeepWordFilterFactory.java | 4 +-
.../miscellaneous/KeywordMarkerFilter.java | 4 +-
.../KeywordMarkerFilterFactory.java | 4 +-
.../miscellaneous/KeywordRepeatFilter.java | 4 +-
.../KeywordRepeatFilterFactory.java | 4 +-
.../analysis/miscellaneous/LengthFilter.java | 4 +-
.../miscellaneous/LengthFilterFactory.java | 4 +-
.../miscellaneous/LimitTokenCountAnalyzer.java | 4 +-
.../miscellaneous/LimitTokenCountFilter.java | 4 +-
.../LimitTokenCountFilterFactory.java | 4 +-
.../miscellaneous/LimitTokenOffsetFilter.java | 4 +-
.../LimitTokenOffsetFilterFactory.java | 4 +-
.../miscellaneous/LimitTokenPositionFilter.java | 2 +-
.../LimitTokenPositionFilterFactory.java | 2 +-
.../PatternKeywordMarkerFilter.java | 4 +-
.../miscellaneous/PerFieldAnalyzerWrapper.java | 4 +-
.../PrefixAndSuffixAwareTokenFilter.java | 4 +-
.../miscellaneous/PrefixAwareTokenFilter.java | 4 +-
.../RemoveDuplicatesTokenFilter.java | 1 -
.../RemoveDuplicatesTokenFilterFactory.java | 4 +-
.../ScandinavianFoldingFilter.java | 4 +-
.../ScandinavianFoldingFilterFactory.java | 4 +-
.../ScandinavianNormalizationFilter.java | 4 +-
.../ScandinavianNormalizationFilterFactory.java | 4 +-
.../miscellaneous/SetKeywordMarkerFilter.java | 2 +-
.../miscellaneous/StemmerOverrideFilter.java | 4 +-
.../StemmerOverrideFilterFactory.java | 4 +-
.../analysis/miscellaneous/TrimFilter.java | 1 -
.../miscellaneous/TrimFilterFactory.java | 4 +-
.../miscellaneous/TruncateTokenFilter.java | 4 +-
.../TruncateTokenFilterFactory.java | 4 +-
.../miscellaneous/WordDelimiterFilter.java | 3 +-
.../WordDelimiterFilterFactory.java | 4 +-
.../miscellaneous/WordDelimiterIterator.java | 4 +-
.../analysis/ngram/EdgeNGramFilterFactory.java | 4 +-
.../analysis/ngram/EdgeNGramTokenFilter.java | 4 +-
.../analysis/ngram/EdgeNGramTokenizer.java | 4 +-
.../ngram/EdgeNGramTokenizerFactory.java | 4 +-
.../analysis/ngram/NGramFilterFactory.java | 4 +-
.../lucene/analysis/ngram/NGramTokenFilter.java | 4 +-
.../lucene/analysis/ngram/NGramTokenizer.java | 4 +-
.../analysis/ngram/NGramTokenizerFactory.java | 4 +-
.../lucene/analysis/nl/DutchAnalyzer.java | 4 +-
.../lucene/analysis/no/NorwegianAnalyzer.java | 4 +-
.../analysis/no/NorwegianLightStemFilter.java | 4 +-
.../no/NorwegianLightStemFilterFactory.java | 4 +-
.../analysis/no/NorwegianLightStemmer.java | 4 +-
.../analysis/no/NorwegianMinimalStemFilter.java | 4 +-
.../no/NorwegianMinimalStemFilterFactory.java | 4 +-
.../analysis/no/NorwegianMinimalStemmer.java | 4 +-
.../analysis/path/PathHierarchyTokenizer.java | 2 +-
.../path/PathHierarchyTokenizerFactory.java | 4 +-
.../path/ReversePathHierarchyTokenizer.java | 2 +-
.../PatternCaptureGroupFilterFactory.java | 4 +-
.../pattern/PatternCaptureGroupTokenFilter.java | 4 +-
.../pattern/PatternReplaceCharFilter.java | 1 -
.../PatternReplaceCharFilterFactory.java | 4 +-
.../analysis/pattern/PatternReplaceFilter.java | 1 -
.../pattern/PatternReplaceFilterFactory.java | 4 +-
.../analysis/pattern/PatternTokenizer.java | 1 -
.../pattern/PatternTokenizerFactory.java | 4 +-
.../analysis/payloads/AbstractEncoder.java | 8 +-
.../payloads/DelimitedPayloadTokenFilter.java | 2 +-
.../DelimitedPayloadTokenFilterFactory.java | 4 +-
.../lucene/analysis/payloads/FloatEncoder.java | 8 +-
.../analysis/payloads/IdentityEncoder.java | 2 +-
.../analysis/payloads/IntegerEncoder.java | 2 +-
.../payloads/NumericPayloadTokenFilter.java | 2 +-
.../NumericPayloadTokenFilterFactory.java | 4 +-
.../analysis/payloads/PayloadEncoder.java | 8 +-
.../lucene/analysis/payloads/PayloadHelper.java | 2 +-
.../payloads/TokenOffsetPayloadTokenFilter.java | 2 +-
.../TokenOffsetPayloadTokenFilterFactory.java | 4 +-
.../payloads/TypeAsPayloadTokenFilter.java | 2 +-
.../TypeAsPayloadTokenFilterFactory.java | 4 +-
.../lucene/analysis/pt/PortugueseAnalyzer.java | 4 +-
.../analysis/pt/PortugueseLightStemFilter.java | 4 +-
.../pt/PortugueseLightStemFilterFactory.java | 4 +-
.../analysis/pt/PortugueseLightStemmer.java | 4 +-
.../pt/PortugueseMinimalStemFilter.java | 4 +-
.../pt/PortugueseMinimalStemFilterFactory.java | 4 +-
.../analysis/pt/PortugueseMinimalStemmer.java | 4 +-
.../analysis/pt/PortugueseStemFilter.java | 4 +-
.../pt/PortugueseStemFilterFactory.java | 4 +-
.../lucene/analysis/pt/PortugueseStemmer.java | 4 +-
.../lucene/analysis/pt/RSLPStemmerBase.java | 4 +-
.../query/QueryAutoStopWordAnalyzer.java | 2 +-
.../analysis/reverse/ReverseStringFilter.java | 1 -
.../reverse/ReverseStringFilterFactory.java | 4 +-
.../lucene/analysis/ro/RomanianAnalyzer.java | 4 +-
.../lucene/analysis/ru/RussianAnalyzer.java | 4 +-
.../analysis/ru/RussianLightStemFilter.java | 4 +-
.../ru/RussianLightStemFilterFactory.java | 4 +-
.../lucene/analysis/ru/RussianLightStemmer.java | 4 +-
.../shingle/ShingleAnalyzerWrapper.java | 4 +-
.../lucene/analysis/shingle/ShingleFilter.java | 4 +-
.../analysis/shingle/ShingleFilterFactory.java | 4 +-
.../analysis/sinks/TeeSinkTokenFilter.java | 4 +-
.../analysis/snowball/SnowballFilter.java | 4 +-
.../snowball/SnowballPorterFilterFactory.java | 4 +-
.../analysis/sr/SerbianNormalizationFilter.java | 4 +-
.../sr/SerbianNormalizationFilterFactory.java | 4 +-
.../sr/SerbianNormalizationRegularFilter.java | 4 +-
.../analysis/standard/ClassicAnalyzer.java | 4 +-
.../lucene/analysis/standard/ClassicFilter.java | 4 +-
.../analysis/standard/ClassicFilterFactory.java | 4 +-
.../analysis/standard/ClassicTokenizer.java | 1 -
.../standard/ClassicTokenizerFactory.java | 4 +-
.../analysis/standard/ClassicTokenizerImpl.java | 3 +-
.../standard/ClassicTokenizerImpl.jflex | 3 +-
.../analysis/standard/StandardAnalyzer.java | 4 +-
.../analysis/standard/StandardFilter.java | 4 +-
.../standard/StandardFilterFactory.java | 4 +-
.../analysis/standard/StandardTokenizer.java | 1 -
.../standard/StandardTokenizerFactory.java | 4 +-
.../standard/StandardTokenizerImpl.java | 3 +-
.../standard/StandardTokenizerImpl.jflex | 3 +-
.../standard/UAX29URLEmailAnalyzer.java | 4 +-
.../standard/UAX29URLEmailTokenizer.java | 4 +-
.../standard/UAX29URLEmailTokenizerFactory.java | 4 +-
.../standard/UAX29URLEmailTokenizerImpl.java | 3 +-
.../standard/UAX29URLEmailTokenizerImpl.jflex | 3 +-
.../lucene/analysis/sv/SwedishAnalyzer.java | 4 +-
.../analysis/sv/SwedishLightStemFilter.java | 4 +-
.../sv/SwedishLightStemFilterFactory.java | 4 +-
.../lucene/analysis/sv/SwedishLightStemmer.java | 4 +-
.../analysis/synonym/SolrSynonymParser.java | 4 +-
.../lucene/analysis/synonym/SynonymFilter.java | 4 +-
.../analysis/synonym/SynonymFilterFactory.java | 4 +-
.../lucene/analysis/synonym/SynonymMap.java | 4 +-
.../analysis/synonym/WordnetSynonymParser.java | 4 +-
.../apache/lucene/analysis/th/ThaiAnalyzer.java | 14 +-
.../lucene/analysis/th/ThaiTokenizer.java | 4 +-
.../analysis/th/ThaiTokenizerFactory.java | 4 +-
.../lucene/analysis/tr/ApostropheFilter.java | 4 +-
.../analysis/tr/ApostropheFilterFactory.java | 4 +-
.../lucene/analysis/tr/TurkishAnalyzer.java | 4 +-
.../analysis/tr/TurkishLowerCaseFilter.java | 4 +-
.../tr/TurkishLowerCaseFilterFactory.java | 4 +-
.../analysis/util/AbstractAnalysisFactory.java | 4 +-
.../lucene/analysis/util/AnalysisSPILoader.java | 4 +-
.../lucene/analysis/util/CharArrayIterator.java | 4 +-
.../lucene/analysis/util/CharArrayMap.java | 4 +-
.../lucene/analysis/util/CharArraySet.java | 4 +-
.../lucene/analysis/util/CharFilterFactory.java | 4 +-
.../lucene/analysis/util/CharTokenizer.java | 4 +-
.../lucene/analysis/util/CharacterUtils.java | 4 +-
.../analysis/util/ClasspathResourceLoader.java | 4 +-
.../lucene/analysis/util/ElisionFilter.java | 4 +-
.../analysis/util/ElisionFilterFactory.java | 4 +-
.../analysis/util/FilesystemResourceLoader.java | 4 +-
.../analysis/util/FilteringTokenFilter.java | 4 +-
.../analysis/util/MultiTermAwareComponent.java | 4 +-
.../lucene/analysis/util/OpenStringBuilder.java | 4 +-
.../lucene/analysis/util/ResourceLoader.java | 4 +-
.../analysis/util/ResourceLoaderAware.java | 1 -
.../lucene/analysis/util/RollingCharBuffer.java | 4 +-
.../analysis/util/SegmentingTokenizerBase.java | 4 +-
.../lucene/analysis/util/StemmerUtil.java | 4 +-
.../analysis/util/StopwordAnalyzerBase.java | 1 -
.../analysis/util/TokenFilterFactory.java | 4 +-
.../lucene/analysis/util/TokenizerFactory.java | 4 +-
.../lucene/analysis/util/WordlistLoader.java | 4 +-
.../analysis/wikipedia/WikipediaTokenizer.java | 1 -
.../wikipedia/WikipediaTokenizerFactory.java | 4 +-
.../wikipedia/WikipediaTokenizerImpl.java | 3 +-
.../wikipedia/WikipediaTokenizerImpl.jflex | 3 +-
.../collation/CollationAttributeFactory.java | 4 +-
.../collation/CollationDocValuesField.java | 4 +-
.../lucene/collation/CollationKeyAnalyzer.java | 4 +-
.../CollatedTermAttributeImpl.java | 4 +-
.../lucene/analysis/ar/TestArabicAnalyzer.java | 4 +-
.../lucene/analysis/ar/TestArabicFilters.java | 4 +-
.../ar/TestArabicNormalizationFilter.java | 4 +-
.../analysis/ar/TestArabicStemFilter.java | 4 +-
.../analysis/bg/TestBulgarianAnalyzer.java | 4 +-
.../bg/TestBulgarianStemFilterFactory.java | 4 +-
.../analysis/bg/TestBulgarianStemmer.java | 4 +-
.../analysis/br/TestBrazilianAnalyzer.java | 4 +-
.../br/TestBrazilianStemFilterFactory.java | 4 +-
.../lucene/analysis/ca/TestCatalanAnalyzer.java | 4 +-
.../charfilter/HTMLStripCharFilterTest.java | 4 +-
.../TestHTMLStripCharFilterFactory.java | 4 +-
.../charfilter/TestMappingCharFilter.java | 1 -
.../TestMappingCharFilterFactory.java | 4 +-
.../lucene/analysis/cjk/TestCJKAnalyzer.java | 4 +-
.../analysis/cjk/TestCJKBigramFilter.java | 4 +-
.../cjk/TestCJKBigramFilterFactory.java | 4 +-
.../lucene/analysis/cjk/TestCJKWidthFilter.java | 4 +-
.../analysis/cjk/TestCJKWidthFilterFactory.java | 4 +-
.../lucene/analysis/ckb/TestSoraniAnalyzer.java | 4 +-
.../ckb/TestSoraniNormalizationFilter.java | 4 +-
.../TestSoraniNormalizationFilterFactory.java | 4 +-
.../analysis/ckb/TestSoraniStemFilter.java | 4 +-
.../ckb/TestSoraniStemFilterFactory.java | 4 +-
.../TestCommonGramsFilterFactory.java | 4 +-
.../TestCommonGramsQueryFilterFactory.java | 4 +-
.../compound/TestCompoundWordTokenFilter.java | 4 +-
...ictionaryCompoundWordTokenFilterFactory.java | 4 +-
...phenationCompoundWordTokenFilterFactory.java | 4 +-
.../core/TestAllAnalyzersHaveFactories.java | 4 +-
.../lucene/analysis/core/TestAnalyzers.java | 4 +-
.../analysis/core/TestBugInSomething.java | 32 +-
.../analysis/core/TestDecimalDigitFilter.java | 4 +-
.../core/TestDecimalDigitFilterFactory.java | 4 +-
.../analysis/core/TestDuelingAnalyzers.java | 4 +-
.../lucene/analysis/core/TestFactories.java | 4 +-
.../analysis/core/TestKeywordAnalyzer.java | 4 +-
.../lucene/analysis/core/TestRandomChains.java | 4 +-
.../lucene/analysis/core/TestStopAnalyzer.java | 4 +-
.../lucene/analysis/core/TestStopFilter.java | 15 +-
.../analysis/core/TestStopFilterFactory.java | 4 +-
.../analysis/core/TestTypeTokenFilter.java | 4 +-
.../core/TestTypeTokenFilterFactory.java | 4 +-
.../core/TestUnicodeWhitespaceTokenizer.java | 4 +-
.../analysis/custom/TestCustomAnalyzer.java | 4 +-
.../lucene/analysis/cz/TestCzechAnalyzer.java | 4 +-
.../analysis/cz/TestCzechStemFilterFactory.java | 4 +-
.../lucene/analysis/cz/TestCzechStemmer.java | 4 +-
.../lucene/analysis/da/TestDanishAnalyzer.java | 4 +-
.../lucene/analysis/de/TestGermanAnalyzer.java | 4 +-
.../analysis/de/TestGermanLightStemFilter.java | 4 +-
.../de/TestGermanLightStemFilterFactory.java | 4 +-
.../de/TestGermanMinimalStemFilter.java | 4 +-
.../de/TestGermanMinimalStemFilterFactory.java | 4 +-
.../de/TestGermanNormalizationFilter.java | 4 +-
.../TestGermanNormalizationFilterFactory.java | 4 +-
.../analysis/de/TestGermanStemFilter.java | 4 +-
.../de/TestGermanStemFilterFactory.java | 4 +-
.../lucene/analysis/el/GreekAnalyzerTest.java | 14 +-
.../el/TestGreekLowerCaseFilterFactory.java | 4 +-
.../analysis/el/TestGreekStemFilterFactory.java | 4 +-
.../lucene/analysis/el/TestGreekStemmer.java | 4 +-
.../lucene/analysis/en/TestEnglishAnalyzer.java | 4 +-
.../en/TestEnglishMinimalStemFilter.java | 4 +-
.../en/TestEnglishMinimalStemFilterFactory.java | 4 +-
.../analysis/en/TestKStemFilterFactory.java | 4 +-
.../apache/lucene/analysis/en/TestKStemmer.java | 4 +-
.../analysis/en/TestPorterStemFilter.java | 4 +-
.../en/TestPorterStemFilterFactory.java | 4 +-
.../lucene/analysis/es/TestSpanishAnalyzer.java | 4 +-
.../analysis/es/TestSpanishLightStemFilter.java | 4 +-
.../es/TestSpanishLightStemFilterFactory.java | 4 +-
.../lucene/analysis/eu/TestBasqueAnalyzer.java | 4 +-
.../lucene/analysis/fa/TestPersianAnalyzer.java | 4 +-
.../analysis/fa/TestPersianCharFilter.java | 4 +-
.../fa/TestPersianNormalizationFilter.java | 4 +-
.../TestPersianNormalizationFilterFactory.java | 4 +-
.../lucene/analysis/fi/TestFinnishAnalyzer.java | 4 +-
.../analysis/fi/TestFinnishLightStemFilter.java | 4 +-
.../fi/TestFinnishLightStemFilterFactory.java | 4 +-
.../lucene/analysis/fr/TestFrenchAnalyzer.java | 4 +-
.../analysis/fr/TestFrenchLightStemFilter.java | 4 +-
.../fr/TestFrenchLightStemFilterFactory.java | 4 +-
.../fr/TestFrenchMinimalStemFilter.java | 4 +-
.../fr/TestFrenchMinimalStemFilterFactory.java | 4 +-
.../lucene/analysis/ga/TestIrishAnalyzer.java | 4 +-
.../analysis/ga/TestIrishLowerCaseFilter.java | 4 +-
.../ga/TestIrishLowerCaseFilterFactory.java | 4 +-
.../analysis/gl/TestGalicianAnalyzer.java | 4 +-
.../gl/TestGalicianMinimalStemFilter.java | 4 +-
.../TestGalicianMinimalStemFilterFactory.java | 4 +-
.../analysis/gl/TestGalicianStemFilter.java | 4 +-
.../gl/TestGalicianStemFilterFactory.java | 4 +-
.../lucene/analysis/hi/TestHindiAnalyzer.java | 14 +-
.../lucene/analysis/hi/TestHindiFilters.java | 4 +-
.../lucene/analysis/hi/TestHindiNormalizer.java | 4 +-
.../lucene/analysis/hi/TestHindiStemmer.java | 4 +-
.../analysis/hu/TestHungarianAnalyzer.java | 4 +-
.../hu/TestHungarianLightStemFilter.java | 4 +-
.../hu/TestHungarianLightStemFilterFactory.java | 4 +-
.../analysis/hunspell/StemmerTestBase.java | 4 +-
.../analysis/hunspell/Test64kAffixes.java | 8 +-
.../analysis/hunspell/TestAllDictionaries.java | 10 +-
.../analysis/hunspell/TestAllDictionaries2.java | 10 +-
.../analysis/hunspell/TestAlternateCasing.java | 4 +-
.../analysis/hunspell/TestCaseInsensitive.java | 4 +-
.../analysis/hunspell/TestCaseSensitive.java | 4 +-
.../lucene/analysis/hunspell/TestCircumfix.java | 4 +-
.../analysis/hunspell/TestComplexPrefix.java | 4 +-
.../lucene/analysis/hunspell/TestCondition.java | 4 +-
.../analysis/hunspell/TestCondition2.java | 8 +-
.../lucene/analysis/hunspell/TestConv.java | 8 +-
.../analysis/hunspell/TestDependencies.java | 4 +-
.../analysis/hunspell/TestDictionary.java | 10 +-
.../analysis/hunspell/TestDoubleEscape.java | 4 +-
.../lucene/analysis/hunspell/TestEscaped.java | 4 +-
.../lucene/analysis/hunspell/TestFlagLong.java | 4 +-
.../lucene/analysis/hunspell/TestFlagNum.java | 4 +-
.../lucene/analysis/hunspell/TestFullStrip.java | 4 +-
.../lucene/analysis/hunspell/TestHomonyms.java | 4 +-
.../hunspell/TestHunspellStemFilter.java | 10 +-
.../hunspell/TestHunspellStemFilterFactory.java | 4 +-
.../lucene/analysis/hunspell/TestIgnore.java | 4 +-
.../lucene/analysis/hunspell/TestKeepCase.java | 4 +-
.../lucene/analysis/hunspell/TestMorph.java | 4 +-
.../analysis/hunspell/TestMorphAlias.java | 4 +-
.../lucene/analysis/hunspell/TestMorphData.java | 4 +-
.../lucene/analysis/hunspell/TestNeedAffix.java | 4 +-
.../analysis/hunspell/TestOnlyInCompound.java | 4 +-
.../hunspell/TestOptionalCondition.java | 4 +-
.../lucene/analysis/hunspell/TestSpaces.java | 4 +-
.../lucene/analysis/hunspell/TestStemmer.java | 4 +-
.../hunspell/TestStrangeOvergeneration.java | 4 +-
.../lucene/analysis/hunspell/TestTwoFold.java | 4 +-
.../analysis/hunspell/TestTwoSuffixes.java | 4 +-
.../lucene/analysis/hunspell/TestZeroAffix.java | 4 +-
.../analysis/hunspell/TestZeroAffix2.java | 4 +-
.../analysis/hy/TestArmenianAnalyzer.java | 4 +-
.../analysis/id/TestIndonesianAnalyzer.java | 4 +-
.../id/TestIndonesianStemFilterFactory.java | 4 +-
.../analysis/id/TestIndonesianStemmer.java | 4 +-
.../lucene/analysis/in/TestIndicNormalizer.java | 4 +-
.../lucene/analysis/it/TestItalianAnalyzer.java | 4 +-
.../analysis/it/TestItalianLightStemFilter.java | 4 +-
.../it/TestItalianLightStemFilterFactory.java | 4 +-
.../analysis/lt/TestLithuanianAnalyzer.java | 4 +-
.../analysis/lt/TestLithuanianStemming.java | 4 +-
.../lucene/analysis/lv/TestLatvianAnalyzer.java | 4 +-
.../lv/TestLatvianStemFilterFactory.java | 4 +-
.../lucene/analysis/lv/TestLatvianStemmer.java | 4 +-
.../DateRecognizerFilterFactoryTest.java | 14 +-
.../miscellaneous/DateRecognizerFilterTest.java | 14 +-
.../miscellaneous/TestASCIIFoldingFilter.java | 4 +-
.../miscellaneous/TestCapitalizationFilter.java | 1 -
.../TestCapitalizationFilterFactory.java | 4 +-
.../miscellaneous/TestCodepointCountFilter.java | 4 +-
.../TestCodepointCountFilterFactory.java | 4 +-
.../miscellaneous/TestEmptyTokenStream.java | 4 +-
.../miscellaneous/TestFingerprintFilter.java | 4 +-
.../TestFingerprintFilterFactory.java | 4 +-
.../TestHyphenatedWordsFilter.java | 1 -
.../miscellaneous/TestKeepFilterFactory.java | 4 +-
.../miscellaneous/TestKeepWordFilter.java | 1 -
.../miscellaneous/TestKeywordMarkerFilter.java | 32 +-
.../TestKeywordMarkerFilterFactory.java | 4 +-
.../miscellaneous/TestKeywordRepeatFilter.java | 4 +-
.../miscellaneous/TestLengthFilter.java | 4 +-
.../miscellaneous/TestLengthFilterFactory.java | 4 +-
.../TestLimitTokenCountAnalyzer.java | 4 +-
.../TestLimitTokenCountFilter.java | 4 +-
.../TestLimitTokenCountFilterFactory.java | 4 +-
.../TestLimitTokenOffsetFilter.java | 4 +-
.../TestLimitTokenOffsetFilterFactory.java | 4 +-
.../TestLimitTokenPositionFilter.java | 2 +-
.../TestLimitTokenPositionFilterFactory.java | 2 +-
.../TestPerFieldAnalyzerWrapper.java | 32 +-
.../TestPrefixAndSuffixAwareTokenFilter.java | 4 +-
.../TestPrefixAwareTokenFilter.java | 4 +-
.../TestRemoveDuplicatesTokenFilter.java | 1 -
.../TestRemoveDuplicatesTokenFilterFactory.java | 4 +-
.../TestScandinavianFoldingFilter.java | 4 +-
.../TestScandinavianFoldingFilterFactory.java | 15 +-
.../TestScandinavianNormalizationFilter.java | 4 +-
...tScandinavianNormalizationFilterFactory.java | 18 +-
.../TestStemmerOverrideFilter.java | 2 +-
.../TestStemmerOverrideFilterFactory.java | 4 +-
.../analysis/miscellaneous/TestTrimFilter.java | 1 -
.../miscellaneous/TestTrimFilterFactory.java | 4 +-
.../miscellaneous/TestTruncateTokenFilter.java | 4 +-
.../TestTruncateTokenFilterFactory.java | 4 +-
.../miscellaneous/TestWordDelimiterFilter.java | 1 -
.../ngram/EdgeNGramTokenFilterTest.java | 4 +-
.../analysis/ngram/EdgeNGramTokenizerTest.java | 4 +-
.../analysis/ngram/NGramTokenFilterTest.java | 4 +-
.../analysis/ngram/NGramTokenizerTest.java | 4 +-
.../lucene/analysis/ngram/TestNGramFilters.java | 4 +-
.../lucene/analysis/nl/TestDutchAnalyzer.java | 4 +-
.../analysis/no/TestNorwegianAnalyzer.java | 4 +-
.../no/TestNorwegianLightStemFilter.java | 4 +-
.../no/TestNorwegianLightStemFilterFactory.java | 4 +-
.../no/TestNorwegianMinimalStemFilter.java | 4 +-
.../TestNorwegianMinimalStemFilterFactory.java | 4 +-
.../path/TestPathHierarchyTokenizer.java | 4 +-
.../path/TestReversePathHierarchyTokenizer.java | 4 +-
.../TestPatternCaptureGroupTokenFilter.java | 4 +-
.../pattern/TestPatternReplaceCharFilter.java | 1 -
.../TestPatternReplaceCharFilterFactory.java | 4 +-
.../pattern/TestPatternReplaceFilter.java | 1 -
.../TestPatternReplaceFilterFactory.java | 4 +-
.../analysis/pattern/TestPatternTokenizer.java | 1 -
.../pattern/TestPatternTokenizerFactory.java | 4 +-
.../DelimitedPayloadTokenFilterTest.java | 2 +-
.../payloads/NumericPayloadTokenFilterTest.java | 14 +-
.../TestDelimitedPayloadTokenFilterFactory.java | 4 +-
.../TokenOffsetPayloadTokenFilterTest.java | 15 +-
.../payloads/TypeAsPayloadTokenFilterTest.java | 17 +-
.../analysis/pt/TestPortugueseAnalyzer.java | 4 +-
.../pt/TestPortugueseLightStemFilter.java | 4 +-
.../TestPortugueseLightStemFilterFactory.java | 4 +-
.../pt/TestPortugueseMinimalStemFilter.java | 4 +-
.../TestPortugueseMinimalStemFilterFactory.java | 4 +-
.../analysis/pt/TestPortugueseStemFilter.java | 4 +-
.../pt/TestPortugueseStemFilterFactory.java | 4 +-
.../query/QueryAutoStopWordAnalyzerTest.java | 2 +-
.../reverse/TestReverseStringFilter.java | 1 -
.../reverse/TestReverseStringFilterFactory.java | 4 +-
.../analysis/ro/TestRomanianAnalyzer.java | 4 +-
.../lucene/analysis/ru/TestRussianAnalyzer.java | 4 +-
.../analysis/ru/TestRussianLightStemFilter.java | 4 +-
.../ru/TestRussianLightStemFilterFactory.java | 4 +-
.../shingle/ShingleAnalyzerWrapperTest.java | 4 +-
.../analysis/shingle/ShingleFilterTest.java | 4 +-
.../shingle/TestShingleFilterFactory.java | 4 +-
.../analysis/sinks/TestTeeSinkTokenFilter.java | 14 +-
.../lucene/analysis/snowball/TestSnowball.java | 4 +-
.../TestSnowballPorterFilterFactory.java | 15 +-
.../analysis/snowball/TestSnowballVocab.java | 4 +-
.../sr/TestSerbianNormalizationFilter.java | 4 +-
.../TestSerbianNormalizationFilterFactory.java | 4 +-
.../TestSerbianNormalizationRegularFilter.java | 4 +-
.../analysis/standard/TestClassicAnalyzer.java | 4 +-
.../analysis/standard/TestStandardAnalyzer.java | 4 +-
.../standard/TestStandardFactories.java | 4 +-
.../standard/TestUAX29URLEmailAnalyzer.java | 4 +-
.../standard/TestUAX29URLEmailTokenizer.java | 32 +-
.../TestUAX29URLEmailTokenizerFactory.java | 4 +-
.../standard/WordBreakTestUnicode_6_3_0.java | 4 +-
.../lucene/analysis/sv/TestSwedishAnalyzer.java | 4 +-
.../analysis/sv/TestSwedishLightStemFilter.java | 4 +-
.../sv/TestSwedishLightStemFilterFactory.java | 4 +-
.../synonym/BaseSynonymParserTestCase.java | 4 +-
.../analysis/synonym/TestMultiWordSynonyms.java | 4 +-
.../analysis/synonym/TestSolrSynonymParser.java | 4 +-
.../synonym/TestSynonymFilterFactory.java | 4 +-
.../analysis/synonym/TestSynonymMapFilter.java | 1 -
.../synonym/TestWordnetSynonymParser.java | 1 -
.../lucene/analysis/th/TestThaiAnalyzer.java | 4 +-
.../analysis/th/TestThaiTokenizerFactory.java | 4 +-
.../analysis/tr/TestApostropheFilter.java | 4 +-
.../tr/TestApostropheFilterFactory.java | 4 +-
.../lucene/analysis/tr/TestTurkishAnalyzer.java | 4 +-
.../analysis/tr/TestTurkishLowerCaseFilter.java | 4 +-
.../tr/TestTurkishLowerCaseFilterFactory.java | 4 +-
.../util/BaseTokenStreamFactoryTestCase.java | 4 +-
.../analysis/util/StringMockResourceLoader.java | 4 +-
.../analysis/util/TestAnalysisSPILoader.java | 4 +-
.../analysis/util/TestCharArrayIterator.java | 4 +-
.../lucene/analysis/util/TestCharArrayMap.java | 1 -
.../lucene/analysis/util/TestCharArraySet.java | 4 +-
.../analysis/util/TestCharTokenizers.java | 4 +-
.../analysis/util/TestCharacterUtils.java | 4 +-
.../lucene/analysis/util/TestElision.java | 4 +-
.../analysis/util/TestElisionFilterFactory.java | 4 +-
.../util/TestFilesystemResourceLoader.java | 32 +-
.../analysis/util/TestRollingCharBuffer.java | 4 +-
.../util/TestSegmentingTokenizerBase.java | 4 +-
.../analysis/util/TestWordlistLoader.java | 4 +-
.../TestWikipediaTokenizerFactory.java | 4 +-
.../wikipedia/WikipediaTokenizerTest.java | 1 -
.../collation/TestCollationDocValuesField.java | 4 +-
.../collation/TestCollationKeyAnalyzer.java | 4 +-
.../standard/GenerateJflexTLDMacros.java | 16 +-
.../lucene/analysis/icu/ICUFoldingFilter.java | 4 +-
.../analysis/icu/ICUFoldingFilterFactory.java | 4 +-
.../analysis/icu/ICUNormalizer2CharFilter.java | 4 +-
.../icu/ICUNormalizer2CharFilterFactory.java | 4 +-
.../analysis/icu/ICUNormalizer2Filter.java | 4 +-
.../icu/ICUNormalizer2FilterFactory.java | 4 +-
.../lucene/analysis/icu/ICUTransformFilter.java | 4 +-
.../analysis/icu/ICUTransformFilterFactory.java | 4 +-
.../icu/segmentation/BreakIteratorWrapper.java | 4 +-
.../icu/segmentation/CharArrayIterator.java | 4 +-
.../segmentation/CompositeBreakIterator.java | 4 +-
.../segmentation/DefaultICUTokenizerConfig.java | 4 +-
.../analysis/icu/segmentation/ICUTokenizer.java | 4 +-
.../icu/segmentation/ICUTokenizerConfig.java | 4 +-
.../icu/segmentation/ICUTokenizerFactory.java | 4 +-
.../icu/segmentation/ScriptIterator.java | 3 +-
.../icu/tokenattributes/ScriptAttribute.java | 4 +-
.../tokenattributes/ScriptAttributeImpl.java | 4 +-
.../collation/ICUCollationAttributeFactory.java | 4 +-
.../collation/ICUCollationDocValuesField.java | 4 +-
.../collation/ICUCollationKeyAnalyzer.java | 4 +-
.../ICUCollatedTermAttributeImpl.java | 4 +-
.../analysis/icu/TestICUFoldingFilter.java | 4 +-
.../icu/TestICUFoldingFilterFactory.java | 4 +-
.../icu/TestICUNormalizer2CharFilter.java | 4 +-
.../TestICUNormalizer2CharFilterFactory.java | 4 +-
.../analysis/icu/TestICUNormalizer2Filter.java | 4 +-
.../icu/TestICUNormalizer2FilterFactory.java | 4 +-
.../analysis/icu/TestICUTransformFilter.java | 4 +-
.../icu/TestICUTransformFilterFactory.java | 4 +-
.../icu/segmentation/TestCharArrayIterator.java | 4 +-
.../icu/segmentation/TestICUTokenizer.java | 4 +-
.../icu/segmentation/TestICUTokenizerCJK.java | 4 +-
.../segmentation/TestICUTokenizerFactory.java | 4 +-
.../segmentation/TestWithCJKBigramFilter.java | 4 +-
.../TestICUCollationDocValuesField.java | 4 +-
.../collation/TestICUCollationKeyAnalyzer.java | 4 +-
.../analysis/icu/GenerateUTR30DataFiles.java | 4 +-
.../lucene/analysis/icu/RBBIRuleCompiler.java | 4 +-
.../lucene/analysis/ja/GraphvizFormatter.java | 4 +-
.../lucene/analysis/ja/JapaneseAnalyzer.java | 4 +-
.../analysis/ja/JapaneseBaseFormFilter.java | 4 +-
.../ja/JapaneseBaseFormFilterFactory.java | 4 +-
.../ja/JapaneseIterationMarkCharFilter.java | 4 +-
.../JapaneseIterationMarkCharFilterFactory.java | 4 +-
.../analysis/ja/JapaneseKatakanaStemFilter.java | 4 +-
.../ja/JapaneseKatakanaStemFilterFactory.java | 4 +-
.../analysis/ja/JapaneseNumberFilter.java | 4 +-
.../ja/JapaneseNumberFilterFactory.java | 4 +-
.../ja/JapanesePartOfSpeechStopFilter.java | 4 +-
.../JapanesePartOfSpeechStopFilterFactory.java | 4 +-
.../analysis/ja/JapaneseReadingFormFilter.java | 4 +-
.../ja/JapaneseReadingFormFilterFactory.java | 4 +-
.../lucene/analysis/ja/JapaneseTokenizer.java | 4 +-
.../analysis/ja/JapaneseTokenizerFactory.java | 4 +-
.../org/apache/lucene/analysis/ja/Token.java | 4 +-
.../analysis/ja/dict/BinaryDictionary.java | 4 +-
.../analysis/ja/dict/CharacterDefinition.java | 4 +-
.../analysis/ja/dict/ConnectionCosts.java | 4 +-
.../lucene/analysis/ja/dict/Dictionary.java | 4 +-
.../analysis/ja/dict/TokenInfoDictionary.java | 4 +-
.../lucene/analysis/ja/dict/TokenInfoFST.java | 4 +-
.../analysis/ja/dict/UnknownDictionary.java | 4 +-
.../lucene/analysis/ja/dict/UserDictionary.java | 4 +-
.../ja/tokenattributes/BaseFormAttribute.java | 4 +-
.../tokenattributes/BaseFormAttributeImpl.java | 4 +-
.../ja/tokenattributes/InflectionAttribute.java | 4 +-
.../InflectionAttributeImpl.java | 4 +-
.../tokenattributes/PartOfSpeechAttribute.java | 4 +-
.../PartOfSpeechAttributeImpl.java | 4 +-
.../ja/tokenattributes/ReadingAttribute.java | 4 +-
.../tokenattributes/ReadingAttributeImpl.java | 4 +-
.../apache/lucene/analysis/ja/util/CSVUtil.java | 4 +-
.../lucene/analysis/ja/util/ToStringUtil.java | 4 +-
.../analysis/ja/StringMockResourceLoader.java | 4 +-
.../lucene/analysis/ja/TestExtendedMode.java | 4 +-
.../analysis/ja/TestJapaneseAnalyzer.java | 4 +-
.../analysis/ja/TestJapaneseBaseFormFilter.java | 4 +-
.../ja/TestJapaneseBaseFormFilterFactory.java | 4 +-
.../ja/TestJapaneseIterationMarkCharFilter.java | 4 +-
...tJapaneseIterationMarkCharFilterFactory.java | 4 +-
.../ja/TestJapaneseKatakanaStemFilter.java | 4 +-
.../TestJapaneseKatakanaStemFilterFactory.java | 4 +-
.../analysis/ja/TestJapaneseNumberFilter.java | 4 +-
.../ja/TestJapaneseNumberFilterFactory.java | 4 +-
...stJapanesePartOfSpeechStopFilterFactory.java | 4 +-
.../ja/TestJapaneseReadingFormFilter.java | 4 +-
.../TestJapaneseReadingFormFilterFactory.java | 4 +-
.../analysis/ja/TestJapaneseTokenizer.java | 4 +-
.../ja/TestJapaneseTokenizerFactory.java | 4 +-
.../lucene/analysis/ja/TestSearchMode.java | 4 +-
.../ja/dict/TestTokenInfoDictionary.java | 4 +-
.../analysis/ja/dict/UserDictionaryTest.java | 4 +-
.../analysis/ja/util/TestToStringUtil.java | 4 +-
.../ja/util/BinaryDictionaryWriter.java | 4 +-
.../ja/util/CharacterDefinitionWriter.java | 4 +-
.../ja/util/ConnectionCostsBuilder.java | 4 +-
.../analysis/ja/util/ConnectionCostsWriter.java | 4 +-
.../analysis/ja/util/DictionaryBuilder.java | 4 +-
.../ja/util/TokenInfoDictionaryBuilder.java | 4 +-
.../ja/util/TokenInfoDictionaryWriter.java | 4 +-
.../ja/util/UnknownDictionaryBuilder.java | 4 +-
.../ja/util/UnknownDictionaryWriter.java | 4 +-
.../analysis/ja/dict/UnknownDictionaryTest.java | 4 +-
.../analysis/morfologik/MorfologikAnalyzer.java | 3 +-
.../analysis/morfologik/MorfologikFilter.java | 4 +-
.../morfologik/MorfologikFilterFactory.java | 4 +-
.../MorphosyntacticTagsAttribute.java | 3 +-
.../MorphosyntacticTagsAttributeImpl.java | 3 +-
.../morfologik/TestMorfologikAnalyzer.java | 4 +-
.../morfologik/TestMorfologikFilterFactory.java | 4 +-
.../analysis/phonetic/BeiderMorseFilter.java | 4 +-
.../phonetic/BeiderMorseFilterFactory.java | 4 +-
.../phonetic/DaitchMokotoffSoundexFilter.java | 2 +-
.../DaitchMokotoffSoundexFilterFactory.java | 2 +-
.../phonetic/DoubleMetaphoneFilterFactory.java | 4 +-
.../analysis/phonetic/PhoneticFilter.java | 1 -
.../phonetic/PhoneticFilterFactory.java | 4 +-
.../phonetic/TestBeiderMorseFilter.java | 4 +-
.../phonetic/TestBeiderMorseFilterFactory.java | 4 +-
.../TestDaitchMokotoffSoundexFilter.java | 2 +-
.../TestDaitchMokotoffSoundexFilterFactory.java | 4 +-
.../TestDoubleMetaphoneFilterFactory.java | 4 +-
.../analysis/phonetic/TestPhoneticFilter.java | 1 -
.../phonetic/TestPhoneticFilterFactory.java | 4 +-
.../analysis/cn/smart/AnalyzerProfile.java | 1 -
.../lucene/analysis/cn/smart/CharType.java | 1 -
.../analysis/cn/smart/HMMChineseTokenizer.java | 4 +-
.../cn/smart/HMMChineseTokenizerFactory.java | 1 -
.../analysis/cn/smart/SmartChineseAnalyzer.java | 1 -
.../lucene/analysis/cn/smart/Utility.java | 1 -
.../lucene/analysis/cn/smart/WordSegmenter.java | 1 -
.../lucene/analysis/cn/smart/WordType.java | 1 -
.../cn/smart/hhmm/AbstractDictionary.java | 1 -
.../analysis/cn/smart/hhmm/BiSegGraph.java | 1 -
.../cn/smart/hhmm/BigramDictionary.java | 1 -
.../analysis/cn/smart/hhmm/HHMMSegmenter.java | 1 -
.../lucene/analysis/cn/smart/hhmm/PathNode.java | 1 -
.../lucene/analysis/cn/smart/hhmm/SegGraph.java | 1 -
.../lucene/analysis/cn/smart/hhmm/SegToken.java | 1 -
.../analysis/cn/smart/hhmm/SegTokenFilter.java | 1 -
.../analysis/cn/smart/hhmm/SegTokenPair.java | 1 -
.../analysis/cn/smart/hhmm/WordDictionary.java | 1 -
.../smart/TestHMMChineseTokenizerFactory.java | 4 +-
.../cn/smart/TestSmartChineseAnalyzer.java | 1 -
.../lucene/analysis/pl/PolishAnalyzer.java | 4 +-
.../lucene/analysis/stempel/StempelFilter.java | 26 +-
.../stempel/StempelPolishStemFilterFactory.java | 4 +-
.../lucene/analysis/stempel/StempelStemmer.java | 26 +-
.../lucene/analysis/pl/TestPolishAnalyzer.java | 4 +-
.../TestStempelPolishStemFilterFactory.java | 4 +-
.../lucene/analysis/uima/BaseUIMATokenizer.java | 4 +-
.../analysis/uima/UIMAAnnotationsTokenizer.java | 4 +-
.../uima/UIMAAnnotationsTokenizerFactory.java | 4 +-
.../lucene/analysis/uima/UIMABaseAnalyzer.java | 4 +-
.../analysis/uima/UIMATypeAwareAnalyzer.java | 4 +-
.../uima/UIMATypeAwareAnnotationsTokenizer.java | 4 +-
...IMATypeAwareAnnotationsTokenizerFactory.java | 4 +-
.../lucene/analysis/uima/ae/AEProvider.java | 4 +-
.../analysis/uima/ae/AEProviderFactory.java | 4 +-
.../analysis/uima/ae/BasicAEProvider.java | 4 +-
.../uima/ae/OverridingParamsAEProvider.java | 4 +-
.../analysis/uima/UIMABaseAnalyzerTest.java | 4 +-
.../uima/UIMATypeAwareAnalyzerTest.java | 4 +-
.../analysis/uima/ae/AEProviderFactoryTest.java | 4 +-
.../analysis/uima/ae/BasicAEProviderTest.java | 4 +-
.../uima/ae/OverridingParamsAEProviderTest.java | 4 +-
.../analysis/uima/an/SampleEntityAnnotator.java | 4 +-
.../analysis/uima/an/SamplePoSTagger.java | 4 +-
.../uima/an/SampleWSTokenizerAnnotator.java | 4 +-
.../org/apache/lucene/codecs/Placeholder.java | 4 +-
.../lucene/codecs/lucene50/Lucene50Codec.java | 4 +-
.../lucene50/Lucene50DocValuesConsumer.java | 4 +-
.../lucene50/Lucene50DocValuesFormat.java | 4 +-
.../lucene50/Lucene50DocValuesProducer.java | 4 +-
.../codecs/lucene50/Lucene50NormsFormat.java | 4 +-
.../codecs/lucene50/Lucene50NormsProducer.java | 4 +-
.../lucene/codecs/lucene53/Lucene53Codec.java | 4 +-
.../lucene/codecs/lucene54/Lucene54Codec.java | 4 +-
.../codecs/lucene50/Lucene50NormsConsumer.java | 4 +-
.../lucene/codecs/lucene50/Lucene50RWCodec.java | 4 +-
.../codecs/lucene50/Lucene50RWNormsFormat.java | 4 +-
.../lucene50/TestLucene50DocValuesFormat.java | 4 +-
.../lucene50/TestLucene50NormsFormat.java | 4 +-
.../index/TestBackwardsCompatibility.java | 12 +-
.../lucene/index/TestMaxPositionInOldIndex.java | 4 +-
lucene/benchmark/.gitignore | 2 +
.../org/apache/lucene/benchmark/Constants.java | 19 +-
.../lucene/benchmark/byTask/Benchmark.java | 4 +-
.../lucene/benchmark/byTask/PerfRunData.java | 4 +-
.../byTask/feeds/AbstractQueryMaker.java | 13 +-
.../byTask/feeds/ContentItemsSource.java | 4 +-
.../benchmark/byTask/feeds/ContentSource.java | 4 +-
.../benchmark/byTask/feeds/DemoHTMLParser.java | 4 +-
.../byTask/feeds/DirContentSource.java | 4 +-
.../lucene/benchmark/byTask/feeds/DocData.java | 4 +-
.../lucene/benchmark/byTask/feeds/DocMaker.java | 4 +-
.../byTask/feeds/EnwikiContentSource.java | 4 +-
.../byTask/feeds/EnwikiQueryMaker.java | 4 +-
.../benchmark/byTask/feeds/FacetSource.java | 4 +-
.../byTask/feeds/FileBasedQueryMaker.java | 18 +-
.../byTask/feeds/GeonamesLineParser.java | 4 +-
.../benchmark/byTask/feeds/HTMLParser.java | 4 +-
.../benchmark/byTask/feeds/LineDocSource.java | 4 +-
.../feeds/LongToEnglishContentSource.java | 4 +-
.../byTask/feeds/LongToEnglishQueryMaker.java | 4 +-
.../byTask/feeds/NoMoreDataException.java | 4 +-
.../benchmark/byTask/feeds/QueryMaker.java | 4 +-
.../byTask/feeds/RandomFacetSource.java | 4 +-
.../byTask/feeds/ReutersContentSource.java | 4 +-
.../byTask/feeds/ReutersQueryMaker.java | 4 +-
.../byTask/feeds/SimpleQueryMaker.java | 4 +-
.../feeds/SimpleSloppyPhraseQueryMaker.java | 4 +-
.../benchmark/byTask/feeds/SingleDocSource.java | 8 +-
.../byTask/feeds/SortableSingleDocSource.java | 4 +-
.../benchmark/byTask/feeds/SpatialDocMaker.java | 4 +-
.../byTask/feeds/SpatialFileQueryMaker.java | 4 +-
.../byTask/feeds/TrecContentSource.java | 4 +-
.../benchmark/byTask/feeds/TrecDocParser.java | 4 +-
.../benchmark/byTask/feeds/TrecFBISParser.java | 4 +-
.../benchmark/byTask/feeds/TrecFR94Parser.java | 4 +-
.../benchmark/byTask/feeds/TrecFTParser.java | 4 +-
.../benchmark/byTask/feeds/TrecGov2Parser.java | 4 +-
.../byTask/feeds/TrecLATimesParser.java | 4 +-
.../byTask/feeds/TrecParserByPath.java | 4 +-
.../benchmark/byTask/programmatic/Sample.java | 15 +-
.../lucene/benchmark/byTask/stats/Points.java | 4 +-
.../lucene/benchmark/byTask/stats/Report.java | 4 +-
.../benchmark/byTask/stats/TaskStats.java | 4 +-
.../benchmark/byTask/tasks/AddDocTask.java | 4 +-
.../byTask/tasks/AddFacetedDocTask.java | 4 +-
.../benchmark/byTask/tasks/AddIndexesTask.java | 4 +-
.../byTask/tasks/AnalyzerFactoryTask.java | 4 +-
.../byTask/tasks/BenchmarkHighlighter.java | 4 +-
.../benchmark/byTask/tasks/ClearStatsTask.java | 8 +-
.../benchmark/byTask/tasks/CloseIndexTask.java | 4 +-
.../benchmark/byTask/tasks/CloseReaderTask.java | 4 +-
.../byTask/tasks/CloseTaxonomyIndexTask.java | 4 +-
.../byTask/tasks/CloseTaxonomyReaderTask.java | 4 +-
.../benchmark/byTask/tasks/CommitIndexTask.java | 4 +-
.../byTask/tasks/CommitTaxonomyIndexTask.java | 2 +-
.../byTask/tasks/ConsumeContentSourceTask.java | 4 +-
.../benchmark/byTask/tasks/CreateIndexTask.java | 4 +-
.../byTask/tasks/CreateTaxonomyIndexTask.java | 4 +-
.../benchmark/byTask/tasks/ForceMergeTask.java | 4 +-
.../byTask/tasks/NearRealtimeReaderTask.java | 4 +-
.../benchmark/byTask/tasks/NewAnalyzerTask.java | 13 +-
.../byTask/tasks/NewCollationAnalyzerTask.java | 4 +-
.../benchmark/byTask/tasks/NewLocaleTask.java | 4 +-
.../benchmark/byTask/tasks/NewRoundTask.java | 8 +-
.../benchmark/byTask/tasks/OpenIndexTask.java | 4 +-
.../benchmark/byTask/tasks/OpenReaderTask.java | 4 +-
.../byTask/tasks/OpenTaxonomyIndexTask.java | 4 +-
.../byTask/tasks/OpenTaxonomyReaderTask.java | 4 +-
.../lucene/benchmark/byTask/tasks/PerfTask.java | 4 +-
.../benchmark/byTask/tasks/PrintReaderTask.java | 4 +-
.../lucene/benchmark/byTask/tasks/ReadTask.java | 4 +-
.../benchmark/byTask/tasks/ReadTokensTask.java | 4 +-
.../byTask/tasks/ReopenReaderTask.java | 2 +-
.../benchmark/byTask/tasks/RepAllTask.java | 4 +-
.../byTask/tasks/RepSelectByPrefTask.java | 4 +-
.../byTask/tasks/RepSumByNameRoundTask.java | 4 +-
.../byTask/tasks/RepSumByNameTask.java | 4 +-
.../byTask/tasks/RepSumByPrefRoundTask.java | 4 +-
.../byTask/tasks/RepSumByPrefTask.java | 4 +-
.../benchmark/byTask/tasks/ReportTask.java | 18 +-
.../benchmark/byTask/tasks/ResetInputsTask.java | 8 +-
.../byTask/tasks/ResetSystemEraseTask.java | 8 +-
.../byTask/tasks/ResetSystemSoftTask.java | 8 +-
.../byTask/tasks/RollbackIndexTask.java | 4 +-
.../benchmark/byTask/tasks/SearchTask.java | 4 +-
.../tasks/SearchTravRetHighlightTask.java | 4 +-
.../SearchTravRetLoadFieldSelectorTask.java | 14 +-
.../byTask/tasks/SearchTravRetTask.java | 4 +-
.../tasks/SearchTravRetVectorHighlightTask.java | 4 +-
.../benchmark/byTask/tasks/SearchTravTask.java | 4 +-
.../byTask/tasks/SearchWithCollectorTask.java | 2 +-
.../byTask/tasks/SearchWithSortTask.java | 2 +-
.../benchmark/byTask/tasks/SetPropTask.java | 8 +-
.../benchmark/byTask/tasks/TaskSequence.java | 4 +-
.../benchmark/byTask/tasks/UpdateDocTask.java | 4 +-
.../lucene/benchmark/byTask/tasks/WaitTask.java | 4 +-
.../lucene/benchmark/byTask/tasks/WarmTask.java | 4 +-
.../byTask/tasks/WriteEnwikiLineDocTask.java | 32 +-
.../byTask/tasks/WriteLineDocTask.java | 4 +-
.../benchmark/byTask/utils/Algorithm.java | 4 +-
.../benchmark/byTask/utils/AnalyzerFactory.java | 4 +-
.../lucene/benchmark/byTask/utils/Config.java | 4 +-
.../lucene/benchmark/byTask/utils/Format.java | 4 +-
.../benchmark/byTask/utils/StreamUtils.java | 4 +-
.../benchmark/quality/trec/QueryDriver.java | 4 +-
.../lucene/benchmark/utils/ExtractReuters.java | 15 +-
.../benchmark/utils/ExtractWikipedia.java | 4 +-
lucene/benchmark/src/test/conf/ConfLoader.java | 3 +-
.../lucene/benchmark/BenchmarkTestCase.java | 4 +-
.../benchmark/byTask/TestPerfTasksLogic.java | 3 -
.../benchmark/byTask/TestPerfTasksParse.java | 1 -
.../benchmark/byTask/feeds/DocMakerTest.java | 4 +-
.../byTask/feeds/EnwikiContentSourceTest.java | 4 +-
.../byTask/feeds/LineDocSourceTest.java | 4 +-
.../benchmark/byTask/feeds/TestHtmlParser.java | 4 +-
.../byTask/feeds/TrecContentSourceTest.java | 4 +-
.../byTask/tasks/AddIndexesTaskTest.java | 4 +-
.../byTask/tasks/CommitIndexTaskTest.java | 4 +-
.../tasks/CountingHighlighterTestTask.java | 1 -
.../byTask/tasks/CountingSearchTestTask.java | 1 -
.../byTask/tasks/CreateIndexTaskTest.java | 4 +-
.../benchmark/byTask/tasks/PerfTaskTest.java | 4 +-
.../byTask/tasks/SearchWithSortTaskTest.java | 4 +-
.../tasks/WriteEnwikiLineDocTaskTest.java | 4 +-
.../byTask/tasks/WriteLineDocTaskTest.java | 4 +-
.../byTask/tasks/alt/AltPackageTaskTest.java | 4 +-
.../benchmark/byTask/tasks/alt/AltTestTask.java | 4 +-
.../benchmark/byTask/utils/StreamUtilsTest.java | 18 +-
.../benchmark/byTask/utils/TestConfig.java | 4 +-
.../benchmark/quality/TestQualityRun.java | 4 +-
.../CachingNaiveBayesClassifier.java | 32 +-
.../document/DocumentClassifier.java | 4 +-
.../KNearestNeighborDocumentClassifier.java | 4 +-
.../SimpleNaiveBayesDocumentClassifier.java | 4 +-
.../utils/ConfusionMatrixGenerator.java | 4 +-
.../classification/utils/DatasetSplitter.java | 4 +-
.../CachingNaiveBayesClassifierTest.java | 4 +-
.../DocumentClassificationTestBase.java | 4 +-
.../KNearestNeighborDocumentClassifierTest.java | 4 +-
.../SimpleNaiveBayesDocumentClassifierTest.java | 4 +-
.../utils/ConfusionMatrixGeneratorTest.java | 4 +-
.../classification/utils/DataSplitterTest.java | 4 +-
.../autoprefix/AutoPrefixPostingsFormat.java | 4 +-
.../codecs/blockterms/BlockTermsReader.java | 4 +-
.../codecs/blockterms/BlockTermsWriter.java | 4 +-
.../blockterms/FixedGapTermsIndexReader.java | 4 +-
.../blockterms/FixedGapTermsIndexWriter.java | 4 +-
.../codecs/blockterms/TermsIndexReaderBase.java | 4 +-
.../codecs/blockterms/TermsIndexWriterBase.java | 4 +-
.../blockterms/VariableGapTermsIndexReader.java | 4 +-
.../blockterms/VariableGapTermsIndexWriter.java | 4 +-
.../BlockTreeOrdsPostingsFormat.java | 4 +-
.../codecs/blocktreeords/FSTOrdsOutputs.java | 4 +-
.../blocktreeords/OrdsBlockTreeTermsReader.java | 4 +-
.../blocktreeords/OrdsBlockTreeTermsWriter.java | 4 +-
.../codecs/blocktreeords/OrdsFieldReader.java | 4 +-
.../blocktreeords/OrdsIntersectTermsEnum.java | 4 +-
.../OrdsIntersectTermsEnumFrame.java | 4 +-
.../blocktreeords/OrdsSegmentTermsEnum.java | 4 +-
.../OrdsSegmentTermsEnumFrame.java | 4 +-
.../lucene/codecs/bloom/BloomFilterFactory.java | 2 +-
.../bloom/BloomFilteringPostingsFormat.java | 4 +-
.../codecs/bloom/DefaultBloomFilterFactory.java | 2 +-
.../apache/lucene/codecs/bloom/FuzzySet.java | 4 +-
.../lucene/codecs/bloom/HashFunction.java | 2 +-
.../apache/lucene/codecs/bloom/MurmurHash2.java | 2 +-
.../codecs/memory/DirectDocValuesConsumer.java | 4 +-
.../codecs/memory/DirectDocValuesFormat.java | 4 +-
.../codecs/memory/DirectDocValuesProducer.java | 4 +-
.../codecs/memory/DirectPostingsFormat.java | 4 +-
.../codecs/memory/FSTOrdPostingsFormat.java | 6 +-
.../lucene/codecs/memory/FSTOrdTermsReader.java | 4 +-
.../lucene/codecs/memory/FSTOrdTermsWriter.java | 4 +-
.../lucene/codecs/memory/FSTPostingsFormat.java | 6 +-
.../lucene/codecs/memory/FSTTermOutputs.java | 4 +-
.../lucene/codecs/memory/FSTTermsReader.java | 4 +-
.../lucene/codecs/memory/FSTTermsWriter.java | 4 +-
.../codecs/memory/MemoryDocValuesConsumer.java | 4 +-
.../codecs/memory/MemoryDocValuesFormat.java | 4 +-
.../codecs/memory/MemoryDocValuesProducer.java | 4 +-
.../codecs/memory/MemoryPostingsFormat.java | 4 +-
.../codecs/simpletext/SimpleTextBKDReader.java | 4 +-
.../codecs/simpletext/SimpleTextCodec.java | 4 +-
.../simpletext/SimpleTextCompoundFormat.java | 4 +-
.../simpletext/SimpleTextDocValuesFormat.java | 4 +-
.../simpletext/SimpleTextDocValuesReader.java | 4 +-
.../simpletext/SimpleTextDocValuesWriter.java | 4 +-
.../simpletext/SimpleTextFieldInfosFormat.java | 4 +-
.../simpletext/SimpleTextFieldsReader.java | 4 +-
.../simpletext/SimpleTextFieldsWriter.java | 4 +-
.../simpletext/SimpleTextLiveDocsFormat.java | 4 +-
.../simpletext/SimpleTextNormsFormat.java | 4 +-
.../simpletext/SimpleTextPointFormat.java | 4 +-
.../simpletext/SimpleTextPointReader.java | 28 +-
.../simpletext/SimpleTextPointWriter.java | 47 +-
.../simpletext/SimpleTextPostingsFormat.java | 4 +-
.../simpletext/SimpleTextSegmentInfoFormat.java | 4 +-
.../SimpleTextStoredFieldsFormat.java | 4 +-
.../SimpleTextStoredFieldsReader.java | 4 +-
.../SimpleTextStoredFieldsWriter.java | 4 +-
.../simpletext/SimpleTextTermVectorsFormat.java | 4 +-
.../simpletext/SimpleTextTermVectorsReader.java | 4 +-
.../simpletext/SimpleTextTermVectorsWriter.java | 4 +-
.../codecs/simpletext/SimpleTextUtil.java | 4 +-
.../TestAutoPrefixPostingsFormat.java | 4 +-
.../codecs/autoprefix/TestAutoPrefixTerms.java | 4 +-
.../blockterms/TestFixedGapPostingsFormat.java | 4 +-
...TestVarGapDocFreqIntervalPostingsFormat.java | 4 +-
.../TestVarGapFixedIntervalPostingsFormat.java | 4 +-
.../codecs/blocktreeords/TestOrdsBlockTree.java | 4 +-
.../codecs/bloom/TestBloomPostingsFormat.java | 4 +-
.../memory/TestDirectDocValuesFormat.java | 4 +-
.../codecs/memory/TestDirectPostingsFormat.java | 4 +-
.../codecs/memory/TestFSTOrdPostingsFormat.java | 4 +-
.../codecs/memory/TestFSTPostingsFormat.java | 4 +-
.../memory/TestMemoryDocValuesFormat.java | 4 +-
.../codecs/memory/TestMemoryPostingsFormat.java | 4 +-
.../TestSimpleTextCompoundFormat.java | 4 +-
.../TestSimpleTextDocValuesFormat.java | 4 +-
.../TestSimpleTextFieldInfoFormat.java | 4 +-
.../simpletext/TestSimpleTextNormsFormat.java | 4 +-
.../simpletext/TestSimpleTextPointFormat.java | 33 +
.../TestSimpleTextPostingsFormat.java | 4 +-
.../TestSimpleTextSegmentInfoFormat.java | 4 +-
.../TestSimpleTextStoredFieldsFormat.java | 4 +-
.../TestSimpleTextTermVectorsFormat.java | 4 +-
lucene/common-build.xml | 35 +-
.../java/org/apache/lucene/LucenePackage.java | 4 +-
.../org/apache/lucene/analysis/Analyzer.java | 4 +-
.../apache/lucene/analysis/AnalyzerWrapper.java | 4 +-
.../lucene/analysis/CachingTokenFilter.java | 4 +-
.../org/apache/lucene/analysis/CharFilter.java | 4 +-
.../analysis/DelegatingAnalyzerWrapper.java | 8 +-
.../analysis/LegacyNumericTokenStream.java | 4 +-
.../lucene/analysis/ReusableStringReader.java | 8 +-
.../java/org/apache/lucene/analysis/Token.java | 4 +-
.../org/apache/lucene/analysis/TokenFilter.java | 4 +-
.../org/apache/lucene/analysis/TokenStream.java | 4 +-
.../lucene/analysis/TokenStreamToAutomaton.java | 4 +-
.../org/apache/lucene/analysis/Tokenizer.java | 4 +-
.../tokenattributes/BytesTermAttribute.java | 4 +-
.../tokenattributes/BytesTermAttributeImpl.java | 4 +-
.../tokenattributes/CharTermAttribute.java | 4 +-
.../tokenattributes/CharTermAttributeImpl.java | 4 +-
.../tokenattributes/FlagsAttribute.java | 4 +-
.../tokenattributes/FlagsAttributeImpl.java | 4 +-
.../tokenattributes/KeywordAttribute.java | 4 +-
.../tokenattributes/KeywordAttributeImpl.java | 4 +-
.../tokenattributes/OffsetAttribute.java | 4 +-
.../tokenattributes/OffsetAttributeImpl.java | 4 +-
.../PackedTokenAttributeImpl.java | 4 +-
.../tokenattributes/PayloadAttribute.java | 4 +-
.../tokenattributes/PayloadAttributeImpl.java | 4 +-
.../PositionIncrementAttribute.java | 4 +-
.../PositionIncrementAttributeImpl.java | 4 +-
.../PositionLengthAttribute.java | 4 +-
.../PositionLengthAttributeImpl.java | 4 +-
.../TermToBytesRefAttribute.java | 4 +-
.../analysis/tokenattributes/TypeAttribute.java | 4 +-
.../tokenattributes/TypeAttributeImpl.java | 4 +-
.../apache/lucene/codecs/BlockTermState.java | 2 +-
.../java/org/apache/lucene/codecs/Codec.java | 4 +-
.../org/apache/lucene/codecs/CodecUtil.java | 6 +-
.../apache/lucene/codecs/CompoundFormat.java | 4 +-
.../apache/lucene/codecs/DocValuesConsumer.java | 4 +-
.../apache/lucene/codecs/DocValuesFormat.java | 4 +-
.../apache/lucene/codecs/DocValuesProducer.java | 4 +-
.../apache/lucene/codecs/FieldInfosFormat.java | 4 +-
.../apache/lucene/codecs/FieldsConsumer.java | 4 +-
.../apache/lucene/codecs/FieldsProducer.java | 4 +-
.../org/apache/lucene/codecs/FilterCodec.java | 4 +-
.../apache/lucene/codecs/LiveDocsFormat.java | 4 +-
.../lucene/codecs/MultiLevelSkipListReader.java | 4 +-
.../lucene/codecs/MultiLevelSkipListWriter.java | 4 +-
.../org/apache/lucene/codecs/NormsConsumer.java | 4 +-
.../org/apache/lucene/codecs/NormsFormat.java | 4 +-
.../org/apache/lucene/codecs/NormsProducer.java | 18 +-
.../org/apache/lucene/codecs/PointFormat.java | 4 +-
.../org/apache/lucene/codecs/PointReader.java | 4 +-
.../org/apache/lucene/codecs/PointWriter.java | 8 +-
.../apache/lucene/codecs/PostingsFormat.java | 4 +-
.../lucene/codecs/PostingsReaderBase.java | 4 +-
.../lucene/codecs/PostingsWriterBase.java | 4 +-
.../lucene/codecs/PushPostingsWriterBase.java | 4 +-
.../apache/lucene/codecs/SegmentInfoFormat.java | 4 +-
.../lucene/codecs/StoredFieldsFormat.java | 4 +-
.../lucene/codecs/StoredFieldsReader.java | 24 +-
.../lucene/codecs/StoredFieldsWriter.java | 26 +-
.../org/apache/lucene/codecs/TermStats.java | 4 +-
.../apache/lucene/codecs/TermVectorsFormat.java | 4 +-
.../apache/lucene/codecs/TermVectorsReader.java | 4 +-
.../apache/lucene/codecs/TermVectorsWriter.java | 4 +-
.../codecs/blocktree/AutoPrefixTermsWriter.java | 4 +-
.../codecs/blocktree/BitSetPostingsEnum.java | 4 +-
.../codecs/blocktree/BitSetTermsEnum.java | 4 +-
.../codecs/blocktree/BlockTreeTermsReader.java | 4 +-
.../codecs/blocktree/BlockTreeTermsWriter.java | 4 +-
.../lucene/codecs/blocktree/FieldReader.java | 4 +-
.../codecs/blocktree/IntersectTermsEnum.java | 4 +-
.../blocktree/IntersectTermsEnumFrame.java | 4 +-
.../codecs/blocktree/SegmentTermsEnum.java | 4 +-
.../codecs/blocktree/SegmentTermsEnumFrame.java | 4 +-
.../apache/lucene/codecs/blocktree/Stats.java | 4 +-
.../CompressingStoredFieldsFormat.java | 4 +-
.../CompressingStoredFieldsIndexReader.java | 4 +-
.../CompressingStoredFieldsIndexWriter.java | 4 +-
.../CompressingStoredFieldsReader.java | 4 +-
.../CompressingStoredFieldsWriter.java | 4 +-
.../CompressingTermVectorsFormat.java | 4 +-
.../CompressingTermVectorsReader.java | 4 +-
.../CompressingTermVectorsWriter.java | 4 +-
.../codecs/compressing/CompressionMode.java | 4 +-
.../lucene/codecs/compressing/Compressor.java | 4 +-
.../lucene/codecs/compressing/Decompressor.java | 4 +-
.../GrowableByteArrayDataOutput.java | 4 +-
.../apache/lucene/codecs/compressing/LZ4.java | 4 +-
.../codecs/compressing/MatchingReaders.java | 4 +-
.../apache/lucene/codecs/lucene50/ForUtil.java | 2 +-
.../codecs/lucene50/Lucene50CompoundFormat.java | 4 +-
.../codecs/lucene50/Lucene50CompoundReader.java | 32 +-
.../lucene50/Lucene50FieldInfosFormat.java | 4 +-
.../codecs/lucene50/Lucene50LiveDocsFormat.java | 4 +-
.../codecs/lucene50/Lucene50PostingsFormat.java | 6 +-
.../codecs/lucene50/Lucene50PostingsReader.java | 4 +-
.../codecs/lucene50/Lucene50PostingsWriter.java | 4 +-
.../lucene50/Lucene50SegmentInfoFormat.java | 4 +-
.../codecs/lucene50/Lucene50SkipReader.java | 4 +-
.../codecs/lucene50/Lucene50SkipWriter.java | 4 +-
.../lucene50/Lucene50StoredFieldsFormat.java | 4 +-
.../lucene50/Lucene50TermVectorsFormat.java | 4 +-
.../codecs/lucene53/Lucene53NormsConsumer.java | 4 +-
.../codecs/lucene53/Lucene53NormsFormat.java | 4 +-
.../codecs/lucene53/Lucene53NormsProducer.java | 4 +-
.../lucene54/Lucene54DocValuesConsumer.java | 4 +-
.../lucene54/Lucene54DocValuesFormat.java | 4 +-
.../lucene54/Lucene54DocValuesProducer.java | 4 +-
.../lucene/codecs/lucene60/Lucene60Codec.java | 4 +-
.../lucene60/Lucene60FieldInfosFormat.java | 4 +-
.../codecs/lucene60/Lucene60PointFormat.java | 7 +-
.../codecs/lucene60/Lucene60PointReader.java | 68 +-
.../codecs/lucene60/Lucene60PointWriter.java | 91 +-
.../perfield/PerFieldDocValuesFormat.java | 4 +-
.../codecs/perfield/PerFieldPostingsFormat.java | 4 +-
.../lucene/document/BinaryDocValuesField.java | 4 +-
.../org/apache/lucene/document/BinaryPoint.java | 4 +-
.../lucene/document/CompressionTools.java | 4 +-
.../org/apache/lucene/document/DateTools.java | 4 +-
.../org/apache/lucene/document/Document.java | 4 +-
.../document/DocumentStoredFieldVisitor.java | 4 +-
.../lucene/document/DoubleDocValuesField.java | 4 +-
.../org/apache/lucene/document/DoublePoint.java | 10 +-
.../java/org/apache/lucene/document/Field.java | 7 +-
.../org/apache/lucene/document/FieldType.java | 4 +-
.../lucene/document/FloatDocValuesField.java | 4 +-
.../org/apache/lucene/document/FloatPoint.java | 10 +-
.../org/apache/lucene/document/IntPoint.java | 10 +-
.../lucene/document/LegacyDoubleField.java | 4 +-
.../lucene/document/LegacyFloatField.java | 4 +-
.../apache/lucene/document/LegacyIntField.java | 4 +-
.../apache/lucene/document/LegacyLongField.java | 4 +-
.../org/apache/lucene/document/LongPoint.java | 10 +-
.../lucene/document/NumericDocValuesField.java | 4 +-
.../lucene/document/SortedDocValuesField.java | 4 +-
.../document/SortedNumericDocValuesField.java | 4 +-
.../document/SortedSetDocValuesField.java | 4 +-
.../org/apache/lucene/document/StoredField.java | 4 +-
.../org/apache/lucene/document/StringField.java | 4 +-
.../org/apache/lucene/document/TextField.java | 4 +-
.../apache/lucene/index/AbortingException.java | 4 +-
.../apache/lucene/index/AutomatonTermsEnum.java | 4 +-
.../lucene/index/BaseCompositeReader.java | 4 +-
.../apache/lucene/index/BinaryDocValues.java | 4 +-
.../index/BinaryDocValuesFieldUpdates.java | 4 +-
.../lucene/index/BinaryDocValuesWriter.java | 4 +-
.../java/org/apache/lucene/index/BitsSlice.java | 8 +-
.../apache/lucene/index/BufferedUpdates.java | 4 +-
.../lucene/index/BufferedUpdatesStream.java | 4 +-
.../apache/lucene/index/ByteSliceReader.java | 4 +-
.../apache/lucene/index/ByteSliceWriter.java | 10 +-
.../org/apache/lucene/index/CheckIndex.java | 4 +-
.../apache/lucene/index/CoalescedUpdates.java | 4 +-
.../org/apache/lucene/index/CodecReader.java | 4 +-
.../apache/lucene/index/CompositeReader.java | 4 +-
.../lucene/index/CompositeReaderContext.java | 4 +-
.../lucene/index/ConcurrentMergeScheduler.java | 4 +-
.../lucene/index/CorruptIndexException.java | 4 +-
.../lucene/index/DefaultIndexingChain.java | 33 +-
.../apache/lucene/index/DirectoryReader.java | 4 +-
.../org/apache/lucene/index/DocConsumer.java | 4 +-
.../java/org/apache/lucene/index/DocValues.java | 4 +-
.../lucene/index/DocValuesFieldUpdates.java | 4 +-
.../org/apache/lucene/index/DocValuesType.java | 4 +-
.../apache/lucene/index/DocValuesUpdate.java | 4 +-
.../apache/lucene/index/DocValuesWriter.java | 4 +-
.../apache/lucene/index/DocumentsWriter.java | 4 +-
.../index/DocumentsWriterDeleteQueue.java | 27 +-
.../index/DocumentsWriterFlushControl.java | 10 +-
.../lucene/index/DocumentsWriterFlushQueue.java | 28 +-
.../lucene/index/DocumentsWriterPerThread.java | 10 +-
.../index/DocumentsWriterPerThreadPool.java | 8 +-
.../index/DocumentsWriterStallControl.java | 10 +-
.../lucene/index/ExitableDirectoryReader.java | 4 +-
.../java/org/apache/lucene/index/FieldInfo.java | 4 +-
.../org/apache/lucene/index/FieldInfos.java | 4 +-
.../apache/lucene/index/FieldTermIterator.java | 4 +-
.../java/org/apache/lucene/index/Fields.java | 4 +-
.../apache/lucene/index/FilterCodecReader.java | 4 +-
.../lucene/index/FilterDirectoryReader.java | 4 +-
.../apache/lucene/index/FilterLeafReader.java | 4 +-
.../apache/lucene/index/FilteredTermsEnum.java | 4 +-
.../lucene/index/FlushByRamOrCountsPolicy.java | 10 +-
.../org/apache/lucene/index/FlushPolicy.java | 10 +-
.../org/apache/lucene/index/FreqProxFields.java | 4 +-
.../lucene/index/FreqProxTermsWriter.java | 4 +-
.../index/FreqProxTermsWriterPerField.java | 4 +-
.../lucene/index/FrozenBufferedUpdates.java | 4 +-
.../org/apache/lucene/index/IndexCommit.java | 5 +-
.../lucene/index/IndexDeletionPolicy.java | 4 +-
.../apache/lucene/index/IndexFileDeleter.java | 141 +--
.../org/apache/lucene/index/IndexFileNames.java | 4 +-
.../index/IndexFormatTooNewException.java | 1 -
.../index/IndexFormatTooOldException.java | 1 -
.../lucene/index/IndexNotFoundException.java | 4 +-
.../org/apache/lucene/index/IndexOptions.java | 4 +-
.../org/apache/lucene/index/IndexReader.java | 8 +-
.../apache/lucene/index/IndexReaderContext.java | 4 +-
.../org/apache/lucene/index/IndexUpgrader.java | 4 +-
.../org/apache/lucene/index/IndexWriter.java | 18 +-
.../apache/lucene/index/IndexWriterConfig.java | 4 +-
.../org/apache/lucene/index/IndexableField.java | 12 +-
.../apache/lucene/index/IndexableFieldType.java | 4 +-
.../index/KeepOnlyLastCommitDeletionPolicy.java | 4 +-
.../org/apache/lucene/index/LeafReader.java | 4 +-
.../apache/lucene/index/LeafReaderContext.java | 4 +-
.../lucene/index/LiveIndexWriterConfig.java | 4 +-
.../lucene/index/LogByteSizeMergePolicy.java | 4 +-
.../apache/lucene/index/LogDocMergePolicy.java | 8 +-
.../org/apache/lucene/index/LogMergePolicy.java | 4 +-
.../apache/lucene/index/MappedMultiFields.java | 4 +-
.../lucene/index/MappingMultiPostingsEnum.java | 4 +-
.../org/apache/lucene/index/MergePolicy.java | 10 +-
.../apache/lucene/index/MergePolicyWrapper.java | 89 ++
.../apache/lucene/index/MergeRateLimiter.java | 4 +-
.../org/apache/lucene/index/MergeScheduler.java | 4 +-
.../org/apache/lucene/index/MergeState.java | 4 +-
.../org/apache/lucene/index/MergeTrigger.java | 4 +-
.../index/MergedPrefixCodedTermsIterator.java | 4 +-
.../java/org/apache/lucene/index/MultiBits.java | 8 +-
.../org/apache/lucene/index/MultiDocValues.java | 4 +-
.../org/apache/lucene/index/MultiFields.java | 4 +-
.../apache/lucene/index/MultiPointValues.java | 8 +-
.../apache/lucene/index/MultiPostingsEnum.java | 4 +-
.../org/apache/lucene/index/MultiReader.java | 4 +-
.../org/apache/lucene/index/MultiTerms.java | 4 +-
.../org/apache/lucene/index/MultiTermsEnum.java | 4 +-
.../apache/lucene/index/NoDeletionPolicy.java | 4 +-
.../org/apache/lucene/index/NoMergePolicy.java | 22 +-
.../apache/lucene/index/NoMergeScheduler.java | 4 +-
.../apache/lucene/index/NormValuesWriter.java | 4 +-
.../apache/lucene/index/NumericDocValues.java | 4 +-
.../index/NumericDocValuesFieldUpdates.java | 18 +-
.../lucene/index/NumericDocValuesWriter.java | 4 +-
.../org/apache/lucene/index/OrdTermState.java | 4 +-
.../lucene/index/ParallelCompositeReader.java | 4 +-
.../apache/lucene/index/ParallelLeafReader.java | 4 +-
.../lucene/index/ParallelPostingsArray.java | 4 +-
.../index/PersistentSnapshotDeletionPolicy.java | 29 +-
.../org/apache/lucene/index/PointValues.java | 12 +-
.../apache/lucene/index/PointValuesWriter.java | 4 +-
.../org/apache/lucene/index/PostingsEnum.java | 4 +-
.../apache/lucene/index/PrefixCodedTerms.java | 25 +-
.../org/apache/lucene/index/QueryTimeout.java | 4 +-
.../apache/lucene/index/QueryTimeoutImpl.java | 4 +-
.../apache/lucene/index/RandomAccessOrds.java | 4 +-
.../org/apache/lucene/index/ReaderManager.java | 4 +-
.../org/apache/lucene/index/ReaderSlice.java | 4 +-
.../org/apache/lucene/index/ReaderUtil.java | 4 +-
.../apache/lucene/index/ReadersAndUpdates.java | 4 +-
.../apache/lucene/index/SegmentCommitInfo.java | 4 +-
.../apache/lucene/index/SegmentCoreReaders.java | 8 +-
.../apache/lucene/index/SegmentDocValues.java | 4 +-
.../lucene/index/SegmentDocValuesProducer.java | 4 +-
.../org/apache/lucene/index/SegmentInfo.java | 4 +-
.../org/apache/lucene/index/SegmentInfos.java | 4 +-
.../org/apache/lucene/index/SegmentMerger.java | 4 +-
.../apache/lucene/index/SegmentReadState.java | 4 +-
.../org/apache/lucene/index/SegmentReader.java | 4 +-
.../apache/lucene/index/SegmentWriteState.java | 4 +-
.../lucene/index/SerialMergeScheduler.java | 4 +-
.../lucene/index/SimpleMergedSegmentWarmer.java | 4 +-
.../apache/lucene/index/SingleTermsEnum.java | 4 +-
.../index/SingletonSortedNumericDocValues.java | 4 +-
.../index/SingletonSortedSetDocValues.java | 4 +-
.../lucene/index/SlowCodecReaderWrapper.java | 4 +-
.../index/SlowCompositeReaderWrapper.java | 4 +-
.../lucene/index/SnapshotDeletionPolicy.java | 4 +-
.../apache/lucene/index/SortedDocValues.java | 4 +-
.../lucene/index/SortedDocValuesTermsEnum.java | 4 +-
.../lucene/index/SortedDocValuesWriter.java | 4 +-
.../lucene/index/SortedNumericDocValues.java | 4 +-
.../index/SortedNumericDocValuesWriter.java | 4 +-
.../apache/lucene/index/SortedSetDocValues.java | 4 +-
.../index/SortedSetDocValuesTermsEnum.java | 4 +-
.../lucene/index/SortedSetDocValuesWriter.java | 4 +-
.../lucene/index/StandardDirectoryReader.java | 4 +-
.../apache/lucene/index/StoredFieldVisitor.java | 4 +-
.../src/java/org/apache/lucene/index/Term.java | 4 +-
.../org/apache/lucene/index/TermContext.java | 4 +-
.../java/org/apache/lucene/index/TermState.java | 4 +-
.../lucene/index/TermVectorsConsumer.java | 4 +-
.../index/TermVectorsConsumerPerField.java | 4 +-
.../src/java/org/apache/lucene/index/Terms.java | 4 +-
.../java/org/apache/lucene/index/TermsEnum.java | 4 +-
.../java/org/apache/lucene/index/TermsHash.java | 4 +-
.../apache/lucene/index/TermsHashPerField.java | 4 +-
.../apache/lucene/index/TieredMergePolicy.java | 8 +-
.../lucene/index/TrackingIndexWriter.java | 4 +-
.../org/apache/lucene/index/TwoPhaseCommit.java | 8 +-
.../apache/lucene/index/TwoPhaseCommitTool.java | 8 +-
.../lucene/index/UpgradeIndexMergePolicy.java | 34 +-
.../apache/lucene/search/AutomatonQuery.java | 4 +-
.../apache/lucene/search/BlendedTermQuery.java | 4 +-
.../org/apache/lucene/search/BooleanClause.java | 4 +-
.../org/apache/lucene/search/BooleanQuery.java | 4 +-
.../org/apache/lucene/search/BooleanScorer.java | 4 +-
.../lucene/search/BooleanTopLevelScorers.java | 4 +-
.../org/apache/lucene/search/BooleanWeight.java | 4 +-
.../apache/lucene/search/BoostAttribute.java | 4 +-
.../lucene/search/BoostAttributeImpl.java | 4 +-
.../org/apache/lucene/search/BoostQuery.java | 4 +-
.../org/apache/lucene/search/BulkScorer.java | 4 +-
.../apache/lucene/search/CachingCollector.java | 4 +-
.../lucene/search/CollectionStatistics.java | 10 +-
.../search/CollectionTerminatedException.java | 4 +-
.../org/apache/lucene/search/Collector.java | 4 +-
.../apache/lucene/search/CollectorManager.java | 4 +-
.../apache/lucene/search/ConjunctionDISI.java | 4 +-
.../apache/lucene/search/ConjunctionScorer.java | 4 +-
.../lucene/search/ConstantScoreQuery.java | 4 +-
.../lucene/search/ConstantScoreScorer.java | 4 +-
.../lucene/search/ConstantScoreWeight.java | 4 +-
.../search/ControlledRealTimeReopenThread.java | 4 +-
.../apache/lucene/search/DisiPriorityQueue.java | 4 +-
.../org/apache/lucene/search/DisiWrapper.java | 4 +-
.../search/DisjunctionDISIApproximation.java | 2 +-
.../lucene/search/DisjunctionMaxQuery.java | 14 +-
.../lucene/search/DisjunctionMaxScorer.java | 14 +-
.../apache/lucene/search/DisjunctionScorer.java | 4 +-
.../lucene/search/DisjunctionSumScorer.java | 4 +-
.../java/org/apache/lucene/search/DocIdSet.java | 4 +-
.../apache/lucene/search/DocIdSetIterator.java | 4 +-
.../apache/lucene/search/DocValuesDocIdSet.java | 2 +-
.../lucene/search/DocValuesRewriteMethod.java | 4 +-
.../apache/lucene/search/ExactPhraseScorer.java | 4 +-
.../apache/lucene/search/ExactPointQuery.java | 4 +-
.../org/apache/lucene/search/Explanation.java | 4 +-
.../org/apache/lucene/search/FakeScorer.java | 4 +-
.../apache/lucene/search/FieldComparator.java | 4 +-
.../lucene/search/FieldComparatorSource.java | 4 +-
.../java/org/apache/lucene/search/FieldDoc.java | 4 +-
.../lucene/search/FieldValueHitQueue.java | 4 +-
.../apache/lucene/search/FieldValueQuery.java | 4 +-
.../apache/lucene/search/FilterCollector.java | 12 +-
.../lucene/search/FilterLeafCollector.java | 4 +-
.../org/apache/lucene/search/FilterScorer.java | 4 +-
.../lucene/search/FilteredDocIdSetIterator.java | 4 +-
.../org/apache/lucene/search/FuzzyQuery.java | 4 +-
.../apache/lucene/search/FuzzyTermsEnum.java | 4 +-
.../java/org/apache/lucene/search/HitQueue.java | 4 +-
.../org/apache/lucene/search/IndexSearcher.java | 4 +-
.../org/apache/lucene/search/LRUQueryCache.java | 4 +-
.../org/apache/lucene/search/LeafCollector.java | 4 +-
.../lucene/search/LeafFieldComparator.java | 4 +-
.../lucene/search/LegacyNumericRangeQuery.java | 4 +-
.../apache/lucene/search/LiveFieldValues.java | 4 +-
.../apache/lucene/search/MatchAllDocsQuery.java | 4 +-
.../apache/lucene/search/MatchNoDocsQuery.java | 4 +-
.../search/MaxNonCompetitiveBoostAttribute.java | 4 +-
.../MaxNonCompetitiveBoostAttributeImpl.java | 4 +-
.../lucene/search/MinShouldMatchSumScorer.java | 4 +-
.../apache/lucene/search/MultiCollector.java | 4 +-
.../apache/lucene/search/MultiPhraseQuery.java | 4 +-
.../apache/lucene/search/MultiTermQuery.java | 4 +-
.../MultiTermQueryConstantScoreWrapper.java | 4 +-
.../java/org/apache/lucene/search/Multiset.java | 4 +-
.../apache/lucene/search/NGramPhraseQuery.java | 4 +-
.../apache/lucene/search/PhrasePositions.java | 4 +-
.../org/apache/lucene/search/PhraseQuery.java | 4 +-
.../org/apache/lucene/search/PhraseQueue.java | 4 +-
.../apache/lucene/search/PointRangeQuery.java | 4 +-
.../search/PositiveScoresOnlyCollector.java | 4 +-
.../org/apache/lucene/search/PrefixQuery.java | 4 +-
.../java/org/apache/lucene/search/Query.java | 4 +-
.../org/apache/lucene/search/QueryCache.java | 4 +-
.../lucene/search/QueryCachingPolicy.java | 4 +-
.../org/apache/lucene/search/QueryRescorer.java | 4 +-
.../lucene/search/RandomAccessWeight.java | 4 +-
.../apache/lucene/search/ReferenceManager.java | 4 +-
.../org/apache/lucene/search/RegexpQuery.java | 4 +-
.../apache/lucene/search/ReqExclBulkScorer.java | 4 +-
.../org/apache/lucene/search/ReqExclScorer.java | 4 +-
.../apache/lucene/search/ReqOptSumScorer.java | 2 +-
.../java/org/apache/lucene/search/Rescorer.java | 4 +-
.../search/ScoreCachingWrappingScorer.java | 4 +-
.../java/org/apache/lucene/search/ScoreDoc.java | 4 +-
.../java/org/apache/lucene/search/Scorer.java | 4 +-
.../apache/lucene/search/ScoringRewrite.java | 4 +-
.../apache/lucene/search/SearcherFactory.java | 4 +-
.../lucene/search/SearcherLifetimeManager.java | 4 +-
.../apache/lucene/search/SearcherManager.java | 4 +-
.../apache/lucene/search/SimpleCollector.java | 4 +-
.../lucene/search/SimpleFieldComparator.java | 4 +-
.../lucene/search/SloppyPhraseScorer.java | 4 +-
.../src/java/org/apache/lucene/search/Sort.java | 4 +-
.../org/apache/lucene/search/SortField.java | 4 +-
.../org/apache/lucene/search/SortRescorer.java | 4 +-
.../lucene/search/SortedNumericSelector.java | 4 +-
.../lucene/search/SortedNumericSortField.java | 4 +-
.../apache/lucene/search/SortedSetSelector.java | 4 +-
.../lucene/search/SortedSetSortField.java | 4 +-
.../org/apache/lucene/search/SynonymQuery.java | 4 +-
.../lucene/search/TermCollectingRewrite.java | 4 +-
.../org/apache/lucene/search/TermQuery.java | 4 +-
.../apache/lucene/search/TermRangeQuery.java | 4 +-
.../org/apache/lucene/search/TermScorer.java | 4 +-
.../apache/lucene/search/TermStatistics.java | 4 +-
.../lucene/search/TimeLimitingCollector.java | 4 +-
.../java/org/apache/lucene/search/TopDocs.java | 4 +-
.../apache/lucene/search/TopDocsCollector.java | 4 +-
.../apache/lucene/search/TopFieldCollector.java | 4 +-
.../org/apache/lucene/search/TopFieldDocs.java | 4 +-
.../lucene/search/TopScoreDocCollector.java | 4 +-
.../apache/lucene/search/TopTermsRewrite.java | 4 +-
.../lucene/search/TotalHitCountCollector.java | 4 +-
.../apache/lucene/search/TwoPhaseIterator.java | 4 +-
.../search/UsageTrackingQueryCachingPolicy.java | 4 +-
.../java/org/apache/lucene/search/Weight.java | 4 +-
.../org/apache/lucene/search/WildcardQuery.java | 4 +-
.../lucene/search/similarities/AfterEffect.java | 4 +-
.../search/similarities/AfterEffectB.java | 4 +-
.../search/similarities/AfterEffectL.java | 4 +-
.../search/similarities/BM25Similarity.java | 4 +-
.../lucene/search/similarities/BasicModel.java | 4 +-
.../search/similarities/BasicModelBE.java | 4 +-
.../lucene/search/similarities/BasicModelD.java | 4 +-
.../lucene/search/similarities/BasicModelG.java | 4 +-
.../search/similarities/BasicModelIF.java | 4 +-
.../search/similarities/BasicModelIn.java | 4 +-
.../search/similarities/BasicModelIne.java | 4 +-
.../lucene/search/similarities/BasicModelP.java | 4 +-
.../lucene/search/similarities/BasicStats.java | 4 +-
.../search/similarities/ClassicSimilarity.java | 4 +-
.../search/similarities/DFISimilarity.java | 4 +-
.../search/similarities/DFRSimilarity.java | 4 +-
.../search/similarities/Distribution.java | 4 +-
.../search/similarities/DistributionLL.java | 4 +-
.../search/similarities/DistributionSPL.java | 4 +-
.../search/similarities/IBSimilarity.java | 4 +-
.../search/similarities/Independence.java | 4 +-
.../similarities/IndependenceChiSquared.java | 4 +-
.../similarities/IndependenceSaturated.java | 4 +-
.../similarities/IndependenceStandardized.java | 4 +-
.../similarities/LMDirichletSimilarity.java | 4 +-
.../similarities/LMJelinekMercerSimilarity.java | 4 +-
.../search/similarities/LMSimilarity.java | 4 +-
.../lucene/search/similarities/Lambda.java | 4 +-
.../lucene/search/similarities/LambdaDF.java | 4 +-
.../lucene/search/similarities/LambdaTTF.java | 4 +-
.../search/similarities/MultiSimilarity.java | 4 +-
.../search/similarities/Normalization.java | 4 +-
.../search/similarities/NormalizationH1.java | 4 +-
.../search/similarities/NormalizationH2.java | 4 +-
.../search/similarities/NormalizationH3.java | 4 +-
.../search/similarities/NormalizationZ.java | 4 +-
.../similarities/PerFieldSimilarityWrapper.java | 4 +-
.../lucene/search/similarities/Similarity.java | 4 +-
.../search/similarities/SimilarityBase.java | 4 +-
.../search/similarities/TFIDFSimilarity.java | 4 +-
.../lucene/search/spans/ConjunctionSpans.java | 4 +-
.../lucene/search/spans/ContainSpans.java | 4 +-
.../search/spans/FieldMaskingSpanQuery.java | 4 +-
.../apache/lucene/search/spans/FilterSpans.java | 4 +-
.../lucene/search/spans/NearSpansOrdered.java | 4 +-
.../lucene/search/spans/NearSpansUnordered.java | 4 +-
.../search/spans/ScoringWrapperSpans.java | 4 +-
.../lucene/search/spans/SpanBoostQuery.java | 4 +-
.../lucene/search/spans/SpanCollector.java | 4 +-
.../lucene/search/spans/SpanContainQuery.java | 4 +-
.../search/spans/SpanContainingQuery.java | 4 +-
.../lucene/search/spans/SpanFirstQuery.java | 4 +-
.../search/spans/SpanMultiTermQueryWrapper.java | 4 +-
.../lucene/search/spans/SpanNearQuery.java | 4 +-
.../lucene/search/spans/SpanNotQuery.java | 4 +-
.../apache/lucene/search/spans/SpanOrQuery.java | 4 +-
.../search/spans/SpanPositionCheckQuery.java | 2 +-
.../lucene/search/spans/SpanPositionQueue.java | 4 +-
.../search/spans/SpanPositionRangeQuery.java | 2 +-
.../apache/lucene/search/spans/SpanQuery.java | 4 +-
.../apache/lucene/search/spans/SpanScorer.java | 4 +-
.../lucene/search/spans/SpanTermQuery.java | 4 +-
.../apache/lucene/search/spans/SpanWeight.java | 4 +-
.../lucene/search/spans/SpanWithinQuery.java | 4 +-
.../org/apache/lucene/search/spans/Spans.java | 4 +-
.../apache/lucene/search/spans/TermSpans.java | 14 +-
.../lucene/store/AlreadyClosedException.java | 4 +-
.../org/apache/lucene/store/BaseDirectory.java | 4 +-
.../apache/lucene/store/BufferedChecksum.java | 4 +-
.../store/BufferedChecksumIndexInput.java | 4 +-
.../apache/lucene/store/BufferedIndexInput.java | 4 +-
.../apache/lucene/store/ByteArrayDataInput.java | 4 +-
.../lucene/store/ByteArrayDataOutput.java | 4 +-
.../lucene/store/ByteBufferIndexInput.java | 4 +-
.../apache/lucene/store/ChecksumIndexInput.java | 8 +-
.../java/org/apache/lucene/store/DataInput.java | 4 +-
.../org/apache/lucene/store/DataOutput.java | 4 +-
.../java/org/apache/lucene/store/Directory.java | 7 +-
.../org/apache/lucene/store/FSDirectory.java | 152 ++-
.../org/apache/lucene/store/FSLockFactory.java | 4 +-
.../lucene/store/FileSwitchDirectory.java | 10 +-
.../apache/lucene/store/FilterDirectory.java | 4 +-
.../java/org/apache/lucene/store/FlushInfo.java | 4 +-
.../java/org/apache/lucene/store/IOContext.java | 4 +-
.../org/apache/lucene/store/IndexInput.java | 4 +-
.../org/apache/lucene/store/IndexOutput.java | 4 +-
.../lucene/store/InputStreamDataInput.java | 4 +-
.../src/java/org/apache/lucene/store/Lock.java | 4 +-
.../org/apache/lucene/store/LockFactory.java | 4 +-
.../lucene/store/LockObtainFailedException.java | 1 -
.../store/LockReleaseFailedException.java | 1 -
.../org/apache/lucene/store/LockStressTest.java | 4 +-
.../store/LockValidatingDirectoryWrapper.java | 4 +-
.../apache/lucene/store/LockVerifyServer.java | 4 +-
.../org/apache/lucene/store/MMapDirectory.java | 5 +-
.../java/org/apache/lucene/store/MergeInfo.java | 2 +-
.../org/apache/lucene/store/NIOFSDirectory.java | 29 +-
.../lucene/store/NRTCachingDirectory.java | 6 +-
.../lucene/store/NativeFSLockFactory.java | 4 +-
.../org/apache/lucene/store/NoLockFactory.java | 4 +-
.../lucene/store/OutputStreamDataOutput.java | 4 +-
.../lucene/store/OutputStreamIndexOutput.java | 4 +-
.../org/apache/lucene/store/RAMDirectory.java | 13 +-
.../java/org/apache/lucene/store/RAMFile.java | 4 +-
.../org/apache/lucene/store/RAMInputStream.java | 4 +-
.../apache/lucene/store/RAMOutputStream.java | 6 +-
.../apache/lucene/store/RandomAccessInput.java | 4 +-
.../lucene/store/RateLimitedIndexOutput.java | 4 +-
.../org/apache/lucene/store/RateLimiter.java | 4 +-
.../apache/lucene/store/SimpleFSDirectory.java | 5 +-
.../lucene/store/SimpleFSLockFactory.java | 4 +-
.../lucene/store/SingleInstanceLockFactory.java | 4 +-
.../lucene/store/SleepingLockWrapper.java | 4 +-
.../lucene/store/TrackingDirectoryWrapper.java | 4 +-
.../lucene/store/VerifyingLockFactory.java | 4 +-
.../org/apache/lucene/util/Accountable.java | 4 +-
.../org/apache/lucene/util/Accountables.java | 4 +-
.../lucene/util/ArrayInPlaceMergeSorter.java | 4 +-
.../apache/lucene/util/ArrayIntroSorter.java | 4 +-
.../org/apache/lucene/util/ArrayTimSorter.java | 4 +-
.../java/org/apache/lucene/util/ArrayUtil.java | 4 +-
.../java/org/apache/lucene/util/Attribute.java | 4 +-
.../apache/lucene/util/AttributeFactory.java | 4 +-
.../org/apache/lucene/util/AttributeImpl.java | 4 +-
.../apache/lucene/util/AttributeReflector.java | 4 +-
.../org/apache/lucene/util/AttributeSource.java | 4 +-
.../org/apache/lucene/util/BitDocIdSet.java | 4 +-
.../src/java/org/apache/lucene/util/BitSet.java | 4 +-
.../org/apache/lucene/util/BitSetIterator.java | 4 +-
.../java/org/apache/lucene/util/BitUtil.java | 1 -
.../src/java/org/apache/lucene/util/Bits.java | 4 +-
.../org/apache/lucene/util/ByteBlockPool.java | 4 +-
.../java/org/apache/lucene/util/BytesRef.java | 4 +-
.../org/apache/lucene/util/BytesRefArray.java | 27 +-
.../org/apache/lucene/util/BytesRefBuilder.java | 4 +-
.../org/apache/lucene/util/BytesRefHash.java | 4 +-
.../apache/lucene/util/BytesRefIterator.java | 4 +-
.../java/org/apache/lucene/util/CharsRef.java | 4 +-
.../org/apache/lucene/util/CharsRefBuilder.java | 4 +-
.../lucene/util/CloseableThreadLocal.java | 4 +-
.../org/apache/lucene/util/CollectionUtil.java | 4 +-
.../org/apache/lucene/util/CommandLineUtil.java | 4 +-
.../java/org/apache/lucene/util/Constants.java | 4 +-
.../java/org/apache/lucene/util/Counter.java | 8 +-
.../org/apache/lucene/util/DocIdSetBuilder.java | 4 +-
.../org/apache/lucene/util/FilterIterator.java | 27 +-
.../org/apache/lucene/util/FixedBitSet.java | 4 +-
.../util/FrequencyTrackingRingBuffer.java | 4 +-
.../java/org/apache/lucene/util/IOUtils.java | 12 +-
.../apache/lucene/util/InPlaceMergeSorter.java | 4 +-
.../java/org/apache/lucene/util/InfoStream.java | 4 +-
.../apache/lucene/util/IntArrayDocIdSet.java | 4 +-
.../org/apache/lucene/util/IntBlockPool.java | 4 +-
.../org/apache/lucene/util/IntroSorter.java | 4 +-
.../java/org/apache/lucene/util/IntsRef.java | 8 +-
.../org/apache/lucene/util/IntsRefBuilder.java | 4 +-
.../org/apache/lucene/util/LSBRadixSorter.java | 4 +-
.../apache/lucene/util/LegacyNumericUtils.java | 4 +-
.../java/org/apache/lucene/util/LongBitSet.java | 4 +-
.../java/org/apache/lucene/util/LongValues.java | 4 +-
.../java/org/apache/lucene/util/LongsRef.java | 8 +-
.../java/org/apache/lucene/util/MapOfSets.java | 4 +-
.../java/org/apache/lucene/util/MathUtil.java | 4 +-
.../org/apache/lucene/util/MergedIterator.java | 4 +-
.../org/apache/lucene/util/MutableBits.java | 4 +-
.../org/apache/lucene/util/NamedSPILoader.java | 4 +-
.../apache/lucene/util/NamedThreadFactory.java | 4 +-
.../org/apache/lucene/util/NotDocIdSet.java | 4 +-
.../org/apache/lucene/util/NumericUtils.java | 4 +-
.../org/apache/lucene/util/OfflineSorter.java | 4 +-
.../java/org/apache/lucene/util/PagedBytes.java | 4 +-
.../lucene/util/PrintStreamInfoStream.java | 4 +-
.../org/apache/lucene/util/PriorityQueue.java | 10 +-
.../org/apache/lucene/util/QueryBuilder.java | 4 +-
.../apache/lucene/util/RamUsageEstimator.java | 4 +-
.../util/RecyclingByteBlockAllocator.java | 8 +-
.../lucene/util/RecyclingIntBlockAllocator.java | 8 +-
.../java/org/apache/lucene/util/RefCount.java | 10 +-
.../org/apache/lucene/util/RoaringDocIdSet.java | 4 +-
.../org/apache/lucene/util/RollingBuffer.java | 4 +-
.../apache/lucene/util/SPIClassIterator.java | 4 +-
.../org/apache/lucene/util/SentinelIntSet.java | 4 +-
.../java/org/apache/lucene/util/SetOnce.java | 8 +-
.../java/org/apache/lucene/util/SloppyMath.java | 10 +-
.../java/org/apache/lucene/util/SmallFloat.java | 14 +-
.../src/java/org/apache/lucene/util/Sorter.java | 4 +-
.../apache/lucene/util/SparseFixedBitSet.java | 4 +-
.../lucene/util/StrictStringTokenizer.java | 4 +-
.../org/apache/lucene/util/StringHelper.java | 4 +-
.../apache/lucene/util/SuppressForbidden.java | 4 +-
.../lucene/util/ThreadInterruptedException.java | 9 +-
.../java/org/apache/lucene/util/TimSorter.java | 4 +-
.../org/apache/lucene/util/ToStringUtils.java | 4 +-
.../org/apache/lucene/util/UnicodeUtil.java | 6 +-
.../java/org/apache/lucene/util/Version.java | 4 +-
.../org/apache/lucene/util/VirtualMethod.java | 4 +-
.../org/apache/lucene/util/WeakIdentityMap.java | 4 +-
.../apache/lucene/util/automaton/Automaton.java | 4 +-
.../lucene/util/automaton/ByteRunAutomaton.java | 4 +-
.../util/automaton/CharacterRunAutomaton.java | 4 +-
.../util/automaton/CompiledAutomaton.java | 4 +-
.../automaton/DaciukMihovAutomatonBuilder.java | 4 +-
.../util/automaton/FiniteStringsIterator.java | 4 +-
.../automaton/Lev1ParametricDescription.java | 3 +-
.../automaton/Lev1TParametricDescription.java | 3 +-
.../automaton/Lev2ParametricDescription.java | 3 +-
.../automaton/Lev2TParametricDescription.java | 3 +-
.../util/automaton/LevenshteinAutomata.java | 4 +-
.../automaton/LimitedFiniteStringsIterator.java | 4 +-
.../lucene/util/automaton/SortedIntSet.java | 4 +-
.../TooComplexToDeterminizeException.java | 4 +-
.../lucene/util/automaton/Transition.java | 4 +-
.../lucene/util/automaton/UTF32ToUTF8.java | 4 +-
.../lucene/util/automaton/createLevAutomata.py | 89 +-
.../org/apache/lucene/util/bkd/BKDReader.java | 4 +-
.../org/apache/lucene/util/bkd/BKDWriter.java | 4 +-
.../apache/lucene/util/bkd/HeapPointReader.java | 4 +-
.../apache/lucene/util/bkd/HeapPointWriter.java | 4 +-
.../lucene/util/bkd/OfflinePointReader.java | 4 +-
.../lucene/util/bkd/OfflinePointWriter.java | 5 +-
.../org/apache/lucene/util/bkd/PointReader.java | 4 +-
.../org/apache/lucene/util/bkd/PointWriter.java | 4 +-
.../org/apache/lucene/util/fst/Builder.java | 4 +-
.../lucene/util/fst/ByteSequenceOutputs.java | 4 +-
.../apache/lucene/util/fst/BytesRefFSTEnum.java | 4 +-
.../org/apache/lucene/util/fst/BytesStore.java | 4 +-
.../lucene/util/fst/CharSequenceOutputs.java | 4 +-
.../java/org/apache/lucene/util/fst/FST.java | 4 +-
.../org/apache/lucene/util/fst/FSTEnum.java | 4 +-
.../lucene/util/fst/ForwardBytesReader.java | 4 +-
.../lucene/util/fst/IntSequenceOutputs.java | 4 +-
.../apache/lucene/util/fst/IntsRefFSTEnum.java | 4 +-
.../org/apache/lucene/util/fst/NoOutputs.java | 4 +-
.../org/apache/lucene/util/fst/NodeHash.java | 4 +-
.../org/apache/lucene/util/fst/Outputs.java | 4 +-
.../org/apache/lucene/util/fst/PairOutputs.java | 4 +-
.../lucene/util/fst/PositiveIntOutputs.java | 4 +-
.../lucene/util/fst/ReverseBytesReader.java | 4 +-
.../java/org/apache/lucene/util/fst/Util.java | 4 +-
.../util/packed/AbstractBlockPackedWriter.java | 4 +-
.../util/packed/AbstractPagedMutable.java | 4 +-
.../lucene/util/packed/BlockPackedReader.java | 4 +-
.../util/packed/BlockPackedReaderIterator.java | 4 +-
.../lucene/util/packed/BlockPackedWriter.java | 4 +-
.../lucene/util/packed/BulkOperation.java | 3 +-
.../lucene/util/packed/BulkOperationPacked.java | 6 +-
.../util/packed/BulkOperationPacked1.java | 3 +-
.../util/packed/BulkOperationPacked10.java | 3 +-
.../util/packed/BulkOperationPacked11.java | 3 +-
.../util/packed/BulkOperationPacked12.java | 3 +-
.../util/packed/BulkOperationPacked13.java | 3 +-
.../util/packed/BulkOperationPacked14.java | 3 +-
.../util/packed/BulkOperationPacked15.java | 3 +-
.../util/packed/BulkOperationPacked16.java | 3 +-
.../util/packed/BulkOperationPacked17.java | 3 +-
.../util/packed/BulkOperationPacked18.java | 3 +-
.../util/packed/BulkOperationPacked19.java | 3 +-
.../util/packed/BulkOperationPacked2.java | 3 +-
.../util/packed/BulkOperationPacked20.java | 3 +-
.../util/packed/BulkOperationPacked21.java | 3 +-
.../util/packed/BulkOperationPacked22.java | 3 +-
.../util/packed/BulkOperationPacked23.java | 3 +-
.../util/packed/BulkOperationPacked24.java | 3 +-
.../util/packed/BulkOperationPacked3.java | 3 +-
.../util/packed/BulkOperationPacked4.java | 3 +-
.../util/packed/BulkOperationPacked5.java | 3 +-
.../util/packed/BulkOperationPacked6.java | 3 +-
.../util/packed/BulkOperationPacked7.java | 3 +-
.../util/packed/BulkOperationPacked8.java | 3 +-
.../util/packed/BulkOperationPacked9.java | 3 +-
.../packed/BulkOperationPackedSingleBlock.java | 4 +-
.../util/packed/DeltaPackedLongValues.java | 4 +-
.../org/apache/lucene/util/packed/Direct16.java | 8 +-
.../org/apache/lucene/util/packed/Direct32.java | 8 +-
.../org/apache/lucene/util/packed/Direct64.java | 3 +-
.../org/apache/lucene/util/packed/Direct8.java | 8 +-
.../util/packed/DirectMonotonicReader.java | 4 +-
.../util/packed/DirectMonotonicWriter.java | 4 +-
.../packed/DirectPacked64SingleBlockReader.java | 4 +-
.../lucene/util/packed/DirectPackedReader.java | 4 +-
.../apache/lucene/util/packed/DirectReader.java | 4 +-
.../apache/lucene/util/packed/DirectWriter.java | 4 +-
.../lucene/util/packed/GrowableWriter.java | 4 +-
.../util/packed/MonotonicBlockPackedReader.java | 4 +-
.../util/packed/MonotonicBlockPackedWriter.java | 4 +-
.../lucene/util/packed/MonotonicLongValues.java | 4 +-
.../lucene/util/packed/Packed16ThreeBlocks.java | 3 +-
.../org/apache/lucene/util/packed/Packed64.java | 4 +-
.../lucene/util/packed/Packed64SingleBlock.java | 3 +-
.../lucene/util/packed/Packed8ThreeBlocks.java | 3 +-
.../lucene/util/packed/PackedDataInput.java | 4 +-
.../lucene/util/packed/PackedDataOutput.java | 4 +-
.../apache/lucene/util/packed/PackedInts.java | 4 +-
.../lucene/util/packed/PackedLongValues.java | 4 +-
.../util/packed/PackedReaderIterator.java | 4 +-
.../apache/lucene/util/packed/PackedWriter.java | 4 +-
.../lucene/util/packed/PagedGrowableWriter.java | 4 +-
.../apache/lucene/util/packed/PagedMutable.java | 4 +-
.../lucene/util/packed/gen_BulkOperation.py | 79 +-
.../org/apache/lucene/util/packed/gen_Direct.py | 29 +-
.../util/packed/gen_Packed64SingleBlock.py | 45 +-
.../lucene/util/packed/gen_PackedThreeBlocks.py | 27 +-
.../test/org/apache/lucene/TestAssertions.java | 4 +-
.../src/test/org/apache/lucene/TestDemo.java | 4 +-
.../org/apache/lucene/TestExternalCodecs.java | 4 +-
.../lucene/TestMergeSchedulerExternal.java | 4 +-
.../src/test/org/apache/lucene/TestSearch.java | 4 +-
.../apache/lucene/TestSearchForDuplicates.java | 4 +-
.../lucene/analysis/TestCachingTokenFilter.java | 4 +-
.../apache/lucene/analysis/TestCharFilter.java | 4 +-
.../lucene/analysis/TestNumericTokenStream.java | 4 +-
.../analysis/TestReusableStringReader.java | 12 +-
.../org/apache/lucene/analysis/TestToken.java | 4 +-
.../tokenattributes/TestBytesRefAttImpl.java | 4 +-
.../TestCharTermAttributeImpl.java | 4 +-
.../TestPackedTokenAttributeImpl.java | 4 +-
.../TestSimpleAttributeImpl.java | 4 +-
.../lucene/codecs/TestCodecLoadingDeadlock.java | 4 +-
.../AbstractTestCompressionMode.java | 4 +-
.../AbstractTestLZ4CompressionMode.java | 4 +-
.../compressing/TestFastCompressionMode.java | 4 +-
.../compressing/TestFastDecompressionMode.java | 4 +-
.../TestGrowableByteArrayDataOutput.java | 4 +-
.../compressing/TestHighCompressionMode.java | 4 +-
.../lucene50/TestBlockPostingsFormat.java | 4 +-
.../lucene50/TestBlockPostingsFormat2.java | 4 +-
.../lucene50/TestBlockPostingsFormat3.java | 4 +-
.../lucene/codecs/lucene50/TestForUtil.java | 4 +-
.../lucene50/TestLucene50CompoundFormat.java | 4 +-
.../lucene50/TestLucene50FieldInfoFormat.java | 4 +-
.../lucene50/TestLucene50SegmentInfoFormat.java | 4 +-
.../TestLucene50StoredFieldsFormat.java | 4 +-
...cene50StoredFieldsFormatHighCompression.java | 4 +-
.../lucene50/TestLucene50TermVectorsFormat.java | 4 +-
.../lucene53/TestLucene53NormsFormat.java | 4 +-
.../lucene54/TestLucene54DocValuesFormat.java | 4 +-
.../lucene60/TestLucene60PointFormat.java | 83 ++
.../perfield/TestPerFieldDocValuesFormat.java | 4 +-
.../perfield/TestPerFieldPostingsFormat.java | 4 +-
.../perfield/TestPerFieldPostingsFormat2.java | 4 +-
.../lucene/document/TestBinaryDocument.java | 4 +-
.../apache/lucene/document/TestDateTools.java | 16 +-
.../apache/lucene/document/TestDocument.java | 4 +-
.../org/apache/lucene/document/TestField.java | 4 +-
.../apache/lucene/document/TestFieldType.java | 4 +-
.../lucene/index/Test2BBinaryDocValues.java | 4 +-
.../lucene/index/Test2BNumericDocValues.java | 4 +-
.../apache/lucene/index/Test2BPositions.java | 4 +-
.../org/apache/lucene/index/Test2BPostings.java | 4 +-
.../lucene/index/Test2BPostingsBytes.java | 4 +-
.../index/Test2BSortedDocValuesFixedSorted.java | 4 +-
.../lucene/index/Test2BSortedDocValuesOrds.java | 4 +-
.../org/apache/lucene/index/Test2BTerms.java | 4 +-
.../lucene/index/Test4GBStoredFields.java | 4 +-
.../org/apache/lucene/index/TestAddIndexes.java | 26 +-
.../index/TestAllFilesCheckIndexHeader.java | 142 +++
.../index/TestAllFilesDetectTruncation.java | 126 +++
.../index/TestAllFilesHaveChecksumFooter.java | 28 +-
.../index/TestAllFilesHaveCodecHeader.java | 35 +-
.../apache/lucene/index/TestAtomicUpdate.java | 18 +-
.../apache/lucene/index/TestBagOfPositions.java | 4 +-
.../apache/lucene/index/TestBagOfPostings.java | 4 +-
.../index/TestBinaryDocValuesUpdates.java | 36 +-
.../apache/lucene/index/TestBinaryTerms.java | 4 +-
.../org/apache/lucene/index/TestByteSlices.java | 12 +-
.../org/apache/lucene/index/TestCheckIndex.java | 4 +-
.../lucene/index/TestCodecHoldsOpenFiles.java | 19 +-
.../org/apache/lucene/index/TestCodecUtil.java | 4 +-
.../org/apache/lucene/index/TestCodecs.java | 4 +-
.../index/TestConcurrentMergeScheduler.java | 8 +-
.../index/TestConsistentFieldNumbers.java | 4 +-
.../test/org/apache/lucene/index/TestCrash.java | 4 +-
.../index/TestCrashCausesCorruptIndex.java | 4 +-
.../apache/lucene/index/TestCustomNorms.java | 4 +-
.../apache/lucene/index/TestDeletionPolicy.java | 38 +-
.../index/TestDemoParallelLeafReader.java | 18 +-
.../lucene/index/TestDirectoryReader.java | 388 ++++----
.../lucene/index/TestDirectoryReaderReopen.java | 14 +-
.../test/org/apache/lucene/index/TestDoc.java | 12 +-
.../org/apache/lucene/index/TestDocCount.java | 4 +-
.../index/TestDocInverterPerFieldErrorInfo.java | 4 +-
.../org/apache/lucene/index/TestDocValues.java | 4 +-
.../lucene/index/TestDocValuesIndexing.java | 4 +-
.../lucene/index/TestDocsAndPositions.java | 4 +-
.../apache/lucene/index/TestDocumentWriter.java | 4 +-
.../index/TestDocumentsWriterDeleteQueue.java | 29 +-
.../index/TestDocumentsWriterStallControl.java | 27 +-
.../apache/lucene/index/TestDuelingCodecs.java | 4 +-
.../lucene/index/TestDuelingCodecsAtNight.java | 4 +-
.../lucene/index/TestExceedMaxTermLength.java | 4 +-
.../index/TestExitableDirectoryReader.java | 4 +-
.../org/apache/lucene/index/TestFieldReuse.java | 6 +-
.../apache/lucene/index/TestFieldsReader.java | 69 +-
.../lucene/index/TestFilterDirectoryReader.java | 4 +-
.../lucene/index/TestFilterLeafReader.java | 4 +-
.../test/org/apache/lucene/index/TestFlex.java | 4 +-
.../index/TestFlushByRamOrCountsPolicy.java | 4 +-
.../lucene/index/TestForTooMuchCloning.java | 4 +-
.../lucene/index/TestForceMergeForever.java | 4 +-
.../apache/lucene/index/TestIndexCommit.java | 4 +-
.../lucene/index/TestIndexFileDeleter.java | 35 +-
.../org/apache/lucene/index/TestIndexInput.java | 4 +-
.../lucene/index/TestIndexReaderClose.java | 4 +-
.../apache/lucene/index/TestIndexWriter.java | 951 +++++++++---------
.../lucene/index/TestIndexWriterCommit.java | 24 +-
.../lucene/index/TestIndexWriterConfig.java | 4 +-
.../lucene/index/TestIndexWriterDelete.java | 13 +-
.../index/TestIndexWriterDeleteByQuery.java | 6 +-
.../lucene/index/TestIndexWriterExceptions.java | 21 +-
.../index/TestIndexWriterExceptions2.java | 7 +-
.../lucene/index/TestIndexWriterForceMerge.java | 5 +-
.../lucene/index/TestIndexWriterFromReader.java | 21 +-
.../index/TestIndexWriterLockRelease.java | 4 +-
.../lucene/index/TestIndexWriterMaxDocs.java | 4 +-
.../index/TestIndexWriterMergePolicy.java | 4 +-
.../lucene/index/TestIndexWriterMerging.java | 17 +-
.../index/TestIndexWriterNRTIsCurrent.java | 28 +-
.../lucene/index/TestIndexWriterOnDiskFull.java | 10 +-
.../lucene/index/TestIndexWriterOnJRECrash.java | 28 +-
.../lucene/index/TestIndexWriterOnVMError.java | 7 +-
.../TestIndexWriterOutOfFileDescriptors.java | 6 +-
.../lucene/index/TestIndexWriterReader.java | 14 +-
.../index/TestIndexWriterThreadsToSegments.java | 4 +-
.../lucene/index/TestIndexWriterUnicode.java | 4 +-
.../index/TestIndexWriterWithThreads.java | 4 +-
.../apache/lucene/index/TestIndexableField.java | 7 +-
.../org/apache/lucene/index/TestInfoStream.java | 4 +-
.../apache/lucene/index/TestIntBlockPool.java | 4 +-
.../org/apache/lucene/index/TestIsCurrent.java | 4 +-
.../lucene/index/TestLazyProxSkipping.java | 4 +-
.../apache/lucene/index/TestLogMergePolicy.java | 4 +-
.../apache/lucene/index/TestLongPostings.java | 4 +-
.../org/apache/lucene/index/TestManyFields.java | 4 +-
.../apache/lucene/index/TestMaxPosition.java | 4 +-
.../lucene/index/TestMaxTermFrequency.java | 4 +-
.../lucene/index/TestMergePolicyWrapper.java | 37 +
.../lucene/index/TestMergeRateLimiter.java | 4 +-
.../apache/lucene/index/TestMixedCodecs.java | 4 +-
.../lucene/index/TestMixedDocValuesUpdates.java | 32 +-
.../apache/lucene/index/TestMultiDocValues.java | 4 +-
.../apache/lucene/index/TestMultiFields.java | 4 +-
.../lucene/index/TestMultiLevelSkipList.java | 4 +-
.../apache/lucene/index/TestMultiTermsEnum.java | 4 +-
.../lucene/index/TestNRTReaderCleanup.java | 14 +-
.../lucene/index/TestNRTReaderWithThreads.java | 4 +-
.../org/apache/lucene/index/TestNRTThreads.java | 4 +-
.../apache/lucene/index/TestNeverDelete.java | 12 +-
.../apache/lucene/index/TestNewestSegment.java | 4 +-
.../lucene/index/TestNoDeletionPolicy.java | 4 +-
.../apache/lucene/index/TestNoMergePolicy.java | 12 +-
.../lucene/index/TestNoMergeScheduler.java | 4 +-
.../test/org/apache/lucene/index/TestNorms.java | 4 +-
.../index/TestNumericDocValuesUpdates.java | 36 +-
.../org/apache/lucene/index/TestOmitNorms.java | 4 +-
.../apache/lucene/index/TestOmitPositions.java | 9 +-
.../org/apache/lucene/index/TestOmitTf.java | 9 +-
.../org/apache/lucene/index/TestOrdinalMap.java | 4 +-
.../index/TestParallelCompositeReader.java | 4 +-
.../lucene/index/TestParallelLeafReader.java | 4 +-
.../index/TestParallelReaderEmptyIndex.java | 4 +-
.../lucene/index/TestParallelTermEnum.java | 4 +-
.../org/apache/lucene/index/TestPayloads.java | 4 +-
.../lucene/index/TestPayloadsOnVectors.java | 4 +-
.../lucene/index/TestPerSegmentDeletes.java | 4 +-
.../TestPersistentSnapshotDeletionPolicy.java | 29 +-
.../apache/lucene/index/TestPointValues.java | 990 +++----------------
.../lucene/index/TestPostingsOffsets.java | 4 +-
.../lucene/index/TestPrefixCodedTerms.java | 4 +-
.../apache/lucene/index/TestReadOnlyIndex.java | 4 +-
.../apache/lucene/index/TestReaderClosed.java | 4 +-
.../index/TestReaderWrapperDVTypeCheck.java | 1 -
.../org/apache/lucene/index/TestRollback.java | 4 +-
.../apache/lucene/index/TestRollingUpdates.java | 8 +-
.../lucene/index/TestSameTokenSamePosition.java | 4 +-
.../apache/lucene/index/TestSegmentInfos.java | 4 +-
.../apache/lucene/index/TestSegmentMerger.java | 4 +-
.../apache/lucene/index/TestSegmentReader.java | 4 +-
.../lucene/index/TestSegmentTermDocs.java | 4 +-
.../lucene/index/TestSegmentTermEnum.java | 4 +-
.../lucene/index/TestSizeBoundedForceMerge.java | 4 +-
.../index/TestSnapshotDeletionPolicy.java | 26 +-
.../apache/lucene/index/TestStressAdvance.java | 4 +-
.../apache/lucene/index/TestStressDeletes.java | 14 +-
.../apache/lucene/index/TestStressIndexing.java | 16 +-
.../lucene/index/TestStressIndexing2.java | 18 +-
.../org/apache/lucene/index/TestStressNRT.java | 6 +-
.../org/apache/lucene/index/TestSumDocFreq.java | 4 +-
.../lucene/index/TestSwappedIndexFiles.java | 128 +++
.../test/org/apache/lucene/index/TestTerm.java | 4 +-
.../apache/lucene/index/TestTermVectors.java | 4 +-
.../lucene/index/TestTermVectorsReader.java | 4 +-
.../lucene/index/TestTermVectorsWriter.java | 4 +-
.../apache/lucene/index/TestTermdocPerf.java | 15 +-
.../test/org/apache/lucene/index/TestTerms.java | 4 +-
.../org/apache/lucene/index/TestTermsEnum.java | 4 +-
.../org/apache/lucene/index/TestTermsEnum2.java | 4 +-
.../lucene/index/TestThreadedForceMerge.java | 4 +-
.../lucene/index/TestTieredMergePolicy.java | 4 +-
.../index/TestTragicIndexWriterDeadlock.java | 4 +-
.../lucene/index/TestTransactionRollback.java | 4 +-
.../apache/lucene/index/TestTransactions.java | 4 +-
.../org/apache/lucene/index/TestTryDelete.java | 4 +-
.../lucene/index/TestTwoPhaseCommitTool.java | 4 +-
.../lucene/index/TestUniqueTermCount.java | 4 +-
.../index/TestUpgradeIndexMergePolicy.java | 26 +
.../lucene/search/BaseTestRangeFilter.java | 4 +-
.../apache/lucene/search/JustCompileSearch.java | 4 +-
.../lucene/search/MultiCollectorTest.java | 4 +-
.../TestApproximationSearchEquivalence.java | 4 +-
.../lucene/search/TestAutomatonQuery.java | 4 +-
.../search/TestAutomatonQueryUnicode.java | 4 +-
.../lucene/search/TestBlendedTermQuery.java | 4 +-
.../org/apache/lucene/search/TestBoolean2.java | 4 +-
.../apache/lucene/search/TestBooleanCoord.java | 4 +-
.../search/TestBooleanMinShouldMatch.java | 4 +-
.../org/apache/lucene/search/TestBooleanOr.java | 2 +-
.../apache/lucene/search/TestBooleanQuery.java | 4 +-
.../search/TestBooleanQueryVisitSubscorers.java | 4 +-
.../lucene/search/TestBooleanRewrites.java | 4 +-
.../apache/lucene/search/TestBooleanScorer.java | 4 +-
.../apache/lucene/search/TestBoostQuery.java | 4 +-
.../lucene/search/TestCachingCollector.java | 4 +-
.../lucene/search/TestComplexExplanations.java | 4 +-
.../TestComplexExplanationsOfNonMatches.java | 4 +-
.../lucene/search/TestConjunctionDISI.java | 4 +-
.../apache/lucene/search/TestConjunctions.java | 4 +-
.../lucene/search/TestConstantScoreQuery.java | 4 +-
.../TestControlledRealTimeReopenThread.java | 4 +-
.../lucene/search/TestCustomSearcherSort.java | 14 +-
.../org/apache/lucene/search/TestDateSort.java | 4 +-
.../lucene/search/TestDisjunctionMaxQuery.java | 4 +-
.../org/apache/lucene/search/TestDocBoost.java | 4 +-
.../search/TestDocValuesRewriteMethod.java | 4 +-
.../lucene/search/TestDocValuesScoring.java | 4 +-
.../lucene/search/TestEarlyTermination.java | 4 +-
.../lucene/search/TestElevationComparator.java | 4 +-
.../search/TestFieldCacheRewriteMethod.java | 4 +-
.../lucene/search/TestFieldValueQuery.java | 4 +-
.../apache/lucene/search/TestFuzzyQuery.java | 4 +-
.../apache/lucene/search/TestIndexSearcher.java | 4 +-
.../apache/lucene/search/TestLRUQueryCache.java | 4 +-
.../lucene/search/TestLiveFieldValues.java | 4 +-
.../lucene/search/TestMinShouldMatch2.java | 4 +-
.../lucene/search/TestMultiCollector.java | 4 +-
.../lucene/search/TestMultiPhraseEnum.java | 4 +-
.../lucene/search/TestMultiPhraseQuery.java | 4 +-
.../search/TestMultiTermConstantScore.java | 4 +-
.../search/TestMultiTermQueryRewrites.java | 4 +-
.../search/TestMultiThreadTermVectors.java | 4 +-
.../TestMultiValuedNumericRangeQuery.java | 4 +-
.../org/apache/lucene/search/TestMultiset.java | 4 +-
.../lucene/search/TestNGramPhraseQuery.java | 4 +-
.../apache/lucene/search/TestNeedsScores.java | 4 +-
.../test/org/apache/lucene/search/TestNot.java | 4 +-
.../lucene/search/TestNumericRangeQuery32.java | 4 +-
.../lucene/search/TestNumericRangeQuery64.java | 4 +-
.../lucene/search/TestPhrasePrefixQuery.java | 4 +-
.../apache/lucene/search/TestPhraseQuery.java | 4 +-
.../apache/lucene/search/TestPointQueries.java | 45 +-
.../lucene/search/TestPositionIncrement.java | 4 +-
.../search/TestPositiveScoresOnlyCollector.java | 4 +-
.../lucene/search/TestPrefixInBooleanQuery.java | 4 +-
.../apache/lucene/search/TestPrefixQuery.java | 4 +-
.../apache/lucene/search/TestPrefixRandom.java | 4 +-
.../lucene/search/TestQueryCachingPolicy.java | 4 +-
.../apache/lucene/search/TestQueryRescorer.java | 4 +-
.../apache/lucene/search/TestRegexpQuery.java | 4 +-
.../apache/lucene/search/TestRegexpRandom.java | 4 +-
.../apache/lucene/search/TestRegexpRandom2.java | 4 +-
.../lucene/search/TestReqExclBulkScorer.java | 4 +-
.../search/TestSameScoresWithThreads.java | 4 +-
.../search/TestScoreCachingWrappingScorer.java | 4 +-
.../apache/lucene/search/TestScorerPerf.java | 32 +-
.../apache/lucene/search/TestSearchAfter.java | 4 +-
.../lucene/search/TestSearchWithThreads.java | 4 +-
.../lucene/search/TestSearcherManager.java | 4 +-
.../lucene/search/TestShardSearching.java | 4 +-
.../apache/lucene/search/TestSimilarity.java | 4 +-
.../lucene/search/TestSimilarityProvider.java | 4 +-
.../lucene/search/TestSimpleExplanations.java | 4 +-
.../TestSimpleExplanationsOfNonMatches.java | 4 +-
.../search/TestSimpleSearchEquivalence.java | 14 +-
.../lucene/search/TestSloppyPhraseQuery.java | 4 +-
.../lucene/search/TestSloppyPhraseQuery2.java | 4 +-
.../test/org/apache/lucene/search/TestSort.java | 4 +-
.../apache/lucene/search/TestSortRandom.java | 4 +-
.../apache/lucene/search/TestSortRescorer.java | 4 +-
.../search/TestSortedNumericSortField.java | 4 +-
.../lucene/search/TestSortedSetSelector.java | 4 +-
.../lucene/search/TestSortedSetSortField.java | 4 +-
.../lucene/search/TestSubScorerFreqs.java | 4 +-
.../apache/lucene/search/TestSynonymQuery.java | 4 +-
.../lucene/search/TestTermRangeQuery.java | 4 +-
.../apache/lucene/search/TestTermScorer.java | 4 +-
.../search/TestTimeLimitingCollector.java | 4 +-
.../lucene/search/TestTopDocsCollector.java | 4 +-
.../apache/lucene/search/TestTopDocsMerge.java | 4 +-
.../lucene/search/TestTopFieldCollector.java | 4 +-
.../search/TestTotalHitCountCollector.java | 4 +-
.../TestUsageTrackingFilterCachingPolicy.java | 4 +-
.../org/apache/lucene/search/TestWildcard.java | 4 +-
.../lucene/search/TestWildcardRandom.java | 4 +-
.../search/similarities/TestBM25Similarity.java | 4 +-
.../similarities/TestClassicSimilarity.java | 4 +-
.../search/similarities/TestSimilarity2.java | 4 +-
.../search/similarities/TestSimilarityBase.java | 4 +-
.../search/spans/JustCompileSearchSpans.java | 4 +-
.../apache/lucene/search/spans/TestBasics.java | 4 +-
.../search/spans/TestFieldMaskingSpanQuery.java | 4 +-
.../lucene/search/spans/TestFilterSpans.java | 27 +-
.../search/spans/TestNearSpansOrdered.java | 4 +-
.../lucene/search/spans/TestSpanBoostQuery.java | 10 +-
.../lucene/search/spans/TestSpanCollection.java | 4 +-
.../search/spans/TestSpanContainQuery.java | 4 +-
.../search/spans/TestSpanExplanations.java | 4 +-
.../spans/TestSpanExplanationsOfNonMatches.java | 4 +-
.../lucene/search/spans/TestSpanFirstQuery.java | 4 +-
.../spans/TestSpanMultiTermQueryWrapper.java | 4 +-
.../lucene/search/spans/TestSpanNearQuery.java | 4 +-
.../lucene/search/spans/TestSpanNotQuery.java | 4 +-
.../lucene/search/spans/TestSpanOrQuery.java | 4 +-
.../search/spans/TestSpanSearchEquivalence.java | 4 +-
.../lucene/search/spans/TestSpanTermQuery.java | 4 +-
.../apache/lucene/search/spans/TestSpans.java | 4 +-
.../lucene/search/spans/TestSpansEnum.java | 4 +-
.../lucene/store/TestBufferedChecksum.java | 4 +-
.../lucene/store/TestBufferedIndexInput.java | 90 +-
.../lucene/store/TestByteArrayDataInput.java | 4 +-
.../org/apache/lucene/store/TestDirectory.java | 23 +-
.../lucene/store/TestFileSwitchDirectory.java | 5 +-
.../lucene/store/TestFilterDirectory.java | 4 +-
.../apache/lucene/store/TestHugeRamFile.java | 4 +-
.../apache/lucene/store/TestLockFactory.java | 4 +-
.../apache/lucene/store/TestMmapDirectory.java | 4 +-
.../org/apache/lucene/store/TestMultiMMap.java | 4 +-
.../apache/lucene/store/TestNIOFSDirectory.java | 4 +-
.../lucene/store/TestNRTCachingDirectory.java | 4 +-
.../lucene/store/TestNativeFSLockFactory.java | 25 +-
.../apache/lucene/store/TestRAMDirectory.java | 5 +-
.../apache/lucene/store/TestRateLimiter.java | 4 +-
.../lucene/store/TestSimpleFSDirectory.java | 4 +-
.../lucene/store/TestSimpleFSLockFactory.java | 5 +-
.../store/TestSingleInstanceLockFactory.java | 4 +-
.../lucene/store/TestSleepingLockWrapper.java | 4 +-
.../store/TestTrackingDirectoryWrapper.java | 4 +-
.../apache/lucene/store/TestWindowsMMap.java | 4 +-
.../apache/lucene/util/BaseSortTestCase.java | 4 +-
.../lucene/util/StressRamUsageEstimator.java | 4 +-
.../apache/lucene/util/Test2BPagedBytes.java | 4 +-
.../org/apache/lucene/util/TestArrayUtil.java | 4 +-
.../apache/lucene/util/TestAttributeSource.java | 4 +-
.../apache/lucene/util/TestByteBlockPool.java | 32 +-
.../org/apache/lucene/util/TestBytesRef.java | 4 +-
.../apache/lucene/util/TestBytesRefArray.java | 27 +-
.../apache/lucene/util/TestBytesRefHash.java | 4 +-
.../org/apache/lucene/util/TestCharsRef.java | 8 +-
.../apache/lucene/util/TestCharsRefBuilder.java | 4 +-
.../lucene/util/TestCloseableThreadLocal.java | 1 -
.../apache/lucene/util/TestCollectionUtil.java | 4 +-
.../apache/lucene/util/TestDocIdSetBuilder.java | 4 +-
.../apache/lucene/util/TestFilterIterator.java | 1 -
.../lucene/util/TestFixedBitDocIdSet.java | 1 -
.../org/apache/lucene/util/TestFixedBitSet.java | 4 +-
.../util/TestFrequencyTrackingRingBuffer.java | 4 +-
.../org/apache/lucene/util/TestIOUtils.java | 5 +-
.../lucene/util/TestInPlaceMergeSorter.java | 4 +-
.../lucene/util/TestIntArrayDocIdSet.java | 4 +-
.../org/apache/lucene/util/TestIntroSorter.java | 4 +-
.../org/apache/lucene/util/TestIntsRef.java | 4 +-
.../apache/lucene/util/TestLSBRadixSorter.java | 4 +-
.../lucene/util/TestLegacyNumericUtils.java | 4 +-
.../org/apache/lucene/util/TestLongBitSet.java | 4 +-
.../org/apache/lucene/util/TestMathUtil.java | 4 +-
.../apache/lucene/util/TestMergedIterator.java | 4 +-
.../apache/lucene/util/TestNamedSPILoader.java | 12 +-
.../org/apache/lucene/util/TestNotDocIdSet.java | 4 +-
.../apache/lucene/util/TestOfflineSorter.java | 32 +-
.../org/apache/lucene/util/TestPagedBytes.java | 1 -
.../apache/lucene/util/TestPriorityQueue.java | 4 +-
.../apache/lucene/util/TestQueryBuilder.java | 4 +-
.../lucene/util/TestRamUsageEstimator.java | 4 +-
.../util/TestRecyclingByteBlockAllocator.java | 18 +-
.../util/TestRecyclingIntBlockAllocator.java | 18 +-
.../apache/lucene/util/TestRoaringDocIdSet.java | 4 +-
.../apache/lucene/util/TestRollingBuffer.java | 8 +-
.../lucene/util/TestSPIClassIterator.java | 4 +-
.../apache/lucene/util/TestSentinelIntSet.java | 6 +-
.../org/apache/lucene/util/TestSetOnce.java | 4 +-
.../org/apache/lucene/util/TestSloppyMath.java | 4 +-
.../org/apache/lucene/util/TestSmallFloat.java | 14 +-
.../lucene/util/TestSparseFixedBitDocIdSet.java | 4 +-
.../lucene/util/TestSparseFixedBitSet.java | 4 +-
.../apache/lucene/util/TestStringHelper.java | 4 +-
.../org/apache/lucene/util/TestTimSorter.java | 4 +-
.../lucene/util/TestTimSorterWorstCase.java | 4 +-
.../org/apache/lucene/util/TestUnicodeUtil.java | 4 +-
.../org/apache/lucene/util/TestVersion.java | 4 +-
.../apache/lucene/util/TestVirtualMethod.java | 4 +-
.../apache/lucene/util/TestWeakIdentityMap.java | 1 -
.../automaton/FiniteStringsIteratorTest.java | 4 +-
.../LimitedFiniteStringsIteratorTest.java | 4 +-
.../lucene/util/automaton/TestAutomaton.java | 4 +-
.../util/automaton/TestCompiledAutomaton.java | 4 +-
.../lucene/util/automaton/TestDeterminism.java | 4 +-
.../util/automaton/TestDeterminizeLexicon.java | 4 +-
.../util/automaton/TestLevenshteinAutomata.java | 4 +-
.../lucene/util/automaton/TestMinimize.java | 4 +-
.../lucene/util/automaton/TestOperations.java | 4 +-
.../lucene/util/automaton/TestRegExp.java | 4 +-
.../lucene/util/automaton/TestUTF32ToUTF8.java | 4 +-
.../org/apache/lucene/util/bkd/TestBKD.java | 10 +-
.../org/apache/lucene/util/fst/Test2BFST.java | 5 +-
.../apache/lucene/util/fst/TestBytesStore.java | 4 +-
.../org/apache/lucene/util/fst/TestFSTs.java | 7 +-
.../lucene/util/packed/TestDirectMonotonic.java | 4 +-
.../lucene/util/packed/TestDirectPacked.java | 4 +-
.../lucene/util/packed/TestPackedInts.java | 4 +-
.../java/org/apache/lucene/demo/IndexFiles.java | 4 +-
.../org/apache/lucene/demo/SearchFiles.java | 4 +-
.../demo/facet/AssociationsFacetsExample.java | 4 +-
.../demo/facet/DistanceFacetsExample.java | 4 +-
.../ExpressionAggregationFacetsExample.java | 32 +-
.../facet/MultiCategoryListsFacetsExample.java | 4 +-
.../lucene/demo/facet/RangeFacetsExample.java | 4 +-
.../lucene/demo/facet/SimpleFacetsExample.java | 4 +-
.../facet/SimpleSortedSetFacetsExample.java | 4 +-
.../demo/xmlparser/FormBasedXmlQueryDemo.java | 1 -
.../test/org/apache/lucene/demo/TestDemo.java | 4 +-
.../facet/TestAssociationsFacetsExample.java | 4 +-
.../demo/facet/TestDistanceFacetsExample.java | 4 +-
.../TestExpressionAggregationFacetsExample.java | 4 +-
.../TestMultiCategoryListsFacetsExample.java | 4 +-
.../demo/facet/TestRangeFacetsExample.java | 4 +-
.../demo/facet/TestSimpleFacetsExample.java | 4 +-
.../facet/TestSimpleSortedSetFacetsExample.java | 4 +-
.../org/apache/lucene/expressions/Bindings.java | 2 +-
.../apache/lucene/expressions/Expression.java | 2 +-
.../expressions/ExpressionComparator.java | 2 +-
.../expressions/ExpressionFunctionValues.java | 2 +-
.../lucene/expressions/ExpressionRescorer.java | 4 +-
.../lucene/expressions/ExpressionSortField.java | 2 +-
.../expressions/ExpressionValueSource.java | 2 +-
.../apache/lucene/expressions/FakeScorer.java | 4 +-
.../lucene/expressions/ScoreFunctionValues.java | 2 +-
.../lucene/expressions/ScoreValueSource.java | 4 +-
.../lucene/expressions/SimpleBindings.java | 4 +-
.../expressions/js/JavascriptCompiler.java | 2 +-
.../js/JavascriptErrorHandlingLexer.java | 4 +-
.../js/JavascriptParserErrorStrategy.java | 4 +-
.../lucene/expressions/js/VariableContext.java | 4 +-
.../lucene/expressions/TestDemoExpressions.java | 32 +-
.../expressions/TestExpressionRescorer.java | 4 +-
.../expressions/TestExpressionSortField.java | 4 +-
.../lucene/expressions/TestExpressionSorts.java | 4 +-
.../expressions/TestExpressionValidation.java | 4 +-
.../expressions/TestExpressionValueSource.java | 4 +-
.../expressions/js/TestCustomFunctions.java | 4 +-
.../expressions/js/TestJavascriptCompiler.java | 2 +-
.../expressions/js/TestJavascriptFunction.java | 2 +-
.../js/TestJavascriptOperations.java | 2 +-
.../expressions/js/TestVariableContext.java | 4 +-
.../org/apache/lucene/facet/DrillDownQuery.java | 3 +-
.../org/apache/lucene/facet/DrillSideways.java | 3 +-
.../apache/lucene/facet/DrillSidewaysQuery.java | 4 +-
.../lucene/facet/DrillSidewaysScorer.java | 3 +-
.../org/apache/lucene/facet/FacetField.java | 3 +-
.../org/apache/lucene/facet/FacetResult.java | 3 +-
.../java/org/apache/lucene/facet/Facets.java | 3 +-
.../apache/lucene/facet/FacetsCollector.java | 3 +-
.../org/apache/lucene/facet/FacetsConfig.java | 4 +-
.../org/apache/lucene/facet/LabelAndValue.java | 3 +-
.../org/apache/lucene/facet/MultiFacets.java | 3 +-
.../facet/RandomSamplingFacetsCollector.java | 3 +-
.../lucene/facet/TopOrdAndFloatQueue.java | 3 +-
.../apache/lucene/facet/TopOrdAndIntQueue.java | 3 +-
.../apache/lucene/facet/range/DoubleRange.java | 3 +-
.../facet/range/DoubleRangeFacetCounts.java | 3 +-
.../apache/lucene/facet/range/LongRange.java | 3 +-
.../lucene/facet/range/LongRangeCounter.java | 3 +-
.../facet/range/LongRangeFacetCounts.java | 3 +-
.../org/apache/lucene/facet/range/Range.java | 3 +-
.../lucene/facet/range/RangeFacetCounts.java | 3 +-
.../DefaultSortedSetDocValuesReaderState.java | 3 +-
.../SortedSetDocValuesFacetCounts.java | 3 +-
.../sortedset/SortedSetDocValuesFacetField.java | 3 +-
.../SortedSetDocValuesReaderState.java | 3 +-
.../facet/taxonomy/AssociationFacetField.java | 3 +-
.../facet/taxonomy/CachedOrdinalsReader.java | 3 +-
.../facet/taxonomy/DocValuesOrdinalsReader.java | 4 +-
.../lucene/facet/taxonomy/FacetLabel.java | 3 +-
.../lucene/facet/taxonomy/FakeScorer.java | 3 +-
.../facet/taxonomy/FastTaxonomyFacetCounts.java | 3 +-
.../taxonomy/FloatAssociationFacetField.java | 3 +-
.../facet/taxonomy/FloatTaxonomyFacets.java | 3 +-
.../taxonomy/IntAssociationFacetField.java | 3 +-
.../facet/taxonomy/IntTaxonomyFacets.java | 3 +-
.../lucene/facet/taxonomy/LRUHashMap.java | 3 +-
.../taxonomy/OrdinalMappingLeafReader.java | 3 +-
.../lucene/facet/taxonomy/OrdinalsReader.java | 3 +-
.../facet/taxonomy/ParallelTaxonomyArrays.java | 4 +-
.../facet/taxonomy/PrintTaxonomyStats.java | 3 +-
.../facet/taxonomy/SearcherTaxonomyManager.java | 3 +-
.../facet/taxonomy/TaxonomyFacetCounts.java | 3 +-
.../TaxonomyFacetSumFloatAssociations.java | 3 +-
.../TaxonomyFacetSumIntAssociations.java | 3 +-
.../taxonomy/TaxonomyFacetSumValueSource.java | 3 +-
.../lucene/facet/taxonomy/TaxonomyFacets.java | 3 +-
.../facet/taxonomy/TaxonomyMergeUtils.java | 31 +-
.../lucene/facet/taxonomy/TaxonomyReader.java | 17 +-
.../lucene/facet/taxonomy/TaxonomyWriter.java | 15 +-
.../lucene/facet/taxonomy/directory/Consts.java | 7 +-
.../directory/DirectoryTaxonomyReader.java | 33 +-
.../directory/DirectoryTaxonomyWriter.java | 33 +-
.../taxonomy/directory/TaxonomyIndexArrays.java | 25 +-
.../taxonomy/writercache/CategoryPathUtils.java | 7 +-
.../taxonomy/writercache/CharBlockArray.java | 21 +-
.../writercache/Cl2oTaxonomyWriterCache.java | 15 +-
.../taxonomy/writercache/CollisionMap.java | 13 +-
.../writercache/CompactLabelToOrdinal.java | 3 +-
.../taxonomy/writercache/LabelToOrdinal.java | 7 +-
.../writercache/LruTaxonomyWriterCache.java | 9 +-
.../writercache/NameHashIntCacheLRU.java | 7 +-
.../taxonomy/writercache/NameIntCacheLRU.java | 13 +-
.../writercache/TaxonomyWriterCache.java | 9 +-
.../facet/AssertingSubDocsAtOnceCollector.java | 3 +-
.../org/apache/lucene/facet/FacetTestCase.java | 3 +-
.../apache/lucene/facet/SlowRAMDirectory.java | 3 +-
.../apache/lucene/facet/TestDrillDownQuery.java | 3 +-
.../apache/lucene/facet/TestDrillSideways.java | 3 +-
.../apache/lucene/facet/TestFacetsConfig.java | 3 +-
.../lucene/facet/TestMultipleIndexFields.java | 3 +-
.../TestRandomSamplingFacetsCollector.java | 33 +-
.../facet/range/TestRangeFacetCounts.java | 3 +-
.../sortedset/TestSortedSetDocValuesFacets.java | 3 +-
.../taxonomy/TestCachedOrdinalsReader.java | 3 +-
.../lucene/facet/taxonomy/TestFacetLabel.java | 21 +-
.../lucene/facet/taxonomy/TestLRUHashMap.java | 3 +-
.../taxonomy/TestOrdinalMappingLeafReader.java | 33 +-
.../taxonomy/TestSearcherTaxonomyManager.java | 3 +-
.../facet/taxonomy/TestTaxonomyCombined.java | 33 +-
.../taxonomy/TestTaxonomyFacetAssociations.java | 4 +-
.../facet/taxonomy/TestTaxonomyFacetCounts.java | 3 +-
.../taxonomy/TestTaxonomyFacetCounts2.java | 3 +-
.../TestTaxonomyFacetSumValueSource.java | 3 +-
.../taxonomy/directory/TestAddTaxonomy.java | 42 +-
.../TestConcurrentFacetedIndexing.java | 33 +-
.../directory/TestDirectoryTaxonomyReader.java | 33 +-
.../directory/TestDirectoryTaxonomyWriter.java | 32 +-
.../writercache/TestCharBlockArray.java | 30 +-
.../writercache/TestCompactLabelToOrdinal.java | 33 +-
.../AbstractAllGroupHeadsCollector.java | 3 +-
.../grouping/AbstractAllGroupsCollector.java | 3 +-
.../AbstractDistinctValuesCollector.java | 3 +-
.../AbstractFirstPassGroupingCollector.java | 3 +-
.../grouping/AbstractGroupFacetCollector.java | 3 +-
.../AbstractSecondPassGroupingCollector.java | 3 +-
.../search/grouping/BlockGroupingCollector.java | 3 +-
.../search/grouping/CollectedSearchGroup.java | 1 -
.../lucene/search/grouping/FakeScorer.java | 3 +-
.../lucene/search/grouping/GroupDocs.java | 3 +-
.../lucene/search/grouping/GroupingSearch.java | 3 +-
.../lucene/search/grouping/SearchGroup.java | 3 +-
.../lucene/search/grouping/TopGroups.java | 3 +-
.../FunctionAllGroupHeadsCollector.java | 3 +-
.../function/FunctionAllGroupsCollector.java | 3 +-
.../FunctionDistinctValuesCollector.java | 3 +-
.../FunctionFirstPassGroupingCollector.java | 3 +-
.../FunctionSecondPassGroupingCollector.java | 3 +-
.../term/TermAllGroupHeadsCollector.java | 3 +-
.../grouping/term/TermAllGroupsCollector.java | 3 +-
.../term/TermDistinctValuesCollector.java | 3 +-
.../term/TermFirstPassGroupingCollector.java | 3 +-
.../grouping/term/TermGroupFacetCollector.java | 3 +-
.../term/TermSecondPassGroupingCollector.java | 3 +-
.../grouping/AbstractGroupingTestCase.java | 3 +-
.../grouping/AllGroupHeadsCollectorTest.java | 3 +-
.../search/grouping/AllGroupsCollectorTest.java | 5 +-
.../grouping/DistinctValuesCollectorTest.java | 3 +-
.../grouping/GroupFacetCollectorTest.java | 3 +-
.../search/grouping/GroupingSearchTest.java | 3 +-
.../lucene/search/grouping/TestGrouping.java | 1 -
.../lucene/search/highlight/DefaultEncoder.java | 22 +-
.../apache/lucene/search/highlight/Encoder.java | 17 +-
.../lucene/search/highlight/Formatter.java | 4 +-
.../lucene/search/highlight/Fragmenter.java | 3 +-
.../search/highlight/GradientFormatter.java | 3 +-
.../lucene/search/highlight/Highlighter.java | 3 +-
.../highlight/InvalidTokenOffsetsException.java | 3 +-
.../lucene/search/highlight/NullFragmenter.java | 3 +-
.../highlight/OffsetLimitTokenFilter.java | 3 +-
.../lucene/search/highlight/PositionSpan.java | 3 +-
.../lucene/search/highlight/QueryScorer.java | 3 +-
.../search/highlight/QueryTermExtractor.java | 3 +-
.../search/highlight/QueryTermScorer.java | 3 +-
.../apache/lucene/search/highlight/Scorer.java | 3 +-
.../search/highlight/SimpleFragmenter.java | 3 +-
.../search/highlight/SimpleHTMLEncoder.java | 19 +-
.../search/highlight/SimpleHTMLFormatter.java | 3 +-
.../search/highlight/SimpleSpanFragmenter.java | 6 +-
.../search/highlight/SpanGradientFormatter.java | 3 +-
.../search/highlight/TermVectorLeafReader.java | 3 +-
.../lucene/search/highlight/TextFragment.java | 4 +-
.../lucene/search/highlight/TokenGroup.java | 3 +-
.../lucene/search/highlight/TokenSources.java | 3 +-
.../highlight/TokenStreamFromTermVector.java | 3 +-
.../search/highlight/WeightedSpanTerm.java | 6 +-
.../highlight/WeightedSpanTermExtractor.java | 3 +-
.../lucene/search/highlight/WeightedTerm.java | 3 +-
.../CustomSeparatorBreakIterator.java | 9 +-
.../DefaultPassageFormatter.java | 3 +-
.../MultiTermHighlighting.java | 3 +-
.../search/postingshighlight/Passage.java | 3 +-
.../postingshighlight/PassageFormatter.java | 3 +-
.../search/postingshighlight/PassageScorer.java | 3 +-
.../postingshighlight/PostingsHighlighter.java | 3 +-
.../postingshighlight/WholeBreakIterator.java | 3 +-
.../vectorhighlight/BaseFragListBuilder.java | 3 +-
.../vectorhighlight/BaseFragmentsBuilder.java | 3 +-
.../search/vectorhighlight/BoundaryScanner.java | 3 +-
.../BreakIteratorBoundaryScanner.java | 3 +-
.../vectorhighlight/FastVectorHighlighter.java | 3 +-
.../search/vectorhighlight/FieldFragList.java | 3 +-
.../search/vectorhighlight/FieldPhraseList.java | 3 +-
.../search/vectorhighlight/FieldQuery.java | 3 +-
.../search/vectorhighlight/FieldTermStack.java | 3 +-
.../search/vectorhighlight/FragListBuilder.java | 3 +-
.../vectorhighlight/FragmentsBuilder.java | 3 +-
.../ScoreOrderFragmentsBuilder.java | 3 +-
.../vectorhighlight/SimpleBoundaryScanner.java | 3 +-
.../vectorhighlight/SimpleFieldFragList.java | 3 +-
.../vectorhighlight/SimpleFragListBuilder.java | 4 +-
.../vectorhighlight/SimpleFragmentsBuilder.java | 3 +-
.../vectorhighlight/SingleFragListBuilder.java | 3 +-
.../vectorhighlight/WeightedFieldFragList.java | 3 +-
.../WeightedFragListBuilder.java | 3 +-
.../search/highlight/HighlighterPhraseTest.java | 3 +-
.../search/highlight/HighlighterTest.java | 3 +-
.../lucene/search/highlight/MissesTest.java | 3 +-
.../highlight/OffsetLimitTokenFilterTest.java | 3 +-
.../search/highlight/TokenSourcesTest.java | 3 +-
.../custom/HighlightCustomQueryTest.java | 4 +-
.../TestCustomSeparatorBreakIterator.java | 3 +-
.../TestMultiTermHighlighting.java | 3 +-
.../TestPostingsHighlighter.java | 3 +-
.../TestPostingsHighlighterRanking.java | 3 +-
.../TestWholeBreakIterator.java | 3 +-
.../vectorhighlight/AbstractTestCase.java | 3 +-
.../BreakIteratorBoundaryScannerTest.java | 3 +-
.../FastVectorHighlighterTest.java | 2 +-
.../vectorhighlight/FieldPhraseListTest.java | 3 +-
.../search/vectorhighlight/FieldQueryTest.java | 3 +-
.../vectorhighlight/FieldTermStackTest.java | 3 +-
.../vectorhighlight/IndexTimeSynonymTest.java | 3 +-
.../ScoreOrderFragmentsBuilderTest.java | 3 +-
.../SimpleBoundaryScannerTest.java | 3 +-
.../SimpleFragListBuilderTest.java | 3 +-
.../SimpleFragmentsBuilderTest.java | 3 +-
.../SingleFragListBuilderTest.java | 11 +-
.../WeightedFragListBuilderTest.java | 3 +-
.../search/join/BaseGlobalOrdinalScorer.java | 3 +-
.../lucene/search/join/BitSetProducer.java | 3 +-
.../lucene/search/join/BlockJoinSelector.java | 3 +-
.../lucene/search/join/CheckJoinIndex.java | 3 +-
.../search/join/DocValuesTermsCollector.java | 33 +-
.../apache/lucene/search/join/FakeScorer.java | 3 +-
.../search/join/GenericTermsCollector.java | 31 +-
.../search/join/GlobalOrdinalsCollector.java | 3 +-
.../lucene/search/join/GlobalOrdinalsQuery.java | 3 +-
.../join/GlobalOrdinalsWithScoreCollector.java | 3 +-
.../join/GlobalOrdinalsWithScoreQuery.java | 3 +-
.../org/apache/lucene/search/join/JoinUtil.java | 17 +-
.../lucene/search/join/QueryBitSetProducer.java | 3 +-
.../apache/lucene/search/join/ScoreMode.java | 3 +-
.../lucene/search/join/TermsCollector.java | 3 +-
.../search/join/TermsIncludingScoreQuery.java | 3 +-
.../apache/lucene/search/join/TermsQuery.java | 3 +-
.../search/join/TermsWithScoreCollector.java | 9 +-
.../search/join/ToChildBlockJoinQuery.java | 3 +-
.../search/join/ToParentBlockJoinCollector.java | 3 +-
.../join/ToParentBlockJoinIndexSearcher.java | 3 +-
.../search/join/ToParentBlockJoinQuery.java | 3 +-
.../search/join/ToParentBlockJoinSortField.java | 3 +-
.../lucene/search/join/TestBlockJoin.java | 3 +-
.../search/join/TestBlockJoinSelector.java | 4 +-
.../search/join/TestBlockJoinSorting.java | 3 +-
.../search/join/TestBlockJoinValidation.java | 3 +-
.../lucene/search/join/TestCheckJoinIndex.java | 3 +-
.../apache/lucene/search/join/TestJoinUtil.java | 33 +-
.../apache/lucene/index/memory/MemoryIndex.java | 110 ++-
.../lucene/index/memory/TestMemoryIndex.java | 30 +-
.../memory/TestMemoryIndexAgainstRAMDir.java | 3 +-
.../apache/lucene/document/LazyDocument.java | 17 +-
.../apache/lucene/index/MergeReaderWrapper.java | 3 +-
.../lucene/index/MultiPassIndexSplitter.java | 3 +-
.../apache/lucene/index/PKIndexSplitter.java | 3 +-
.../java/org/apache/lucene/index/Sorter.java | 3 +-
.../apache/lucene/index/SortingLeafReader.java | 3 +-
.../apache/lucene/index/SortingMergePolicy.java | 20 +-
.../org/apache/lucene/misc/GetTermInfo.java | 3 +-
.../org/apache/lucene/misc/HighFreqTerms.java | 3 +-
.../org/apache/lucene/misc/IndexMergeTool.java | 14 +-
.../apache/lucene/misc/SweetSpotSimilarity.java | 1 -
.../java/org/apache/lucene/misc/TermStats.java | 3 +-
.../search/BlockJoinComparatorSource.java | 3 +-
.../search/DiversifiedTopDocsCollector.java | 4 +-
.../EarlyTerminatingSortingCollector.java | 3 +-
.../apache/lucene/store/NativePosixUtil.java | 3 +-
.../lucene/store/NativeUnixDirectory.java | 28 +-
.../org/apache/lucene/store/RAFDirectory.java | 4 +-
.../apache/lucene/store/WindowsDirectory.java | 27 +-
.../apache/lucene/uninverting/DocTermOrds.java | 1 -
.../apache/lucene/uninverting/FieldCache.java | 3 +-
.../lucene/uninverting/FieldCacheImpl.java | 3 +-
.../uninverting/FieldCacheSanityChecker.java | 13 +-
.../lucene/uninverting/UninvertingReader.java | 3 +-
.../apache/lucene/util/fst/ListOfOutputs.java | 3 +-
.../util/fst/UpToTwoPositiveIntOutputs.java | 3 +-
.../apache/lucene/index/IndexSortingTest.java | 3 +-
.../org/apache/lucene/index/SorterTestBase.java | 3 +-
.../lucene/index/SortingLeafReaderTest.java | 3 +-
.../lucene/index/TestBlockJoinSorter.java | 3 +-
.../index/TestMultiPassIndexSplitter.java | 3 +-
.../lucene/index/TestPKIndexSplitter.java | 27 +-
.../lucene/index/TestSortingMergePolicy.java | 22 +-
.../lucene/misc/SweetSpotSimilarityTest.java | 3 -
.../apache/lucene/misc/TestHighFreqTerms.java | 3 +-
.../search/TestDiversifiedTopDocsCollector.java | 3 +-
.../TestEarlyTerminatingSortingCollector.java | 3 +-
.../apache/lucene/store/TestRAFDirectory.java | 3 +-
.../lucene/uninverting/TestDocTermOrds.java | 3 +-
.../lucene/uninverting/TestFieldCache.java | 14 +-
.../uninverting/TestFieldCacheReopen.java | 3 +-
.../TestFieldCacheSanityChecker.java | 14 +-
.../lucene/uninverting/TestFieldCacheSort.java | 3 +-
.../uninverting/TestFieldCacheSortRandom.java | 3 +-
.../uninverting/TestFieldCacheVsDocValues.java | 3 +-
.../uninverting/TestFieldCacheWithThreads.java | 3 +-
.../lucene/uninverting/TestNumericTerms32.java | 3 +-
.../lucene/uninverting/TestNumericTerms64.java | 3 +-
.../uninverting/TestUninvertingReader.java | 3 +-
.../apache/lucene/util/fst/TestFSTsMisc.java | 4 +-
.../apache/lucene/queries/BoostingQuery.java | 3 +-
.../apache/lucene/queries/CommonTermsQuery.java | 4 +-
.../lucene/queries/CustomScoreProvider.java | 3 +-
.../apache/lucene/queries/CustomScoreQuery.java | 3 +-
.../org/apache/lucene/queries/TermsQuery.java | 55 +-
.../lucene/queries/function/BoostedQuery.java | 3 +-
.../lucene/queries/function/FunctionQuery.java | 3 +-
.../queries/function/FunctionRangeQuery.java | 3 +-
.../lucene/queries/function/FunctionValues.java | 3 +-
.../lucene/queries/function/ValueSource.java | 3 +-
.../queries/function/ValueSourceScorer.java | 3 +-
.../function/docvalues/BoolDocValues.java | 3 +-
.../docvalues/DocTermsIndexDocValues.java | 1 -
.../function/docvalues/DoubleDocValues.java | 3 +-
.../function/docvalues/FloatDocValues.java | 3 +-
.../function/docvalues/IntDocValues.java | 3 +-
.../function/docvalues/LongDocValues.java | 3 +-
.../function/docvalues/StrDocValues.java | 3 +-
.../function/valuesource/BoolFunction.java | 1 -
.../valuesource/BytesRefFieldSource.java | 3 +-
.../function/valuesource/ConstNumberSource.java | 1 -
.../function/valuesource/ConstValueSource.java | 1 -
.../function/valuesource/DefFunction.java | 3 +-
.../function/valuesource/DivFloatFunction.java | 1 -
.../valuesource/DocFreqValueSource.java | 1 -
.../valuesource/DoubleConstValueSource.java | 1 -
.../function/valuesource/DoubleFieldSource.java | 1 -
.../function/valuesource/DualFloatFunction.java | 1 -
.../function/valuesource/EnumFieldSource.java | 3 +-
.../function/valuesource/FieldCacheSource.java | 1 -
.../function/valuesource/FloatFieldSource.java | 1 -
.../function/valuesource/IDFValueSource.java | 1 -
.../function/valuesource/IfFunction.java | 1 -
.../function/valuesource/IntFieldSource.java | 1 -
.../valuesource/JoinDocFreqValueSource.java | 1 -
.../valuesource/LinearFloatFunction.java | 1 -
.../valuesource/LiteralValueSource.java | 3 +-
.../function/valuesource/LongFieldSource.java | 1 -
.../function/valuesource/MaxFloatFunction.java | 1 -
.../function/valuesource/MinFloatFunction.java | 1 -
.../function/valuesource/MultiBoolFunction.java | 1 -
.../valuesource/MultiFloatFunction.java | 3 +-
.../function/valuesource/MultiFunction.java | 3 +-
.../function/valuesource/MultiValueSource.java | 3 +-
.../function/valuesource/NormValueSource.java | 1 -
.../function/valuesource/PowFloatFunction.java | 1 -
.../valuesource/ProductFloatFunction.java | 1 -
.../function/valuesource/QueryValueSource.java | 1 -
.../valuesource/RangeMapFloatFunction.java | 1 -
.../valuesource/ReciprocalFloatFunction.java | 1 -
.../valuesource/ScaleFloatFunction.java | 1 -
.../valuesource/SimpleBoolFunction.java | 1 -
.../valuesource/SimpleFloatFunction.java | 1 -
.../function/valuesource/SingleFunction.java | 1 -
.../valuesource/SortedSetFieldSource.java | 3 +-
.../function/valuesource/SumFloatFunction.java | 1 -
.../SumTotalTermFreqValueSource.java | 1 -
.../function/valuesource/TFValueSource.java | 3 +-
.../valuesource/TermFreqValueSource.java | 1 -
.../valuesource/TotalTermFreqValueSource.java | 1 -
.../function/valuesource/VectorValueSource.java | 3 +-
.../apache/lucene/queries/mlt/MoreLikeThis.java | 14 +-
.../lucene/queries/mlt/MoreLikeThisQuery.java | 6 +-
.../payloads/AveragePayloadFunction.java | 4 +-
.../queries/payloads/MaxPayloadFunction.java | 4 +-
.../queries/payloads/MinPayloadFunction.java | 3 +-
.../queries/payloads/PayloadFunction.java | 3 +-
.../queries/payloads/PayloadScoreQuery.java | 3 +-
.../queries/payloads/SpanPayloadCheckQuery.java | 3 +-
.../lucene/queries/BoostingQueryTest.java | 3 +-
.../lucene/queries/CommonTermsQueryTest.java | 9 +-
.../apache/lucene/queries/TermsQueryTest.java | 3 +-
.../queries/TestCustomScoreExplanations.java | 3 +-
.../lucene/queries/TestCustomScoreQuery.java | 3 +-
.../queries/function/FunctionTestSetup.java | 32 +-
.../queries/function/TestBoostedQuery.java | 33 +-
.../function/TestDocValuesFieldSources.java | 3 +-
.../queries/function/TestFieldScoreQuery.java | 3 +-
.../function/TestFunctionQueryExplanations.java | 3 +-
.../queries/function/TestFunctionQuerySort.java | 3 +-
.../function/TestFunctionRangeQuery.java | 3 +-
.../function/TestLongNormValueSource.java | 3 +-
.../function/TestSortedSetFieldSource.java | 3 +-
.../queries/function/TestValueSources.java | 3 +-
.../lucene/queries/mlt/TestMoreLikeThis.java | 3 +-
.../lucene/queries/payloads/PayloadHelper.java | 3 +-
.../queries/payloads/TestPayloadCheckQuery.java | 3 +-
.../payloads/TestPayloadExplanations.java | 3 +-
.../queries/payloads/TestPayloadScoreQuery.java | 3 +-
.../queries/payloads/TestPayloadSpans.java | 14 +-
.../queries/payloads/TestPayloadTermQuery.java | 3 +-
.../analyzing/AnalyzingQueryParser.java | 3 +-
.../queryparser/classic/FastCharStream.java | 5 +-
.../classic/MultiFieldQueryParser.java | 3 +-
.../queryparser/classic/QueryParserBase.java | 1 -
.../complexPhrase/ComplexPhraseQueryParser.java | 3 +-
.../queryparser/ext/ExtendableQueryParser.java | 3 +-
.../lucene/queryparser/ext/ExtensionQuery.java | 7 +-
.../lucene/queryparser/ext/Extensions.java | 4 +-
.../lucene/queryparser/ext/ParserExtension.java | 3 +-
.../flexible/core/QueryNodeError.java | 3 +-
.../flexible/core/QueryNodeException.java | 3 +-
.../flexible/core/QueryNodeParseException.java | 3 +-
.../flexible/core/QueryParserHelper.java | 16 +-
.../flexible/core/builders/QueryBuilder.java | 8 +-
.../core/builders/QueryTreeBuilder.java | 3 +-
.../core/config/AbstractQueryConfig.java | 3 +-
.../flexible/core/config/ConfigurationKey.java | 3 +-
.../flexible/core/config/FieldConfig.java | 3 +-
.../core/config/FieldConfigListener.java | 3 +-
.../core/config/QueryConfigHandler.java | 3 +-
.../core/messages/QueryParserMessages.java | 3 +-
.../flexible/core/nodes/AndQueryNode.java | 3 +-
.../flexible/core/nodes/AnyQueryNode.java | 3 +-
.../flexible/core/nodes/BooleanQueryNode.java | 3 +-
.../flexible/core/nodes/BoostQueryNode.java | 3 +-
.../flexible/core/nodes/DeletedQueryNode.java | 7 +-
.../flexible/core/nodes/FieldQueryNode.java | 3 +-
.../core/nodes/FieldValuePairQueryNode.java | 3 +-
.../flexible/core/nodes/FieldableNode.java | 3 +-
.../flexible/core/nodes/FuzzyQueryNode.java | 3 +-
.../flexible/core/nodes/GroupQueryNode.java | 3 +-
.../core/nodes/MatchAllDocsQueryNode.java | 3 +-
.../core/nodes/MatchNoDocsQueryNode.java | 3 +-
.../flexible/core/nodes/ModifierQueryNode.java | 3 +-
.../core/nodes/NoTokenFoundQueryNode.java | 3 +-
.../flexible/core/nodes/OpaqueQueryNode.java | 3 +-
.../flexible/core/nodes/OrQueryNode.java | 3 +-
.../flexible/core/nodes/PathQueryNode.java | 3 +-
.../core/nodes/PhraseSlopQueryNode.java | 3 +-
.../flexible/core/nodes/ProximityQueryNode.java | 3 +-
.../flexible/core/nodes/QueryNode.java | 3 +-
.../flexible/core/nodes/QueryNodeImpl.java | 3 +-
.../core/nodes/QuotedFieldQueryNode.java | 3 +-
.../flexible/core/nodes/RangeQueryNode.java | 7 +-
.../flexible/core/nodes/SlopQueryNode.java | 3 +-
.../flexible/core/nodes/TextableQueryNode.java | 27 +-
.../core/nodes/TokenizedPhraseQueryNode.java | 3 +-
.../flexible/core/nodes/ValueQueryNode.java | 27 +-
.../flexible/core/parser/EscapeQuerySyntax.java | 3 +-
.../flexible/core/parser/SyntaxParser.java | 3 +-
.../NoChildOptimizationQueryNodeProcessor.java | 3 +-
.../core/processors/QueryNodeProcessor.java | 3 +-
.../core/processors/QueryNodeProcessorImpl.java | 3 +-
.../processors/QueryNodeProcessorPipeline.java | 3 +-
.../RemoveDeletedQueryNodesProcessor.java | 3 +-
.../flexible/core/util/QueryNodeOperation.java | 3 +-
.../flexible/core/util/StringUtils.java | 3 +-
.../core/util/UnescapedCharSequence.java | 7 +-
.../queryparser/flexible/messages/Message.java | 3 +-
.../flexible/messages/MessageImpl.java | 3 +-
.../queryparser/flexible/messages/NLS.java | 3 +-
.../flexible/messages/NLSException.java | 3 +-
.../precedence/PrecedenceQueryParser.java | 3 +-
.../BooleanModifiersQueryNodeProcessor.java | 3 +-
.../PrecedenceQueryNodeProcessorPipeline.java | 3 +-
.../CommonQueryParserConfiguration.java | 23 +-
.../flexible/standard/QueryParserUtil.java | 3 +-
.../flexible/standard/StandardQueryParser.java | 3 +-
.../standard/builders/AnyQueryNodeBuilder.java | 3 +-
.../builders/BooleanQueryNodeBuilder.java | 3 +-
.../builders/BoostQueryNodeBuilder.java | 3 +-
.../builders/DummyQueryNodeBuilder.java | 3 +-
.../builders/FieldQueryNodeBuilder.java | 3 +-
.../builders/FuzzyQueryNodeBuilder.java | 3 +-
.../builders/GroupQueryNodeBuilder.java | 3 +-
.../builders/MatchAllDocsQueryNodeBuilder.java | 3 +-
.../builders/MatchNoDocsQueryNodeBuilder.java | 3 +-
.../builders/ModifierQueryNodeBuilder.java | 3 +-
.../builders/MultiPhraseQueryNodeBuilder.java | 3 +-
.../builders/NumericRangeQueryNodeBuilder.java | 3 +-
.../builders/PhraseQueryNodeBuilder.java | 3 +-
.../PrefixWildcardQueryNodeBuilder.java | 3 +-
.../builders/RegexpQueryNodeBuilder.java | 3 +-
.../standard/builders/SlopQueryNodeBuilder.java | 3 +-
.../StandardBooleanQueryNodeBuilder.java | 3 +-
.../standard/builders/StandardQueryBuilder.java | 3 +-
.../builders/StandardQueryTreeBuilder.java | 3 +-
.../builders/TermRangeQueryNodeBuilder.java | 3 +-
.../builders/WildcardQueryNodeBuilder.java | 3 +-
.../config/FieldBoostMapFCListener.java | 3 +-
.../config/FieldDateResolutionFCListener.java | 3 +-
.../flexible/standard/config/FuzzyConfig.java | 3 +-
.../standard/config/NumberDateFormat.java | 3 +-
.../flexible/standard/config/NumericConfig.java | 3 +-
.../config/NumericFieldConfigListener.java | 3 +-
.../config/StandardQueryConfigHandler.java | 3 +-
.../standard/nodes/AbstractRangeQueryNode.java | 3 +-
.../standard/nodes/BooleanModifierNode.java | 3 +-
.../standard/nodes/MultiPhraseQueryNode.java | 3 +-
.../standard/nodes/NumericQueryNode.java | 3 +-
.../standard/nodes/NumericRangeQueryNode.java | 27 +-
.../standard/nodes/PrefixWildcardQueryNode.java | 7 +-
.../standard/nodes/RegexpQueryNode.java | 3 +-
.../nodes/StandardBooleanQueryNode.java | 3 +-
.../standard/nodes/TermRangeQueryNode.java | 3 +-
.../standard/nodes/WildcardQueryNode.java | 3 +-
.../standard/parser/EscapeQuerySyntaxImpl.java | 3 +-
.../standard/parser/FastCharStream.java | 5 +-
.../AllowLeadingWildcardProcessor.java | 3 +-
.../processors/AnalyzerQueryNodeProcessor.java | 3 +-
.../BooleanQuery2ModifierNodeProcessor.java | 3 +-
...ngleChildOptimizationQueryNodeProcessor.java | 3 +-
.../processors/BoostQueryNodeProcessor.java | 3 +-
.../DefaultPhraseSlopQueryNodeProcessor.java | 3 +-
.../processors/FuzzyQueryNodeProcessor.java | 3 +-
...owercaseExpandedTermsQueryNodeProcessor.java | 3 +-
.../MatchAllDocsQueryNodeProcessor.java | 3 +-
.../MultiFieldQueryNodeProcessor.java | 3 +-
.../MultiTermRewriteMethodProcessor.java | 3 +-
.../processors/NumericQueryNodeProcessor.java | 3 +-
.../NumericRangeQueryNodeProcessor.java | 3 +-
.../processors/OpenRangeQueryNodeProcessor.java | 3 +-
.../PhraseSlopQueryNodeProcessor.java | 3 +-
.../RemoveEmptyNonLeafQueryNodeProcessor.java | 3 +-
.../StandardQueryNodeProcessorPipeline.java | 3 +-
.../processors/TermRangeQueryNodeProcessor.java | 3 +-
.../processors/WildcardQueryNodeProcessor.java | 3 +-
.../queryparser/simple/SimpleQueryParser.java | 3 +-
.../surround/parser/FastCharStream.java | 3 +-
.../queryparser/surround/query/AndQuery.java | 4 +-
.../surround/query/BasicQueryFactory.java | 3 +-
.../surround/query/ComposedQuery.java | 3 +-
.../surround/query/DistanceQuery.java | 3 +-
.../surround/query/DistanceRewriteQuery.java | 3 +-
.../surround/query/DistanceSubQuery.java | 3 +-
.../queryparser/surround/query/FieldsQuery.java | 3 +-
.../queryparser/surround/query/NotQuery.java | 3 +-
.../queryparser/surround/query/OrQuery.java | 3 +-
.../surround/query/RewriteQuery.java | 2 +-
.../queryparser/surround/query/SimpleTerm.java | 3 +-
.../surround/query/SimpleTermRewriteQuery.java | 2 +-
.../surround/query/SpanNearClauseFactory.java | 3 +-
.../surround/query/SrndBooleanQuery.java | 3 +-
.../surround/query/SrndPrefixQuery.java | 3 +-
.../queryparser/surround/query/SrndQuery.java | 3 +-
.../surround/query/SrndTermQuery.java | 3 +-
.../surround/query/SrndTruncQuery.java | 3 +-
.../surround/query/TooManyBasicQueries.java | 3 +-
.../lucene/queryparser/xml/CoreParser.java | 27 +-
.../xml/CorePlusExtensionsParser.java | 11 +-
.../queryparser/xml/CorePlusQueriesParser.java | 13 +-
.../apache/lucene/queryparser/xml/DOMUtils.java | 19 +-
.../lucene/queryparser/xml/ParserException.java | 5 +-
.../lucene/queryparser/xml/QueryBuilder.java | 8 +-
.../queryparser/xml/QueryBuilderFactory.java | 15 +-
.../queryparser/xml/QueryTemplateManager.java | 33 +-
.../xml/builders/BooleanQueryBuilder.java | 27 +-
.../xml/builders/BoostingQueryBuilder.java | 17 +-
.../xml/builders/BoostingTermBuilder.java | 23 +-
.../xml/builders/ConstantScoreQueryBuilder.java | 19 +-
.../builders/DisjunctionMaxQueryBuilder.java | 3 +-
.../xml/builders/FuzzyLikeThisQueryBuilder.java | 25 +-
.../LegacyNumericRangeQueryBuilder.java | 3 +-
.../xml/builders/LikeThisQueryBuilder.java | 31 +-
.../xml/builders/MatchAllDocsQueryBuilder.java | 13 +-
.../xml/builders/RangeQueryBuilder.java | 6 +-
.../xml/builders/SpanBuilderBase.java | 11 +-
.../xml/builders/SpanFirstBuilder.java | 15 +-
.../xml/builders/SpanNearBuilder.java | 23 +-
.../xml/builders/SpanNotBuilder.java | 15 +-
.../queryparser/xml/builders/SpanOrBuilder.java | 23 +-
.../xml/builders/SpanOrTermsBuilder.java | 33 +-
.../xml/builders/SpanQueryBuilder.java | 3 +-
.../xml/builders/SpanQueryBuilderFactory.java | 17 +-
.../xml/builders/SpanTermBuilder.java | 17 +-
.../xml/builders/TermQueryBuilder.java | 19 +-
.../xml/builders/TermsQueryBuilder.java | 33 +-
.../xml/builders/UserInputQueryBuilder.java | 23 +-
.../analyzing/TestAnalyzingQueryParser.java | 3 +-
.../queryparser/classic/TestMultiAnalyzer.java | 3 +-
.../classic/TestMultiFieldQueryParser.java | 3 +-
.../classic/TestMultiPhraseQueryParsing.java | 3 +-
.../queryparser/classic/TestQueryParser.java | 3 +-
.../complexPhrase/TestComplexPhraseQuery.java | 3 +-
.../lucene/queryparser/ext/ExtensionStub.java | 13 +-
.../ext/TestExtendableQueryParser.java | 3 +-
.../lucene/queryparser/ext/TestExtensions.java | 3 +-
.../core/builders/TestQueryTreeBuilder.java | 3 +-
.../flexible/core/nodes/TestQueryNode.java | 3 +-
.../flexible/messages/MessagesTestBundle.java | 27 +-
.../queryparser/flexible/messages/TestNLS.java | 3 +-
.../precedence/TestPrecedenceQueryParser.java | 3 +-
.../flexible/spans/SpanOrQueryNodeBuilder.java | 3 +-
.../spans/SpanTermQueryNodeBuilder.java | 3 +-
.../flexible/spans/SpansQueryConfigHandler.java | 3 +-
.../flexible/spans/SpansQueryTreeBuilder.java | 3 +-
.../spans/SpansValidatorQueryNodeProcessor.java | 3 +-
.../flexible/spans/TestSpanQueryParser.java | 3 +-
.../spans/TestSpanQueryParserSimpleSample.java | 3 +-
.../flexible/spans/UniqueFieldAttribute.java | 3 +-
.../spans/UniqueFieldAttributeImpl.java | 3 +-
.../spans/UniqueFieldQueryNodeProcessor.java | 3 +-
.../standard/TestMultiAnalyzerQPHelper.java | 3 +-
.../standard/TestMultiFieldQPHelper.java | 3 +-
.../standard/TestNumericQueryParser.java | 3 +-
.../flexible/standard/TestQPHelper.java | 3 +-
.../flexible/standard/TestStandardQP.java | 3 +-
.../simple/TestSimpleQueryParser.java | 3 +-
.../surround/query/BooleanQueryTst.java | 3 +-
.../surround/query/ExceptionQueryTst.java | 3 +-
.../surround/query/SingleFieldTestDb.java | 3 +-
.../surround/query/SrndQueryTest.java | 3 +-
.../surround/query/Test01Exceptions.java | 3 +-
.../surround/query/Test02Boolean.java | 3 +-
.../surround/query/Test03Distance.java | 3 +-
.../queryparser/util/QueryParserTestBase.java | 3 +-
.../lucene/queryparser/xml/TestCoreParser.java | 9 +-
.../xml/TestCorePlusExtensionsParser.java | 9 +-
.../xml/TestCorePlusQueriesParser.java | 9 +-
.../xml/TestQueryTemplateManager.java | 3 +-
.../builders/TestNumericRangeQueryBuilder.java | 3 +-
.../IndexAndTaxonomyReplicationHandler.java | 3 +-
.../replicator/IndexAndTaxonomyRevision.java | 3 +-
.../replicator/IndexInputInputStream.java | 3 +-
.../replicator/IndexReplicationHandler.java | 3 +-
.../apache/lucene/replicator/IndexRevision.java | 3 +-
.../lucene/replicator/LocalReplicator.java | 3 +-
.../replicator/PerSessionDirectoryFactory.java | 3 +-
.../lucene/replicator/ReplicationClient.java | 3 +-
.../apache/lucene/replicator/Replicator.java | 3 +-
.../org/apache/lucene/replicator/Revision.java | 3 +-
.../apache/lucene/replicator/RevisionFile.java | 3 +-
.../replicator/SessionExpiredException.java | 3 +-
.../apache/lucene/replicator/SessionToken.java | 3 +-
.../lucene/replicator/http/HttpClientBase.java | 3 +-
.../lucene/replicator/http/HttpReplicator.java | 3 +-
.../replicator/http/ReplicationService.java | 3 +-
.../lucene/replicator/nrt/ReplicaNode.java | 5 +
.../IndexAndTaxonomyReplicationClientTest.java | 20 +-
.../IndexAndTaxonomyRevisionTest.java | 11 +-
.../replicator/IndexReplicationClientTest.java | 3 +-
.../lucene/replicator/IndexRevisionTest.java | 7 +-
.../lucene/replicator/LocalReplicatorTest.java | 93 +-
.../lucene/replicator/ReplicatorTestCase.java | 3 +-
.../lucene/replicator/SessionTokenTest.java | 3 +-
.../replicator/http/HttpReplicatorTest.java | 8 +-
.../replicator/http/ReplicationServlet.java | 3 +-
.../replicator/nrt/TestNRTReplication.java | 6 +-
lucene/sandbox/build.xml | 25 +
.../idversion/IDVersionPostingsFormat.java | 3 +-
.../idversion/IDVersionPostingsReader.java | 3 +-
.../idversion/IDVersionPostingsWriter.java | 3 +-
.../idversion/IDVersionSegmentTermsEnum.java | 3 +-
.../IDVersionSegmentTermsEnumFrame.java | 3 +-
.../codecs/idversion/IDVersionTermState.java | 3 +-
.../lucene/codecs/idversion/SingleDocsEnum.java | 3 +-
.../codecs/idversion/SinglePostingsEnum.java | 3 +-
.../idversion/VersionBlockTreeTermsReader.java | 3 +-
.../idversion/VersionBlockTreeTermsWriter.java | 3 +-
.../codecs/idversion/VersionFieldReader.java | 3 +-
.../apache/lucene/document/GeoPointField.java | 132 ---
.../org/apache/lucene/document/LatLonPoint.java | 5 +-
.../lucene/payloads/PayloadSpanCollector.java | 3 +-
.../apache/lucene/payloads/PayloadSpanUtil.java | 3 +-
.../sandbox/queries/FuzzyLikeThisQuery.java | 3 +-
.../lucene/sandbox/queries/SlowFuzzyQuery.java | 3 +-
.../sandbox/queries/SlowFuzzyTermsEnum.java | 3 +-
.../lucene/search/DocValuesNumbersQuery.java | 3 +-
.../lucene/search/DocValuesRangeQuery.java | 3 +-
.../lucene/search/DocValuesTermsQuery.java | 3 +-
.../apache/lucene/search/GeoBoundingBox.java | 47 -
.../lucene/search/GeoPointDistanceQuery.java | 171 ----
.../search/GeoPointDistanceQueryImpl.java | 132 ---
.../search/GeoPointDistanceRangeQuery.java | 103 --
.../lucene/search/GeoPointInBBoxQuery.java | 153 ---
.../lucene/search/GeoPointInBBoxQueryImpl.java | 163 ---
.../lucene/search/GeoPointInPolygonQuery.java | 195 ----
.../apache/lucene/search/GeoPointTermQuery.java | 71 --
.../GeoPointTermQueryConstantScoreWrapper.java | 130 ---
.../apache/lucene/search/GeoPointTermsEnum.java | 250 -----
.../lucene/search/PointInPolygonQuery.java | 11 +-
.../apache/lucene/search/PointInRectQuery.java | 5 +-
.../lucene/search/TermAutomatonQuery.java | 3 +-
.../lucene/search/TermAutomatonScorer.java | 3 +-
.../search/TokenStreamToTermAutomatonQuery.java | 3 +-
.../apache/lucene/util/GeoDistanceUtils.java | 216 ----
.../org/apache/lucene/util/GeoHashUtils.java | 274 -----
.../apache/lucene/util/GeoProjectionUtils.java | 438 --------
.../java/org/apache/lucene/util/GeoRect.java | 67 --
.../apache/lucene/util/GeoRelationUtils.java | 428 --------
.../java/org/apache/lucene/util/GeoUtils.java | 238 -----
.../java/org/apache/lucene/util/package.html | 28 -
.../codecs/idversion/StringAndPayloadField.java | 7 +-
.../idversion/TestIDVersionPostingsFormat.java | 3 +-
.../lucene/payloads/TestPayloadSpanUtil.java | 3 +-
.../sandbox/queries/FuzzyLikeThisQueryTest.java | 3 +-
.../sandbox/queries/TestSlowFuzzyQuery.java | 3 +-
.../sandbox/queries/TestSlowFuzzyQuery2.java | 3 +-
.../search/TestDocValuesNumbersQuery.java | 3 +-
.../lucene/search/TestDocValuesRangeQuery.java | 3 +-
.../lucene/search/TestDocValuesTermsQuery.java | 3 +-
.../search/TestFieldCacheTermsFilter.java | 3 +-
.../apache/lucene/search/TestGeoPointQuery.java | 351 -------
.../lucene/search/TestLatLonPointQueries.java | 61 +-
.../lucene/search/TestTermAutomatonQuery.java | 3 +-
.../lucene/util/BaseGeoPointTestCase.java | 782 ---------------
.../org/apache/lucene/util/TestGeoUtils.java | 537 ----------
.../apache/lucene/spatial/SpatialStrategy.java | 3 +-
.../spatial/bbox/BBoxSimilarityValueSource.java | 3 +-
.../lucene/spatial/bbox/BBoxStrategy.java | 3 +-
.../lucene/spatial/bbox/BBoxValueSource.java | 3 +-
.../composite/CompositeSpatialStrategy.java | 3 +-
.../spatial/composite/CompositeVerifyQuery.java | 3 +-
.../composite/IntersectsRPTVerifyQuery.java | 3 +-
.../lucene/spatial/document/GeoPointField.java | 250 +++++
.../spatial/document/GeoPointTokenStream.java | 233 +++++
.../lucene/spatial/document/package-info.java | 21 +
.../spatial/prefix/AbstractPrefixTreeQuery.java | 3 +-
.../prefix/AbstractVisitingPrefixTreeQuery.java | 3 +-
.../prefix/BytesRefIteratorTokenStream.java | 3 +-
.../spatial/prefix/CellToBytesRefIterator.java | 3 +-
.../spatial/prefix/ContainsPrefixTreeQuery.java | 3 +-
.../spatial/prefix/HeatmapFacetCounter.java | 3 +-
.../prefix/IntersectsPrefixTreeQuery.java | 3 +-
.../prefix/NumberRangePrefixTreeStrategy.java | 3 +-
.../PointPrefixTreeFieldCacheProvider.java | 3 +-
.../spatial/prefix/PrefixTreeFacetCounter.java | 3 +-
.../spatial/prefix/PrefixTreeStrategy.java | 3 +-
.../prefix/RecursivePrefixTreeStrategy.java | 3 +-
.../prefix/TermQueryPrefixTreeStrategy.java | 3 +-
.../spatial/prefix/WithinPrefixTreeQuery.java | 3 +-
.../apache/lucene/spatial/prefix/tree/Cell.java | 3 +-
.../spatial/prefix/tree/CellIterator.java | 3 +-
.../prefix/tree/DateRangePrefixTree.java | 3 +-
.../spatial/prefix/tree/FilterCellIterator.java | 3 +-
.../spatial/prefix/tree/GeohashPrefixTree.java | 3 +-
.../lucene/spatial/prefix/tree/LegacyCell.java | 3 +-
.../spatial/prefix/tree/LegacyPrefixTree.java | 3 +-
.../prefix/tree/NumberRangePrefixTree.java | 3 +-
.../prefix/tree/PackedQuadPrefixTree.java | 3 +-
.../spatial/prefix/tree/QuadPrefixTree.java | 3 +-
.../prefix/tree/SingletonCellIterator.java | 3 +-
.../spatial/prefix/tree/SpatialPrefixTree.java | 3 +-
.../prefix/tree/SpatialPrefixTreeFactory.java | 1 -
.../spatial/prefix/tree/TreeCellIterator.java | 3 +-
.../lucene/spatial/query/SpatialArgs.java | 3 +-
.../lucene/spatial/query/SpatialArgsParser.java | 3 +-
.../lucene/spatial/query/SpatialOperation.java | 3 +-
.../query/UnsupportedSpatialOperation.java | 3 +-
.../lucene/spatial/search/GeoBoundingBox.java | 53 +
.../spatial/search/GeoPointDistanceQuery.java | 189 ++++
.../search/GeoPointDistanceQueryImpl.java | 107 ++
.../search/GeoPointDistanceRangeQuery.java | 116 +++
.../spatial/search/GeoPointInBBoxQuery.java | 173 ++++
.../spatial/search/GeoPointInBBoxQueryImpl.java | 156 +++
.../spatial/search/GeoPointInPolygonQuery.java | 153 +++
.../search/GeoPointInPolygonQueryImpl.java | 103 ++
.../spatial/search/GeoPointMultiTermQuery.java | 166 ++++
.../search/GeoPointNumericTermsEnum.java | 161 +++
.../spatial/search/GeoPointPrefixTermsEnum.java | 237 +++++
.../GeoPointTermQueryConstantScoreWrapper.java | 141 +++
.../spatial/search/GeoPointTermsEnum.java | 140 +++
.../lucene/spatial/search/package-info.java | 21 +
.../serialized/SerializedDVStrategy.java | 3 +-
.../lucene/spatial/spatial4j/Geo3dShape.java | 3 +-
.../spatial/util/CachingDoubleValueSource.java | 1 -
.../util/DistanceToShapeValueSource.java | 3 +-
.../lucene/spatial/util/GeoDistanceUtils.java | 223 +++++
.../lucene/spatial/util/GeoEncodingUtils.java | 157 +++
.../lucene/spatial/util/GeoHashUtils.java | 283 ++++++
.../lucene/spatial/util/GeoProjectionUtils.java | 465 +++++++++
.../org/apache/lucene/spatial/util/GeoRect.java | 66 ++
.../lucene/spatial/util/GeoRelationUtils.java | 520 ++++++++++
.../apache/lucene/spatial/util/GeoUtils.java | 187 ++++
.../spatial/util/ShapeAreaValueSource.java | 3 +-
.../lucene/spatial/util/ShapeFieldCache.java | 1 -
.../ShapeFieldCacheDistanceValueSource.java | 3 +-
.../spatial/util/ShapeFieldCacheProvider.java | 1 -
.../spatial/util/ShapePredicateValueSource.java | 3 +-
.../spatial/vector/DistanceValueSource.java | 3 +-
.../spatial/vector/PointVectorStrategy.java | 3 +-
.../lucene/spatial/DistanceStrategyTest.java | 3 +-
.../apache/lucene/spatial/PortedSolr3Test.java | 3 +-
.../lucene/spatial/QueryEqualsHashCodeTest.java | 3 +-
.../apache/lucene/spatial/SpatialArgsTest.java | 3 +-
.../apache/lucene/spatial/SpatialExample.java | 3 +-
.../lucene/spatial/SpatialMatchConcern.java | 1 -
.../apache/lucene/spatial/SpatialTestCase.java | 3 +-
.../apache/lucene/spatial/SpatialTestData.java | 3 +-
.../apache/lucene/spatial/SpatialTestQuery.java | 3 +-
.../apache/lucene/spatial/StrategyTestCase.java | 5 +-
.../lucene/spatial/TestTestFramework.java | 3 +-
.../lucene/spatial/bbox/TestBBoxStrategy.java | 3 +-
.../composite/CompositeStrategyTest.java | 3 +-
.../prefix/CellToBytesRefIterator50.java | 3 +-
.../spatial/prefix/DateNRStrategyTest.java | 3 +-
.../spatial/prefix/HeatmapFacetCounterTest.java | 3 +-
.../lucene/spatial/prefix/JtsPolygonTest.java | 3 +-
.../spatial/prefix/NumberRangeFacetsTest.java | 3 +-
.../RandomSpatialOpFuzzyPrefixTree50Test.java | 3 +-
.../RandomSpatialOpFuzzyPrefixTreeTest.java | 3 +-
.../prefix/RandomSpatialOpStrategyTestCase.java | 3 +-
.../prefix/TestRecursivePrefixTreeStrategy.java | 3 +-
.../prefix/TestTermQueryPrefixGridStrategy.java | 3 +-
.../prefix/tree/DateRangePrefixTreeTest.java | 3 +-
.../prefix/tree/SpatialPrefixTreeTest.java | 3 +-
.../spatial/query/SpatialArgsParserTest.java | 3 +-
.../spatial/search/TestGeoPointQuery.java | 404 ++++++++
.../serialized/SerializedStrategyTest.java | 3 +-
.../lucene/spatial/spatial4j/Geo3dRptTest.java | 3 +-
.../Geo3dShapeRectRelationTestCase.java | 3 +-
.../Geo3dShapeSphereModelRectRelationTest.java | 3 +-
.../Geo3dShapeWGS84ModelRectRelationTest.java | 3 +-
.../spatial4j/RandomizedShapeTestCase.java | 4 +-
.../spatial/spatial4j/geo3d/GeoPointTest.java | 3 +-
.../spatial/util/BaseGeoPointTestCase.java | 760 ++++++++++++++
.../lucene/spatial/util/TestGeoUtils.java | 565 +++++++++++
.../spatial/vector/TestPointVectorStrategy.java | 3 +-
.../org/apache/lucene/geo3d/ArcDistance.java | 3 +-
.../apache/lucene/geo3d/BasePlanetObject.java | 3 +-
.../org/apache/lucene/geo3d/BaseXYZSolid.java | 3 +-
.../java/org/apache/lucene/geo3d/Bounds.java | 3 +-
.../org/apache/lucene/geo3d/DistanceStyle.java | 3 +-
.../org/apache/lucene/geo3d/Geo3DPoint.java | 3 +-
.../java/org/apache/lucene/geo3d/Geo3DUtil.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoArea.java | 3 +-
.../org/apache/lucene/geo3d/GeoAreaFactory.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoBBox.java | 3 +-
.../org/apache/lucene/geo3d/GeoBBoxFactory.java | 3 +-
.../org/apache/lucene/geo3d/GeoBaseBBox.java | 3 +-
.../org/apache/lucene/geo3d/GeoBaseCircle.java | 3 +-
.../lucene/geo3d/GeoBaseDistanceShape.java | 3 +-
.../lucene/geo3d/GeoBaseMembershipShape.java | 3 +-
.../org/apache/lucene/geo3d/GeoBasePolygon.java | 3 +-
.../org/apache/lucene/geo3d/GeoBaseShape.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoCircle.java | 3 +-
.../apache/lucene/geo3d/GeoCircleFactory.java | 3 +-
.../geo3d/GeoCompositeMembershipShape.java | 3 +-
.../lucene/geo3d/GeoCompositePolygon.java | 3 +-
.../apache/lucene/geo3d/GeoConvexPolygon.java | 3 +-
.../geo3d/GeoDegenerateHorizontalLine.java | 3 +-
.../lucene/geo3d/GeoDegenerateLatitudeZone.java | 3 +-
.../geo3d/GeoDegenerateLongitudeSlice.java | 3 +-
.../apache/lucene/geo3d/GeoDegeneratePoint.java | 5 +-
.../lucene/geo3d/GeoDegenerateVerticalLine.java | 3 +-
.../org/apache/lucene/geo3d/GeoDistance.java | 3 +-
.../apache/lucene/geo3d/GeoDistanceShape.java | 3 +-
.../apache/lucene/geo3d/GeoLatitudeZone.java | 3 +-
.../apache/lucene/geo3d/GeoLongitudeSlice.java | 3 +-
.../apache/lucene/geo3d/GeoMembershipShape.java | 3 +-
.../lucene/geo3d/GeoNorthLatitudeZone.java | 3 +-
.../apache/lucene/geo3d/GeoNorthRectangle.java | 3 +-
.../apache/lucene/geo3d/GeoOutsideDistance.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoPath.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoPoint.java | 5 +-
.../org/apache/lucene/geo3d/GeoPolygon.java | 3 +-
.../apache/lucene/geo3d/GeoPolygonFactory.java | 3 +-
.../org/apache/lucene/geo3d/GeoRectangle.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoShape.java | 3 +-
.../org/apache/lucene/geo3d/GeoSizeable.java | 3 +-
.../lucene/geo3d/GeoSouthLatitudeZone.java | 3 +-
.../apache/lucene/geo3d/GeoSouthRectangle.java | 3 +-
.../apache/lucene/geo3d/GeoStandardCircle.java | 3 +-
.../geo3d/GeoWideDegenerateHorizontalLine.java | 3 +-
.../lucene/geo3d/GeoWideLongitudeSlice.java | 3 +-
.../lucene/geo3d/GeoWideNorthRectangle.java | 3 +-
.../apache/lucene/geo3d/GeoWideRectangle.java | 3 +-
.../lucene/geo3d/GeoWideSouthRectangle.java | 3 +-
.../java/org/apache/lucene/geo3d/GeoWorld.java | 3 +-
.../org/apache/lucene/geo3d/LatLonBounds.java | 3 +-
.../org/apache/lucene/geo3d/LinearDistance.java | 3 +-
.../lucene/geo3d/LinearSquaredDistance.java | 3 +-
.../org/apache/lucene/geo3d/Membership.java | 3 +-
.../org/apache/lucene/geo3d/NormalDistance.java | 3 +-
.../lucene/geo3d/NormalSquaredDistance.java | 3 +-
.../src/java/org/apache/lucene/geo3d/Plane.java | 3 +-
.../org/apache/lucene/geo3d/PlanetModel.java | 3 +-
.../lucene/geo3d/PointInGeo3DShapeQuery.java | 3 +-
.../org/apache/lucene/geo3d/SidedPlane.java | 3 +-
.../apache/lucene/geo3d/StandardXYZSolid.java | 3 +-
.../src/java/org/apache/lucene/geo3d/Tools.java | 3 +-
.../java/org/apache/lucene/geo3d/Vector.java | 3 +-
.../java/org/apache/lucene/geo3d/XYZBounds.java | 3 +-
.../java/org/apache/lucene/geo3d/XYZSolid.java | 3 +-
.../apache/lucene/geo3d/XYZSolidFactory.java | 3 +-
.../java/org/apache/lucene/geo3d/XYdZSolid.java | 3 +-
.../java/org/apache/lucene/geo3d/XdYZSolid.java | 3 +-
.../org/apache/lucene/geo3d/XdYdZSolid.java | 3 +-
.../java/org/apache/lucene/geo3d/dXYZSolid.java | 3 +-
.../org/apache/lucene/geo3d/dXYdZSolid.java | 3 +-
.../org/apache/lucene/geo3d/dXdYZSolid.java | 3 +-
.../org/apache/lucene/geo3d/dXdYdZSolid.java | 3 +-
.../org/apache/lucene/geo3d/GeoBBoxTest.java | 3 +-
.../org/apache/lucene/geo3d/GeoCircleTest.java | 3 +-
.../lucene/geo3d/GeoConvexPolygonTest.java | 3 +-
.../org/apache/lucene/geo3d/GeoModelTest.java | 3 +-
.../org/apache/lucene/geo3d/GeoPathTest.java | 3 +-
.../org/apache/lucene/geo3d/GeoPolygonTest.java | 3 +-
.../test/org/apache/lucene/geo3d/PlaneTest.java | 3 +-
.../org/apache/lucene/geo3d/TestGeo3DPoint.java | 14 +-
.../org/apache/lucene/geo3d/XYZSolidTest.java | 3 +-
.../lucene/search/spell/CombineSuggestion.java | 3 +-
.../apache/lucene/search/spell/Dictionary.java | 3 +-
.../lucene/search/spell/DirectSpellChecker.java | 3 +-
.../search/spell/HighFrequencyDictionary.java | 1 -
.../search/spell/JaroWinklerDistance.java | 3 +-
.../lucene/search/spell/LevensteinDistance.java | 3 +-
.../lucene/search/spell/LuceneDictionary.java | 3 +-
.../search/spell/LuceneLevenshteinDistance.java | 3 +-
.../lucene/search/spell/NGramDistance.java | 33 +-
.../search/spell/PlainTextDictionary.java | 3 +-
.../lucene/search/spell/SpellChecker.java | 3 +-
.../lucene/search/spell/StringDistance.java | 3 +-
.../apache/lucene/search/spell/SuggestMode.java | 3 +-
.../apache/lucene/search/spell/SuggestWord.java | 5 +-
.../spell/SuggestWordFrequencyComparator.java | 6 +-
.../lucene/search/spell/SuggestWordQueue.java | 5 +-
.../spell/SuggestWordScoreComparator.java | 3 +-
.../search/spell/WordBreakSpellChecker.java | 3 +-
.../lucene/search/suggest/BitsProducer.java | 3 +-
.../search/suggest/BufferedInputIterator.java | 3 +-
.../search/suggest/DocumentDictionary.java | 3 +-
.../suggest/DocumentValueSourceDictionary.java | 3 +-
.../lucene/search/suggest/FileDictionary.java | 4 +-
.../lucene/search/suggest/InMemorySorter.java | 3 +-
.../lucene/search/suggest/InputIterator.java | 3 +-
.../apache/lucene/search/suggest/Lookup.java | 3 +-
.../search/suggest/SortedInputIterator.java | 3 +-
.../search/suggest/UnsortedInputIterator.java | 3 +-
.../analyzing/AnalyzingInfixSuggester.java | 6 +-
.../suggest/analyzing/AnalyzingSuggester.java | 3 +-
.../analyzing/BlendedInfixSuggester.java | 3 +-
.../search/suggest/analyzing/FSTUtil.java | 3 +-
.../suggest/analyzing/FreeTextSuggester.java | 3 +-
.../suggest/analyzing/FuzzySuggester.java | 3 +-
.../suggest/analyzing/SuggestStopFilter.java | 3 +-
.../analyzing/SuggestStopFilterFactory.java | 3 +-
.../document/Completion50PostingsFormat.java | 4 +-
.../suggest/document/CompletionAnalyzer.java | 3 +-
.../document/CompletionFieldsConsumer.java | 3 +-
.../document/CompletionFieldsProducer.java | 3 +-
.../document/CompletionPostingsFormat.java | 3 +-
.../suggest/document/CompletionQuery.java | 3 +-
.../suggest/document/CompletionScorer.java | 3 +-
.../suggest/document/CompletionTerms.java | 3 +-
.../suggest/document/CompletionTokenStream.java | 3 +-
.../suggest/document/CompletionWeight.java | 3 +-
.../document/CompletionsTermsReader.java | 3 +-
.../search/suggest/document/ContextQuery.java | 3 +-
.../suggest/document/ContextSuggestField.java | 3 +-
.../suggest/document/FuzzyCompletionQuery.java | 3 +-
.../search/suggest/document/NRTSuggester.java | 3 +-
.../suggest/document/NRTSuggesterBuilder.java | 3 +-
.../suggest/document/PrefixCompletionQuery.java | 3 +-
.../suggest/document/RegexCompletionQuery.java | 3 +-
.../search/suggest/document/SuggestField.java | 9 +-
.../suggest/document/SuggestIndexSearcher.java | 3 +-
.../document/SuggestScoreDocPriorityQueue.java | 3 +-
.../search/suggest/document/TopSuggestDocs.java | 3 +-
.../document/TopSuggestDocsCollector.java | 3 +-
.../search/suggest/fst/BytesRefSorter.java | 3 +-
.../search/suggest/fst/ExternalRefSorter.java | 4 +-
.../search/suggest/fst/FSTCompletion.java | 3 +-
.../suggest/fst/FSTCompletionBuilder.java | 3 +-
.../search/suggest/fst/FSTCompletionLookup.java | 3 +-
.../suggest/fst/WFSTCompletionLookup.java | 3 +-
.../search/suggest/jaspell/JaspellLookup.java | 3 +-
.../jaspell/JaspellTernarySearchTrie.java | 3 +-
.../search/suggest/tst/TSTAutocomplete.java | 3 +-
.../lucene/search/suggest/tst/TSTLookup.java | 3 +-
.../search/suggest/tst/TernaryTreeNode.java | 7 +-
.../search/spell/TestDirectSpellChecker.java | 3 +-
.../search/spell/TestJaroWinklerDistance.java | 3 +-
.../search/spell/TestLevenshteinDistance.java | 3 +-
.../search/spell/TestLuceneDictionary.java | 3 +-
.../lucene/search/spell/TestNGramDistance.java | 3 +-
.../search/spell/TestPlainTextDictionary.java | 3 +-
.../lucene/search/spell/TestSpellChecker.java | 3 +-
.../search/spell/TestWordBreakSpellChecker.java | 3 +-
.../apache/lucene/search/suggest/Average.java | 5 +-
.../search/suggest/DocumentDictionaryTest.java | 33 +-
.../DocumentValueSourceDictionaryTest.java | 3 +-
.../search/suggest/FileDictionaryTest.java | 33 +-
.../org/apache/lucene/search/suggest/Input.java | 3 +-
.../search/suggest/InputArrayIterator.java | 3 +-
.../search/suggest/LookupBenchmarkTest.java | 3 +-
.../lucene/search/suggest/PersistenceTest.java | 6 +-
.../suggest/TestHighFrequencyDictionary.java | 27 +-
.../search/suggest/TestInputIterator.java | 33 +-
.../analyzing/AnalyzingInfixSuggesterTest.java | 3 +-
.../analyzing/AnalyzingSuggesterTest.java | 9 +-
.../analyzing/BlendedInfixSuggesterTest.java | 3 +-
.../suggest/analyzing/FuzzySuggesterTest.java | 9 +-
.../analyzing/TestFreeTextSuggester.java | 3 +-
.../analyzing/TestSuggestStopFilter.java | 3 +-
.../analyzing/TestSuggestStopFilterFactory.java | 3 +-
.../document/CompletionTokenStreamTest.java | 3 +-
.../suggest/document/TestContextQuery.java | 3 +-
.../document/TestContextSuggestField.java | 3 +-
.../document/TestFuzzyCompletionQuery.java | 3 +-
.../document/TestPrefixCompletionQuery.java | 3 +-
.../document/TestRegexCompletionQuery.java | 3 +-
.../suggest/document/TestSuggestField.java | 3 +-
.../search/suggest/fst/BytesRefSortersTest.java | 6 +-
.../search/suggest/fst/FSTCompletionTest.java | 9 +-
.../search/suggest/fst/WFSTCompletionTest.java | 9 +-
.../analysis/BaseTokenStreamTestCase.java | 3 +-
.../analysis/CannedBinaryTokenStream.java | 3 +-
.../lucene/analysis/CannedTokenStream.java | 3 +-
.../lucene/analysis/CollationTestBase.java | 4 +-
.../lucene/analysis/CrankyTokenFilter.java | 3 +-
.../lucene/analysis/LookaheadTokenFilter.java | 3 +-
.../apache/lucene/analysis/MockAnalyzer.java | 3 +-
.../lucene/analysis/MockBytesAnalyzer.java | 3 +-
.../apache/lucene/analysis/MockCharFilter.java | 3 +-
.../analysis/MockFixedLengthPayloadFilter.java | 3 +-
.../lucene/analysis/MockGraphTokenFilter.java | 3 +-
.../analysis/MockHoleInjectingTokenFilter.java | 4 +-
.../lucene/analysis/MockPayloadAnalyzer.java | 4 +-
.../MockRandomLookaheadTokenFilter.java | 3 +-
.../lucene/analysis/MockReaderWrapper.java | 3 +-
.../apache/lucene/analysis/MockTokenFilter.java | 3 +-
.../apache/lucene/analysis/MockTokenizer.java | 3 +-
.../analysis/MockUTF16TermAttributeImpl.java | 3 +-
.../MockVariableLengthPayloadFilter.java | 3 +-
.../lucene/analysis/SimplePayloadFilter.java | 3 +-
.../lucene/analysis/TokenStreamToDot.java | 3 +-
.../lucene/analysis/ValidatingTokenFilter.java | 3 +-
.../lucene/analysis/VocabularyAssert.java | 5 +-
.../lucene/codecs/MissingOrdRemapper.java | 11 +-
.../lucene/codecs/asserting/AssertingCodec.java | 3 +-
.../asserting/AssertingDocValuesFormat.java | 3 +-
.../asserting/AssertingLiveDocsFormat.java | 3 +-
.../codecs/asserting/AssertingNormsFormat.java | 3 +-
.../codecs/asserting/AssertingPointFormat.java | 8 +-
.../asserting/AssertingPostingsFormat.java | 3 +-
.../asserting/AssertingStoredFieldsFormat.java | 3 +-
.../asserting/AssertingTermVectorsFormat.java | 3 +-
.../codecs/blockterms/LuceneFixedGap.java | 3 +-
.../blockterms/LuceneVarGapDocFreqInterval.java | 3 +-
.../blockterms/LuceneVarGapFixedInterval.java | 3 +-
.../bloom/TestBloomFilteredLucenePostings.java | 3 +-
.../codecs/cheapbastard/CheapBastardCodec.java | 3 +-
.../codecs/compressing/CompressingCodec.java | 3 +-
.../compressing/FastCompressingCodec.java | 3 +-
.../FastDecompressionCompressingCodec.java | 3 +-
.../HighCompressionCompressingCodec.java | 3 +-
.../dummy/DummyCompressingCodec.java | 3 +-
.../lucene/codecs/cranky/CrankyCodec.java | 9 +-
.../codecs/cranky/CrankyCompoundFormat.java | 3 +-
.../codecs/cranky/CrankyDocValuesFormat.java | 3 +-
.../codecs/cranky/CrankyFieldInfosFormat.java | 3 +-
.../codecs/cranky/CrankyLiveDocsFormat.java | 3 +-
.../lucene/codecs/cranky/CrankyNormsFormat.java | 3 +-
.../lucene/codecs/cranky/CrankyPointFormat.java | 175 ++++
.../codecs/cranky/CrankyPostingsFormat.java | 3 +-
.../codecs/cranky/CrankySegmentInfoFormat.java | 3 +-
.../codecs/cranky/CrankyStoredFieldsFormat.java | 3 +-
.../codecs/cranky/CrankyTermVectorsFormat.java | 3 +-
.../mockrandom/MockRandomPostingsFormat.java | 3 +-
.../codecs/ramonly/RAMOnlyPostingsFormat.java | 3 +-
.../lucene/index/AlcoholicMergePolicy.java | 3 +-
.../lucene/index/AllDeletedFilterReader.java | 7 +-
.../lucene/index/AssertingDirectoryReader.java | 3 +-
.../lucene/index/AssertingLeafReader.java | 21 +-
.../index/BaseCompoundFormatTestCase.java | 8 +-
.../BaseCompressingDocValuesFormatTestCase.java | 3 +-
.../index/BaseDocValuesFormatTestCase.java | 3 +-
.../index/BaseFieldInfoFormatTestCase.java | 3 +-
.../index/BaseIndexFileFormatTestCase.java | 19 +-
.../lucene/index/BaseMergePolicyTestCase.java | 3 +-
.../lucene/index/BaseNormsFormatTestCase.java | 3 +-
.../lucene/index/BasePointFormatTestCase.java | 917 +++++++++++++++++
.../index/BasePostingsFormatTestCase.java | 5 +-
.../index/BaseSegmentInfoFormatTestCase.java | 3 +-
.../index/BaseStoredFieldsFormatTestCase.java | 3 +-
.../index/BaseTermVectorsFormatTestCase.java | 3 +-
.../apache/lucene/index/BaseTestCheckIndex.java | 4 +-
.../java/org/apache/lucene/index/DocHelper.java | 3 +-
.../lucene/index/FieldFilterLeafReader.java | 3 +-
.../apache/lucene/index/ForceMergePolicy.java | 48 +-
.../lucene/index/IndexWriterMaxDocsChanger.java | 7 +-
.../lucene/index/MismatchedDirectoryReader.java | 3 +-
.../lucene/index/MismatchedLeafReader.java | 3 +-
.../lucene/index/MockRandomMergePolicy.java | 3 +-
.../apache/lucene/index/PerThreadPKLookup.java | 3 +-
.../org/apache/lucene/index/RandomCodec.java | 3 +-
.../apache/lucene/index/RandomIndexWriter.java | 3 +-
.../lucene/index/RandomPostingsTester.java | 4 +-
.../SuppressingConcurrentMergeScheduler.java | 7 +-
.../ThreadedIndexingAndSearchingTestCase.java | 4 +-
.../apache/lucene/mockfile/DisableFsyncFS.java | 23 +-
.../org/apache/lucene/mockfile/ExtrasFS.java | 3 +-
.../mockfile/FilterAsynchronousFileChannel.java | 3 +-
.../lucene/mockfile/FilterDirectoryStream.java | 3 +-
.../lucene/mockfile/FilterFileChannel.java | 3 +-
.../apache/lucene/mockfile/FilterFileStore.java | 3 +-
.../lucene/mockfile/FilterFileSystem.java | 3 +-
.../mockfile/FilterFileSystemProvider.java | 3 +-
.../lucene/mockfile/FilterInputStream2.java | 3 +-
.../lucene/mockfile/FilterOutputStream2.java | 3 +-
.../org/apache/lucene/mockfile/FilterPath.java | 3 +-
.../mockfile/FilterSeekableByteChannel.java | 3 +-
.../apache/lucene/mockfile/HandleLimitFS.java | 3 +-
.../lucene/mockfile/HandleTrackingFS.java | 3 +-
.../java/org/apache/lucene/mockfile/LeakFS.java | 3 +-
.../lucene/mockfile/MockFileSystemTestCase.java | 3 +-
.../org/apache/lucene/mockfile/ShuffleFS.java | 3 +-
.../org/apache/lucene/mockfile/VerboseFS.java | 3 +-
.../apache/lucene/mockfile/VirusCheckingFS.java | 80 ++
.../org/apache/lucene/mockfile/WindowsFS.java | 3 +-
.../lucene/search/AssertingBulkScorer.java | 3 +-
.../lucene/search/AssertingCollector.java | 3 +-
.../lucene/search/AssertingIndexSearcher.java | 3 +-
.../lucene/search/AssertingLeafCollector.java | 3 +-
.../apache/lucene/search/AssertingQuery.java | 3 +-
.../apache/lucene/search/AssertingScorer.java | 3 +-
.../apache/lucene/search/AssertingWeight.java | 3 +-
.../lucene/search/BaseExplanationTestCase.java | 3 +-
.../lucene/search/BulkScorerWrapperScorer.java | 3 +-
.../org/apache/lucene/search/CheckHits.java | 3 +-
.../org/apache/lucene/search/QueryUtils.java | 3 +-
.../lucene/search/RandomApproximationQuery.java | 3 +-
.../lucene/search/ScorerIndexSearcher.java | 3 +-
.../search/SearchEquivalenceTestBase.java | 3 +-
.../lucene/search/ShardSearchingTestBase.java | 3 +-
.../search/similarities/RandomSimilarity.java | 3 +-
.../lucene/search/spans/AssertingSpanQuery.java | 3 +-
.../search/spans/AssertingSpanWeight.java | 3 +-
.../lucene/search/spans/AssertingSpans.java | 3 +-
.../lucene/search/spans/MultiSpansWrapper.java | 3 +-
.../lucene/search/spans/SpanTestUtil.java | 3 +-
.../lucene/store/BaseDirectoryTestCase.java | 77 +-
.../lucene/store/BaseDirectoryWrapper.java | 3 +-
.../lucene/store/BaseLockFactoryTestCase.java | 49 +-
.../lucene/store/MockDirectoryWrapper.java | 229 +----
.../lucene/store/MockIndexInputWrapper.java | 13 +-
.../lucene/store/MockIndexOutputWrapper.java | 3 +-
.../lucene/store/RawDirectoryWrapper.java | 3 +-
.../store/SlowClosingMockIndexInputWrapper.java | 3 +-
.../store/SlowOpeningMockIndexInputWrapper.java | 3 +-
.../lucene/util/AbstractBeforeAfterRule.java | 23 +-
.../apache/lucene/util/BaseBitSetTestCase.java | 3 +-
.../lucene/util/BaseDocIdSetTestCase.java | 3 +-
.../apache/lucene/util/CloseableDirectory.java | 15 +-
.../java/org/apache/lucene/util/English.java | 9 +-
.../util/FailOnNonBulkMergesInfoStream.java | 3 +-
.../org/apache/lucene/util/FailureMarker.java | 13 +-
.../org/apache/lucene/util/LineFileDocs.java | 3 +-
.../lucene/util/LuceneJUnit3MethodProvider.java | 3 +-
.../org/apache/lucene/util/LuceneTestCase.java | 121 ++-
.../org/apache/lucene/util/NullInfoStream.java | 3 +-
.../lucene/util/QuickPatchThreadsFilter.java | 7 +-
.../org/apache/lucene/util/RamUsageTester.java | 3 +-
.../org/apache/lucene/util/RemoveUponClose.java | 19 +-
.../java/org/apache/lucene/util/Rethrow.java | 3 +-
.../util/RunListenerPrintReproduceInfo.java | 33 +-
.../lucene/util/TestRuleAssertionsRequired.java | 3 +-
.../apache/lucene/util/TestRuleDelegate.java | 15 +-
.../util/TestRuleIgnoreAfterMaxFailures.java | 21 +-
.../lucene/util/TestRuleIgnoreTestSuites.java | 11 +-
.../lucene/util/TestRuleLimitSysouts.java | 33 +-
.../apache/lucene/util/TestRuleMarkFailure.java | 19 +-
.../util/TestRuleRestoreSystemProperties.java | 17 +-
.../util/TestRuleSetupAndRestoreClassEnv.java | 3 +-
.../TestRuleSetupAndRestoreInstanceEnv.java | 7 +-
.../util/TestRuleSetupTeardownChained.java | 13 +-
.../lucene/util/TestRuleStoreClassName.java | 3 +-
.../util/TestRuleTemporaryFilesCleanup.java | 5 +-
.../lucene/util/TestRuleThreadAndTestName.java | 11 +-
.../apache/lucene/util/TestSecurityManager.java | 9 +-
.../java/org/apache/lucene/util/TestUtil.java | 105 +-
.../lucene/util/ThrottledIndexOutput.java | 9 +-
.../java/org/apache/lucene/util/TimeUnits.java | 3 +-
.../util/automaton/AutomatonTestUtil.java | 3 +-
.../org/apache/lucene/util/fst/FSTTester.java | 9 +-
.../lucene/analysis/TestGraphTokenizers.java | 3 +-
.../analysis/TestLookaheadTokenFilter.java | 3 +-
.../lucene/analysis/TestMockAnalyzer.java | 3 +-
.../lucene/analysis/TestMockCharFilter.java | 9 +-
.../apache/lucene/analysis/TestPosition.java | 5 +-
.../lucene/analysis/TrivialLookaheadFilter.java | 3 +-
.../asserting/TestAssertingDocValuesFormat.java | 3 +-
.../asserting/TestAssertingNormsFormat.java | 3 +-
.../asserting/TestAssertingPointFormat.java | 30 +
.../asserting/TestAssertingPostingsFormat.java | 3 +-
.../TestAssertingStoredFieldsFormat.java | 3 +-
.../TestAssertingTermVectorsFormat.java | 3 +-
.../TestCompressingStoredFieldsFormat.java | 7 +-
.../TestCompressingTermVectorsFormat.java | 33 +-
.../lucene/index/TestAssertingLeafReader.java | 3 +-
.../lucene/index/TestForceMergePolicy.java | 3 +-
.../lucene/mockfile/TestDisableFsyncFS.java | 3 +-
.../apache/lucene/mockfile/TestExtrasFS.java | 3 +-
.../lucene/mockfile/TestHandleLimitFS.java | 3 +-
.../lucene/mockfile/TestHandleTrackingFS.java | 3 +-
.../org/apache/lucene/mockfile/TestLeakFS.java | 3 +-
.../apache/lucene/mockfile/TestShuffleFS.java | 3 +-
.../apache/lucene/mockfile/TestVerboseFS.java | 3 +-
.../lucene/mockfile/TestVirusCheckingFS.java | 62 ++
.../apache/lucene/mockfile/TestWindowsFS.java | 5 +-
.../lucene/store/TestMockDirectoryWrapper.java | 4 +-
.../test/org/apache/lucene/util/SorePoint.java | 3 +-
.../test/org/apache/lucene/util/SoreType.java | 3 +-
.../lucene/util/TestBeforeAfterOverrides.java | 17 +-
.../apache/lucene/util/TestCodecReported.java | 15 +-
.../util/TestExceptionInBeforeClassHooks.java | 3 +-
.../util/TestFailIfDirectoryNotClosed.java | 3 +-
.../util/TestFailIfUnreferencedFiles.java | 3 +-
.../apache/lucene/util/TestGroupFiltering.java | 21 +-
.../apache/lucene/util/TestJUnitRuleOrder.java | 3 +-
.../apache/lucene/util/TestMaxFailuresRule.java | 3 +-
.../util/TestRamUsageTesterOnWildAnimals.java | 3 +-
.../lucene/util/TestReproduceMessage.java | 3 +-
.../util/TestReproduceMessageWithRepeated.java | 3 +-
.../util/TestRunWithRestrictedPermissions.java | 3 +-
.../lucene/util/TestSeedFromUncaught.java | 3 +-
.../lucene/util/TestSetupTeardownChaining.java | 3 +-
.../lucene/util/TestWorstCaseTestBehavior.java | 15 +-
.../org/apache/lucene/util/WithNestedTests.java | 3 +-
.../dependencies/GetMavenDependenciesTask.java | 3 +-
.../dependencies/InterpolatedProperties.java | 3 +-
.../lucene/validation/LibVersionsCheckTask.java | 3 +-
.../lucene/validation/LicenseCheckTask.java | 3 +-
.../apache/lucene/validation/LicenseType.java | 5 +-
.../lucene/validation/ivyde/IvyNodeElement.java | 28 +-
.../validation/ivyde/IvyNodeElementAdapter.java | 28 +-
solr/CHANGES.txt | 34 +
.../apache/solr/schema/ICUCollationField.java | 3 +-
.../TestFoldingMultitermExtrasQuery.java | 3 +-
.../solr/schema/TestICUCollationField.java | 1 -
.../schema/TestICUCollationFieldDocValues.java | 1 -
.../schema/TestICUCollationFieldOptions.java | 3 +-
.../analytics/accumulator/BasicAccumulator.java | 1 -
.../accumulator/FacetingAccumulator.java | 1 -
.../analytics/accumulator/ValueAccumulator.java | 1 -
.../facet/FacetValueAccumulator.java | 1 -
.../facet/FieldFacetAccumulator.java | 1 -
.../facet/QueryFacetAccumulator.java | 1 -
.../facet/RangeFacetAccumulator.java | 1 -
.../analytics/expression/BaseExpression.java | 1 -
.../expression/DualDelegateExpression.java | 1 -
.../solr/analytics/expression/Expression.java | 1 -
.../analytics/expression/ExpressionFactory.java | 1 -
.../expression/MultiDelegateExpression.java | 1 -
.../expression/SingleDelegateExpression.java | 1 -
.../plugin/AnalyticsStatisticsCollector.java | 1 -
.../request/AbstractFieldFacetRequest.java | 1 -
.../request/AnalyticsContentHandler.java | 1 -
.../analytics/request/AnalyticsRequest.java | 1 -
.../request/AnalyticsRequestFactory.java | 1 -
.../solr/analytics/request/AnalyticsStats.java | 1 -
.../analytics/request/ExpressionRequest.java | 1 -
.../solr/analytics/request/FacetRequest.java | 1 -
.../analytics/request/FieldFacetRequest.java | 1 -
.../analytics/request/QueryFacetRequest.java | 1 -
.../analytics/request/RangeFacetRequest.java | 1 -
.../AbstractDelegatingStatsCollector.java | 1 -
.../statistics/MedianStatsCollector.java | 1 -
.../statistics/MinMaxStatsCollector.java | 1 -
.../statistics/NumericStatsCollector.java | 1 -
.../statistics/PercentileStatsCollector.java | 1 -
.../analytics/statistics/StatsCollector.java | 1 -
.../StatsCollectorSupplierFactory.java | 1 -
.../statistics/UniqueStatsCollector.java | 1 -
.../solr/analytics/util/AnalyticsParams.java | 1 -
.../solr/analytics/util/AnalyticsParsers.java | 1 -
.../solr/analytics/util/MedianCalculator.java | 1 -
.../analytics/util/PercentileCalculator.java | 1 -
.../analytics/util/RangeEndpointCalculator.java | 1 -
.../AbsoluteValueDoubleFunction.java | 1 -
.../util/valuesource/AddDoubleFunction.java | 1 -
.../util/valuesource/ConcatStringFunction.java | 1 -
.../util/valuesource/ConstDateSource.java | 1 -
.../util/valuesource/ConstDoubleSource.java | 1 -
.../util/valuesource/ConstStringSource.java | 1 -
.../util/valuesource/DateFieldSource.java | 1 -
.../util/valuesource/DateMathFunction.java | 1 -
.../util/valuesource/DivDoubleFunction.java | 1 -
.../util/valuesource/DualDoubleFunction.java | 1 -
.../util/valuesource/FilterFieldSource.java | 1 -
.../util/valuesource/LogDoubleFunction.java | 1 -
.../util/valuesource/MultiDateFunction.java | 1 -
.../util/valuesource/MultiDoubleFunction.java | 1 -
.../util/valuesource/MultiStringFunction.java | 1 -
.../valuesource/MultiplyDoubleFunction.java | 1 -
.../util/valuesource/NegateDoubleFunction.java | 1 -
.../util/valuesource/PowDoubleFunction.java | 1 -
.../util/valuesource/ReverseStringFunction.java | 1 -
.../util/valuesource/SingleDoubleFunction.java | 1 -
.../util/valuesource/SingleStringFunction.java | 1 -
.../handler/component/AnalyticsComponent.java | 1 -
.../analytics/AbstractAnalyticsStatsTest.java | 1 -
.../org/apache/solr/analytics/NoFacetTest.java | 1 -
.../analytics/expression/ExpressionTest.java | 1 -
.../facet/AbstractAnalyticsFacetTest.java | 1 -
.../analytics/facet/FieldFacetExtrasTest.java | 1 -
.../solr/analytics/facet/FieldFacetTest.java | 1 -
.../solr/analytics/facet/QueryFacetTest.java | 1 -
.../solr/analytics/facet/RangeFacetTest.java | 1 -
.../util/valuesource/FunctionTest.java | 1 -
.../handler/clustering/ClusteringComponent.java | 3 +-
.../handler/clustering/ClusteringEngine.java | 2 +-
.../handler/clustering/ClusteringParams.java | 3 +-
.../clustering/DocumentClusteringEngine.java | 3 +-
.../clustering/SearchClusteringEngine.java | 3 +-
.../carrot2/CarrotClusteringEngine.java | 3 +-
.../clustering/carrot2/CarrotParams.java | 10 +-
.../carrot2/LuceneCarrot2StemmerFactory.java | 7 +-
.../carrot2/LuceneCarrot2TokenizerFactory.java | 3 +-
.../clustering/carrot2/SolrResourceLocator.java | 1 -
.../SolrStopwordsCarrot2LexicalDataFactory.java | 3 +-
.../clustering/AbstractClusteringTestCase.java | 3 +-
.../clustering/ClusteringComponentTest.java | 3 +-
.../DistributedClusteringComponentTest.java | 3 +-
.../MockDocumentClusteringEngine.java | 2 +-
.../carrot2/CarrotClusteringEngineTest.java | 3 +-
.../carrot2/DuplicatingStemmerFactory.java | 3 +-
.../carrot2/DuplicatingTokenizerFactory.java | 3 +-
.../carrot2/EchoClusteringAlgorithm.java | 2 +-
.../carrot2/EchoStemsClusteringAlgorithm.java | 4 +-
.../carrot2/EchoTokensClusteringAlgorithm.java | 4 +-
...exicalResourcesCheckClusteringAlgorithm.java | 4 +-
.../carrot2/MockClusteringAlgorithm.java | 2 +-
.../handler/dataimport/CachePropertyUtil.java | 3 +-
.../apache/solr/handler/dataimport/Context.java | 1 -
.../solr/handler/dataimport/ContextImpl.java | 4 +-
.../solr/handler/dataimport/DIHCache.java | 3 +-
.../handler/dataimport/DIHCacheSupport.java | 3 +-
.../solr/handler/dataimport/DIHLogLevels.java | 3 +-
.../solr/handler/dataimport/DIHProperties.java | 5 +-
.../solr/handler/dataimport/DIHWriter.java | 2 +-
.../solr/handler/dataimport/DIHWriterBase.java | 3 +-
.../dataimport/DataImportHandlerException.java | 1 -
.../solr/handler/dataimport/DataImporter.java | 1 -
.../solr/handler/dataimport/DataSource.java | 1 -
.../handler/dataimport/DateFormatEvaluator.java | 33 +-
.../dataimport/DateFormatTransformer.java | 1 -
.../solr/handler/dataimport/DebugLogger.java | 3 +-
.../solr/handler/dataimport/DocBuilder.java | 1 -
.../solr/handler/dataimport/JdbcDataSource.java | 70 +-
.../dataimport/NumberFormatTransformer.java | 3 +-
.../dataimport/SimplePropertiesWriter.java | 3 +-
.../handler/dataimport/SolrEntityProcessor.java | 3 +-
.../dataimport/SolrQueryEscapingEvaluator.java | 15 +-
.../dataimport/SortedMapBackedCache.java | 3 +-
.../dataimport/SqlEscapingEvaluator.java | 11 +-
.../solr/handler/dataimport/UrlEvaluator.java | 17 +-
.../apache/solr/handler/dataimport/Zipper.java | 27 +-
.../dataimport/config/DIHConfiguration.java | 33 +-
.../solr/handler/dataimport/config/Entity.java | 26 +-
.../handler/dataimport/config/EntityField.java | 1 -
.../solr/handler/dataimport/config/Field.java | 1 -
.../dataimport/config/PropertyWriter.java | 9 +-
.../solr/handler/dataimport/config/Script.java | 25 +-
.../dataimport/AbstractDIHCacheTestCase.java | 3 +-
.../dataimport/AbstractDIHJdbcTestCase.java | 30 +-
.../AbstractSqlEntityProcessorTestCase.java | 33 +-
.../dataimport/AddAColumnTransformer.java | 7 +-
.../handler/dataimport/DestroyCountCache.java | 3 +-
.../dataimport/MockSolrEntityProcessor.java | 3 +-
.../handler/dataimport/TestEphemeralCache.java | 3 +-
.../handler/dataimport/TestErrorHandling.java | 3 +-
.../dataimport/TestFileListEntityProcessor.java | 3 +-
.../TestFileListWithLineEntityProcessor.java | 19 +-
.../dataimport/TestHierarchicalDocBuilder.java | 3 +-
.../TestJdbcDataSourceConvertType.java | 3 +-
.../dataimport/TestLineEntityProcessor.java | 3 +-
.../handler/dataimport/TestNestedChildren.java | 15 +-
.../dataimport/TestSimplePropertiesWriter.java | 33 +-
.../TestSolrEntityProcessorEndToEnd.java | 3 +-
.../dataimport/TestSolrEntityProcessorUnit.java | 2 +-
.../dataimport/TestSortedMapBackedCache.java | 7 +-
.../dataimport/TestSqlEntityProcessor.java | 9 +-
.../dataimport/TestSqlEntityProcessorDelta.java | 21 +-
.../TestVariableResolverEndToEnd.java | 31 +-
.../dataimport/TestXPathRecordReader.java | 2 +-
.../dataimport/TestZKPropertiesWriter.java | 7 +-
.../dataimport/TripleThreatTransformer.java | 16 +-
.../handler/extraction/ParseContextConfig.java | 3 +-
.../extraction/RegexRulesPasswordProvider.java | 1 -
.../handler/extraction/SolrContentHandler.java | 1 -
.../ExtractingRequestHandlerTest.java | 3 +-
.../extraction/ParseContextConfigTest.java | 3 +-
.../solr/update/processor/DetectedLanguage.java | 3 +-
...DetectLanguageIdentifierUpdateProcessor.java | 3 +-
...anguageIdentifierUpdateProcessorFactory.java | 3 +-
.../solr/update/processor/LangIdParams.java | 3 +-
.../LanguageIdentifierUpdateProcessor.java | 3 +-
.../TikaLanguageIdentifierUpdateProcessor.java | 3 +-
...anguageIdentifierUpdateProcessorFactory.java | 3 +-
...ageIdentifierUpdateProcessorFactoryTest.java | 3 +-
...dentifierUpdateProcessorFactoryTestCase.java | 1 -
...ageIdentifierUpdateProcessorFactoryTest.java | 3 +-
.../solr/hadoop/AlphaNumericComparator.java | 1 -
.../solr/hadoop/DataInputInputStream.java | 1 -
.../hadoop/UnbufferedDataInputInputStream.java | 1 -
.../apache/solr/hadoop/ZooKeeperInspector.java | 1 -
.../solr/hadoop/MorphlineGoLiveMiniMRTest.java | 1 -
.../solr/AbstractSolrMorphlineZkTestBase.java | 1 -
.../uima/processor/FieldMappingException.java | 3 +-
.../uima/processor/SolrUIMAConfiguration.java | 3 +-
.../processor/SolrUIMAConfigurationReader.java | 3 +-
.../solr/uima/processor/UIMAToSolrMapper.java | 3 +-
.../processor/UIMAUpdateRequestProcessor.java | 3 +-
.../UIMAUpdateRequestProcessorFactory.java | 3 +-
.../UIMATokenizersSolrIntegrationTest.java | 3 +-
.../UIMAUpdateRequestProcessorTest.java | 3 +-
.../uima/processor/an/DummyEntityAnnotator.java | 17 +-
.../processor/an/DummyExceptionAnnotator.java | 11 +-
.../processor/an/DummySentimentAnnotator.java | 21 +-
.../java/org/apache/solr/response/PageTool.java | 1 -
.../solr/response/SolrParamResourceLoader.java | 1 -
.../solr/response/SolrVelocityLogger.java | 3 +-
.../solr/response/VelocityResponseWriter.java | 1 -
.../test/org/apache/solr/velocity/MockTool.java | 3 +-
.../velocity/VelocityResponseWriterTest.java | 1 -
.../solr/analysis/ReversedWildcardFilter.java | 3 +-
.../analysis/ReversedWildcardFilterFactory.java | 3 +-
.../org/apache/solr/analysis/SolrAnalyzer.java | 1 -
.../apache/solr/analysis/TokenizerChain.java | 1 -
.../solrj/embedded/EmbeddedSolrServer.java | 1 -
.../solr/client/solrj/embedded/JettyConfig.java | 1 -
.../client/solrj/embedded/JettySolrRunner.java | 1 -
.../solr/client/solrj/embedded/SSLConfig.java | 3 +-
.../org/apache/solr/cloud/ActionThrottle.java | 7 +-
.../src/java/org/apache/solr/cloud/Assign.java | 27 +-
.../solr/cloud/CloudConfigSetService.java | 1 -
.../org/apache/solr/cloud/CloudDescriptor.java | 3 +-
.../java/org/apache/solr/cloud/CloudUtil.java | 3 +-
.../cloud/CurrentCoreDescriptorProvider.java | 3 +-
.../org/apache/solr/cloud/DistributedMap.java | 3 +-
.../org/apache/solr/cloud/DistributedQueue.java | 4 +-
.../org/apache/solr/cloud/ElectionContext.java | 3 +-
.../org/apache/solr/cloud/LeaderElector.java | 3 +-
.../cloud/LeaderInitiatedRecoveryThread.java | 33 +-
.../java/org/apache/solr/cloud/Overseer.java | 27 +-
.../OverseerAutoReplicaFailoverThread.java | 3 +-
.../OverseerCollectionConfigSetProcessor.java | 3 +-
.../cloud/OverseerCollectionMessageHandler.java | 3 +-
.../cloud/OverseerConfigSetMessageHandler.java | 3 +-
.../solr/cloud/OverseerMessageHandler.java | 3 +-
.../solr/cloud/OverseerNodePrioritizer.java | 7 +-
.../apache/solr/cloud/OverseerSolrResponse.java | 3 +-
.../solr/cloud/OverseerTaskProcessor.java | 3 +-
.../apache/solr/cloud/OverseerTaskQueue.java | 7 +-
.../org/apache/solr/cloud/RecoveryStrategy.java | 3 +-
.../solr/cloud/SizeLimitedDistributedMap.java | 3 +-
.../org/apache/solr/cloud/SolrZkServer.java | 27 +-
.../org/apache/solr/cloud/SyncStrategy.java | 19 +-
.../src/java/org/apache/solr/cloud/ZkCLI.java | 33 +-
.../org/apache/solr/cloud/ZkController.java | 3 +-
.../apache/solr/cloud/ZkSolrResourceLoader.java | 3 +-
.../cloud/overseer/ClusterStateMutator.java | 7 +-
.../solr/cloud/overseer/CollectionMutator.java | 7 +-
.../solr/cloud/overseer/OverseerAction.java | 3 +-
.../solr/cloud/overseer/ReplicaMutator.java | 7 +-
.../solr/cloud/overseer/SliceMutator.java | 7 +-
.../solr/cloud/overseer/ZkStateWriter.java | 3 +-
.../solr/cloud/overseer/ZkWriteCommand.java | 3 +-
.../apache/solr/cloud/rule/ImplicitSnitch.java | 4 +-
.../apache/solr/cloud/rule/RemoteCallback.java | 3 +-
.../apache/solr/cloud/rule/ReplicaAssigner.java | 7 +-
.../java/org/apache/solr/cloud/rule/Rule.java | 4 +-
.../java/org/apache/solr/cloud/rule/Snitch.java | 3 +-
.../apache/solr/cloud/rule/SnitchContext.java | 3 +-
.../solr/core/AbstractSolrEventListener.java | 1 -
.../solr/core/CachingDirectoryFactory.java | 3 +-
.../java/org/apache/solr/core/CloseHook.java | 4 +-
.../java/org/apache/solr/core/CloudConfig.java | 3 +-
.../java/org/apache/solr/core/CodecFactory.java | 3 +-
.../src/java/org/apache/solr/core/Config.java | 1 -
.../org/apache/solr/core/ConfigOverlay.java | 3 +-
.../java/org/apache/solr/core/ConfigSet.java | 1 -
.../apache/solr/core/ConfigSetProperties.java | 1 -
.../org/apache/solr/core/ConfigSetService.java | 1 -
.../org/apache/solr/core/CoreContainer.java | 59 +-
.../org/apache/solr/core/CoreDescriptor.java | 1 -
.../apache/solr/core/CorePropertiesLocator.java | 3 +-
.../java/org/apache/solr/core/CoresLocator.java | 3 +-
.../java/org/apache/solr/core/Diagnostics.java | 4 +-
.../org/apache/solr/core/DirectoryFactory.java | 3 +-
.../solr/core/EphemeralDirectoryFactory.java | 3 +-
.../apache/solr/core/HdfsDirectoryFactory.java | 3 +-
.../org/apache/solr/core/ImplicitPlugins.java | 4 +-
.../solr/core/IndexDeletionPolicyWrapper.java | 3 +-
.../apache/solr/core/IndexReaderFactory.java | 3 +-
.../java/org/apache/solr/core/InitParams.java | 3 +-
.../org/apache/solr/core/JarRepository.java | 3 +-
.../apache/solr/core/MMapDirectoryFactory.java | 4 +-
.../org/apache/solr/core/MapSerializable.java | 4 +-
.../org/apache/solr/core/MemClassLoader.java | 3 +-
.../apache/solr/core/NIOFSDirectoryFactory.java | 3 +-
.../solr/core/NRTCachingDirectoryFactory.java | 1 -
.../java/org/apache/solr/core/NodeConfig.java | 3 +-
.../java/org/apache/solr/core/PluginBag.java | 4 +-
.../apache/solr/core/QuerySenderListener.java | 1 -
.../apache/solr/core/RAMDirectoryFactory.java | 1 -
.../org/apache/solr/core/RequestHandlers.java | 1 -
.../org/apache/solr/core/RequestParams.java | 3 +-
.../apache/solr/core/RunExecutableListener.java | 1 -
.../apache/solr/core/SchemaCodecFactory.java | 33 +-
.../solr/core/SimpleFSDirectoryFactory.java | 3 +-
.../java/org/apache/solr/core/SolrConfig.java | 1 -
.../src/java/org/apache/solr/core/SolrCore.java | 1 -
.../java/org/apache/solr/core/SolrCores.java | 3 +-
.../apache/solr/core/SolrDeletionPolicy.java | 3 +-
.../org/apache/solr/core/SolrEventListener.java | 1 -
.../org/apache/solr/core/SolrInfoMBean.java | 1 -
.../apache/solr/core/SolrResourceLoader.java | 1 -
.../core/SolrResourceNotFoundException.java | 1 -
.../org/apache/solr/core/SolrXmlConfig.java | 3 +-
.../solr/core/StandardDirectoryFactory.java | 3 +-
.../solr/core/StandardIndexReaderFactory.java | 3 +-
.../java/org/apache/solr/core/ZkContainer.java | 3 +-
.../handler/AnalysisRequestHandlerBase.java | 1 -
.../org/apache/solr/handler/BlobHandler.java | 3 +-
.../apache/solr/handler/CdcrBufferManager.java | 3 +-
.../solr/handler/CdcrBufferStateManager.java | 3 +-
.../solr/handler/CdcrLeaderStateManager.java | 7 +-
.../org/apache/solr/handler/CdcrParams.java | 3 +-
.../solr/handler/CdcrProcessStateManager.java | 7 +-
.../org/apache/solr/handler/CdcrReplicator.java | 3 +-
.../solr/handler/CdcrReplicatorManager.java | 3 +-
.../solr/handler/CdcrReplicatorScheduler.java | 3 +-
.../solr/handler/CdcrReplicatorState.java | 3 +-
.../apache/solr/handler/CdcrRequestHandler.java | 3 +-
.../apache/solr/handler/CdcrStateManager.java | 3 +-
.../solr/handler/CdcrUpdateLogSynchronizer.java | 15 +-
.../solr/handler/ContentStreamHandlerBase.java | 3 +-
.../solr/handler/ContentStreamLoader.java | 4 +-
.../handler/DocumentAnalysisRequestHandler.java | 1 -
.../apache/solr/handler/DumpRequestHandler.java | 1 -
.../handler/FieldAnalysisRequestHandler.java | 1 -
.../solr/handler/MoreLikeThisHandler.java | 1 -
.../solr/handler/NestedRequestHandler.java | 3 +-
.../solr/handler/NotFoundRequestHandler.java | 3 +-
.../apache/solr/handler/OldBackupDirectory.java | 3 +-
.../apache/solr/handler/PingRequestHandler.java | 1 -
.../apache/solr/handler/RealTimeGetHandler.java | 1 -
.../apache/solr/handler/RequestHandlerBase.java | 1 -
.../solr/handler/RequestHandlerUtils.java | 1 -
.../org/apache/solr/handler/RestoreCore.java | 4 +-
.../org/apache/solr/handler/SQLHandler.java | 200 +++-
.../org/apache/solr/handler/SchemaHandler.java | 4 +-
.../apache/solr/handler/SolrConfigHandler.java | 4 +-
.../solr/handler/StandardRequestHandler.java | 1 -
.../org/apache/solr/handler/StreamHandler.java | 1 -
.../solr/handler/UpdateRequestHandler.java | 1 -
.../solr/handler/admin/AdminHandlers.java | 1 -
.../solr/handler/admin/ClusterStatus.java | 4 +-
.../solr/handler/admin/ConfigSetsHandler.java | 7 +-
.../solr/handler/admin/CoreAdminHandler.java | 1 -
.../solr/handler/admin/CoreAdminOperation.java | 3 +-
.../apache/solr/handler/admin/InfoHandler.java | 3 +-
.../solr/handler/admin/LoggingHandler.java | 3 +-
.../solr/handler/admin/LukeRequestHandler.java | 21 +-
.../solr/handler/admin/PluginInfoHandler.java | 1 -
.../handler/admin/PropertiesRequestHandler.java | 1 -
.../solr/handler/admin/RebalanceLeaders.java | 33 +-
.../solr/handler/admin/SecurityConfHandler.java | 3 +-
.../admin/SegmentsInfoRequestHandler.java | 33 +-
.../handler/admin/ShowFileRequestHandler.java | 1 -
.../handler/admin/SolrInfoMBeanHandler.java | 3 +-
.../solr/handler/admin/SystemInfoHandler.java | 1 -
.../solr/handler/admin/ThreadDumpHandler.java | 1 -
.../handler/admin/ZookeeperInfoHandler.java | 1 -
.../handler/component/DateFacetProcessor.java | 3 +-
.../solr/handler/component/DebugComponent.java | 1 -
.../solr/handler/component/ExpandComponent.java | 1 -
.../solr/handler/component/FacetComponent.java | 1 -
.../solr/handler/component/FieldFacetStats.java | 3 +-
.../handler/component/HighlightComponent.java | 1 -
.../handler/component/HttpShardHandler.java | 3 +-
.../component/HttpShardHandlerFactory.java | 3 +-
.../component/IterativeMergeStrategy.java | 31 +-
.../solr/handler/component/MergeStrategy.java | 31 +-
.../component/MoreLikeThisComponent.java | 1 -
.../solr/handler/component/PivotFacet.java | 1 -
.../solr/handler/component/PivotFacetField.java | 1 -
.../PivotFacetFieldValueCollection.java | 3 +-
.../handler/component/PivotFacetHelper.java | 1 -
.../handler/component/PivotFacetProcessor.java | 3 +-
.../solr/handler/component/PivotFacetValue.java | 3 +-
.../solr/handler/component/QueryComponent.java | 49 +-
.../component/QueryElevationComponent.java | 1 -
.../handler/component/RangeFacetProcessor.java | 3 +-
.../handler/component/RangeFacetRequest.java | 7 +-
.../handler/component/RealTimeGetComponent.java | 11 +-
.../solr/handler/component/ResponseBuilder.java | 5 +-
.../handler/component/ResponseLogComponent.java | 3 +-
.../solr/handler/component/SearchComponent.java | 1 -
.../solr/handler/component/SearchHandler.java | 1 -
.../apache/solr/handler/component/ShardDoc.java | 166 +---
.../component/ShardFieldSortedHitQueue.java | 178 ++++
.../solr/handler/component/ShardHandler.java | 4 +-
.../handler/component/ShardHandlerFactory.java | 4 +-
.../solr/handler/component/ShardResponse.java | 3 +-
.../handler/component/SpatialHeatmapFacets.java | 3 +-
.../handler/component/SpellCheckComponent.java | 1 -
.../handler/component/SpellCheckMergeData.java | 3 +-
.../solr/handler/component/StatsComponent.java | 1 -
.../solr/handler/component/StatsField.java | 1 -
.../solr/handler/component/StatsValues.java | 2 -
.../handler/component/StatsValuesFactory.java | 3 +-
.../handler/component/SuggestComponent.java | 3 +-
.../handler/component/TermVectorComponent.java | 34 +-
.../solr/handler/component/TermsComponent.java | 3 +-
.../apache/solr/handler/loader/CSVLoader.java | 1 -
.../solr/handler/loader/CSVLoaderBase.java | 1 -
.../handler/loader/ContentStreamLoader.java | 4 +-
.../solr/handler/loader/JavabinLoader.java | 1 -
.../apache/solr/handler/loader/JsonLoader.java | 3 +-
.../apache/solr/handler/loader/XMLLoader.java | 3 +-
.../highlight/BreakIteratorBoundaryScanner.java | 1 -
.../apache/solr/highlight/DefaultEncoder.java | 1 -
.../solr/highlight/HighlightingPluginBase.java | 1 -
.../org/apache/solr/highlight/HtmlEncoder.java | 1 -
.../solr/highlight/PostingsSolrHighlighter.java | 3 +-
.../highlight/ScoreOrderFragmentsBuilder.java | 1 -
.../solr/highlight/SimpleBoundaryScanner.java | 1 -
.../solr/highlight/SimpleFragListBuilder.java | 1 -
.../solr/highlight/SimpleFragmentsBuilder.java | 1 -
.../solr/highlight/SingleFragListBuilder.java | 1 -
.../solr/highlight/SolrBoundaryScanner.java | 1 -
.../org/apache/solr/highlight/SolrEncoder.java | 1 -
.../apache/solr/highlight/SolrFormatter.java | 1 -
.../solr/highlight/SolrFragListBuilder.java | 1 -
.../apache/solr/highlight/SolrFragmenter.java | 1 -
.../solr/highlight/SolrFragmentsBuilder.java | 1 -
.../apache/solr/highlight/SolrHighlighter.java | 3 +-
.../solr/highlight/WeightedFragListBuilder.java | 1 -
.../apache/solr/index/hdfs/CheckHdfsIndex.java | 3 +-
.../org/apache/solr/internal/csv/CSVParser.java | 6 +-
.../apache/solr/internal/csv/CSVPrinter.java | 6 +-
.../apache/solr/internal/csv/CSVStrategy.java | 6 +-
.../org/apache/solr/internal/csv/CSVUtils.java | 6 +-
.../apache/solr/internal/csv/CharBuffer.java | 26 +-
.../internal/csv/ExtendedBufferedReader.java | 6 +-
.../solr/internal/csv/writer/CSVConfig.java | 26 +-
.../internal/csv/writer/CSVConfigGuesser.java | 26 +-
.../solr/internal/csv/writer/CSVField.java | 26 +-
.../solr/internal/csv/writer/CSVWriter.java | 26 +-
.../org/apache/solr/logging/CircularList.java | 1 -
.../org/apache/solr/logging/ListenerConfig.java | 5 +-
.../org/apache/solr/logging/LogWatcher.java | 1 -
.../apache/solr/logging/LogWatcherConfig.java | 3 +-
.../org/apache/solr/logging/LoggerInfo.java | 1 -
.../apache/solr/logging/MDCLoggingContext.java | 3 +-
.../org/apache/solr/parser/FastCharStream.java | 5 +-
.../java/org/apache/solr/parser/QueryParser.jj | 2 +-
.../solr/parser/QueryParserTokenManager.java | 426 ++++----
.../apache/solr/parser/SolrQueryParserBase.java | 1 -
.../java/org/apache/solr/query/FilterQuery.java | 3 +-
.../org/apache/solr/query/SolrRangeQuery.java | 3 +-
.../apache/solr/request/DocValuesFacets.java | 3 +-
.../org/apache/solr/request/DocValuesStats.java | 3 +-
.../org/apache/solr/request/IntervalFacets.java | 33 +-
.../solr/request/LocalSolrQueryRequest.java | 1 -
.../org/apache/solr/request/NumericFacets.java | 3 +-
.../request/PerSegmentSingleValuedFaceting.java | 1 -
.../org/apache/solr/request/SimpleFacets.java | 1 -
.../apache/solr/request/SolrQueryRequest.java | 1 -
.../solr/request/SolrQueryRequestBase.java | 1 -
.../apache/solr/request/SolrRequestHandler.java | 1 -
.../apache/solr/request/SolrRequestInfo.java | 1 -
.../org/apache/solr/request/json/JSONUtil.java | 4 +-
.../apache/solr/request/json/ObjectUtil.java | 4 +-
.../apache/solr/request/json/RequestUtil.java | 4 +-
.../solr/request/macro/MacroExpander.java | 3 +-
.../solr/response/BasicResultContext.java | 3 +-
.../response/BinaryQueryResponseWriter.java | 4 +-
.../apache/solr/response/CSVResponseWriter.java | 1 -
.../org/apache/solr/response/DocsStreamer.java | 3 +-
.../solr/response/JSONResponseWriter.java | 1 -
.../apache/solr/response/PHPResponseWriter.java | 1 -
.../response/PHPSerializedResponseWriter.java | 1 -
.../solr/response/PythonResponseWriter.java | 1 -
.../solr/response/QueryResponseWriter.java | 1 -
.../solr/response/QueryResponseWriterUtil.java | 1 -
.../apache/solr/response/RawResponseWriter.java | 1 -
.../org/apache/solr/response/ResultContext.java | 1 -
.../solr/response/SchemaXmlResponseWriter.java | 1 -
.../apache/solr/response/SchemaXmlWriter.java | 3 +-
.../solr/response/SmileResponseWriter.java | 4 +-
.../apache/solr/response/SolrQueryResponse.java | 1 -
.../solr/response/SortingResponseWriter.java | 101 +-
.../solr/response/TextResponseWriter.java | 1 -
.../apache/solr/response/XMLResponseWriter.java | 1 -
.../org/apache/solr/response/XMLWriter.java | 1 -
.../solr/response/XSLTResponseWriter.java | 1 -
.../transform/BaseEditorialTransformer.java | 5 +-
.../solr/response/transform/DocTransformer.java | 1 -
.../response/transform/DocTransformers.java | 1 -
.../transform/ElevatedMarkerFactory.java | 3 +-
.../transform/ExcludedMarkerFactory.java | 3 +-
.../response/transform/TransformerFactory.java | 1 -
.../org/apache/solr/rest/BaseSolrResource.java | 3 +-
.../java/org/apache/solr/rest/DELETEable.java | 3 +-
.../src/java/org/apache/solr/rest/GETable.java | 3 +-
.../org/apache/solr/rest/ManagedResource.java | 3 +-
.../solr/rest/ManagedResourceObserver.java | 3 +-
.../solr/rest/ManagedResourceStorage.java | 3 +-
.../src/java/org/apache/solr/rest/POSTable.java | 3 +-
.../src/java/org/apache/solr/rest/PUTable.java | 3 +-
.../java/org/apache/solr/rest/RestManager.java | 3 +-
.../org/apache/solr/rest/SolrSchemaRestApi.java | 3 +-
.../solr/rest/schema/BaseFieldResource.java | 3 +-
.../solr/rest/schema/BaseFieldTypeResource.java | 3 +-
.../schema/CopyFieldCollectionResource.java | 4 +-
.../schema/DynamicFieldCollectionResource.java | 3 +-
.../solr/rest/schema/DynamicFieldResource.java | 4 +-
.../rest/schema/FieldCollectionResource.java | 4 +-
.../apache/solr/rest/schema/FieldResource.java | 3 +-
.../schema/FieldTypeCollectionResource.java | 3 +-
.../solr/rest/schema/FieldTypeResource.java | 3 +-
.../solr/rest/schema/FieldTypeXmlAdapter.java | 3 +-
.../analysis/BaseManagedTokenFilterFactory.java | 3 +-
.../analysis/ManagedStopFilterFactory.java | 3 +-
.../analysis/ManagedSynonymFilterFactory.java | 3 +-
.../schema/analysis/ManagedWordSetResource.java | 3 +-
.../solr/schema/AbstractSpatialFieldType.java | 3 +-
.../AbstractSpatialPrefixTreeFieldType.java | 3 +-
.../solr/schema/AbstractSubTypeFieldType.java | 3 +-
.../java/org/apache/solr/schema/BBoxField.java | 3 +-
.../org/apache/solr/schema/BinaryField.java | 1 -
.../java/org/apache/solr/schema/BoolField.java | 1 -
.../solr/schema/ClassicIndexSchemaFactory.java | 3 +-
.../org/apache/solr/schema/CollationField.java | 3 +-
.../apache/solr/schema/CoordinateFieldType.java | 1 -
.../java/org/apache/solr/schema/CopyField.java | 1 -
.../org/apache/solr/schema/CurrencyField.java | 3 +-
.../org/apache/solr/schema/DateRangeField.java | 3 +-
.../apache/solr/schema/DateValueFieldType.java | 1 -
.../solr/schema/DoubleValueFieldType.java | 1 -
.../java/org/apache/solr/schema/EnumField.java | 3 +-
.../solr/schema/ExchangeRateProvider.java | 3 +-
.../solr/schema/ExternalFileFieldReloader.java | 1 -
.../org/apache/solr/schema/FieldProperties.java | 1 -
.../java/org/apache/solr/schema/FieldType.java | 1 -
.../solr/schema/FieldTypePluginLoader.java | 1 -
.../apache/solr/schema/FloatValueFieldType.java | 1 -
.../org/apache/solr/schema/GeoHashField.java | 1 -
.../org/apache/solr/schema/IndexSchema.java | 1 -
.../apache/solr/schema/IndexSchemaFactory.java | 3 +-
.../apache/solr/schema/IntValueFieldType.java | 1 -
.../solr/schema/JsonPreAnalyzedParser.java | 33 +-
.../java/org/apache/solr/schema/LatLonType.java | 3 +-
.../apache/solr/schema/LongValueFieldType.java | 1 -
.../apache/solr/schema/ManagedIndexSchema.java | 7 +-
.../solr/schema/ManagedIndexSchemaFactory.java | 3 +-
.../solr/schema/NumericValueFieldType.java | 1 -
.../schema/OpenExchangeRatesOrgProvider.java | 3 +-
.../java/org/apache/solr/schema/PointType.java | 1 -
.../apache/solr/schema/PreAnalyzedField.java | 3 +-
.../apache/solr/schema/PrimitiveFieldType.java | 1 -
.../org/apache/solr/schema/RandomSortField.java | 1 -
.../schema/RptWithGeometrySpatialField.java | 3 +-
.../org/apache/solr/schema/SchemaAware.java | 4 +-
.../org/apache/solr/schema/SchemaField.java | 1 -
.../org/apache/solr/schema/SchemaManager.java | 4 +-
.../apache/solr/schema/SimilarityFactory.java | 3 +-
.../solr/schema/SimplePreAnalyzedParser.java | 3 +-
.../schema/SpatialPointVectorFieldType.java | 3 +-
.../apache/solr/schema/SpatialQueryable.java | 4 +-
.../SpatialRecursivePrefixTreeFieldType.java | 3 +-
.../SpatialTermQueryPrefixTreeFieldType.java | 1 -
.../java/org/apache/solr/schema/StrField.java | 1 -
.../org/apache/solr/schema/StrFieldSource.java | 1 -
.../java/org/apache/solr/schema/TextField.java | 1 -
.../org/apache/solr/schema/TrieDateField.java | 1 -
.../org/apache/solr/schema/TrieDoubleField.java | 1 -
.../org/apache/solr/schema/TrieFloatField.java | 1 -
.../org/apache/solr/schema/TrieIntField.java | 1 -
.../org/apache/solr/schema/TrieLongField.java | 1 -
.../java/org/apache/solr/schema/UUIDField.java | 3 +-
.../apache/solr/schema/ZkIndexSchemaReader.java | 3 +-
.../org/apache/solr/search/AnalyticsQuery.java | 31 +-
.../java/org/apache/solr/search/BitDocSet.java | 1 -
.../solr/search/BitsFilteredDocIdSet.java | 3 +-
.../solr/search/BitsFilteredPostingsEnum.java | 3 +-
.../apache/solr/search/BoostQParserPlugin.java | 4 -
.../org/apache/solr/search/CacheConfig.java | 1 -
.../apache/solr/search/CacheRegenerator.java | 1 -
.../solr/search/CollapsingQParserPlugin.java | 5 -
.../solr/search/ComplexPhraseQParserPlugin.java | 4 +-
.../java/org/apache/solr/search/CursorMark.java | 1 -
.../apache/solr/search/DelegatingCollector.java | 1 -
.../apache/solr/search/DisMaxQParserPlugin.java | 4 -
.../org/apache/solr/search/DocIterator.java | 1 -
.../java/org/apache/solr/search/DocList.java | 1 -
.../org/apache/solr/search/DocListAndSet.java | 1 -
.../src/java/org/apache/solr/search/DocSet.java | 1 -
.../java/org/apache/solr/search/DocSetBase.java | 1 -
.../org/apache/solr/search/DocSetBuilder.java | 3 +-
.../org/apache/solr/search/DocSetCollector.java | 3 +-
.../org/apache/solr/search/DocSetProducer.java | 3 +-
.../java/org/apache/solr/search/DocSetUtil.java | 4 +-
.../java/org/apache/solr/search/DocSlice.java | 1 -
.../solr/search/EarlyTerminatingCollector.java | 3 +-
.../EarlyTerminatingCollectorException.java | 4 +-
.../apache/solr/search/ExportQParserPlugin.java | 4 -
.../solr/search/ExtendedDismaxQParser.java | 1 -
.../search/ExtendedDismaxQParserPlugin.java | 5 -
.../org/apache/solr/search/ExtendedQuery.java | 1 -
.../apache/solr/search/ExtendedQueryBase.java | 1 -
.../org/apache/solr/search/FastLRUCache.java | 3 +-
.../org/apache/solr/search/FieldParams.java | 3 +-
.../apache/solr/search/FieldQParserPlugin.java | 4 -
.../src/java/org/apache/solr/search/Filter.java | 3 +-
.../apache/solr/search/FilteredDocIdSet.java | 3 +-
.../solr/search/FunctionQParserPlugin.java | 4 -
.../solr/search/FunctionRangeQParserPlugin.java | 4 -
.../apache/solr/search/FunctionRangeQuery.java | 1 -
.../java/org/apache/solr/search/Grouping.java | 1 -
.../java/org/apache/solr/search/HashDocSet.java | 1 -
.../apache/solr/search/HashQParserPlugin.java | 5 -
.../java/org/apache/solr/search/Insanity.java | 3 +-
.../apache/solr/search/JoinQParserPlugin.java | 4 -
.../java/org/apache/solr/search/LFUCache.java | 3 +-
.../java/org/apache/solr/search/LRUCache.java | 1 -
.../apache/solr/search/LuceneQParserPlugin.java | 4 -
.../apache/solr/search/NestedQParserPlugin.java | 4 -
.../org/apache/solr/search/NoOpRegenerator.java | 3 +-
.../solr/search/OldLuceneQParserPlugin.java | 4 -
.../java/org/apache/solr/search/PostFilter.java | 1 -
.../apache/solr/search/PrefixQParserPlugin.java | 4 -
.../org/apache/solr/search/QParserPlugin.java | 4 +
.../org/apache/solr/search/QueryCommand.java | 3 +-
.../org/apache/solr/search/QueryContext.java | 3 +-
.../org/apache/solr/search/QueryParsing.java | 1 -
.../org/apache/solr/search/QueryResult.java | 3 +-
.../org/apache/solr/search/QueryResultKey.java | 1 -
.../java/org/apache/solr/search/QueryUtils.java | 1 -
.../apache/solr/search/QueryWrapperFilter.java | 3 +-
.../java/org/apache/solr/search/RankQuery.java | 31 +-
.../apache/solr/search/RawQParserPlugin.java | 4 -
.../apache/solr/search/ReRankQParserPlugin.java | 29 +-
.../org/apache/solr/search/ScoreFilter.java | 1 -
.../apache/solr/search/SimpleQParserPlugin.java | 8 +-
.../java/org/apache/solr/search/SolrCache.java | 1 -
.../org/apache/solr/search/SolrCacheBase.java | 1 -
.../solr/search/SolrConstantScoreQuery.java | 31 +-
.../org/apache/solr/search/SolrCoreParser.java | 3 +-
.../apache/solr/search/SolrFieldCacheMBean.java | 1 -
.../java/org/apache/solr/search/SolrFilter.java | 1 -
.../apache/solr/search/SolrIndexSearcher.java | 1 -
.../org/apache/solr/search/SolrQueryParser.java | 1 -
.../solr/search/SolrQueryTimeoutImpl.java | 3 +-
.../java/org/apache/solr/search/SortSpec.java | 1 -
.../org/apache/solr/search/SortSpecParsing.java | 31 +-
.../org/apache/solr/search/SortedIntDocSet.java | 1 -
.../java/org/apache/solr/search/Sorting.java | 1 -
.../solr/search/SpatialBoxQParserPlugin.java | 6 -
.../solr/search/SpatialFilterQParser.java | 4 +-
.../solr/search/SpatialFilterQParserPlugin.java | 8 +-
.../org/apache/solr/search/SpatialOptions.java | 3 +-
.../java/org/apache/solr/search/StrParser.java | 7 +-
.../solr/search/SurroundQParserPlugin.java | 8 +-
.../apache/solr/search/SwitchQParserPlugin.java | 4 -
.../org/apache/solr/search/SyntaxError.java | 1 -
.../apache/solr/search/TermQParserPlugin.java | 4 -
.../apache/solr/search/TermsQParserPlugin.java | 7 +-
.../org/apache/solr/search/WrappedQuery.java | 1 -
.../apache/solr/search/XmlQParserPlugin.java | 7 +-
.../solr/search/facet/AggValueSource.java | 3 +-
.../org/apache/solr/search/facet/AvgAgg.java | 3 +-
.../org/apache/solr/search/facet/BlockJoin.java | 4 +-
.../org/apache/solr/search/facet/CountAgg.java | 3 +-
.../solr/search/facet/FacetDebugInfo.java | 1 -
.../apache/solr/search/facet/FacetField.java | 3 +-
.../search/facet/FacetFieldProcessorDV.java | 3 +-
.../facet/FacetFieldProcessorNumeric.java | 3 +-
.../apache/solr/search/facet/FacetMerger.java | 3 +-
.../apache/solr/search/facet/FacetModule.java | 3 +-
.../solr/search/facet/FacetProcessor.java | 3 +-
.../apache/solr/search/facet/FacetQuery.java | 3 +-
.../apache/solr/search/facet/FacetRange.java | 3 +-
.../apache/solr/search/facet/FacetRequest.java | 13 +-
.../org/apache/solr/search/facet/FieldUtil.java | 3 +-
.../org/apache/solr/search/facet/HLLAgg.java | 3 +-
.../apache/solr/search/facet/LegacyFacet.java | 4 +-
.../org/apache/solr/search/facet/MaxAgg.java | 3 +-
.../org/apache/solr/search/facet/MinAgg.java | 3 +-
.../apache/solr/search/facet/PercentileAgg.java | 3 +-
.../solr/search/facet/SimpleAggValueSource.java | 3 +-
.../org/apache/solr/search/facet/SlotAcc.java | 3 +-
.../solr/search/facet/StrAggValueSource.java | 3 +-
.../org/apache/solr/search/facet/SumAgg.java | 3 +-
.../org/apache/solr/search/facet/SumsqAgg.java | 3 +-
.../solr/search/facet/UnInvertedField.java | 1 -
.../org/apache/solr/search/facet/UniqueAgg.java | 3 +-
.../apache/solr/search/facet/UniqueSlotAcc.java | 3 +-
.../search/function/CollapseScoreFunction.java | 1 -
.../solr/search/function/OrdFieldSource.java | 1 -
.../search/function/ReverseOrdFieldSource.java | 1 -
.../search/function/ValueSourceRangeFilter.java | 1 -
.../distance/GeoDistValueSourceParser.java | 3 +-
.../function/distance/GeohashFunction.java | 3 +-
.../distance/GeohashHaversineFunction.java | 4 +-
.../distance/HaversineConstFunction.java | 3 +-
.../function/distance/HaversineFunction.java | 3 +-
.../distance/SquaredEuclideanFunction.java | 3 +-
.../distance/StringDistanceFunction.java | 3 +-
.../distance/VectorDistanceFunction.java | 3 +-
.../apache/solr/search/grouping/Command.java | 3 +-
.../solr/search/grouping/CommandHandler.java | 3 +-
.../search/grouping/GroupingSpecification.java | 3 +-
.../grouping/collector/FilterCollector.java | 3 +-
.../distributed/ShardRequestFactory.java | 3 +-
.../distributed/ShardResponseProcessor.java | 3 +-
.../distributed/command/GroupConverter.java | 3 +-
.../distributed/command/QueryCommand.java | 3 +-
.../distributed/command/QueryCommandResult.java | 3 +-
.../command/SearchGroupsFieldCommand.java | 3 +-
.../command/SearchGroupsFieldCommandResult.java | 3 +-
.../command/TopGroupsFieldCommand.java | 3 +-
.../SearchGroupsRequestFactory.java | 3 +-
.../StoredFieldsShardRequestFactory.java | 3 +-
.../TopGroupsShardRequestFactory.java | 3 +-
.../SearchGroupShardResponseProcessor.java | 3 +-
.../StoredFieldsShardResponseProcessor.java | 3 +-
.../TopGroupsShardResponseProcessor.java | 3 +-
.../SearchGroupsResultTransformer.java | 3 +-
.../ShardResultTransformer.java | 3 +-
.../TopGroupsResultTransformer.java | 3 +-
.../EndResultTransformer.java | 3 +-
.../GroupedEndResultTransformer.java | 3 +-
.../MainEndResultTransformer.java | 3 +-
.../SimpleEndResultTransformer.java | 3 +-
.../apache/solr/search/join/BitSetSlice.java | 1 -
.../solr/search/join/BlockJoinChildQParser.java | 1 -
.../join/BlockJoinChildQParserPlugin.java | 1 -
.../join/BlockJoinDocSetFacetComponent.java | 33 +-
.../search/join/BlockJoinFacetCollector.java | 3 +-
.../search/join/BlockJoinFacetComponent.java | 3 +-
.../solr/search/join/BlockJoinFacetFilter.java | 3 +-
.../join/BlockJoinFieldFacetAccumulator.java | 3 +-
.../search/join/BlockJoinParentQParser.java | 1 -
.../join/BlockJoinParentQParserPlugin.java | 5 -
.../apache/solr/search/join/FrontierQuery.java | 3 +-
.../solr/search/join/GraphQParserPlugin.java | 7 +-
.../org/apache/solr/search/join/GraphQuery.java | 92 +-
.../solr/search/join/GraphQueryParser.java | 3 +-
.../solr/search/join/GraphTermsCollector.java | 5 +-
.../search/join/ScoreJoinQParserPlugin.java | 5 -
.../solr/search/join/ScoreModeParser.java | 19 +-
.../apache/solr/search/mlt/CloudMLTQParser.java | 3 +-
.../solr/search/mlt/MLTQParserPlugin.java | 8 +-
.../solr/search/mlt/SimpleMLTQParser.java | 3 +-
.../similarities/BM25SimilarityFactory.java | 3 +-
.../similarities/ClassicSimilarityFactory.java | 3 +-
.../similarities/DFISimilarityFactory.java | 3 +-
.../similarities/DFRSimilarityFactory.java | 3 +-
.../similarities/IBSimilarityFactory.java | 3 +-
.../LMDirichletSimilarityFactory.java | 3 +-
.../LMJelinekMercerSimilarityFactory.java | 3 +-
.../similarities/SchemaSimilarityFactory.java | 3 +-
.../SweetSpotSimilarityFactory.java | 3 +-
.../solr/search/stats/CachedSearcherStats.java | 3 +-
.../solr/search/stats/CollectionStats.java | 5 +-
.../search/stats/ExactSharedStatsCache.java | 3 +-
.../solr/search/stats/ExactStatsCache.java | 3 +-
.../apache/solr/search/stats/LRUStatsCache.java | 3 +-
.../solr/search/stats/LocalStatsCache.java | 7 +-
.../solr/search/stats/LocalStatsSource.java | 3 +-
.../apache/solr/search/stats/StatsCache.java | 3 +-
.../apache/solr/search/stats/StatsSource.java | 3 +-
.../org/apache/solr/search/stats/StatsUtil.java | 7 +-
.../org/apache/solr/search/stats/TermStats.java | 3 +-
.../solr/security/AuthenticationPlugin.java | 33 +-
.../solr/security/AuthorizationContext.java | 3 +-
.../solr/security/AuthorizationPlugin.java | 3 +-
.../solr/security/AuthorizationResponse.java | 3 +-
.../apache/solr/security/BasicAuthPlugin.java | 4 +-
.../solr/security/ConfigEditablePlugin.java | 3 +-
.../security/HttpClientInterceptorPlugin.java | 4 +-
.../apache/solr/security/KerberosFilter.java | 25 +-
.../apache/solr/security/KerberosPlugin.java | 33 +-
.../solr/security/PKIAuthenticationPlugin.java | 3 +-
.../security/RuleBasedAuthorizationPlugin.java | 3 +-
.../solr/security/SecurityPluginHolder.java | 4 +-
.../security/Sha256AuthenticationProvider.java | 3 +-
.../org/apache/solr/servlet/BaseSolrFilter.java | 1 -
.../apache/solr/servlet/BaseSolrServlet.java | 1 -
.../solr/servlet/CheckLoggingConfiguration.java | 1 -
.../solr/servlet/DirectSolrConnection.java | 1 -
.../org/apache/solr/servlet/HttpSolrCall.java | 7 +-
.../apache/solr/servlet/LoadAdminUiServlet.java | 1 -
.../apache/solr/servlet/RedirectServlet.java | 1 -
.../org/apache/solr/servlet/ResponseUtils.java | 3 +-
.../apache/solr/servlet/SolrDispatchFilter.java | 1 -
.../apache/solr/servlet/SolrRequestParsers.java | 1 -
.../solr/servlet/cache/HttpCacheHeaderUtil.java | 1 -
.../org/apache/solr/servlet/cache/Method.java | 1 -
.../spelling/AbstractLuceneSpellChecker.java | 5 +-
.../spelling/ConjunctionSolrSpellChecker.java | 3 +-
.../solr/spelling/DirectSolrSpellChecker.java | 3 +-
.../solr/spelling/IndexBasedSpellChecker.java | 3 +-
.../solr/spelling/PossibilityIterator.java | 3 +-
.../apache/solr/spelling/QueryConverter.java | 3 +-
.../org/apache/solr/spelling/ResultEntry.java | 3 +-
.../apache/solr/spelling/SolrSpellChecker.java | 3 +-
.../solr/spelling/SpellCheckCollation.java | 3 +-
.../solr/spelling/SpellCheckCollator.java | 3 +-
.../solr/spelling/SpellCheckCorrection.java | 3 +-
.../apache/solr/spelling/SpellingOptions.java | 17 +-
.../solr/spelling/SpellingQueryConverter.java | 1 -
.../apache/solr/spelling/SpellingResult.java | 3 +-
.../solr/spelling/SuggestQueryConverter.java | 3 +-
.../spelling/WordBreakSolrSpellChecker.java | 3 +-
.../spelling/suggest/DictionaryFactory.java | 3 +-
.../suggest/DocumentDictionaryFactory.java | 3 +-
.../DocumentExpressionDictionaryFactory.java | 3 +-
.../spelling/suggest/FileDictionaryFactory.java | 3 +-
.../suggest/HighFrequencyDictionaryFactory.java | 3 +-
.../solr/spelling/suggest/LookupFactory.java | 3 +-
.../solr/spelling/suggest/SolrSuggester.java | 3 +-
.../apache/solr/spelling/suggest/Suggester.java | 1 -
.../solr/spelling/suggest/SuggesterOptions.java | 3 +-
.../solr/spelling/suggest/SuggesterParams.java | 3 +-
.../solr/spelling/suggest/SuggesterResult.java | 3 +-
.../fst/AnalyzingInfixLookupFactory.java | 3 +-
.../suggest/fst/AnalyzingLookupFactory.java | 3 +-
.../suggest/fst/BlendedInfixLookupFactory.java | 3 +-
.../spelling/suggest/fst/FSTLookupFactory.java | 3 +-
.../suggest/fst/FreeTextLookupFactory.java | 23 +-
.../suggest/fst/FuzzyLookupFactory.java | 3 +-
.../spelling/suggest/fst/WFSTLookupFactory.java | 3 +-
.../suggest/jaspell/JaspellLookupFactory.java | 7 +-
.../spelling/suggest/tst/TSTLookupFactory.java | 3 +-
.../solr/store/blockcache/BlockCache.java | 3 +-
.../solr/store/blockcache/BlockCacheKey.java | 4 +-
.../store/blockcache/BlockCacheLocation.java | 3 +-
.../solr/store/blockcache/BlockDirectory.java | 5 +-
.../store/blockcache/BlockDirectoryCache.java | 3 +-
.../solr/store/blockcache/BlockLocks.java | 3 +-
.../solr/store/blockcache/BufferStore.java | 3 +-
.../org/apache/solr/store/blockcache/Cache.java | 3 +-
.../store/blockcache/CachedIndexOutput.java | 3 +-
.../blockcache/CustomBufferedIndexInput.java | 3 +-
.../apache/solr/store/blockcache/Metrics.java | 3 +-
.../blockcache/ReusedBufferedIndexOutput.java | 3 +-
.../org/apache/solr/store/blockcache/Store.java | 3 +-
.../apache/solr/store/hdfs/HdfsDirectory.java | 3 +-
.../apache/solr/store/hdfs/HdfsFileReader.java | 3 +-
.../apache/solr/store/hdfs/HdfsFileWriter.java | 3 +-
.../solr/store/hdfs/HdfsLocalityReporter.java | 3 +-
.../apache/solr/store/hdfs/HdfsLockFactory.java | 3 +-
.../apache/solr/update/AddUpdateCommand.java | 1 -
.../apache/solr/update/CdcrTransactionLog.java | 3 +-
.../org/apache/solr/update/CdcrUpdateLog.java | 3 +-
.../org/apache/solr/update/CommitTracker.java | 7 +-
.../apache/solr/update/CommitUpdateCommand.java | 1 -
.../solr/update/DefaultSolrCoreState.java | 3 +-
.../solr/update/DeleteByQueryWrapper.java | 3 +-
.../apache/solr/update/DeleteUpdateCommand.java | 1 -
.../solr/update/DirectUpdateHandler2.java | 1 -
.../org/apache/solr/update/DocumentBuilder.java | 1 -
.../apache/solr/update/HdfsTransactionLog.java | 23 +-
.../org/apache/solr/update/HdfsUpdateLog.java | 1 -
.../apache/solr/update/IndexFingerprint.java | 200 ++++
.../apache/solr/update/LoggingInfoStream.java | 1 -
.../org/apache/solr/update/MemOutputStream.java | 1 -
.../apache/solr/update/MergeIndexesCommand.java | 1 -
.../java/org/apache/solr/update/PeerSync.java | 79 +-
.../solr/update/RollbackUpdateCommand.java | 1 -
.../apache/solr/update/SolrCmdDistributor.java | 3 +-
.../org/apache/solr/update/SolrCoreState.java | 3 +-
.../org/apache/solr/update/SolrIndexConfig.java | 1 -
.../apache/solr/update/SolrIndexSplitter.java | 1 -
.../org/apache/solr/update/SolrIndexWriter.java | 1 -
.../apache/solr/update/SplitIndexCommand.java | 1 -
.../solr/update/StreamingSolrClients.java | 3 +-
.../org/apache/solr/update/TransactionLog.java | 1 -
.../org/apache/solr/update/UpdateCommand.java | 1 -
.../org/apache/solr/update/UpdateHandler.java | 1 -
.../java/org/apache/solr/update/UpdateLog.java | 7 +-
.../apache/solr/update/UpdateShardHandler.java | 3 +-
.../solr/update/UpdateShardHandlerConfig.java | 3 +-
.../org/apache/solr/update/VersionBucket.java | 1 -
.../org/apache/solr/update/VersionInfo.java | 1 -
...tractDefaultValueUpdateProcessorFactory.java | 1 -
.../AddSchemaFieldsUpdateProcessorFactory.java | 1 -
...aluesOrNoneFieldMutatingUpdateProcessor.java | 1 -
.../processor/AtomicUpdateDocumentMerger.java | 7 +-
.../update/processor/CdcrUpdateProcessor.java | 3 +-
.../processor/CdcrUpdateProcessorFactory.java | 3 +-
.../ConcatFieldUpdateProcessorFactory.java | 1 -
.../CountFieldValuesUpdateProcessorFactory.java | 1 -
.../DefaultValueUpdateProcessorFactory.java | 1 -
.../processor/DistributedUpdateProcessor.java | 7 +-
.../DistributedUpdateProcessorFactory.java | 3 +-
.../DistributingUpdateProcessorFactory.java | 1 -
...BasedVersionConstraintsProcessorFactory.java | 1 -
.../DocExpirationUpdateProcessorFactory.java | 1 -
.../FieldLengthUpdateProcessorFactory.java | 1 -
.../processor/FieldMutatingUpdateProcessor.java | 1 -
.../FieldMutatingUpdateProcessorFactory.java | 1 -
...FieldNameMutatingUpdateProcessorFactory.java | 4 +-
.../FieldValueMutatingUpdateProcessor.java | 1 -
.../FieldValueSubsetUpdateProcessorFactory.java | 1 -
.../FirstFieldValueUpdateProcessorFactory.java | 1 -
.../HTMLStripFieldUpdateProcessorFactory.java | 1 -
...oreCommitOptimizeUpdateProcessorFactory.java | 1 -
.../IgnoreFieldUpdateProcessorFactory.java | 1 -
.../LastFieldValueUpdateProcessorFactory.java | 1 -
.../processor/LogUpdateProcessorFactory.java | 1 -
.../solr/update/processor/Lookup3Signature.java | 3 +-
.../solr/update/processor/MD5Signature.java | 3 +-
.../MaxFieldValueUpdateProcessorFactory.java | 1 -
.../MinFieldValueUpdateProcessorFactory.java | 1 -
.../NoOpDistributingUpdateProcessorFactory.java | 1 -
...ParseBooleanFieldUpdateProcessorFactory.java | 1 -
.../ParseDateFieldUpdateProcessorFactory.java | 1 -
.../ParseDoubleFieldUpdateProcessorFactory.java | 1 -
.../ParseFloatFieldUpdateProcessorFactory.java | 1 -
.../ParseIntFieldUpdateProcessorFactory.java | 1 -
.../ParseLongFieldUpdateProcessorFactory.java | 1 -
...ParseNumericFieldUpdateProcessorFactory.java | 1 -
.../PreAnalyzedUpdateProcessorFactory.java | 33 +-
.../RemoveBlankFieldUpdateProcessorFactory.java | 1 -
.../processor/RunUpdateProcessorFactory.java | 1 -
.../processor/ScriptEngineCustomizer.java | 1 -
.../apache/solr/update/processor/Signature.java | 3 +-
.../SignatureUpdateProcessorFactory.java | 3 +-
.../processor/SimpleUpdateProcessorFactory.java | 3 +-
.../StatelessScriptUpdateProcessorFactory.java | 3 +-
.../update/processor/TextProfileSignature.java | 3 +-
.../TimestampUpdateProcessorFactory.java | 1 -
.../TrimFieldUpdateProcessorFactory.java | 1 -
.../TruncateFieldUpdateProcessorFactory.java | 1 -
.../processor/UUIDUpdateProcessorFactory.java | 1 -
.../UniqFieldsUpdateProcessorFactory.java | 3 +-
.../processor/UpdateRequestProcessor.java | 1 -
.../processor/UpdateRequestProcessorChain.java | 1 -
.../UpdateRequestProcessorFactory.java | 1 -
.../apache/solr/util/AdjustableSemaphore.java | 3 +-
.../org/apache/solr/util/BoundedTreeSet.java | 2 -
.../org/apache/solr/util/CommandOperation.java | 3 +-
.../apache/solr/util/ConcurrentLFUCache.java | 3 +-
.../apache/solr/util/ConcurrentLRUCache.java | 3 +-
.../java/org/apache/solr/util/CryptoKeys.java | 3 +-
.../src/java/org/apache/solr/util/DOMUtil.java | 3 +-
.../org/apache/solr/util/DateFormatUtil.java | 3 +-
.../org/apache/solr/util/DateMathParser.java | 1 -
.../solr/util/DefaultSolrThreadFactory.java | 3 +-
.../org/apache/solr/util/DistanceUnits.java | 19 +-
.../java/org/apache/solr/util/FSHDFSUtils.java | 3 +-
.../java/org/apache/solr/util/FastWriter.java | 3 +-
.../java/org/apache/solr/util/FileUtils.java | 3 +-
.../src/java/org/apache/solr/util/HdfsUtil.java | 17 +-
.../org/apache/solr/util/LongPriorityQueue.java | 7 +-
.../java/org/apache/solr/util/MapListener.java | 3 +-
.../java/org/apache/solr/util/NumberUtils.java | 1 -
.../org/apache/solr/util/PivotListEntry.java | 3 +-
.../java/org/apache/solr/util/PrimUtils.java | 3 +-
.../apache/solr/util/PropertiesInputStream.java | 3 +-
.../solr/util/PropertiesOutputStream.java | 3 +-
.../org/apache/solr/util/PropertiesUtil.java | 3 +-
.../src/java/org/apache/solr/util/RTimer.java | 3 +-
.../java/org/apache/solr/util/RTimerTree.java | 3 +-
.../apache/solr/util/RecordingJSONParser.java | 4 +-
.../java/org/apache/solr/util/RefCounted.java | 1 -
.../org/apache/solr/util/RegexFileFilter.java | 3 +-
.../org/apache/solr/util/SimplePostTool.java | 3 +-
.../src/java/org/apache/solr/util/SolrCLI.java | 3 +-
.../org/apache/solr/util/SolrLogLayout.java | 33 +-
.../org/apache/solr/util/SolrPluginUtils.java | 1 -
.../java/org/apache/solr/util/SpatialUtils.java | 3 +-
.../org/apache/solr/util/SystemIdResolver.java | 3 +-
.../org/apache/solr/util/TestInjection.java | 3 +-
.../src/java/org/apache/solr/util/TimeOut.java | 3 +-
.../org/apache/solr/util/TimeZoneUtils.java | 1 -
.../org/apache/solr/util/VersionedFile.java | 1 -
.../hll/BigEndianAscendingWordDeserializer.java | 3 +-
.../hll/BigEndianAscendingWordSerializer.java | 3 +-
.../java/org/apache/solr/util/hll/BitUtil.java | 3 +-
.../org/apache/solr/util/hll/BitVector.java | 3 +-
.../src/java/org/apache/solr/util/hll/HLL.java | 3 +-
.../org/apache/solr/util/hll/HLLMetadata.java | 3 +-
.../java/org/apache/solr/util/hll/HLLType.java | 3 +-
.../java/org/apache/solr/util/hll/HLLUtil.java | 3 +-
.../org/apache/solr/util/hll/IHLLMetadata.java | 3 +-
.../apache/solr/util/hll/ISchemaVersion.java | 3 +-
.../apache/solr/util/hll/IWordDeserializer.java | 3 +-
.../apache/solr/util/hll/IWordSerializer.java | 3 +-
.../org/apache/solr/util/hll/LongIterator.java | 3 +-
.../org/apache/solr/util/hll/NumberUtil.java | 3 +-
.../apache/solr/util/hll/SchemaVersionOne.java | 3 +-
.../apache/solr/util/hll/SerializationUtil.java | 3 +-
.../solr/util/plugin/AbstractPluginLoader.java | 1 -
.../solr/util/plugin/MapInitializedPlugin.java | 1 -
.../solr/util/plugin/MapPluginLoader.java | 1 -
.../util/plugin/NamedListInitializedPlugin.java | 1 -
.../solr/util/plugin/NamedListPluginLoader.java | 1 -
.../apache/solr/util/plugin/SolrCoreAware.java | 1 -
.../java/org/apache/solr/util/stats/Clock.java | 1 -
.../java/org/apache/solr/util/stats/EWMA.java | 1 -
.../util/stats/ExponentiallyDecayingSample.java | 1 -
.../org/apache/solr/util/stats/Histogram.java | 1 -
.../java/org/apache/solr/util/stats/Meter.java | 1 -
.../java/org/apache/solr/util/stats/Sample.java | 1 -
.../org/apache/solr/util/stats/Snapshot.java | 1 -
.../java/org/apache/solr/util/stats/Timer.java | 1 -
.../apache/solr/util/stats/TimerContext.java | 1 -
.../apache/solr/util/stats/UniformSample.java | 1 -
.../solr/util/xslt/TransformerProvider.java | 1 -
.../runtimecode/RuntimeLibReqHandler.java | 4 +-
.../runtimecode/RuntimeLibResponseWriter.java | 18 +-
.../runtimecode/RuntimeLibSearchComponent.java | 4 +-
.../solr/AnalysisAfterCoreReloadTest.java | 3 +-
.../org/apache/solr/BasicFunctionalityTest.java | 1 -
.../org/apache/solr/ConvertedLegacyTest.java | 1 -
.../test/org/apache/solr/CursorPagingTest.java | 1 -
.../apache/solr/DisMaxRequestHandlerTest.java | 1 -
.../solr/DistributedIntervalFacetingTest.java | 28 +-
.../test/org/apache/solr/EchoParamsTest.java | 1 -
.../test/org/apache/solr/MinimalSchemaTest.java | 1 -
.../test/org/apache/solr/OutputWriterTest.java | 1 -
.../src/test/org/apache/solr/SampleTest.java | 1 -
.../org/apache/solr/SolrTestCaseJ4Test.java | 3 +-
.../test/org/apache/solr/TestCrossCoreJoin.java | 1 -
.../solr/TestCursorMarkWithoutUniqueKey.java | 1 -
.../apache/solr/TestDistributedGrouping.java | 3 +-
.../apache/solr/TestDistributedMissingSort.java | 1 -
.../org/apache/solr/TestDistributedSearch.java | 1 -
.../org/apache/solr/TestDocumentBuilder.java | 4 +-
.../org/apache/solr/TestGroupingSearch.java | 1 -
.../apache/solr/TestHighlightDedupGrouping.java | 1 -
.../core/src/test/org/apache/solr/TestJoin.java | 1 -
.../org/apache/solr/TestRandomDVFaceting.java | 1 -
.../org/apache/solr/TestRandomFaceting.java | 1 -
.../solr/TestSimpleTrackingShardHandler.java | 1 -
.../org/apache/solr/TestTolerantSearch.java | 33 +-
.../PathHierarchyTokenizerFactoryTest.java | 1 -
.../apache/solr/analysis/TestCharFilters.java | 3 +-
.../TestReversedWildcardFilterFactory.java | 3 +-
.../TestWordDelimiterFilterFactory.java | 3 +-
.../ThrowingMockTokenFilterFactory.java | 3 +-
.../solr/client/solrj/ConnectionReuseTest.java | 5 +-
.../TestEmbeddedSolrServerConstructors.java | 3 +-
.../solrj/embedded/TestJettySolrRunner.java | 3 +-
.../apache/solr/cloud/ActionThrottleTest.java | 3 +-
.../apache/solr/cloud/AliasIntegrationTest.java | 3 +-
.../test/org/apache/solr/cloud/AssignTest.java | 3 +-
.../AsyncCallRequestStatusResponseTest.java | 3 +-
.../solr/cloud/AsyncMigrateRouteKeyTest.java | 3 +-
.../solr/cloud/BaseCdcrDistributedZkTest.java | 3 +-
.../solr/cloud/BasicDistributedZk2Test.java | 3 +-
.../solr/cloud/BasicDistributedZkTest.java | 3 +-
.../test/org/apache/solr/cloud/BasicZkTest.java | 3 +-
.../cloud/CdcrReplicationDistributedZkTest.java | 3 +-
.../solr/cloud/CdcrReplicationHandlerTest.java | 3 +-
.../solr/cloud/CdcrRequestHandlerTest.java | 3 +-
.../solr/cloud/CdcrVersionReplicationTest.java | 3 +-
.../cloud/ChaosMonkeyNothingIsSafeTest.java | 3 +-
.../solr/cloud/ChaosMonkeySafeLeaderTest.java | 3 +-
.../solr/cloud/ChaosMonkeyShardSplitTest.java | 3 +-
.../apache/solr/cloud/CleanupOldIndexTest.java | 3 +-
.../cloud/CloudExitableDirectoryReaderTest.java | 34 +-
.../org/apache/solr/cloud/ClusterStateTest.java | 27 +-
.../solr/cloud/ClusterStateUpdateTest.java | 3 +-
.../apache/solr/cloud/CollectionReloadTest.java | 3 +-
.../solr/cloud/CollectionStateFormat2Test.java | 3 +-
.../cloud/CollectionTooManyReplicasTest.java | 3 +-
.../CollectionsAPIAsyncDistributedZkTest.java | 3 +-
.../cloud/CollectionsAPIDistributedZkTest.java | 3 +-
.../solr/cloud/CollectionsAPISolrJTests.java | 3 +-
...ConcurrentDeleteAndCreateCollectionTest.java | 3 +-
.../apache/solr/cloud/ConfigSetsAPITest.java | 3 +-
.../solr/cloud/ConnectionManagerTest.java | 27 +-
.../apache/solr/cloud/CustomCollectionTest.java | 3 +-
.../solr/cloud/DeleteInactiveReplicaTest.java | 3 +-
.../DeleteLastCustomShardedReplicaTest.java | 3 +-
.../apache/solr/cloud/DeleteReplicaTest.java | 3 +-
.../org/apache/solr/cloud/DeleteShardTest.java | 3 +-
.../cloud/DistribJoinFromCollectionTest.java | 3 +-
.../apache/solr/cloud/DistributedQueueTest.java | 22 +-
.../solr/cloud/DistributedVersionInfoTest.java | 3 +-
.../org/apache/solr/cloud/ForceLeaderTest.java | 3 +-
.../cloud/FullSolrCloudDistribCmdsTest.java | 3 +-
.../apache/solr/cloud/HttpPartitionTest.java | 3 +-
.../org/apache/solr/cloud/KerberosTestUtil.java | 29 +-
.../cloud/LeaderElectionIntegrationTest.java | 3 +-
.../apache/solr/cloud/LeaderElectionTest.java | 28 +-
.../cloud/LeaderFailoverAfterPartitionTest.java | 3 +-
.../LeaderInitiatedRecoveryOnCommitTest.java | 3 +-
...aderInitiatedRecoveryOnShardRestartTest.java | 3 +-
.../apache/solr/cloud/MigrateRouteKeyTest.java | 3 +-
.../apache/solr/cloud/MultiThreadedOCPTest.java | 3 +-
...utOfBoxZkACLAndCredentialsProvidersTest.java | 33 +-
...rriddenZkACLAndCredentialsProvidersTest.java | 33 +-
...verseerCollectionConfigSetProcessorTest.java | 3 +-
.../apache/solr/cloud/OverseerRolesTest.java | 4 +-
.../apache/solr/cloud/OverseerStatusTest.java | 3 +-
.../solr/cloud/OverseerTaskQueueTest.java | 22 +-
.../org/apache/solr/cloud/OverseerTest.java | 3 +-
.../solr/cloud/RecoveryAfterSoftCommitTest.java | 3 +-
.../org/apache/solr/cloud/RecoveryZkTest.java | 3 +-
.../apache/solr/cloud/RemoteQueryErrorTest.java | 3 +-
.../solr/cloud/ReplicaPropertiesBase.java | 3 +-
.../solr/cloud/ReplicationFactorTest.java | 3 +-
.../solr/cloud/RestartWhileUpdatingTest.java | 3 +-
.../apache/solr/cloud/RollingRestartTest.java | 3 +-
.../org/apache/solr/cloud/SSLMigrationTest.java | 3 +-
.../solr/cloud/SaslZkACLProviderTest.java | 33 +-
.../solr/cloud/ShardRoutingCustomTest.java | 3 +-
.../org/apache/solr/cloud/ShardRoutingTest.java | 3 +-
.../org/apache/solr/cloud/ShardSplitTest.java | 3 +-
.../cloud/SharedFSAutoReplicaFailoverTest.java | 3 +-
.../SharedFSAutoReplicaFailoverUtilsTest.java | 3 +-
.../cloud/SimpleCollectionCreateDeleteTest.java | 3 +-
.../org/apache/solr/cloud/SliceStateTest.java | 27 +-
.../apache/solr/cloud/SolrCloudExampleTest.java | 3 +-
.../org/apache/solr/cloud/SolrXmlInZkTest.java | 22 +-
.../org/apache/solr/cloud/SyncSliceTest.java | 3 +-
.../solr/cloud/TestAuthenticationFramework.java | 3 +-
.../apache/solr/cloud/TestCloudInspectUtil.java | 7 +-
.../apache/solr/cloud/TestCollectionAPI.java | 4 +-
.../apache/solr/cloud/TestConfigSetsAPI.java | 1 -
.../cloud/TestConfigSetsAPIExclusivity.java | 1 -
.../solr/cloud/TestConfigSetsAPIZkFailure.java | 1 -
.../org/apache/solr/cloud/TestCryptoKeys.java | 3 +-
.../solr/cloud/TestDistribDocBasedVersion.java | 3 +-
.../solr/cloud/TestDownShardTolerantSearch.java | 3 +-
.../TestExclusionRuleCollectionAccess.java | 3 +-
.../apache/solr/cloud/TestHashPartitioner.java | 27 +-
.../solr/cloud/TestLeaderElectionZkExpiry.java | 3 +-
.../TestLeaderInitiatedRecoveryThread.java | 31 +-
.../solr/cloud/TestMiniSolrCloudCluster.java | 3 +-
.../cloud/TestMiniSolrCloudClusterBase.java | 4 +-
.../cloud/TestMiniSolrCloudClusterKerberos.java | 3 +-
.../solr/cloud/TestMiniSolrCloudClusterSSL.java | 3 +-
.../cloud/TestRandomRequestDistribution.java | 3 +-
.../apache/solr/cloud/TestRebalanceLeaders.java | 3 +-
.../solr/cloud/TestReplicaProperties.java | 4 +-
.../solr/cloud/TestRequestForwarding.java | 3 +-
.../cloud/TestRequestStatusCollectionAPI.java | 3 +-
.../solr/cloud/TestShortCircuitedRequests.java | 3 +-
.../cloud/TestSolrCloudWithKerberosAlt.java | 3 +-
.../org/apache/solr/cloud/TestZkChroot.java | 3 +-
.../TlogReplayBufferedWhileIndexingTest.java | 136 +++
.../cloud/TriLevelCompositeIdRoutingTest.java | 3 +-
.../solr/cloud/UnloadDistributedZkTest.java | 3 +-
...MParamsZkACLAndCredentialsProvidersTest.java | 33 +-
.../test/org/apache/solr/cloud/ZkCLITest.java | 3 +-
.../org/apache/solr/cloud/ZkControllerTest.java | 28 +-
.../org/apache/solr/cloud/ZkNodePropsTest.java | 27 +-
.../org/apache/solr/cloud/ZkSolrClientTest.java | 27 +-
.../cloud/hdfs/HdfsBasicDistributedZk2Test.java | 3 +-
.../cloud/hdfs/HdfsBasicDistributedZkTest.java | 3 +-
.../hdfs/HdfsChaosMonkeyNothingIsSafeTest.java | 3 +-
.../hdfs/HdfsChaosMonkeySafeLeaderTest.java | 3 +-
.../HdfsCollectionsAPIDistributedZkTest.java | 3 +-
.../solr/cloud/hdfs/HdfsNNFailoverTest.java | 1 -
.../solr/cloud/hdfs/HdfsRecoverLeaseTest.java | 3 +-
.../solr/cloud/hdfs/HdfsRecoveryZkTest.java | 3 +-
.../hdfs/HdfsRestartWhileUpdatingTest.java | 3 +-
.../solr/cloud/hdfs/HdfsSyncSliceTest.java | 3 +-
.../apache/solr/cloud/hdfs/HdfsTestUtil.java | 33 +-
.../solr/cloud/hdfs/HdfsThreadLeakTest.java | 3 +-
...HdfsTlogReplayBufferedWhileIndexingTest.java | 63 ++
.../cloud/hdfs/HdfsUnloadDistributedZkTest.java | 3 +-
.../HdfsWriteToMultipleCollectionsTest.java | 1 -
.../apache/solr/cloud/hdfs/StressHdfsTest.java | 1 -
.../cloud/overseer/TestClusterStateMutator.java | 3 +-
.../solr/cloud/overseer/ZkStateReaderTest.java | 3 +-
.../solr/cloud/overseer/ZkStateWriterTest.java | 3 +-
.../apache/solr/cloud/rule/RuleEngineTest.java | 3 +-
.../org/apache/solr/cloud/rule/RulesTest.java | 7 +-
.../solr/core/BlobStoreTestRequestHandler.java | 4 +-
.../solr/core/CachingDirectoryFactoryTest.java | 33 +-
.../solr/core/CountUsageValueSourceParser.java | 3 +-
.../apache/solr/core/DirectoryFactoryTest.java | 1 -
.../solr/core/DummyValueSourceParser.java | 3 +-
.../solr/core/ExitableDirectoryReaderTest.java | 4 +-
.../solr/core/HdfsDirectoryFactoryTest.java | 1 -
.../org/apache/solr/core/MockInfoMBean.java | 13 +-
.../solr/core/MockShardHandlerFactory.java | 3 +-
.../solr/core/OpenCloseCoreStressTest.java | 1 -
.../org/apache/solr/core/PluginInfoTest.java | 3 +-
.../apache/solr/core/QueryResultKeyTest.java | 1 -
.../solr/core/RAMDirectoryFactoryTest.java | 1 -
.../apache/solr/core/RequestHandlersTest.java | 1 -
.../apache/solr/core/ResourceLoaderTest.java | 1 -
.../test/org/apache/solr/core/SOLR749Test.java | 3 +-
.../core/SolrCoreCheckLockOnStartupTest.java | 3 +-
.../test/org/apache/solr/core/SolrCoreTest.java | 1 -
.../org/apache/solr/core/TestBadConfig.java | 1 -
.../org/apache/solr/core/TestCodecSupport.java | 3 +-
.../test/org/apache/solr/core/TestConfig.java | 1 -
.../org/apache/solr/core/TestConfigOverlay.java | 3 +-
.../solr/core/TestConfigSetImmutable.java | 3 +-
.../solr/core/TestConfigSetProperties.java | 3 +-
.../org/apache/solr/core/TestConfigSets.java | 3 +-
.../org/apache/solr/core/TestCoreContainer.java | 1 -
.../org/apache/solr/core/TestCoreDiscovery.java | 3 +-
.../apache/solr/core/TestDynamicLoading.java | 4 +-
.../solr/core/TestImplicitCoreProperties.java | 32 +-
.../apache/solr/core/TestInfoStreamLogging.java | 3 +-
.../org/apache/solr/core/TestInitParams.java | 3 +-
.../org/apache/solr/core/TestLazyCores.java | 3 +-
.../apache/solr/core/TestMergePolicyConfig.java | 3 +-
.../test/org/apache/solr/core/TestNRTOpen.java | 3 +-
.../solr/core/TestQuerySenderListener.java | 1 -
.../solr/core/TestQuerySenderNoQuery.java | 3 +-
.../solr/core/TestReloadAndDeleteDocs.java | 1 -
.../solr/core/TestShardHandlerFactory.java | 1 -
.../apache/solr/core/TestSolrConfigHandler.java | 4 +-
.../apache/solr/core/TestSolrIndexConfig.java | 3 +-
.../test/org/apache/solr/core/TestSolrXml.java | 3 +-
.../apache/solr/core/TestXIncludeConfig.java | 3 +-
.../handler/AnalysisRequestHandlerTestBase.java | 1 -
.../solr/handler/CSVRequestHandlerTest.java | 1 -
.../apache/solr/handler/CheckBackupStatus.java | 3 +-
.../DocumentAnalysisRequestHandlerTest.java | 1 -
.../FieldAnalysisRequestHandlerTest.java | 1 -
.../org/apache/solr/handler/JsonLoaderTest.java | 1 -
.../solr/handler/MoreLikeThisHandlerTest.java | 1 -
.../solr/handler/PingRequestHandlerTest.java | 1 -
.../apache/solr/handler/RequestLoggingTest.java | 3 +-
.../handler/StandardRequestHandlerTest.java | 1 -
.../apache/solr/handler/TestBlobHandler.java | 3 +-
.../org/apache/solr/handler/TestCSVLoader.java | 1 -
.../apache/solr/handler/TestConfigReload.java | 3 +-
.../handler/TestReplicationHandlerBackup.java | 3 +-
.../apache/solr/handler/TestReqParamsAPI.java | 33 +-
.../apache/solr/handler/TestRestoreCore.java | 4 +-
.../org/apache/solr/handler/TestSQLHandler.java | 6 +-
.../handler/TestSolrConfigHandlerCloud.java | 4 +-
.../TestSolrConfigHandlerConcurrent.java | 3 +-
.../handler/ThrowErrorOnInitRequestHandler.java | 1 -
.../admin/CoreAdminCreateDiscoverTest.java | 1 -
.../handler/admin/CoreAdminHandlerTest.java | 62 +-
.../admin/CoreAdminRequestStatusTest.java | 3 +-
.../admin/CoreMergeIndexesAdminHandlerTest.java | 3 +-
.../solr/handler/admin/InfoHandlerTest.java | 1 -
.../solr/handler/admin/LoggingHandlerTest.java | 1 -
.../handler/admin/LukeRequestHandlerTest.java | 1 -
.../solr/handler/admin/MBeansHandlerTest.java | 1 -
.../handler/admin/SecurityConfHandlerTest.java | 4 +-
.../admin/SegmentsInfoRequestHandlerTest.java | 3 +-
.../admin/ShowFileRequestHandlerTest.java | 3 +-
.../handler/admin/SystemInfoHandlerTest.java | 1 -
.../handler/component/BadComponentTest.java | 5 +-
.../handler/component/DebugComponentTest.java | 3 +-
.../DistributedDebugComponentTest.java | 33 +-
.../DistributedExpandComponentTest.java | 3 +-
.../DistributedFacetPivotLargeTest.java | 3 +-
.../DistributedFacetPivotLongTailTest.java | 3 +-
.../DistributedFacetPivotSmallAdvancedTest.java | 3 +-
.../DistributedFacetPivotSmallTest.java | 3 +-
.../DistributedFacetPivotWhiteBoxTest.java | 3 +-
.../component/DistributedMLTComponentTest.java | 3 +-
...DistributedQueryComponentCustomSortTest.java | 3 +-
...stributedQueryComponentOptimizationTest.java | 3 +-
.../DistributedQueryElevationComponentTest.java | 3 +-
.../DistributedSpellCheckComponentTest.java | 3 +-
.../DistributedSuggestComponentTest.java | 3 +-
.../DistributedTermsComponentTest.java | 3 +-
.../component/DummyCustomParamSpellChecker.java | 26 +-
.../handler/component/FacetPivotSmallTest.java | 3 +-
.../component/QueryElevationComponentTest.java | 1 -
.../component/ResponseLogComponentTest.java | 17 +-
.../handler/component/SearchHandlerTest.java | 1 -
.../component/SpatialHeatmapFacetsTest.java | 3 +-
.../component/SpellCheckComponentTest.java | 1 -
.../handler/component/StatsComponentTest.java | 3 +-
.../SuggestComponentContextFilterQueryTest.java | 3 +-
.../handler/component/SuggestComponentTest.java | 3 +-
.../TermVectorComponentDistributedTest.java | 1 -
.../component/TermVectorComponentTest.java | 18 +-
.../handler/component/TermsComponentTest.java | 3 +-
...estDistributedStatsComponentCardinality.java | 7 +-
.../handler/component/TestExpandComponent.java | 31 +-
.../handler/component/TestPivotHelperCode.java | 5 +-
.../TestTrackingShardHandlerFactory.java | 3 +-
.../solr/handler/loader/JavabinLoaderTest.java | 3 +-
.../highlight/FastVectorHighlighterTest.java | 1 -
.../highlight/HighlighterMaxOffsetTest.java | 3 +-
.../apache/solr/highlight/HighlighterTest.java | 1 -
.../highlight/TestPostingsSolrHighlighter.java | 3 +-
.../solr/index/hdfs/CheckHdfsIndexTest.java | 3 +-
.../apache/solr/internal/csv/CSVParserTest.java | 6 +-
.../solr/internal/csv/CSVPrinterTest.java | 6 +-
.../solr/internal/csv/CSVStrategyTest.java | 6 +-
.../apache/solr/internal/csv/CSVUtilsTest.java | 6 +-
.../solr/internal/csv/CharBufferTest.java | 26 +-
.../csv/ExtendedBufferedReaderTest.java | 6 +-
.../csv/writer/CSVConfigGuesserTest.java | 26 +-
.../solr/internal/csv/writer/CSVConfigTest.java | 26 +-
.../solr/internal/csv/writer/CSVFieldTest.java | 26 +-
.../solr/internal/csv/writer/CSVWriterTest.java | 26 +-
.../org/apache/solr/logging/TestLogWatcher.java | 3 +-
.../org/apache/solr/request/JSONWriterTest.java | 1 -
.../apache/solr/request/SimpleFacetsTest.java | 1 -
.../apache/solr/request/SmileWriterTest.java | 4 +-
.../org/apache/solr/request/TestFaceting.java | 1 -
.../solr/request/TestIntervalFaceting.java | 1 -
.../solr/request/TestRemoteStreaming.java | 3 +-
.../org/apache/solr/request/TestWriterPerf.java | 1 -
.../apache/solr/request/macro/TestMacros.java | 4 +-
.../solr/response/TestCSVResponseWriter.java | 1 -
.../solr/response/TestChildDocTransformer.java | 3 +-
.../solr/response/TestCustomDocTransformer.java | 3 +-
.../TestPHPSerializedResponseWriter.java | 1 -
.../solr/response/TestRawResponseWriter.java | 1 -
.../solr/response/TestRawTransformer.java | 3 +-
.../solr/response/TestSolrQueryResponse.java | 3 +-
.../response/TestSortingResponseWriter.java | 6 +-
.../apache/solr/rest/SolrRestletTestBase.java | 3 +-
.../apache/solr/rest/TestManagedResource.java | 3 +-
.../solr/rest/TestManagedResourceStorage.java | 3 +-
.../org/apache/solr/rest/TestRestManager.java | 3 +-
.../solr/rest/schema/TestBulkSchemaAPI.java | 86 +-
.../rest/schema/TestClassNameShortening.java | 3 +-
.../schema/TestCopyFieldCollectionResource.java | 3 +-
.../schema/TestDefaultSearchFieldResource.java | 3 +-
.../TestDynamicFieldCollectionResource.java | 3 +-
.../rest/schema/TestDynamicFieldResource.java | 3 +-
.../schema/TestFieldCollectionResource.java | 3 +-
.../solr/rest/schema/TestFieldResource.java | 3 +-
.../schema/TestFieldTypeCollectionResource.java | 3 +-
.../solr/rest/schema/TestFieldTypeResource.java | 4 +-
.../TestManagedSchemaDynamicFieldResource.java | 3 +-
.../schema/TestManagedSchemaFieldResource.java | 3 +-
.../TestManagedSchemaFieldTypeResource.java | 3 +-
.../schema/TestRemoveLastDynamicCopyField.java | 4 +-
.../rest/schema/TestSchemaNameResource.java | 3 +-
.../solr/rest/schema/TestSchemaResource.java | 4 +-
.../schema/TestSchemaSimilarityResource.java | 3 +-
.../rest/schema/TestSchemaVersionResource.java | 3 +-
.../TestSerializedLuceneMatchVersion.java | 3 +-
...tSolrQueryParserDefaultOperatorResource.java | 3 +-
.../schema/TestSolrQueryParserResource.java | 3 +-
.../rest/schema/TestUniqueKeyFieldResource.java | 3 +-
.../analysis/TestManagedStopFilterFactory.java | 3 +-
.../TestManagedSynonymFilterFactory.java | 3 +-
.../solr/schema/AbstractCurrencyFieldTest.java | 3 +-
.../apache/solr/schema/BadCopyFieldTest.java | 1 -
.../apache/solr/schema/BadIndexSchemaTest.java | 1 -
.../solr/schema/ChangedSchemaMergeTest.java | 1 -
.../org/apache/solr/schema/CopyFieldTest.java | 1 -
.../schema/CurrencyFieldOpenExchangeTest.java | 3 +-
.../solr/schema/CurrencyFieldXmlFileTest.java | 3 +-
.../solr/schema/CustomAnalyzerStrField.java | 3 +-
.../org/apache/solr/schema/DateFieldTest.java | 1 -
.../apache/solr/schema/DateRangeFieldTest.java | 5 +-
.../solr/schema/DocValuesMissingTest.java | 3 +-
.../apache/solr/schema/DocValuesMultiTest.java | 3 +-
.../org/apache/solr/schema/DocValuesTest.java | 3 +-
.../org/apache/solr/schema/EnumFieldTest.java | 3 +-
.../solr/schema/ExternalFileFieldSortTest.java | 19 +-
.../schema/IndexSchemaRuntimeFieldTest.java | 3 +-
.../org/apache/solr/schema/IndexSchemaTest.java | 1 -
.../solr/schema/MockExchangeRateProvider.java | 3 +-
.../org/apache/solr/schema/MultiTermTest.java | 3 +-
.../apache/solr/schema/MyCrazyCustomField.java | 3 +-
.../solr/schema/NotRequiredUniqueKeyTest.java | 1 -
.../apache/solr/schema/NumericFieldsTest.java | 1 -
.../OpenExchangeRatesOrgProviderTest.java | 3 +-
.../org/apache/solr/schema/PolyFieldTest.java | 3 +-
.../solr/schema/PreAnalyzedFieldTest.java | 3 +-
.../solr/schema/PrimitiveFieldTypeTest.java | 1 -
.../apache/solr/schema/RequiredFieldsTest.java | 1 -
.../SchemaVersionSpecificBehaviorTest.java | 1 -
.../apache/solr/schema/SortableBinaryField.java | 3 +-
.../solr/schema/SpatialRPTFieldTypeTest.java | 3 +-
.../solr/schema/SynonymTokenizerTest.java | 3 +-
.../org/apache/solr/schema/TestBinaryField.java | 1 -
.../solr/schema/TestBulkSchemaConcurrent.java | 4 +-
.../solr/schema/TestCloudManagedSchema.java | 3 +-
.../TestCloudManagedSchemaConcurrent.java | 3 +-
.../apache/solr/schema/TestCloudSchemaless.java | 3 +-
.../apache/solr/schema/TestCollationField.java | 1 -
.../schema/TestCollationFieldDocValues.java | 1 -
.../apache/solr/schema/TestManagedSchema.java | 3 +-
.../apache/solr/schema/TestOmitPositions.java | 3 +-
.../apache/solr/schema/TestSchemaManager.java | 3 +-
.../solr/schema/TestUseDocValuesAsStored.java | 3 +-
.../solr/schema/TestUseDocValuesAsStored2.java | 3 +-
.../solr/schema/ThrowErrorOnInitFieldType.java | 3 +-
.../TrieIntPrefixActsAsRangeQueryFieldType.java | 3 +-
.../org/apache/solr/schema/WrappedIntField.java | 3 +-
.../solr/search/AnalyticsMergeStrategyTest.java | 3 +-
.../apache/solr/search/AnalyticsQueryTest.java | 1 -
.../org/apache/solr/search/CursorMarkTest.java | 1 -
.../solr/search/DelayingSearchComponent.java | 3 +-
.../test/org/apache/solr/search/DocSetPerf.java | 1 -
.../apache/solr/search/FooQParserPlugin.java | 5 -
.../apache/solr/search/MergeStrategyTest.java | 3 +-
.../apache/solr/search/MockSearchComponent.java | 3 +-
.../apache/solr/search/QueryEqualityTest.java | 16 +-
.../apache/solr/search/QueryParsingTest.java | 3 +-
.../org/apache/solr/search/RankQueryTest.java | 1 -
.../apache/solr/search/ReturnFieldsTest.java | 1 -
.../apache/solr/search/SortSpecParsingTest.java | 3 +-
.../apache/solr/search/SpatialFilterTest.java | 4 +-
.../solr/search/TestAddFieldRealTimeGet.java | 3 +-
.../solr/search/TestAnalyticsQParserPlugin.java | 5 -
.../solr/search/TestCollapseQParserPlugin.java | 1 -
.../search/TestComplexPhraseQParserPlugin.java | 3 +-
.../apache/solr/search/TestComponentsName.java | 3 +-
.../org/apache/solr/search/TestCustomSort.java | 3 +-
.../test/org/apache/solr/search/TestDocSet.java | 1 -
.../solr/search/TestElisionMultitermQuery.java | 11 +-
.../solr/search/TestExtendedDismaxParser.java | 1 -
.../apache/solr/search/TestFieldSortValues.java | 3 +-
.../solr/search/TestFilteredDocIdSet.java | 3 +-
.../org/apache/solr/search/TestFiltering.java | 1 -
.../solr/search/TestFoldingMultitermQuery.java | 3 +-
.../solr/search/TestHashQParserPlugin.java | 1 -
.../org/apache/solr/search/TestInitQParser.java | 3 +-
.../org/apache/solr/search/TestLFUCache.java | 3 +-
.../org/apache/solr/search/TestLRUCache.java | 3 +-
.../solr/search/TestMaxScoreQueryParser.java | 3 +-
.../apache/solr/search/TestMissingGroups.java | 1 -
.../apache/solr/search/TestNoOpRegenerator.java | 3 +-
...OverriddenPrefixQueryForCustomFieldType.java | 3 +-
.../solr/search/TestPseudoReturnFields.java | 1 -
.../org/apache/solr/search/TestQueryUtils.java | 1 -
.../solr/search/TestQueryWrapperFilter.java | 4 +-
.../search/TestRandomCollapseQParserPlugin.java | 1 -
.../apache/solr/search/TestRankQueryPlugin.java | 5 -
.../solr/search/TestReRankQParserPlugin.java | 64 +-
.../apache/solr/search/TestReloadDeadlock.java | 5 +-
.../org/apache/solr/search/TestSearchPerf.java | 1 -
.../solr/search/TestSimpleQParserPlugin.java | 3 +-
.../apache/solr/search/TestSmileRequest.java | 4 +-
.../apache/solr/search/TestSolr4Spatial.java | 3 +-
.../apache/solr/search/TestSolr4Spatial2.java | 3 +-
.../test/org/apache/solr/search/TestSolrJ.java | 1 -
.../apache/solr/search/TestSolrQueryParser.java | 231 ++---
.../test/org/apache/solr/search/TestSort.java | 1 -
.../solr/search/TestStandardQParsers.java | 3 +-
.../solr/search/TestStressUserVersions.java | 5 +-
.../solr/search/TestSurroundQueryParser.java | 3 +-
.../org/apache/solr/search/TestTrieFacet.java | 1 -
.../solr/search/TestValueSourceCache.java | 3 +-
.../org/apache/solr/search/TestXmlQParser.java | 5 +-
.../solr/search/facet/TestJsonFacets.java | 25 +-
.../search/function/NvlValueSourceParser.java | 1 -
.../search/function/SortByFunctionTest.java | 3 +-
.../solr/search/function/TestFunctionQuery.java | 1 -
.../function/TestMinMaxOnMultiValuedField.java | 1 -
.../solr/search/function/TestOrdValues.java | 3 +-
.../function/TestSortByMinMaxFunction.java | 3 +-
.../function/distance/DistanceFunctionTest.java | 3 +-
.../apache/solr/search/join/BJQParserTest.java | 1 -
.../search/join/BlockJoinFacetDistribTest.java | 3 +-
.../search/join/BlockJoinFacetRandomTest.java | 3 +-
.../search/join/BlockJoinFacetSimpleTest.java | 3 +-
.../apache/solr/search/join/GraphQueryTest.java | 26 +-
.../search/join/TestScoreJoinQPNoScore.java | 1 -
.../solr/search/join/TestScoreJoinQPScore.java | 1 -
.../solr/search/json/TestJsonRequest.java | 3 +-
.../solr/search/mlt/CloudMLTQParserTest.java | 3 +-
.../solr/search/mlt/SimpleMLTQParserTest.java | 3 +-
.../similarities/BaseSimilarityTestCase.java | 3 +-
.../similarities/TestBM25SimilarityFactory.java | 3 +-
.../TestClassicSimilarityFactory.java | 3 +-
.../similarities/TestDFISimilarityFactory.java | 3 +-
.../similarities/TestDFRSimilarityFactory.java | 3 +-
.../similarities/TestIBSimilarityFactory.java | 3 +-
.../TestLMDirichletSimilarityFactory.java | 3 +-
.../TestLMJelinekMercerSimilarityFactory.java | 3 +-
.../TestNonDefinedSimilarityFactory.java | 3 +-
.../similarities/TestPerFieldSimilarity.java | 3 +-
.../TestPerFieldSimilarityClassic.java | 3 +-
...stPerFieldSimilarityWithDefaultOverride.java | 3 +-
.../TestSweetSpotSimilarityFactory.java | 3 +-
.../solr/search/stats/TestBaseStatsCache.java | 3 +-
.../search/stats/TestDefaultStatsCache.java | 3 +-
.../solr/search/stats/TestDistribIDF.java | 3 +-
.../search/stats/TestExactSharedStatsCache.java | 4 +-
.../solr/search/stats/TestExactStatsCache.java | 4 +-
.../solr/search/stats/TestLRUStatsCache.java | 4 +-
.../solr/security/BasicAuthIntegrationTest.java | 66 +-
.../solr/security/MockAuthenticationPlugin.java | 4 +-
.../solr/security/MockAuthorizationPlugin.java | 3 +-
.../PKIAuthenticationIntegrationTest.java | 4 +-
.../security/TestAuthorizationFramework.java | 7 +-
.../security/TestPKIAuthenticationPlugin.java | 3 +-
.../TestRuleBasedAuthorizationPlugin.java | 3 +-
.../TestSha256AuthenticationProvider.java | 3 +-
.../solr/servlet/DirectSolrConnectionTest.java | 1 -
.../apache/solr/servlet/ResponseHeaderTest.java | 3 +-
.../solr/servlet/SolrRequestParserTest.java | 1 -
.../ConjunctionSolrSpellCheckerTest.java | 23 +-
.../spelling/DirectSolrSpellCheckerTest.java | 3 +-
.../spelling/FileBasedSpellCheckerTest.java | 1 -
.../apache/solr/spelling/SampleComparator.java | 3 +-
.../solr/spelling/SpellCheckCollatorTest.java | 3 +-
.../spelling/SpellPossibilityIteratorTest.java | 3 +-
.../spelling/SpellingQueryConverterTest.java | 1 -
.../spelling/TestSuggestSpellingConverter.java | 3 +-
.../spelling/WordBreakSolrSpellCheckerTest.java | 3 +-
.../solr/spelling/suggest/SuggesterFSTTest.java | 3 +-
.../solr/spelling/suggest/SuggesterTSTTest.java | 3 +-
.../solr/spelling/suggest/SuggesterTest.java | 1 -
.../spelling/suggest/SuggesterWFSTTest.java | 3 +-
.../suggest/TestAnalyzeInfixSuggestions.java | 11 +-
.../suggest/TestAnalyzedSuggestions.java | 3 +-
.../suggest/TestBlendedInfixSuggestions.java | 3 +-
.../suggest/TestFileDictionaryLookup.java | 3 +-
.../suggest/TestFreeTextSuggestions.java | 3 +-
.../suggest/TestFuzzyAnalyzedSuggestions.java | 11 +-
.../TestHighFrequencyDictionaryFactory.java | 3 +-
.../spelling/suggest/TestPhraseSuggestions.java | 3 +-
.../solr/store/blockcache/BlockCacheTest.java | 3 +-
.../store/blockcache/BlockDirectoryTest.java | 3 +-
.../solr/store/blockcache/BufferStoreTest.java | 3 +-
.../solr/store/hdfs/HdfsDirectoryTest.java | 3 +-
.../solr/store/hdfs/HdfsLockFactoryTest.java | 3 +-
.../apache/solr/update/AddBlockUpdateTest.java | 25 +-
.../solr/update/AnalysisErrorHandlingTest.java | 3 +-
.../org/apache/solr/update/AutoCommitTest.java | 1 -
.../apache/solr/update/CdcrUpdateLogTest.java | 3 +-
.../solr/update/DataDrivenBlockJoinTest.java | 23 +-
.../update/DirectUpdateHandlerOptimizeTest.java | 3 +-
.../solr/update/DirectUpdateHandlerTest.java | 1 -
.../apache/solr/update/DocumentBuilderTest.java | 1 -
.../apache/solr/update/DummyMergePolicy.java | 3 +-
.../apache/solr/update/HardAutoCommitTest.java | 1 -
.../solr/update/MockStreamingSolrClients.java | 3 +-
.../org/apache/solr/update/PeerSyncTest.java | 26 +-
.../apache/solr/update/SoftAutoCommitTest.java | 1 -
.../solr/update/SolrCmdDistributorTest.java | 3 +-
.../apache/solr/update/SolrIndexConfigTest.java | 3 +-
.../solr/update/SolrIndexSplitterTest.java | 3 +-
.../update/TestDocBasedVersionConstraints.java | 1 -
.../solr/update/TestExceedMaxTermLength.java | 3 +-
.../apache/solr/update/TestHdfsUpdateLog.java | 3 +-
.../solr/update/TestIndexingPerformance.java | 1 -
.../apache/solr/update/UpdateParamsTest.java | 1 -
.../org/apache/solr/update/VersionInfoTest.java | 3 +-
...dSchemaFieldsUpdateProcessorFactoryTest.java | 1 -
.../update/processor/AtomicUpdatesTest.java | 31 +-
.../CloneFieldUpdateProcessorFactoryTest.java | 1 -
.../processor/CustomUpdateRequestProcessor.java | 1 -
.../CustomUpdateRequestProcessorFactory.java | 1 -
.../DefaultValueUpdateProcessorTest.java | 1 -
...DocExpirationUpdateProcessorFactoryTest.java | 1 -
.../FieldMutatingUpdateProcessorTest.java | 1 -
...ommitOptimizeUpdateProcessorFactoryTest.java | 3 +-
.../ParsingFieldUpdateProcessorsTest.java | 1 -
.../PreAnalyzedUpdateProcessorTest.java | 13 +-
.../RecordingUpdateProcessorFactory.java | 1 -
.../solr/update/processor/RuntimeUrp.java | 3 +-
.../solr/update/processor/ScriptEngineTest.java | 1 -
.../SignatureUpdateProcessorFactoryTest.java | 1 -
...atelessScriptUpdateProcessorFactoryTest.java | 3 +-
.../processor/TestNamedUpdateProcessors.java | 4 +-
.../TestPartialUpdateDeduplication.java | 3 +-
.../UUIDUpdateProcessorFallbackTest.java | 3 +-
.../UniqFieldsUpdateProcessorFactoryTest.java | 1 -
.../processor/UpdateProcessorTestBase.java | 3 +-
.../UpdateRequestProcessorFactoryTest.java | 1 -
.../test/org/apache/solr/util/BitSetPerf.java | 1 -
.../org/apache/solr/util/CircularListTest.java | 1 -
.../test/org/apache/solr/util/DOMUtilTest.java | 3 +-
.../apache/solr/util/DateMathParserTest.java | 1 -
.../org/apache/solr/util/DistanceUnitsTest.java | 9 +-
.../org/apache/solr/util/FileUtilsTest.java | 3 +-
.../org/apache/solr/util/MockCoreContainer.java | 3 +-
.../org/apache/solr/util/PrimUtilsTest.java | 14 +-
.../apache/solr/util/SimplePostToolTest.java | 3 +-
.../apache/solr/util/SolrPluginUtilsTest.java | 1 -
.../apache/solr/util/TestFastOutputStream.java | 1 -
.../org/apache/solr/util/TestFastWriter.java | 1 -
.../solr/util/TestObjectReleaseTracker.java | 3 +-
.../org/apache/solr/util/TestRTimerTree.java | 3 +-
.../solr/util/TestRandomForceMergePolicy.java | 36 -
.../apache/solr/util/TestRandomMergePolicy.java | 74 --
.../apache/solr/util/TestSolrCLIRunExample.java | 3 +-
.../apache/solr/util/TestSystemIdResolver.java | 3 +-
.../org/apache/solr/util/TestTestInjection.java | 3 +-
.../test/org/apache/solr/util/TestUtils.java | 1 -
.../org/apache/solr/util/TimeZoneUtilsTest.java | 1 -
.../BigEndianAscendingWordDeserializerTest.java | 1 -
.../BigEndianAscendingWordSerializerTest.java | 1 -
.../org/apache/solr/util/hll/BitVectorTest.java | 1 -
.../apache/solr/util/hll/ExplicitHLLTest.java | 1 -
.../org/apache/solr/util/hll/FullHLLTest.java | 1 -
.../solr/util/hll/HLLSerializationTest.java | 1 -
.../org/apache/solr/util/hll/HLLUtilTest.java | 1 -
.../solr/util/hll/IntegrationTestGenerator.java | 1 -
.../solr/util/hll/ProbabilisticTestUtil.java | 1 -
.../org/apache/solr/util/hll/SparseHLLTest.java | 1 -
.../solr/client/solrj/ResponseParser.java | 1 -
.../apache/solr/client/solrj/SolrClient.java | 1 -
.../org/apache/solr/client/solrj/SolrQuery.java | 1 -
.../apache/solr/client/solrj/SolrRequest.java | 1 -
.../apache/solr/client/solrj/SolrResponse.java | 1 -
.../solr/client/solrj/SolrServerException.java | 1 -
.../client/solrj/StreamingResponseCallback.java | 1 -
.../client/solrj/beans/BindingException.java | 3 +-
.../solr/client/solrj/impl/CloudSolrClient.java | 3 +-
.../solrj/impl/ConcurrentUpdateSolrClient.java | 1 -
.../client/solrj/impl/HttpClientConfigurer.java | 4 +-
.../solr/client/solrj/impl/HttpSolrClient.java | 2 +-
.../solrj/impl/InputStreamResponseParser.java | 3 +-
.../solrj/impl/Krb5HttpClientConfigurer.java | 3 +-
.../client/solrj/impl/NoOpResponseParser.java | 3 +-
.../solrj/impl/SolrHttpRequestRetryHandler.java | 3 +-
.../impl/SolrPortAwareCookieSpecFactory.java | 33 +-
.../client/solrj/impl/XMLResponseParser.java | 1 -
.../solr/client/solrj/io/SolrClientCache.java | 3 +-
.../org/apache/solr/client/solrj/io/Tuple.java | 3 +-
.../client/solrj/io/comp/ComparatorLambda.java | 11 +-
.../client/solrj/io/comp/ComparatorOrder.java | 7 +-
.../client/solrj/io/comp/FieldComparator.java | 1 -
.../solr/client/solrj/io/comp/HashKey.java | 14 +-
.../solrj/io/comp/MultipleFieldComparator.java | 1 -
.../client/solrj/io/comp/StreamComparator.java | 1 -
.../solr/client/solrj/io/eq/Equalitor.java | 5 +-
.../solr/client/solrj/io/eq/FieldEqualitor.java | 1 -
.../solrj/io/eq/MultipleFieldEqualitor.java | 1 -
.../client/solrj/io/eq/StreamEqualitor.java | 1 -
.../client/solrj/io/ops/ConcatOperation.java | 23 +-
.../client/solrj/io/ops/DistinctOperation.java | 33 +-
.../client/solrj/io/ops/GroupOperation.java | 33 +-
.../client/solrj/io/ops/ReduceOperation.java | 6 +-
.../client/solrj/io/ops/ReplaceOperation.java | 21 +-
.../solrj/io/ops/ReplaceWithFieldOperation.java | 23 +-
.../solrj/io/ops/ReplaceWithValueOperation.java | 23 +-
.../client/solrj/io/ops/StreamOperation.java | 19 +-
.../client/solrj/io/sql/ConnectionImpl.java | 46 +-
.../solrj/io/sql/DatabaseMetaDataImpl.java | 16 +-
.../solr/client/solrj/io/sql/DriverImpl.java | 7 +-
.../solr/client/solrj/io/sql/ResultSetImpl.java | 245 ++++-
.../solrj/io/sql/ResultSetMetaDataImpl.java | 55 +-
.../solr/client/solrj/io/sql/StatementImpl.java | 41 +-
.../client/solrj/io/stream/BiJoinStream.java | 1 -
.../client/solrj/io/stream/CloudSolrStream.java | 13 +-
.../solrj/io/stream/ComplementStream.java | 1 -
.../client/solrj/io/stream/DaemonStream.java | 2 -
.../client/solrj/io/stream/ExceptionStream.java | 1 -
.../client/solrj/io/stream/FacetStream.java | 3 +-
.../client/solrj/io/stream/HashJoinStream.java | 1 -
.../client/solrj/io/stream/InnerJoinStream.java | 1 -
.../client/solrj/io/stream/IntersectStream.java | 1 -
.../solr/client/solrj/io/stream/JDBCStream.java | 1 -
.../client/solrj/io/stream/JSONTupleStream.java | 33 +-
.../solr/client/solrj/io/stream/JoinStream.java | 33 +-
.../solrj/io/stream/LeftOuterJoinStream.java | 1 -
.../client/solrj/io/stream/MergeStream.java | 1 -
.../solrj/io/stream/OuterHashJoinStream.java | 1 -
.../client/solrj/io/stream/ParallelStream.java | 1 -
.../client/solrj/io/stream/PushBackStream.java | 1 -
.../solr/client/solrj/io/stream/RankStream.java | 1 -
.../client/solrj/io/stream/ReducerStream.java | 1 -
.../client/solrj/io/stream/RollupStream.java | 8 +-
.../client/solrj/io/stream/SelectStream.java | 1 -
.../solr/client/solrj/io/stream/SolrStream.java | 1 -
.../client/solrj/io/stream/StatsStream.java | 3 +-
.../client/solrj/io/stream/StreamContext.java | 1 -
.../client/solrj/io/stream/TupleStream.java | 1 -
.../client/solrj/io/stream/UniqueStream.java | 1 -
.../solrj/io/stream/expr/Expressible.java | 7 +-
.../solrj/io/stream/expr/StreamExpression.java | 9 +-
.../expr/StreamExpressionNamedParameter.java | 7 +-
.../stream/expr/StreamExpressionParameter.java | 3 +-
.../io/stream/expr/StreamExpressionParser.java | 22 +-
.../io/stream/expr/StreamExpressionValue.java | 3 +-
.../solrj/io/stream/expr/StreamFactory.java | 33 +-
.../client/solrj/io/stream/metrics/Bucket.java | 11 +-
.../solrj/io/stream/metrics/CountMetric.java | 3 +-
.../solrj/io/stream/metrics/MaxMetric.java | 3 +-
.../solrj/io/stream/metrics/MeanMetric.java | 3 +-
.../client/solrj/io/stream/metrics/Metric.java | 3 +-
.../solrj/io/stream/metrics/MinMetric.java | 3 +-
.../solrj/io/stream/metrics/SumMetric.java | 3 +-
.../solrj/request/AbstractUpdateRequest.java | 3 +-
.../solrj/request/CollectionAdminRequest.java | 1 -
.../solrj/request/ConfigSetAdminRequest.java | 1 -
.../request/ContentStreamUpdateRequest.java | 4 +-
.../client/solrj/request/CoreAdminRequest.java | 1 -
.../client/solrj/request/DirectXmlRequest.java | 1 -
.../solrj/request/DocumentAnalysisRequest.java | 1 -
.../solrj/request/FieldAnalysisRequest.java | 1 -
.../solrj/request/GenericSolrRequest.java | 3 +-
.../client/solrj/request/IsUpdateRequest.java | 4 +-
.../solr/client/solrj/request/LukeRequest.java | 1 -
.../solr/client/solrj/request/QueryRequest.java | 1 -
.../client/solrj/request/RequestWriter.java | 1 -
.../solr/client/solrj/request/SolrPing.java | 1 -
.../client/solrj/request/UpdateRequest.java | 2 +-
.../request/schema/AbstractSchemaRequest.java | 3 +-
.../request/schema/AnalyzerDefinition.java | 3 +-
.../request/schema/FieldTypeDefinition.java | 3 +-
.../solrj/request/schema/SchemaRequest.java | 3 +-
.../solrj/response/AnalysisResponseBase.java | 1 -
.../solr/client/solrj/response/Cluster.java | 3 +-
.../solrj/response/ClusteringResponse.java | 3 +-
.../solrj/response/CollectionAdminResponse.java | 1 -
.../solrj/response/ConfigSetAdminResponse.java | 1 -
.../solrj/response/CoreAdminResponse.java | 1 -
.../response/DocumentAnalysisResponse.java | 1 -
.../solr/client/solrj/response/FacetField.java | 1 -
.../solrj/response/FieldAnalysisResponse.java | 1 -
.../solr/client/solrj/response/Group.java | 3 +-
.../client/solrj/response/GroupCommand.java | 3 +-
.../client/solrj/response/GroupResponse.java | 3 +-
.../client/solrj/response/IntervalFacet.java | 8 +-
.../client/solrj/response/LukeResponse.java | 1 -
.../solr/client/solrj/response/PivotField.java | 1 -
.../client/solrj/response/QueryResponse.java | 1 -
.../solr/client/solrj/response/RangeFacet.java | 3 +-
.../solrj/response/SimpleSolrResponse.java | 4 +-
.../client/solrj/response/SolrPingResponse.java | 1 -
.../client/solrj/response/SolrResponseBase.java | 1 -
.../solrj/response/SpellCheckResponse.java | 3 +-
.../solrj/response/SuggesterResponse.java | 3 +-
.../solr/client/solrj/response/Suggestion.java | 2 +-
.../client/solrj/response/TermsResponse.java | 3 +-
.../client/solrj/response/UpdateResponse.java | 1 -
.../schema/FieldTypeRepresentation.java | 3 +-
.../response/schema/SchemaRepresentation.java | 3 +-
.../solrj/response/schema/SchemaResponse.java | 3 +-
.../solr/client/solrj/util/ClientUtils.java | 1 -
.../java/org/apache/solr/common/Callable.java | 3 +-
.../apache/solr/common/EmptyEntityResolver.java | 3 +-
.../org/apache/solr/common/EnumFieldValue.java | 3 +-
.../org/apache/solr/common/SolrDocument.java | 1 -
.../apache/solr/common/SolrDocumentBase.java | 13 +-
.../apache/solr/common/SolrDocumentList.java | 1 -
.../org/apache/solr/common/SolrException.java | 1 -
.../apache/solr/common/SolrInputDocument.java | 1 -
.../org/apache/solr/common/SolrInputField.java | 1 -
.../org/apache/solr/common/StringUtils.java | 3 +-
.../org/apache/solr/common/cloud/Aliases.java | 3 +-
.../solr/common/cloud/BeforeReconnect.java | 27 +-
.../solr/common/cloud/ClosableThread.java | 3 +-
.../apache/solr/common/cloud/ClusterState.java | 3 +-
.../solr/common/cloud/ClusterStateUtil.java | 31 +-
.../solr/common/cloud/CompositeIdRouter.java | 3 +-
.../solr/common/cloud/ConnectionManager.java | 7 +-
.../common/cloud/DefaultConnectionStrategy.java | 27 +-
.../solr/common/cloud/DefaultZkACLProvider.java | 13 +-
.../cloud/DefaultZkCredentialsProvider.java | 9 +-
.../apache/solr/common/cloud/DocCollection.java | 3 +-
.../org/apache/solr/common/cloud/DocRouter.java | 3 +-
.../solr/common/cloud/HashBasedRouter.java | 3 +-
.../solr/common/cloud/ImplicitDocRouter.java | 3 +-
.../apache/solr/common/cloud/OnReconnect.java | 27 +-
.../apache/solr/common/cloud/PlainIdRouter.java | 5 +-
.../org/apache/solr/common/cloud/Replica.java | 3 +-
.../apache/solr/common/cloud/RoutingRule.java | 3 +-
.../solr/common/cloud/SaslZkACLProvider.java | 3 +-
.../org/apache/solr/common/cloud/Slice.java | 3 +-
.../apache/solr/common/cloud/SolrZkClient.java | 23 +-
.../apache/solr/common/cloud/SolrZooKeeper.java | 3 +-
...ParamsAllAndReadonlyDigestZkACLProvider.java | 23 +-
...tCredentialsDigestZkCredentialsProvider.java | 17 +-
.../apache/solr/common/cloud/ZkACLProvider.java | 11 +-
.../cloud/ZkClientConnectionStrategy.java | 27 +-
.../apache/solr/common/cloud/ZkCmdExecutor.java | 3 +-
.../solr/common/cloud/ZkConfigManager.java | 1 -
.../solr/common/cloud/ZkCoreNodeProps.java | 27 +-
.../common/cloud/ZkCredentialsProvider.java | 7 +-
.../apache/solr/common/cloud/ZkNodeProps.java | 3 +-
.../apache/solr/common/cloud/ZkOperation.java | 6 +-
.../apache/solr/common/cloud/ZkStateReader.java | 5 +-
.../solr/common/cloud/ZooKeeperException.java | 4 +-
.../org/apache/solr/common/luke/FieldFlag.java | 4 +-
.../solr/common/params/AnalysisParams.java | 1 -
.../solr/common/params/AppendedSolrParams.java | 1 -
.../solr/common/params/CollectionParams.java | 3 +-
.../solr/common/params/CommonAdminParams.java | 1 -
.../apache/solr/common/params/CommonParams.java | 1 -
.../solr/common/params/ConfigSetParams.java | 3 +-
.../solr/common/params/CoreAdminParams.java | 1 -
.../solr/common/params/CursorMarkParams.java | 1 -
.../solr/common/params/DefaultSolrParams.java | 1 -
.../apache/solr/common/params/DisMaxParams.java | 1 -
.../apache/solr/common/params/EventParams.java | 4 +-
.../apache/solr/common/params/ExpandParams.java | 1 -
.../apache/solr/common/params/FacetParams.java | 1 -
.../apache/solr/common/params/GroupParams.java | 1 -
.../solr/common/params/HighlightParams.java | 1 -
.../solr/common/params/MapSolrParams.java | 1 -
.../common/params/ModifiableSolrParams.java | 1 -
.../solr/common/params/MoreLikeThisParams.java | 1 -
.../solr/common/params/MultiMapSolrParams.java | 1 -
.../common/params/QueryElevationParams.java | 4 +-
.../solr/common/params/RequiredSolrParams.java | 1 -
.../apache/solr/common/params/ShardParams.java | 1 -
.../apache/solr/common/params/SimpleParams.java | 3 +-
.../apache/solr/common/params/SolrParams.java | 1 -
.../solr/common/params/SpatialParams.java | 4 +-
.../solr/common/params/SpellingParams.java | 1 -
.../apache/solr/common/params/StatsParams.java | 1 -
.../solr/common/params/TermVectorParams.java | 4 +-
.../apache/solr/common/params/TermsParams.java | 1 -
.../apache/solr/common/params/UpdateParams.java | 1 -
.../org/apache/solr/common/util/ByteUtils.java | 1 -
.../java/org/apache/solr/common/util/Cache.java | 3 +-
.../apache/solr/common/util/ContentStream.java | 1 -
.../solr/common/util/ContentStreamBase.java | 1 -
.../solr/common/util/DataInputInputStream.java | 1 -
.../org/apache/solr/common/util/DateUtil.java | 3 +-
.../apache/solr/common/util/ExecutorUtil.java | 11 +-
.../solr/common/util/FastInputStream.java | 1 -
.../solr/common/util/FastOutputStream.java | 1 -
.../java/org/apache/solr/common/util/Hash.java | 3 +-
.../org/apache/solr/common/util/IOUtils.java | 15 +-
.../apache/solr/common/util/IteratorChain.java | 1 -
.../org/apache/solr/common/util/NamedList.java | 1 -
.../solr/common/util/ObjectReleaseTracker.java | 3 +-
.../java/org/apache/solr/common/util/Pair.java | 3 +-
.../org/apache/solr/common/util/RetryUtil.java | 11 +-
.../solr/common/util/SimpleOrderedMap.java | 3 +-
.../common/util/SolrjNamedThreadFactory.java | 9 +-
.../org/apache/solr/common/util/StrUtils.java | 1 -
.../solr/common/util/SuppressForbidden.java | 3 +-
.../org/apache/solr/common/util/URLUtil.java | 1 -
.../java/org/apache/solr/common/util/Utils.java | 4 +-
.../java/org/apache/solr/common/util/XML.java | 1 -
.../apache/solr/common/util/XMLErrorLogger.java | 1 -
...ollectionAdminRequestRequiredParamsTest.java | 3 +-
.../apache/solr/client/solrj/GetByIdTest.java | 3 +-
.../solr/client/solrj/LargeVolumeTestBase.java | 1 -
.../solrj/MergeIndexesExampleTestBase.java | 1 -
.../client/solrj/SolrExampleBinaryTest.java | 1 -
.../solr/client/solrj/SolrExampleTestBase.java | 1 -
.../solr/client/solrj/SolrExampleTests.java | 1 -
.../solr/client/solrj/SolrExampleTestsBase.java | 1 -
.../solr/client/solrj/SolrExampleXMLTest.java | 1 -
.../solr/client/solrj/SolrExceptionTest.java | 1 -
.../apache/solr/client/solrj/SolrQueryTest.java | 1 -
.../client/solrj/SolrSchemalessExampleTest.java | 1 -
.../solr/client/solrj/StartSolrJetty.java | 1 -
.../solr/client/solrj/TestLBHttpSolrClient.java | 1 -
.../client/solrj/TestSolrJErrorHandling.java | 7 +-
.../AbstractEmbeddedSolrServerTestCase.java | 3 +-
.../client/solrj/embedded/JettyWebappTest.java | 1 -
.../solrj/embedded/LargeVolumeEmbeddedTest.java | 1 -
.../solrj/embedded/LargeVolumeJettyTest.java | 1 -
.../embedded/MergeIndexesEmbeddedTest.java | 1 -
.../solrj/embedded/SolrExampleEmbeddedTest.java | 1 -
.../solrj/embedded/SolrExampleJettyTest.java | 1 -
.../SolrExampleStreamingBinaryTest.java | 3 +-
.../embedded/SolrExampleStreamingTest.java | 1 -
.../solrj/embedded/TestEmbeddedSolrServer.java | 3 +-
.../solrj/embedded/TestSolrProperties.java | 1 -
.../solrj/impl/BasicHttpSolrClientTest.java | 7 +-
.../CloudSolrClientMultiConstructorTest.java | 21 +-
.../client/solrj/impl/CloudSolrClientTest.java | 3 +-
.../impl/ConcurrentUpdateSolrClientTest.java | 3 -
.../solrj/impl/ExternalHttpClientTest.java | 3 +-
.../client/solrj/impl/LBHttpSolrClientTest.java | 1 -
.../solrj/impl/SolrPortAwareCookieSpecTest.java | 3 +-
.../impl/TestCloudSolrClientConnections.java | 3 +-
.../client/solrj/io/sql/JdbcDriverTest.java | 3 +-
.../solr/client/solrj/io/sql/JdbcTest.java | 349 ++++++-
.../client/solrj/io/stream/JDBCStreamTest.java | 3 +-
.../solrj/io/stream/RecordCountStream.java | 2 +-
.../solrj/io/stream/StreamExpressionTest.java | 66 +-
.../stream/StreamExpressionToExpessionTest.java | 22 +-
.../client/solrj/io/stream/StreamingTest.java | 3 +-
.../stream/expr/StreamExpressionParserTest.java | 3 +-
.../io/stream/ops/ConcatOperationTest.java | 3 +-
.../solrj/io/stream/ops/OperationsTest.java | 3 +-
.../solr/client/solrj/request/SchemaTest.java | 3 +-
.../solr/client/solrj/request/SolrPingTest.java | 3 +-
.../request/TestConfigSetAdminRequest.java | 1 -
.../client/solrj/request/TestCoreAdmin.java | 1 -
.../client/solrj/request/TestUpdateRequest.java | 3 +-
.../solrj/response/AnlysisResponseBaseTest.java | 1 -
.../response/DocumentAnalysisResponseTest.java | 1 -
.../client/solrj/response/FacetFieldTest.java | 3 +-
.../response/FieldAnalysisResponseTest.java | 1 -
.../solrj/response/NoOpResponseParserTest.java | 3 +-
.../solrj/response/QueryResponseTest.java | 1 -
.../solrj/response/TermsResponseTest.java | 3 +-
.../solrj/response/TestClusteringResponse.java | 3 +-
.../solrj/response/TestSpellCheckResponse.java | 3 +-
.../solrj/response/TestSuggesterResponse.java | 3 +-
.../solr/client/solrj/util/ClientUtilsTest.java | 1 -
.../apache/solr/common/SolrDocumentTest.java | 1 -
.../solr/common/cloud/SolrZkClientTest.java | 3 +-
.../solr/common/cloud/TestZkConfigManager.java | 1 -
.../common/params/CommonAdminParamsTest.java | 1 -
.../solr/common/params/CommonParamsTest.java | 1 -
.../common/params/ModifiableSolrParamsTest.java | 25 +-
.../solr/common/params/ShardParamsTest.java | 3 +-
.../solr/common/params/SolrParamTest.java | 1 -
.../solr/common/util/ContentStreamTest.java | 1 -
.../solr/common/util/IteratorChainTest.java | 1 -
.../apache/solr/common/util/NamedListTest.java | 1 -
.../apache/solr/common/util/TestDateUtil.java | 3 +-
.../org/apache/solr/common/util/TestHash.java | 3 +-
.../solr/common/util/TestJavaBinCodec.java | 3 +-
.../solr/common/util/TestJsonRecordReader.java | 3 +-
.../apache/solr/common/util/TestRetryUtil.java | 3 +-
.../solr/common/util/TestXMLEscaping.java | 1 -
.../apache/solr/common/util/URLUtilTest.java | 1 -
.../solr/BaseDistributedSearchTestCase.java | 3 +-
.../src/java/org/apache/solr/JSONTestUtil.java | 1 -
.../apache/solr/SolrIgnoredThreadsFilter.java | 13 +-
.../java/org/apache/solr/SolrJettyTestBase.java | 3 +-
.../java/org/apache/solr/SolrTestCaseHS.java | 4 +-
.../java/org/apache/solr/SolrTestCaseJ4.java | 1 -
.../solr/analysis/MockCharFilterFactory.java | 3 +-
.../solr/analysis/MockTokenFilterFactory.java | 3 +-
.../solr/analysis/MockTokenizerFactory.java | 3 +-
.../analysis/StringMockSolrResourceLoader.java | 3 +-
.../solr/cloud/AbstractDistribZkTestBase.java | 3 +-
.../apache/solr/cloud/AbstractZkTestCase.java | 3 +-
.../java/org/apache/solr/cloud/ChaosMonkey.java | 7 +-
.../org/apache/solr/cloud/CloudInspectUtil.java | 33 +-
.../java/org/apache/solr/cloud/IpTables.java | 4 +-
.../apache/solr/cloud/MiniSolrCloudCluster.java | 3 +-
.../org/apache/solr/cloud/MockSolrZkClient.java | 3 +-
.../apache/solr/cloud/MockZkStateReader.java | 3 +-
.../java/org/apache/solr/cloud/SocketProxy.java | 3 +-
.../solr/cloud/StoppableIndexingThread.java | 25 +-
.../solr/cloud/StoppableSearchThread.java | 7 +-
.../org/apache/solr/cloud/ZkTestServer.java | 27 +-
.../solr/core/AbstractBadConfigTestBase.java | 1 -
.../apache/solr/core/MockDirectoryFactory.java | 6 +-
.../solr/core/MockFSDirectoryFactory.java | 4 +-
.../component/TrackingShardHandlerFactory.java | 3 +-
.../processor/BufferingRequestProcessor.java | 1 -
.../apache/solr/util/AbstractSolrTestCase.java | 2 -
.../apache/solr/util/BadHdfsThreadsFilter.java | 3 +-
.../solr/util/BadMrClusterThreadsFilter.java | 3 +-
.../solr/util/BadZookeeperThreadsFilter.java | 3 +-
.../org/apache/solr/util/BaseTestHarness.java | 3 +-
.../org/apache/solr/util/DOMUtilTestBase.java | 1 -
.../org/apache/solr/util/ExternalPaths.java | 3 +-
.../apache/solr/util/RESTfulServerProvider.java | 3 +-
.../solr/util/RandomForceMergePolicy.java | 1 -
.../org/apache/solr/util/RandomMergePolicy.java | 52 +-
.../apache/solr/util/ReadOnlyCoresLocator.java | 3 +-
.../java/org/apache/solr/util/RestTestBase.java | 3 +-
.../org/apache/solr/util/RestTestHarness.java | 3 +-
.../util/RevertDefaultThreadHandlerRule.java | 23 +-
.../org/apache/solr/util/SSLTestConfig.java | 3 +-
.../java/org/apache/solr/util/TestHarness.java | 1 -
5346 files changed, 22854 insertions(+), 22215 deletions(-)
----------------------------------------------------------------------
[12/15] lucene-solr git commit: add more non-stress test cases
Posted by mi...@apache.org.
add more non-stress test cases
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/b1b2c799
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/b1b2c799
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/b1b2c799
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: b1b2c799aa1d6598daa4ea8c63a5fa5484b5052d
Parents: 82ecccf
Author: Mike McCandless <mi...@apache.org>
Authored: Sat Feb 6 09:04:15 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Sat Feb 6 09:04:15 2016 -0500
----------------------------------------------------------------------
.../org/apache/lucene/replicator/nrt/Node.java | 3 +-
.../lucene/replicator/nrt/ReplicaNode.java | 2 +
.../lucene/replicator/nrt/NodeProcess.java | 17 +-
.../replicator/nrt/SimplePrimaryNode.java | 20 +-
.../replicator/nrt/SimpleReplicaNode.java | 12 +-
.../lucene/replicator/nrt/SimpleServer.java | 5 +-
.../replicator/nrt/TestNRTReplication.java | 717 +++++++++++++++----
.../nrt/TestStressNRTReplication.java | 13 +-
8 files changed, 640 insertions(+), 149 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
index 742b19f..e54c01e 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java
@@ -49,7 +49,8 @@ import org.apache.lucene.util.StringHelper;
abstract class Node implements Closeable {
static boolean VERBOSE_FILES = true;
- static boolean VERBOSE_CONNECTIONS = false;
+ // nocommit
+ static boolean VERBOSE_CONNECTIONS = true;
// Keys we store into IndexWriter's commit user data:
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index c7af429..133992f 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -86,6 +86,8 @@ abstract class ReplicaNode extends Node {
// Obtain a write lock on this index since we "act like" an IndexWriter, to prevent any other IndexWriter or ReplicaNode from using it:
writeFileLock = dir.obtainLock(IndexWriter.WRITE_LOCK_NAME);
+
+ // nocommit must check for no pending deletes here, like IW does
state = "init";
deleter = new ReplicaFileDeleter(this, dir);
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
index be0b3df3..9d8b764 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -77,6 +77,7 @@ class NodeProcess implements Closeable {
isOpen = false;
p.destroy();
try {
+ p.waitFor();
pumper.join();
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
@@ -95,6 +96,7 @@ class NodeProcess implements Closeable {
}
return true;
} catch (Throwable t) {
+ // nocommit throw this
// Something wrong with this replica; skip it:
System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
return false;
@@ -106,6 +108,7 @@ class NodeProcess implements Closeable {
c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
c.flush();
} catch (Throwable t) {
+ // nocommit throw this
// Something wrong with this replica; skip it:
System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
}
@@ -118,6 +121,7 @@ class NodeProcess implements Closeable {
c.s.shutdownOutput();
return c.in.readVLong();
} catch (Throwable t) {
+ // nocommit throw this
// Something wrong with this replica; skip it:
System.out.println("PARENT: top: hit SocketException during getSearchingVersion with R" + id + "; skipping");
return -1L;
@@ -145,11 +149,11 @@ class NodeProcess implements Closeable {
public synchronized boolean shutdown() {
lock.lock();
try {
- System.out.println("PARENT: now shutdown node=" + id + " isOpen=" + isOpen);
+ //System.out.println("PARENT: now shutdown node=" + id + " isOpen=" + isOpen);
if (isOpen) {
// Ask the child process to shutdown gracefully:
isOpen = false;
- System.out.println("PARENT: send CMD_CLOSE to node=" + id);
+ //System.out.println("PARENT: send CMD_CLOSE to node=" + id);
try (Connection c = new Connection(tcpPort)) {
c.out.writeByte(SimplePrimaryNode.CMD_CLOSE);
c.flush();
@@ -174,6 +178,15 @@ class NodeProcess implements Closeable {
}
}
+ public void newNRTPoint(long version, int primaryTCPPort) throws IOException {
+ try (Connection c = new Connection(tcpPort)) {
+ c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
+ c.out.writeVLong(version);
+ c.out.writeInt(primaryTCPPort);
+ c.flush();
+ }
+ }
+
public void addOrUpdateDocument(Connection c, Document doc, boolean isUpdate) throws IOException {
if (isPrimary == false) {
throw new IllegalStateException("only primary can index");
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
index b9ecced..7f5634c 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -76,8 +76,8 @@ class SimplePrimaryNode extends PrimaryNode {
final Random random;
// These are updated by parent test process whenever replicas change:
- int[] replicaTCPPorts;
- int[] replicaIDs;
+ int[] replicaTCPPorts = new int[0];
+ int[] replicaIDs = new int[0];
// So we only flip a bit once per file name:
final Set<String> bitFlipped = Collections.synchronizedSet(new HashSet<>());
@@ -115,8 +115,8 @@ class SimplePrimaryNode extends PrimaryNode {
}
public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory,
- boolean doFlipBitsDuringCopy) throws IOException {
- super(initWriter(id, random, indexPath), id, primaryGen, forcePrimaryVersion, searcherFactory);
+ boolean doFlipBitsDuringCopy, boolean doCheckIndexOnClose) throws IOException {
+ super(initWriter(id, random, indexPath, doCheckIndexOnClose), id, primaryGen, forcePrimaryVersion, searcherFactory);
this.tcpPort = tcpPort;
this.random = new Random(random.nextLong());
this.doFlipBitsDuringCopy = doFlipBitsDuringCopy;
@@ -129,8 +129,8 @@ class SimplePrimaryNode extends PrimaryNode {
this.replicaTCPPorts = replicaTCPPorts;
}
- private static IndexWriter initWriter(int id, Random random, Path indexPath) throws IOException {
- Directory dir = SimpleReplicaNode.getDirectory(random, id, indexPath);
+ private static IndexWriter initWriter(int id, Random random, Path indexPath, boolean doCheckIndexOnClose) throws IOException {
+ Directory dir = SimpleReplicaNode.getDirectory(random, id, indexPath, doCheckIndexOnClose);
MockAnalyzer analyzer = new MockAnalyzer(random);
analyzer.setMaxTokenLength(TestUtil.nextInt(random, 1, IndexWriter.MAX_TERM_LENGTH));
@@ -599,13 +599,15 @@ class SimplePrimaryNode extends PrimaryNode {
IndexSearcher searcher = mgr.acquire();
try {
long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
- int hitCount = searcher.search(new TermQuery(new Term("body", "the")), 1).totalHits;
+ int hitCount = searcher.count(new TermQuery(new Term("body", "the")));
//message("version=" + version + " searcher=" + searcher);
out.writeVLong(version);
out.writeVInt(hitCount);
+ bos.flush();
} finally {
mgr.release(searcher);
}
+ bos.flush();
}
continue outer;
@@ -615,10 +617,11 @@ class SimplePrimaryNode extends PrimaryNode {
IndexSearcher searcher = mgr.acquire();
try {
long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
- int hitCount = searcher.search(new MatchAllDocsQuery(), 1).totalHits;
+ int hitCount = searcher.count(new MatchAllDocsQuery());
//message("version=" + version + " searcher=" + searcher);
out.writeVLong(version);
out.writeVInt(hitCount);
+ bos.flush();
} finally {
mgr.release(searcher);
}
@@ -630,6 +633,7 @@ class SimplePrimaryNode extends PrimaryNode {
Thread.currentThread().setName("msearch");
int expectedAtLeastCount = in.readVInt();
verifyAtLeastMarkerCount(expectedAtLeastCount, out);
+ bos.flush();
}
continue outer;
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
index bc8bb03..2510c40 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -64,8 +64,8 @@ class SimpleReplicaNode extends ReplicaNode {
/** Changes over time, as primary node crashes and moves around */
int curPrimaryTCPPort;
- public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory) throws IOException {
- super(id, getDirectory(random, id, indexPath), searcherFactory);
+ public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory, boolean doCheckIndexOnClose) throws IOException {
+ super(id, getDirectory(random, id, indexPath, doCheckIndexOnClose), searcherFactory);
this.tcpPort = tcpPort;
this.random = new Random(random.nextLong());
@@ -131,13 +131,13 @@ class SimpleReplicaNode extends ReplicaNode {
return new SimpleCopyJob(reason, c, copyState, this, files, highPriority, onceDone);
}
- static Directory getDirectory(Random random, int id, Path path) throws IOException {
+ static Directory getDirectory(Random random, int id, Path path, boolean doCheckIndexOnClose) throws IOException {
MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path);
dir.setAssertNoUnrefencedFilesOnClose(true);
- // This is very costly (takes more time to check than it did to index); we do this ourselves in the end instead of each time a replica
- // is restarted:
- dir.setCheckIndexOnClose(false);
+ if (doCheckIndexOnClose) {
+ dir.setCheckIndexOnClose(false);
+ }
// Corrupt any index files not referenced by current commit point; this is important (increases test evilness) because we may have done
// a hard crash of the previous JVM writing to this directory and so MDW's corrupt-unknown-files-on-close never ran:
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
index 3fdc45f..72e33d7 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -238,6 +238,7 @@ public class SimpleServer extends LuceneTestCase {
boolean doRandomCrash = "true".equals(System.getProperty("tests.nrtreplication.doRandomCrash"));
boolean doRandomClose = "true".equals(System.getProperty("tests.nrtreplication.doRandomClose"));
boolean doFlipBitsDuringCopy = "true".equals(System.getProperty("tests.nrtreplication.doFlipBitsDuringCopy"));
+ boolean doCheckIndexOnClose = "true".equals(System.getProperty("tests.nrtreplication.checkonclose"));
// Create server socket that we listen for incoming requests on:
try (final ServerSocket ss = new ServerSocket(0, 0, InetAddress.getLoopbackAddress())) {
@@ -246,11 +247,11 @@ public class SimpleServer extends LuceneTestCase {
System.out.println("\nPORT: " + tcpPort);
final Node node;
if (isPrimary) {
- node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy);
+ node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null, doFlipBitsDuringCopy, doCheckIndexOnClose);
System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
} else {
try {
- node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+ node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null, doCheckIndexOnClose);
} catch (RuntimeException re) {
if (re.getMessage().startsWith("replica cannot start")) {
// this is "OK": it means MDW's refusal to delete a segments_N commit point means we cannot start:
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 7ba3bc2..15e9c8c 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -17,15 +17,6 @@ package org.apache.lucene.replicator.nrt;
* limitations under the License.
*/
-import org.apache.lucene.document.Document;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LineFileDocs;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-import org.apache.lucene.util.LuceneTestCase;
-
-import com.carrotsearch.randomizedtesting.SeedUtils;
-
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
@@ -38,6 +29,16 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
// nocommit make some explicit failure tests
// MockRandom's .sd file has no index header/footer:
@@ -49,9 +50,12 @@ public class TestNRTReplication extends LuceneTestCase {
private Path childTempDir;
final AtomicLong nodeStartCounter = new AtomicLong();
+ private long nextPrimaryGen;
+ private long lastPrimaryGen;
+ LineFileDocs docs;
/** Launches a child "server" (separate JVM), which is either primary or replica node */
- NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
+ private NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, long forcePrimaryVersion, boolean willCrash) throws IOException {
List<String> cmd = new ArrayList<>();
cmd.add(System.getProperty("java.home")
@@ -61,26 +65,30 @@ public class TestNRTReplication extends LuceneTestCase {
+ "java");
cmd.add("-Xmx512m");
+ long myPrimaryGen;
if (primaryTCPPort != -1) {
+ // I am a replica
cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + primaryTCPPort);
- } else if (isPrimary == false) {
- // We cannot start a replica when there is no primary:
- return null;
+ myPrimaryGen = lastPrimaryGen;
+ } else {
+ myPrimaryGen = nextPrimaryGen++;
+ lastPrimaryGen = myPrimaryGen;
}
+ cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
cmd.add("-Dtests.nrtreplication.closeorcrash=false");
cmd.add("-Dtests.nrtreplication.node=true");
cmd.add("-Dtests.nrtreplication.nodeid=" + id);
cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
- if (isPrimary) {
+ cmd.add("-Dtests.nrtreplication.checkonclose=true");
+
+ if (primaryTCPPort == -1) {
+ // We are the primary node
cmd.add("-Dtests.nrtreplication.isPrimary=true");
cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
}
- long myPrimaryGen = 0;
- cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
-
// Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
// new node:
long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
@@ -112,7 +120,6 @@ public class TestNRTReplication extends LuceneTestCase {
long initCommitVersion = -1;
long initInfosVersion = -1;
Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
- boolean willCrash = false;
boolean sawExistingSegmentsFile = false;
while (true) {
@@ -169,7 +176,7 @@ public class TestNRTReplication extends LuceneTestCase {
message("done wait for process " + p);
int exitValue = p.exitValue();
message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
- if (exitValue != 0) {
+ if (exitValue != 0 && finalWillCrash == false) {
// should fail test
throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
}
@@ -178,32 +185,33 @@ public class TestNRTReplication extends LuceneTestCase {
pumper.setName("pump" + id);
message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
- return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion, nodeClosing);
+ return new NodeProcess(p, id, tcpPort, pumper, primaryTCPPort == -1, initCommitVersion, initInfosVersion, nodeClosing);
}
- public void testReplicateDeleteAllDocuments() throws Exception {
-
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
Node.globalStartNS = System.nanoTime();
childTempDir = createTempDir("child");
+ docs = new LineFileDocs(random());
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ super.tearDown();
+ docs.close();
+ }
+
+ public void testReplicateDeleteAllDocuments() throws Exception {
- message("change thread name from " + Thread.currentThread().getName());
- Thread.currentThread().setName("main");
-
Path primaryPath = createTempDir("primary");
- NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
Path replicaPath = createTempDir("replica");
- NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
// Tell primary current replicas:
- try (Connection c = new Connection(primary.tcpPort)) {
- c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
- c.out.writeVInt(1);
- c.out.writeVInt(replica.id);
- c.out.writeVInt(replica.tcpPort);
- c.flush();
- c.in.readByte();
- }
+ sendReplicasToPrimary(primary, replica);
// Index 10 docs into primary:
LineFileDocs docs = new LineFileDocs(random());
@@ -215,33 +223,14 @@ public class TestNRTReplication extends LuceneTestCase {
}
// Nothing in replica index yet
- Connection replicaC = new Connection(replica.tcpPort);
- replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
- replicaC.flush();
- long version1 = replicaC.in.readVLong();
- assertEquals(0L, version1);
- int hitCount = replicaC.in.readVInt();
- assertEquals(0, hitCount);
+ assertVersionAndHits(replica, 0, 0);
// Refresh primary, which also pushes to replica:
long primaryVersion1 = primary.flush(0);
assertTrue(primaryVersion1 > 0);
- long version2;
-
// Wait for replica to show the change
- while (true) {
- replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
- replicaC.flush();
- version2 = replicaC.in.readVLong();
- hitCount = replicaC.in.readVInt();
- if (version2 == primaryVersion1) {
- assertEquals(10, hitCount);
- // good!
- break;
- }
- Thread.sleep(10);
- }
+ waitForVersionAndHits(replica, primaryVersion1, 10);
// Delete all docs from primary
if (random().nextBoolean()) {
@@ -255,32 +244,14 @@ public class TestNRTReplication extends LuceneTestCase {
}
// Replica still shows 10 docs:
- replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
- replicaC.flush();
- long version3 = replicaC.in.readVLong();
- assertEquals(version2, version3);
- hitCount = replicaC.in.readVInt();
- assertEquals(10, hitCount);
+ assertVersionAndHits(replica, primaryVersion1, 10);
// Refresh primary, which also pushes to replica:
long primaryVersion2 = primary.flush(0);
assertTrue(primaryVersion2 > primaryVersion1);
// Wait for replica to show the change
- long version4;
- while (true) {
- replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
- replicaC.flush();
- version4 = replicaC.in.readVLong();
- hitCount = replicaC.in.readVInt();
- if (version4 == primaryVersion2) {
- assertTrue(version4 > version3);
- assertEquals(0, hitCount);
- // good!
- break;
- }
- Thread.sleep(10);
- }
+ waitForVersionAndHits(replica, primaryVersion2, 0);
// Index 10 docs again:
for(int i=0;i<10;i++) {
@@ -293,21 +264,8 @@ public class TestNRTReplication extends LuceneTestCase {
assertTrue(primaryVersion3 > primaryVersion2);
// Wait for replica to show the change
- while (true) {
- replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
- replicaC.flush();
- long version5 = replicaC.in.readVLong();
- hitCount = replicaC.in.readVInt();
- if (version5 == primaryVersion3) {
- assertEquals(10, hitCount);
- assertTrue(version5 > version4);
- // good!
- break;
- }
- Thread.sleep(10);
- }
+ waitForVersionAndHits(replica, primaryVersion3, 10);
- replicaC.close();
primaryC.close();
replica.close();
@@ -316,27 +274,13 @@ public class TestNRTReplication extends LuceneTestCase {
public void testReplicateForceMerge() throws Exception {
- Node.globalStartNS = System.nanoTime();
- childTempDir = createTempDir("child");
-
- message("change thread name from " + Thread.currentThread().getName());
- Thread.currentThread().setName("main");
-
Path primaryPath = createTempDir("primary");
- NodeProcess primary = startNode(-1, 0, primaryPath, true, -1);
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
Path replicaPath = createTempDir("replica");
- NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, false, -1);
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
- // Tell primary current replicas:
- try (Connection c = new Connection(primary.tcpPort)) {
- c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
- c.out.writeVInt(1);
- c.out.writeVInt(replica.id);
- c.out.writeVInt(replica.tcpPort);
- c.flush();
- c.in.readByte();
- }
+ sendReplicasToPrimary(primary, replica);
// Index 10 docs into primary:
LineFileDocs docs = new LineFileDocs(random());
@@ -367,29 +311,560 @@ public class TestNRTReplication extends LuceneTestCase {
long primaryVersion3 = primary.flush(0);
assertTrue(primaryVersion3 > primaryVersion2);
- Connection replicaC = new Connection(replica.tcpPort);
-
// Wait for replica to show the change
- while (true) {
- replicaC.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
- replicaC.flush();
- long version = replicaC.in.readVLong();
- int hitCount = replicaC.in.readVInt();
- if (version == primaryVersion3) {
- assertEquals(20, hitCount);
- // good!
- break;
+ waitForVersionAndHits(replica, primaryVersion3, 20);
+
+ primaryC.close();
+
+ replica.close();
+ primary.close();
+ }
+
+ // Start up, index 10 docs, replicate, but crash and restart the replica without committing it:
+ public void testReplicaCrashNoCommit() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
}
- Thread.sleep(10);
}
- replicaC.close();
- primaryC.close();
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Crash replica:
+ replica.crash();
+
+ // Restart replica:
+ replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ // On startup the replica searches the last commit (empty here):
+ assertVersionAndHits(replica, 0, 0);
+
+ // Ask replica to sync:
+ replica.newNRTPoint(primaryVersion1, primary.tcpPort);
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ replica.close();
+ primary.close();
+ }
+
+ // Start up, index 10 docs, replicate, commit, crash and restart the replica
+ public void testReplicaCrashWithCommit() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Commit and crash replica:
+ replica.commit();
+ replica.crash();
+
+ // Restart replica:
+ replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ // On startup the replica searches the last commit:
+ assertVersionAndHits(replica, primaryVersion1, 10);
replica.close();
primary.close();
}
+ // Start up, index 10 docs, replicate, commit, crash, index more docs, replicate, then restart the replica
+ public void testIndexingWhileReplicaIsDown() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Commit and crash replica:
+ replica.commit();
+ replica.crash();
+
+ sendReplicasToPrimary(primary);
+
+ // Index 10 more docs, while replica is down
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // And flush:
+ long primaryVersion2 = primary.flush(0);
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ // Now restart replica:
+ replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // On startup the replica still searches its last commit:
+ assertVersionAndHits(replica, primaryVersion1, 10);
+
+ // Now ask replica to sync:
+ replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+
+ waitForVersionAndHits(replica, primaryVersion2, 20);
+
+ replica.close();
+ primary.close();
+ }
+
+ // Crash primary and promote a replica
+ public void testCrashPrimary1() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Crash primary:
+ primary.crash();
+
+ // Promote replica:
+ replica.commit();
+ replica.close();
+
+ primary = startNode(-1, 1, path2, -1, false);
+
+ // Should still see 10 docs:
+ assertVersionAndHits(primary, primaryVersion1, 10);
+
+ primary.close();
+ }
+
+ // Crash primary and then restart it
+ public void testCrashPrimary2() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ primary.commit();
+
+ // Index 10 docs, but crash before replicating or committing:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Crash primary:
+ primary.crash();
+
+ // Restart it:
+ primary = startNode(-1, 0, path1, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 more docs
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ long primaryVersion2 = primary.flush(0);
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion2, 20);
+
+ primary.close();
+ replica.close();
+ }
+
+ // Crash primary and then restart it, while a replica node is down, then bring replica node back up and make sure it properly "unforks" itself
+ public void testCrashPrimary3() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ replica.commit();
+
+ replica.close();
+ primary.crash();
+
+ // At this point replica is "in the future": it has 10 docs committed, but the primary crashed before committing so it has 0 docs
+
+ // Restart primary:
+ primary = startNode(-1, 0, path1, -1, true);
+
+ // Index 20 docs into primary:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<20;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Flush primary, but there are no replicas to sync to:
+ long primaryVersion2 = primary.flush(0);
+
+ // Now restart replica, which on init should detect on a "lost branch" because its 10 docs that were committed came from a different
+ // primary node:
+ replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ assertVersionAndHits(replica, primaryVersion2, 20);
+
+ primary.close();
+ replica.close();
+ }
+
+ public void testCrashPrimaryWhileCopying() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 100 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<100;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes (async) to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ Thread.sleep(TestUtil.nextInt(random(), 1, 30));
+
+ // Crash primary, likely/hopefully while replica is still copying
+ primary.crash();
+
+ // Could see either 100 docs (replica finished before crash) or 0 docs:
+ try (Connection c = new Connection(replica.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ c.flush();
+ long version = c.in.readVLong();
+ int hitCount = c.in.readVInt();
+ if (version == 0) {
+ assertEquals(0, hitCount);
+ } else {
+ assertEquals(primaryVersion1, version);
+ assertEquals(100, hitCount);
+ }
+ }
+
+ primary.close();
+ replica.close();
+ }
+
+ public void testCrashReplica() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Crash replica
+ replica.crash();
+
+ sendReplicasToPrimary(primary);
+
+ // Lots of new flushes while replica is down:
+ long primaryVersion2 = 0;
+ for(int iter=0;iter<10;iter++) {
+ // Index 10 docs into primary:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+ primaryVersion2 = primary.flush(0);
+ }
+
+ // Start up replica again:
+ replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Now ask replica to sync:
+ replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+
+ // Make sure it sees all docs that were indexed while it was down:
+ assertVersionAndHits(primary, primaryVersion2, 110);
+
+ replica.close();
+ primary.close();
+ }
+
+ public void testFullClusterCrash() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ Path path3 = createTempDir("3");
+ NodeProcess replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
+
+ sendReplicasToPrimary(primary, replica1, replica2);
+
+ // Index 50 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ long primaryVersion1 = 0;
+ for (int iter=0;iter<5;iter++) {
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replicas:
+ primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+ }
+
+ // Wait for replicas to sync up:
+ waitForVersionAndHits(replica1, primaryVersion1, 50);
+ waitForVersionAndHits(replica2, primaryVersion1, 50);
+
+ primary.commit();
+ replica1.commit();
+ replica2.commit();
+
+ // Index 10 more docs, but don't sync to replicas:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Full cluster crash
+ primary.crash();
+ replica1.crash();
+ replica2.crash();
+
+ // Full cluster restart
+ primary = startNode(-1, 0, path1, -1, true);
+ replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
+ replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
+
+ // Only 50 because we didn't commit primary before the crash:
+
+ // It's -1 because it's unpredictable how IW changes segments version on init:
+ assertVersionAndHits(primary, -1, 50);
+ assertVersionAndHits(replica1, primaryVersion1, 50);
+ assertVersionAndHits(replica2, primaryVersion1, 50);
+
+ primary.close();
+ replica1.close();
+ replica2.close();
+ }
+
+ /** Tell primary current replicas. */
+ private void sendReplicasToPrimary(NodeProcess primary, NodeProcess... replicas) throws IOException {
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+ c.out.writeVInt(replicas.length);
+ for(int id=0;id<replicas.length;id++) {
+ NodeProcess replica = replicas[id];
+ c.out.writeVInt(replica.id);
+ c.out.writeVInt(replica.tcpPort);
+ }
+ c.flush();
+ c.in.readByte();
+ }
+ }
+
+ /** Verifies this node is currently searching the specified version with the specified total hit count, or that it eventually does when
+ * keepTrying is true. */
+ private void assertVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
+ try (Connection c = new Connection(node.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ c.flush();
+ long version = c.in.readVLong();
+ int hitCount = c.in.readVInt();
+ if (expectedVersion != -1) {
+ assertEquals("hitCount=" + hitCount, expectedVersion, version);
+ }
+ assertEquals(expectedHitCount, hitCount);
+ }
+ }
+
+ private void waitForVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
+ try (Connection c = new Connection(node.tcpPort)) {
+ while (true) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ c.flush();
+ long version = c.in.readVLong();
+ int hitCount = c.in.readVInt();
+
+ if (version == expectedVersion) {
+ assertEquals(expectedHitCount, hitCount);
+ break;
+ }
+
+ assertTrue(version < expectedVersion);
+ Thread.sleep(10);
+ }
+ }
+ }
+
static void message(String message) {
long now = System.nanoTime();
System.out.println(String.format(Locale.ROOT,
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b1b2c799/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
index 63ff12a..04bbdc1 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
@@ -518,6 +518,10 @@ public class TestStressNRTReplication extends LuceneTestCase {
return null;
}
+ // This is very costly (takes more time to check than it did to index); we do this ourselves in the end instead of each time a replica
+ // is restarted:
+ // cmd.add("-Dtests.nrtreplication.checkonclose=true");
+
cmd.add("-Dtests.nrtreplication.node=true");
cmd.add("-Dtests.nrtreplication.nodeid=" + id);
cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
@@ -590,7 +594,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
long initInfosVersion = -1;
Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
boolean willCrash = false;
- boolean sawExistingSegmentsFile = false;
while (true) {
String l = r.readLine();
@@ -609,12 +612,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
// Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
if (isPrimary == false) {
- if (sawExistingSegmentsFile) {
- // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
- // and retry again later:
- message("failed to remove segments_N; skipping");
- return null;
- }
for(int i=0;i<100;i++) {
NodeProcess primary2 = primary;
if (primaryGen != myPrimaryGen || primary2 == null || primary2.nodeIsClosing.get()) {
@@ -658,8 +655,6 @@ public class TestStressNRTReplication extends LuceneTestCase {
willCrash = true;
} else if (l.startsWith("NODE STARTED")) {
break;
- } else if (l.contains("replica cannot start: existing segments file=")) {
- sawExistingSegmentsFile = true;
}
}
[09/15] lucene-solr git commit: fix compilation errors
Posted by mi...@apache.org.
fix compilation errors
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/d9f20f53
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/d9f20f53
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/d9f20f53
Branch: refs/heads/jira/lucene-5438-nrt-replication
Commit: d9f20f53edd9dc5895ad18e1844fb4ae9652189e
Parents: 8889469
Author: Michael McCandless <ma...@mikemccandless.com>
Authored: Tue Feb 2 06:16:34 2016 -0500
Committer: Mike McCandless <mi...@apache.org>
Committed: Tue Feb 2 06:16:34 2016 -0500
----------------------------------------------------------------------
.../replicator/nrt/TestNRTReplication.java | 36 +++++++++++---------
1 file changed, 19 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d9f20f53/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index b2240eb..e2824af 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@ -17,6 +17,15 @@ package org.apache.lucene.replicator.nrt;
* limitations under the License.
*/
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
@@ -25,18 +34,10 @@ import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;
+import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Pattern;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.LineFileDocs;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
-
-import com.carrotsearch.randomizedtesting.SeedUtils;
-
// MockRandom's .sd file has no index header/footer:
@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
@@ -151,6 +152,7 @@ public class TestNRTReplication extends LuceneTestCase {
final boolean finalWillCrash = willCrash;
// Baby sits the child process, pulling its stdout and printing to our stdout:
+ AtomicBoolean nodeClosing = new AtomicBoolean();
Thread pumper = ThreadPumper.start(
new Runnable() {
@Override
@@ -170,11 +172,11 @@ public class TestNRTReplication extends LuceneTestCase {
throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
}
}
- }, r, System.out, null);
+ }, r, System.out, null, nodeClosing);
pumper.setName("pump" + id);
message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
- return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+ return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion, nodeClosing);
}
public void testReplicateDeleteAllDocuments() throws Exception {
@@ -220,7 +222,7 @@ public class TestNRTReplication extends LuceneTestCase {
assertEquals(0, hitCount);
// Refresh primary, which also pushes to replica:
- long primaryVersion1 = primary.flush();
+ long primaryVersion1 = primary.flush(0);
assertTrue(primaryVersion1 > 0);
long version2;
@@ -259,7 +261,7 @@ public class TestNRTReplication extends LuceneTestCase {
assertEquals(10, hitCount);
// Refresh primary, which also pushes to replica:
- long primaryVersion2 = primary.flush();
+ long primaryVersion2 = primary.flush(0);
assertTrue(primaryVersion2 > primaryVersion1);
// Wait for replica to show the change
@@ -285,7 +287,7 @@ public class TestNRTReplication extends LuceneTestCase {
}
// Refresh primary, which also pushes to replica:
- long primaryVersion3 = primary.flush();
+ long primaryVersion3 = primary.flush(0);
assertTrue(primaryVersion3 > primaryVersion2);
// Wait for replica to show the change
@@ -344,7 +346,7 @@ public class TestNRTReplication extends LuceneTestCase {
}
// Refresh primary, which also pushes to replica:
- long primaryVersion1 = primary.flush();
+ long primaryVersion1 = primary.flush(0);
assertTrue(primaryVersion1 > 0);
// Index 10 more docs into primary:
@@ -354,13 +356,13 @@ public class TestNRTReplication extends LuceneTestCase {
}
// Refresh primary, which also pushes to replica:
- long primaryVersion2 = primary.flush();
+ long primaryVersion2 = primary.flush(0);
assertTrue(primaryVersion2 > primaryVersion1);
primary.forceMerge(primaryC);
// Refresh primary, which also pushes to replica:
- long primaryVersion3 = primary.flush();
+ long primaryVersion3 = primary.flush(0);
assertTrue(primaryVersion3 > primaryVersion2);
Connection replicaC = new Connection(replica.tcpPort);
[14/15] lucene-solr git commit: Merge branch 'master' into
nrt_replicas
Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/ReaderManager.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/search/SearcherManager.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions2.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnJRECrash.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnVMError.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestRollingUpdates.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/index/TestStressDeletes.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/core/src/test/org/apache/lucene/search/TestSearcherManager.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/SearcherTaxonomyManager.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --cc lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index 133992f,0000000..a7adbe2
mode 100644,000000..100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@@ -1,763 -1,0 +1,768 @@@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ByteArrayIndexInput;
+import org.apache.lucene.store.Directory;
++import org.apache.lucene.store.FSDirectory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.Lock;
+import org.apache.lucene.util.IOUtils;
+
+/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */
+
+abstract class ReplicaNode extends Node {
+
+ ReplicaFileDeleter deleter;
+
+ /** IncRef'd files in the current commit point: */
+ private final Collection<String> lastCommitFiles = new HashSet<>();
+
+ /** IncRef'd files in the current NRT point: */
+ protected final Collection<String> lastNRTFiles = new HashSet<>();
+
+ /** Currently running merge pre-copy jobs */
+ protected final Set<CopyJob> mergeCopyJobs = Collections.synchronizedSet(new HashSet<>());
+
+ /** Non-null when we are currently copying files from a new NRT point: */
+ protected CopyJob curNRTCopy;
+
+ /** We hold this to ensure an external IndexWriter cannot also open on our directory: */
+ private final Lock writeFileLock;
+
+ /** Merged segment files that we pre-copied, but have not yet made visible in a new NRT point. */
+ final Set<String> pendingMergeFiles = Collections.synchronizedSet(new HashSet<String>());
+
+ /** Primary gen last time we successfully replicated: */
+ protected long lastPrimaryGen;
+
+ public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory) throws IOException {
+ super(id, dir, searcherFactory);
+
++ if (dir instanceof FSDirectory && ((FSDirectory) dir).checkPendingDeletions()) {
++ throw new IllegalArgumentException("Directory " + dir + " still has pending deleted files; cannot initialize IndexWriter");
++ }
++
+ boolean success = false;
+
+ try {
+ message("top: init replica dir=" + dir);
+
+ // Obtain a write lock on this index since we "act like" an IndexWriter, to prevent any other IndexWriter or ReplicaNode from using it:
+ writeFileLock = dir.obtainLock(IndexWriter.WRITE_LOCK_NAME);
+
+ // nocommit must check for no pending deletes here, like IW does
+
+ state = "init";
+ deleter = new ReplicaFileDeleter(this, dir);
+ } catch (Throwable t) {
+ message("exc on init:");
+ t.printStackTrace(System.out);
+ throw t;
+ } finally {
+ if (success == false) {
+ IOUtils.closeWhileHandlingException(this);
+ }
+ }
+ }
+
+ /** Start up this replica, which possibly requires heavy copying of files from the primary node, if we were down for a long time */
+ protected synchronized void start(long curPrimaryGen) throws IOException {
+
+ if (state.equals("init") == false) {
+ throw new IllegalStateException("already started");
+ }
+
+ message("top: now start");
+ try {
+
+ // Figure out what state our local index is in now:
+ String segmentsFileName = SegmentInfos.getLastCommitSegmentsFileName(dir);
+
+ // Also look for any pending_segments_N, in case we crashed mid-commit. We must "inflate" our infos gen to at least this, since
+ // otherwise we may wind up re-using the pending_segments_N file name on commit, and then our deleter can get angry because it still
+ // wants to delete this file:
+ long maxPendingGen = -1;
+ for(String fileName : dir.listAll()) {
+ if (fileName.startsWith(IndexFileNames.PENDING_SEGMENTS)) {
+ long gen = Long.parseLong(fileName.substring(IndexFileNames.PENDING_SEGMENTS.length()+1), Character.MAX_RADIX);
+ if (gen > maxPendingGen) {
+ maxPendingGen = gen;
+ }
+ }
+ }
+
+ SegmentInfos infos;
+ if (segmentsFileName == null) {
+ // No index here yet:
+ infos = new SegmentInfos();
+ message("top: init: no segments in index");
+ } else {
+ message("top: init: read existing segments commit " + segmentsFileName);
+ infos = SegmentInfos.readCommit(dir, segmentsFileName);
+ message("top: init: segments: " + infos.toString() + " version=" + infos.getVersion());
+ Collection<String> indexFiles = infos.files(false);
+
+ lastCommitFiles.add(segmentsFileName);
+ lastCommitFiles.addAll(indexFiles);
+
+ // Always protect the last commit:
+ deleter.incRef(lastCommitFiles);
+
+ lastNRTFiles.addAll(indexFiles);
+ deleter.incRef(lastNRTFiles);
+ message("top: commitFiles=" + lastCommitFiles);
+ message("top: nrtFiles=" + lastNRTFiles);
+ }
+
+ message("top: delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
+ deleter.deleteUnknownFiles(segmentsFileName);
+ message("top: done delete unknown files on init: all files=" + Arrays.toString(dir.listAll()));
+
+ String s = infos.getUserData().get(PRIMARY_GEN_KEY);
+ long myPrimaryGen;
+ if (s == null) {
+ assert infos.size() == 0;
+ myPrimaryGen = -1;
+ } else {
+ myPrimaryGen = Long.parseLong(s);
+ }
+ message("top: myPrimaryGen=" + myPrimaryGen);
+
+ boolean doCommit;
+
+ if (infos.size() > 0 && myPrimaryGen != -1 && myPrimaryGen != curPrimaryGen) {
+
+ assert myPrimaryGen < curPrimaryGen;
+
+ // Primary changed while we were down. In this case, we must sync from primary before opening a reader, because it's possible current
+ // files we have will need to be overwritten with different ones (if index rolled back and "forked"), and we can't overwrite open
+ // files on Windows:
+
+ final long initSyncStartNS = System.nanoTime();
+
+ message("top: init: primary changed while we were down myPrimaryGen=" + myPrimaryGen +
+ " vs curPrimaryGen=" + curPrimaryGen +
+ "; sync now before mgr init");
+
+ // Try until we succeed in copying over the latest NRT point:
+ CopyJob job = null;
+
+ // We may need to overwrite files referenced by our latest commit, either right now on initial sync, or on a later sync. To make
+ // sure the index is never even in an "apparently" corrupt state (where an old segments_N references invalid files) we forcefully
+ // remove the commit now, and refuse to start the replica if this delete fails:
+ message("top: now delete starting commit point " + segmentsFileName);
+
+ // If this throws exc (e.g. due to virus checker), we cannot start this replica:
+ assert deleter.getRefCount(segmentsFileName) == 1;
+ deleter.decRef(Collections.singleton(segmentsFileName));
+ if (deleter.isPending(segmentsFileName)) {
+ // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else we can cause corruption:
+ throw new RuntimeException("replica cannot start: existing segments file=" + segmentsFileName + " must be removed in order to start, but the file delete failed");
+ }
+ // So we don't later try to decRef it (illegally) again:
+ boolean didRemove = lastCommitFiles.remove(segmentsFileName);
+ assert didRemove;
+
+ while (true) {
+ job = newCopyJob("sync on startup replica=" + name() + " myVersion=" + infos.getVersion(),
+ null,
+ null,
+ true,
+ null);
+ job.start();
+
+ message("top: init: sync sis.version=" + job.getCopyState().version);
+
+ Collection<String> fileNamesToCopy = job.getFileNamesToCopy();
+
+ // Force this copy job to finish while we wait, now. Note that this can be very time consuming!
+ // NOTE: newNRTPoint detects we are still in init (mgr is null) and does not cancel our copy if a flush happens
+ try {
+ job.runBlocking();
+ job.finish();
+
+ // Success!
+ break;
+ } catch (IOException ioe) {
+ job.cancel("startup failed", ioe);
+ if (ioe.getMessage().contains("checksum mismatch after file copy")) {
+ // OK-ish
+ message("top: failed to copy: " + ioe + "; retrying");
+ } else {
+ throw ioe;
+ }
+ }
+ }
+
+ lastPrimaryGen = job.getCopyState().primaryGen;
+ byte[] infosBytes = job.getCopyState().infosBytes;
+
+ SegmentInfos syncInfos = SegmentInfos.readCommit(dir,
+ new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", job.getCopyState().infosBytes)),
+ job.getCopyState().gen);
+
+ // Must always commit to a larger generation than what's currently in the index:
+ syncInfos.updateGeneration(infos);
+ infos = syncInfos;
+
+ assert infos.getVersion() == job.getCopyState().version;
+ message(" version=" + infos.getVersion() + " segments=" + infos.toString());
+ message("top: init: incRef nrtFiles=" + job.getFileNames());
+ deleter.incRef(job.getFileNames());
+ message("top: init: decRef lastNRTFiles=" + lastNRTFiles);
+ deleter.decRef(lastNRTFiles);
+
+ lastNRTFiles.clear();
+ lastNRTFiles.addAll(job.getFileNames());
+
+ message("top: init: set lastNRTFiles=" + lastNRTFiles);
+ lastFileMetaData = job.getCopyState().files;
+ message(String.format(Locale.ROOT, "top: %d: start: done sync: took %.3fs for %s, opened NRT reader version=%d",
+ id,
+ (System.nanoTime()-initSyncStartNS)/1000000000.0,
+ bytesToString(job.getTotalBytesCopied()),
+ job.getCopyState().version));
+
+ doCommit = true;
+ } else {
+ doCommit = false;
+ lastPrimaryGen = curPrimaryGen;
+ message("top: same primary as before");
+ }
+
+ if (infos.getGeneration() < maxPendingGen) {
+ message("top: move infos generation from " + infos.getGeneration() + " to " + maxPendingGen);
+ infos.setNextWriteGeneration(maxPendingGen);
+ }
+
+ // Notify primary we started, to give it a chance to send any warming merges our way to reduce NRT latency of first sync:
+ sendNewReplica();
+
+ // Finally, we are open for business, since our index now "agrees" with the primary:
+ mgr = new SegmentInfosSearcherManager(dir, this, infos, searcherFactory);
+
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ // TODO: this is test specific:
+ int hitCount = searcher.count(new TermQuery(new Term("marker", "marker")));
+ message("top: marker count=" + hitCount + " version=" + ((DirectoryReader) searcher.getIndexReader()).getVersion());
+ } finally {
+ mgr.release(searcher);
+ }
+
+ // Must commit after init mgr:
+ if (doCommit) {
+ // Very important to commit what we just sync'd over, because we removed the pre-existing commit point above if we had to
+ // overwrite any files it referenced:
+ commit();
+ }
+
+ message("top: done start");
+ state = "idle";
+ } catch (Throwable t) {
+ if (t.getMessage().startsWith("replica cannot start") == false) {
+ message("exc on start:");
+ t.printStackTrace(System.out);
+ } else {
+ dir.close();
+ }
+ IOUtils.reThrow(t);
+ }
+ }
+
+ final Object commitLock = new Object();
+
+ @Override
+ public void commit() throws IOException {
+
+ synchronized(commitLock) {
+
+ SegmentInfos infos;
+ Collection<String> indexFiles;
+
+ synchronized (this) {
+ infos = ((SegmentInfosSearcherManager) mgr).getCurrentInfos();
+ indexFiles = infos.files(false);
+ deleter.incRef(indexFiles);
+ }
+
+ message("top: commit primaryGen=" + lastPrimaryGen + " infos=" + infos.toString() + " files=" + indexFiles);
+
+ // fsync all index files we are now referencing
+ dir.sync(indexFiles);
+
+ Map<String,String> commitData = new HashMap<>();
+ commitData.put(PRIMARY_GEN_KEY, Long.toString(lastPrimaryGen));
+ commitData.put(VERSION_KEY, Long.toString(getCurrentSearchingVersion()));
+ infos.setUserData(commitData, false);
+
+ // write and fsync a new segments_N
+ infos.commit(dir);
+
+ // Notify current infos (which may have changed while we were doing dir.sync above) what generation we are up to; this way future
+ // commits are guaranteed to go to the next (unwritten) generations:
+ if (mgr != null) {
+ ((SegmentInfosSearcherManager) mgr).getCurrentInfos().updateGeneration(infos);
+ }
+ String segmentsFileName = infos.getSegmentsFileName();
+ message("top: commit wrote segments file " + segmentsFileName + " version=" + infos.getVersion() + " sis=" + infos.toString() + " commitData=" + commitData);
+ deleter.incRef(Collections.singletonList(segmentsFileName));
+ message("top: commit decRef lastCommitFiles=" + lastCommitFiles);
+ deleter.decRef(lastCommitFiles);
+ lastCommitFiles.clear();
+ lastCommitFiles.addAll(indexFiles);
+ lastCommitFiles.add(segmentsFileName);
+ message("top: commit version=" + infos.getVersion() + " files now " + lastCommitFiles);
+ }
+ }
+
+ void finishNRTCopy(CopyJob job, long startNS) throws IOException {
+ CopyState copyState = job.getCopyState();
+ message("top: finishNRTCopy: version=" + copyState.version + (job.getFailed() ? " FAILED" : "") + " job=" + job);
+
+ // NOTE: if primary crashed while we were still copying then the job will hit an exc trying to read bytes for the files from the primary node,
+ // and the job will be marked as failed here:
+
+ synchronized (this) {
+
+ if ("syncing".equals(state)) {
+ state = "idle";
+ }
+
+ if (curNRTCopy == job) {
+ message("top: now clear curNRTCopy; job=" + job);
+ curNRTCopy = null;
+ } else {
+ assert job.getFailed();
+ message("top: skip clear curNRTCopy: we were cancelled; job=" + job);
+ }
+
+ if (job.getFailed()) {
+ return;
+ }
+
+ // Does final file renames:
+ job.finish();
+
+ // Turn byte[] back to SegmentInfos:
+ byte[] infosBytes = copyState.infosBytes;
+ SegmentInfos infos = SegmentInfos.readCommit(dir,
+ new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", copyState.infosBytes)),
+ copyState.gen);
+ assert infos.getVersion() == copyState.version;
+
+ message(" version=" + infos.getVersion() + " segments=" + infos.toString());
+
+ // Cutover to new searcher:
+ if (mgr != null) {
+ ((SegmentInfosSearcherManager) mgr).setCurrentInfos(infos);
+ }
+
+ // Must first incRef new NRT files, then decRef old ones, to make sure we don't remove an NRT file that's in common to both:
+ Collection<String> newFiles = copyState.files.keySet();
+ message("top: incRef newNRTFiles=" + newFiles);
+ deleter.incRef(newFiles);
+
+ // If any of our new files were previously copied merges, we clear them now, so we don't try to later delete a non-existent file:
+ pendingMergeFiles.removeAll(newFiles);
+ message("top: after remove from pending merges pendingMergeFiles=" + pendingMergeFiles);
+
+ message("top: decRef lastNRTFiles=" + lastNRTFiles);
+ deleter.decRef(lastNRTFiles);
+ lastNRTFiles.clear();
+ lastNRTFiles.addAll(newFiles);
+ message("top: set lastNRTFiles=" + lastNRTFiles);
+
+ // At this point we can remove any completed merge segment files that we still do not reference. This can happen when a merge
+ // finishes, copies its files out to us, but is then merged away (or dropped due to 100% deletions) before we ever cutover to it
+ // in an NRT point:
+ if (copyState.completedMergeFiles.isEmpty() == false) {
+ message("now remove-if-not-ref'd completed merge files: " + copyState.completedMergeFiles);
+ for(String fileName : copyState.completedMergeFiles) {
+ if (pendingMergeFiles.contains(fileName)) {
+ pendingMergeFiles.remove(fileName);
+ deleter.deleteIfNoRef(fileName);
+ }
+ }
+ }
+
+ lastFileMetaData = copyState.files;
+
+ // It's a good time to delete pending files, since we just refreshed and some previously open files are now closed:
+ deleter.deletePending();
+ }
+
+ int markerCount;
+ IndexSearcher s = mgr.acquire();
+ try {
+ markerCount = s.count(new TermQuery(new Term("marker", "marker")));
+ } finally {
+ mgr.release(s);
+ }
+
+ message(String.format(Locale.ROOT, "top: done sync: took %.3fs for %s, opened NRT reader version=%d markerCount=%d",
+ (System.nanoTime()-startNS)/1000000000.0,
+ bytesToString(job.getTotalBytesCopied()),
+ copyState.version,
+ markerCount));
+ }
+
+ /** Start a background copying job, to copy the specified files from the current primary node. If files is null then the latest copy
+ * state should be copied. If prevJob is not null, then the new copy job is replacing it and should 1) cancel the previous one, and
+ * 2) optionally salvage e.g. partially copied and, shared with the new copy job, files. */
+ protected abstract CopyJob newCopyJob(String reason, Map<String,FileMetaData> files, Map<String,FileMetaData> prevFiles,
+ boolean highPriority, CopyJob.OnceDone onceDone) throws IOException;
+
+ /** Runs this job async'd */
+ protected abstract void launch(CopyJob job);
+
+ /** Tell primary we (replica) just started, so primary can tell us to warm any already warming merges. This lets us keep low nrt refresh
+ * time for the first nrt sync after we started. */
+ protected abstract void sendNewReplica() throws IOException;
+
+ /** Call this to notify this replica node that a new NRT infos is available on the primary.
+ * We kick off a job (runs in the background) to copy files across, and open a new reader once that's done. */
+ public synchronized CopyJob newNRTPoint(long version) throws IOException {
+
+ if (isClosed()) {
+ throw new AlreadyClosedException("this replica is closed: state=" + state);
+ }
+
+ // Caller should not "publish" us until we have finished .start():
+ assert mgr != null;
+
+ if ("idle".equals(state)) {
+ state = "syncing";
+ }
+
+ long curVersion = getCurrentSearchingVersion();
+
+ message("top: start sync sis.version=" + version);
+
+ if (version == curVersion) {
+ // Caller releases the CopyState:
+ message("top: new NRT point has same version as current; skipping");
+ return null;
+ }
+
+ if (version < curVersion) {
+ // This can happen, if two syncs happen close together, and due to thread scheduling, the incoming older version runs after the newer version
+ message("top: new NRT point (version=" + version + ") is older than current (version=" + version + "); skipping");
+ return null;
+ }
+
+ final long startNS = System.nanoTime();
+
+ message("top: newNRTPoint");
+ CopyJob job = null;
+ try {
+ job = newCopyJob("NRT point sync version=" + version,
+ null,
+ lastFileMetaData,
+ true,
+ new CopyJob.OnceDone() {
+ @Override
+ public void run(CopyJob job) {
+ try {
+ finishNRTCopy(job, startNS);
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ }
+ });
+ } catch (NodeCommunicationException nce) {
+ // E.g. primary could crash/close when we are asking it for the copy state:
+ message("top: ignoring communication exception creating CopyJob: " + nce);
+ //nce.printStackTrace(System.out);
+ if (state.equals("syncing")) {
+ state = "idle";
+ }
+ return null;
+ }
+
+ Collection<String> newNRTFiles = job.getFileNames();
+ long newPrimaryGen = job.getCopyState().primaryGen;
+ maybeNewPrimary(newPrimaryGen);
+
+ message("top: newNRTPoint: job files=" + newNRTFiles);
+
+ if (curNRTCopy != null) {
+ job.transferAndCancel(curNRTCopy);
+ assert curNRTCopy.getFailed();
+ }
+
+ curNRTCopy = job;
+
+ for(String fileName : curNRTCopy.getFileNamesToCopy()) {
+ assert lastCommitFiles.contains(fileName) == false: "fileName=" + fileName + " is in lastCommitFiles and is being copied?";
+ synchronized (mergeCopyJobs) {
+ for (CopyJob mergeJob : mergeCopyJobs) {
+ if (mergeJob.getFileNames().contains(fileName)) {
+ // TODO: we could maybe transferAndCancel here? except CopyJob can't transferAndCancel more than one currently
+ message("top: now cancel merge copy job=" + mergeJob + ": file " + fileName + " is now being copied via NRT point");
+ mergeJob.cancel("newNRTPoint is copying over the same file", null);
+ }
+ }
+ }
+ }
+
+ try {
+ job.start();
+ } catch (NodeCommunicationException nce) {
+ // E.g. primary could crash/close when we are asking it for the copy state:
+ message("top: ignoring exception starting CopyJob: " + nce);
+ nce.printStackTrace(System.out);
+ if (state.equals("syncing")) {
+ state = "idle";
+ }
+ return null;
+ }
+
+ // Runs in the background jobs thread, maybe slowly/throttled, and calls finishSync once it's done:
+ launch(curNRTCopy);
+ return curNRTCopy;
+ }
+
+ public synchronized boolean isCopying() {
+ return curNRTCopy != null;
+ }
+
+ @Override
+ public boolean isClosed() {
+ return "closed".equals(state) || "closing".equals(state) || "crashing".equals(state) || "crashed".equals(state);
+ }
+
+ @Override
+ public void close() throws IOException {
+ message("top: now close");
+
+ synchronized (this) {
+ state = "closing";
+ if (curNRTCopy != null) {
+ curNRTCopy.cancel("closing", null);
+ }
+ }
+
+ synchronized (this) {
+ message("top: close mgr");
+ mgr.close();
+
+ message("top: decRef lastNRTFiles=" + lastNRTFiles);
+ deleter.decRef(lastNRTFiles);
+ lastNRTFiles.clear();
+
+ // NOTE: do not decRef these!
+ lastCommitFiles.clear();
+
+ message("top: delete if no ref pendingMergeFiles=" + pendingMergeFiles);
+ for(String fileName : pendingMergeFiles) {
+ deleter.deleteIfNoRef(fileName);
+ }
+ pendingMergeFiles.clear();
+
+ message("top: close dir");
+ IOUtils.close(writeFileLock, dir);
+ }
+ message("top: done close");
+ state = "closed";
+ }
+
+ /** Called when the primary changed */
+ protected synchronized void maybeNewPrimary(long newPrimaryGen) {
+ if (newPrimaryGen != lastPrimaryGen) {
+ message("top: now change lastPrimaryGen from " + lastPrimaryGen + " to " + newPrimaryGen + " pendingMergeFiles=" + pendingMergeFiles);
+ assert newPrimaryGen > lastPrimaryGen: "newPrimaryGen=" + newPrimaryGen + " vs lastPrimaryGen=" + lastPrimaryGen;
+ lastPrimaryGen = newPrimaryGen;
+ pendingMergeFiles.clear();
+ } else {
+ message("top: keep current lastPrimaryGen=" + lastPrimaryGen);
+ }
+ }
+
+ protected synchronized CopyJob launchPreCopyMerge(AtomicBoolean finished, long newPrimaryGen, Map<String,FileMetaData> files) throws IOException {
+
+ CopyJob job;
+
+ maybeNewPrimary(newPrimaryGen);
+ final long primaryGenStart = lastPrimaryGen;
+ Set<String> fileNames = files.keySet();
+ message("now pre-copy warm merge files=" + fileNames + " primaryGen=" + newPrimaryGen);
+
+ for(String fileName : fileNames) {
+ assert pendingMergeFiles.contains(fileName) == false: "file \"" + fileName + "\" is already being warmed!";
+ assert lastNRTFiles.contains(fileName) == false: "file \"" + fileName + "\" is already NRT visible!";
+ }
+
+ job = newCopyJob("warm merge on " + name() + " filesNames=" + fileNames,
+ files, null, false,
+ new CopyJob.OnceDone() {
+
+ @Override
+ public void run(CopyJob job) throws IOException {
+ // Signals that this replica has finished
+ mergeCopyJobs.remove(job);
+ message("done warming merge " + fileNames + " failed?=" + job.getFailed());
+ synchronized(this) {
+ if (job.getFailed() == false) {
+ if (lastPrimaryGen != primaryGenStart) {
+ message("merge pre copy finished but primary has changed; cancelling job files=" + fileNames);
+ job.cancel("primary changed during merge copy", null);
+ } else {
+ boolean abort = false;
+ for (String fileName : fileNames) {
+ if (lastNRTFiles.contains(fileName)) {
+ message("abort merge finish: file " + fileName + " is referenced by last NRT point");
+ abort = true;
+ }
+ if (lastCommitFiles.contains(fileName)) {
+ message("abort merge finish: file " + fileName + " is referenced by last commit point");
+ abort = true;
+ }
+ }
+ if (abort) {
+ // Even though in newNRTPoint we have similar logic, which cancels any merge copy jobs if an NRT point
+ // shows up referencing the files we are warming (because primary got impatient and gave up on us), we also
+ // need it here in case replica is way far behind and fails to even receive the merge pre-copy request
+ // until after the newNRTPoint referenced those files:
+ job.cancel("merged segment was separately copied via NRT point", null);
+ } else {
+ job.finish();
+ message("merge pre copy finished files=" + fileNames);
+ for(String fileName : fileNames) {
+ assert pendingMergeFiles.contains(fileName) == false : "file \"" + fileName + "\" is already in pendingMergeFiles";
+ message("add file " + fileName + " to pendingMergeFiles");
+ pendingMergeFiles.add(fileName);
+ }
+ }
+ }
+ } else {
+ message("merge copy finished with failure");
+ }
+ }
+ finished.set(true);
+ }
+ });
+
+ job.start();
+
+ // When warming a merge we better not already have any of these files copied!
+ assert job.getFileNamesToCopy().size() == files.size();
+
+ mergeCopyJobs.add(job);
+ launch(job);
+
+ return job;
+ }
+
+ public IndexOutput createTempOutput(String prefix, String suffix, IOContext ioContext) throws IOException {
+ return dir.createTempOutput(prefix, suffix, IOContext.DEFAULT);
+ }
+
+ /** Compares incoming per-file identity (id, checksum, header, footer) versus what we have locally and returns the subset of the incoming
+ * files that need copying */
+ public List<Map.Entry<String,FileMetaData>> getFilesToCopy(Map<String,FileMetaData> files) throws IOException {
+
+ boolean doCopyCommitFiles = false;
+ List<Map.Entry<String,FileMetaData>> toCopy = new ArrayList<>();
+ for (Map.Entry<String,FileMetaData> ent : files.entrySet()) {
+ String fileName = ent.getKey();
+ FileMetaData fileMetaData = ent.getValue();
+ if (fileIsIdentical(fileName, fileMetaData) == false) {
+ toCopy.add(ent);
+ }
+ }
+
+ return toCopy;
+ }
+
+ /** Carefully determine if the file on the primary, identified by its {@code String fileName} along with the {@link FileMetaData}
+ * "summarizing" its contents, is precisely the same file that we have locally. If the file does not exist locally, or if its its header
+ * (inclues the segment id), length, footer (including checksum) differ, then this returns false, else true. */
+ private boolean fileIsIdentical(String fileName, FileMetaData srcMetaData) throws IOException {
+
+ if (deleter.isPending(fileName)) {
+ // This was a file we had wanted to delete yet a virus checker prevented us, and now we need to overwrite it.
+ // Such files are in an unknown state, and even if their header and footer and length all
+ // match, since they may not have been fsync'd by the previous node instance on this directory,
+ // they could in theory have corruption internally. So we always force ourselves to copy them here:
+ if (Node.VERBOSE_FILES) {
+ message("file " + fileName + ": will copy [we had wanted to delete this file on init, but failed]");
+ }
+ return false;
+ }
+
+ FileMetaData destMetaData = readLocalFileMetaData(fileName);
+ if (destMetaData == null) {
+ // Something went wrong in reading the file (it's corrupt, truncated, does not exist, etc.):
+ return false;
+ }
+
+ if (Arrays.equals(destMetaData.header, srcMetaData.header) == false ||
+ Arrays.equals(destMetaData.footer, srcMetaData.footer) == false) {
+ // Segment name was reused! This is rare but possible and otherwise devastating:
+ if (Node.VERBOSE_FILES) {
+ message("file " + fileName + ": will copy [header/footer is different]");
+ }
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ private ConcurrentMap<String,Boolean> copying = new ConcurrentHashMap<>();
+
+ // Used only to catch bugs, ensuring a given file name is only ever being copied bye one job:
+ public void startCopyFile(String name) {
+ if (copying.putIfAbsent(name, Boolean.TRUE) != null) {
+ throw new IllegalStateException("file " + name + " is being copied in two places!");
+ }
+ }
+
+ public void finishCopyFile(String name) {
+ if (copying.remove(name) == null) {
+ throw new IllegalStateException("file " + name + " was not actually being copied?");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/bd6804bc/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
----------------------------------------------------------------------
diff --cc lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
index 15e9c8c,0000000..cd98b48
mode 100644,000000..100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java
@@@ -1,876 -1,0 +1,876 @@@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
+// nocommit make some explicit failure tests
+
+// MockRandom's .sd file has no index header/footer:
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class TestNRTReplication extends LuceneTestCase {
+
+ /** cwd where we start each child (server) node */
+ private Path childTempDir;
+
+ final AtomicLong nodeStartCounter = new AtomicLong();
+ private long nextPrimaryGen;
+ private long lastPrimaryGen;
+ LineFileDocs docs;
+
+ /** Launches a child "server" (separate JVM), which is either primary or replica node */
+ private NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, long forcePrimaryVersion, boolean willCrash) throws IOException {
+ List<String> cmd = new ArrayList<>();
+
+ cmd.add(System.getProperty("java.home")
+ + System.getProperty("file.separator")
+ + "bin"
+ + System.getProperty("file.separator")
+ + "java");
+ cmd.add("-Xmx512m");
+
+ long myPrimaryGen;
+ if (primaryTCPPort != -1) {
+ // I am a replica
+ cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + primaryTCPPort);
+ myPrimaryGen = lastPrimaryGen;
+ } else {
+ myPrimaryGen = nextPrimaryGen++;
+ lastPrimaryGen = myPrimaryGen;
+ }
+ cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
+ cmd.add("-Dtests.nrtreplication.closeorcrash=false");
+
+ cmd.add("-Dtests.nrtreplication.node=true");
+ cmd.add("-Dtests.nrtreplication.nodeid=" + id);
+ cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
+ cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
+ cmd.add("-Dtests.nrtreplication.checkonclose=true");
+
+ if (primaryTCPPort == -1) {
+ // We are the primary node
+ cmd.add("-Dtests.nrtreplication.isPrimary=true");
+ cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
+ }
+
+ // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
+ // new node:
+ long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
+
+ cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(seed));
+ cmd.add("-ea");
+ cmd.add("-cp");
+ cmd.add(System.getProperty("java.class.path"));
+ cmd.add("org.junit.runner.JUnitCore");
+ cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
+
+ message("child process command: " + cmd);
+ ProcessBuilder pb = new ProcessBuilder(cmd);
+ pb.redirectErrorStream(true);
+
+ // Important, so that the scary looking hs_err_<pid>.log appear under our test temp dir:
+ pb.directory(childTempDir.toFile());
+
+ Process p = pb.start();
+
+ BufferedReader r;
+ try {
+ r = new BufferedReader(new InputStreamReader(p.getInputStream(), IOUtils.UTF_8));
+ } catch (UnsupportedEncodingException uee) {
+ throw new RuntimeException(uee);
+ }
+
+ int tcpPort = -1;
+ long initCommitVersion = -1;
+ long initInfosVersion = -1;
+ Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+ boolean sawExistingSegmentsFile = false;
+
+ while (true) {
+ String l = r.readLine();
+ if (l == null) {
+ message("top: node=" + id + " failed to start");
+ try {
+ p.waitFor();
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ message("exit value=" + p.exitValue());
+ message("top: now fail test replica R" + id + " failed to start");
+ throw new RuntimeException("replica R" + id + " failed to start");
+ }
+
+ if (logTimeStart.matcher(l).matches()) {
+ // Already a well-formed log output:
+ System.out.println(l);
+ } else {
+ message(l);
+ }
+
+ if (l.startsWith("PORT: ")) {
+ tcpPort = Integer.parseInt(l.substring(6).trim());
+ } else if (l.startsWith("COMMIT VERSION: ")) {
+ initCommitVersion = Integer.parseInt(l.substring(16).trim());
+ } else if (l.startsWith("INFOS VERSION: ")) {
+ initInfosVersion = Integer.parseInt(l.substring(15).trim());
+ } else if (l.contains("will crash after")) {
+ willCrash = true;
+ } else if (l.startsWith("NODE STARTED")) {
+ break;
+ } else if (l.contains("replica cannot start: existing segments file=")) {
+ sawExistingSegmentsFile = true;
+ }
+ }
+
+ final boolean finalWillCrash = willCrash;
+
+ // Baby sits the child process, pulling its stdout and printing to our stdout:
+ AtomicBoolean nodeClosing = new AtomicBoolean();
+ Thread pumper = ThreadPumper.start(
+ new Runnable() {
+ @Override
+ public void run() {
+ message("now wait for process " + p);
+ try {
+ p.waitFor();
+ } catch (Throwable t) {
+ throw new RuntimeException(t);
+ }
+
+ message("done wait for process " + p);
+ int exitValue = p.exitValue();
+ message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
+ if (exitValue != 0 && finalWillCrash == false) {
+ // should fail test
+ throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
+ }
+ }
+ }, r, System.out, null, nodeClosing);
+ pumper.setName("pump" + id);
+
+ message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
+ return new NodeProcess(p, id, tcpPort, pumper, primaryTCPPort == -1, initCommitVersion, initInfosVersion, nodeClosing);
+ }
+
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
+ Node.globalStartNS = System.nanoTime();
+ childTempDir = createTempDir("child");
+ docs = new LineFileDocs(random());
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ super.tearDown();
+ docs.close();
+ }
+
+ public void testReplicateDeleteAllDocuments() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ // Tell primary current replicas:
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ Connection primaryC = new Connection(primary.tcpPort);
+ primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
+
+ // Nothing in replica index yet
+ assertVersionAndHits(replica, 0, 0);
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to show the change
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Delete all docs from primary
+ if (random().nextBoolean()) {
+ // Inefficiently:
+ for(int id=0;id<10;id++) {
+ primary.deleteDocument(primaryC, Integer.toString(id));
+ }
+ } else {
+ // Efficiently:
+ primary.deleteAllDocuments(primaryC);
+ }
+
+ // Replica still shows 10 docs:
+ assertVersionAndHits(replica, primaryVersion1, 10);
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion2 = primary.flush(0);
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ // Wait for replica to show the change
+ waitForVersionAndHits(replica, primaryVersion2, 0);
+
+ // Index 10 docs again:
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion3 = primary.flush(0);
+ assertTrue(primaryVersion3 > primaryVersion2);
+
+ // Wait for replica to show the change
+ waitForVersionAndHits(replica, primaryVersion3, 10);
+
+ primaryC.close();
+
+ replica.close();
+ primary.close();
+ }
+
+ public void testReplicateForceMerge() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ Connection primaryC = new Connection(primary.tcpPort);
+ primaryC.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Index 10 more docs into primary:
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(primaryC, doc, false);
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion2 = primary.flush(0);
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ primary.forceMerge(primaryC);
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion3 = primary.flush(0);
+ assertTrue(primaryVersion3 > primaryVersion2);
+
+ // Wait for replica to show the change
+ waitForVersionAndHits(replica, primaryVersion3, 20);
+
+ primaryC.close();
+
+ replica.close();
+ primary.close();
+ }
+
+ // Start up, index 10 docs, replicate, but crash and restart the replica without committing it:
+ public void testReplicaCrashNoCommit() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Crash replica:
+ replica.crash();
+
+ // Restart replica:
+ replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ // On startup the replica searches the last commit (empty here):
+ assertVersionAndHits(replica, 0, 0);
+
+ // Ask replica to sync:
+ replica.newNRTPoint(primaryVersion1, primary.tcpPort);
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ replica.close();
+ primary.close();
+ }
+
+ // Start up, index 10 docs, replicate, commit, crash and restart the replica
+ public void testReplicaCrashWithCommit() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Commit and crash replica:
+ replica.commit();
+ replica.crash();
+
+ // Restart replica:
+ replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ // On startup the replica searches the last commit:
+ assertVersionAndHits(replica, primaryVersion1, 10);
+
+ replica.close();
+ primary.close();
+ }
+
+ // Start up, index 10 docs, replicate, commit, crash, index more docs, replicate, then restart the replica
+ public void testIndexingWhileReplicaIsDown() throws Exception {
+
+ Path primaryPath = createTempDir("primary");
+ NodeProcess primary = startNode(-1, 0, primaryPath, -1, false);
+
+ Path replicaPath = createTempDir("replica");
+ NodeProcess replica = startNode(primary.tcpPort, 1, replicaPath, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Commit and crash replica:
+ replica.commit();
+ replica.crash();
+
+ sendReplicasToPrimary(primary);
+
+ // Index 10 more docs, while replica is down
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // And flush:
+ long primaryVersion2 = primary.flush(0);
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ // Now restart replica:
+ replica = startNode(primary.tcpPort, 1, replicaPath, -1, false);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // On startup the replica still searches its last commit:
+ assertVersionAndHits(replica, primaryVersion1, 10);
+
+ // Now ask replica to sync:
+ replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+
+ waitForVersionAndHits(replica, primaryVersion2, 20);
+
+ replica.close();
+ primary.close();
+ }
+
+ // Crash primary and promote a replica
+ public void testCrashPrimary1() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Crash primary:
+ primary.crash();
+
+ // Promote replica:
+ replica.commit();
+ replica.close();
+
+ primary = startNode(-1, 1, path2, -1, false);
+
+ // Should still see 10 docs:
+ assertVersionAndHits(primary, primaryVersion1, 10);
+
+ primary.close();
+ }
+
+ // Crash primary and then restart it
+ public void testCrashPrimary2() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ primary.commit();
+
+ // Index 10 docs, but crash before replicating or committing:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Crash primary:
+ primary.crash();
+
+ // Restart it:
+ primary = startNode(-1, 0, path1, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 more docs
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ long primaryVersion2 = primary.flush(0);
+ assertTrue(primaryVersion2 > primaryVersion1);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion2, 20);
+
+ primary.close();
+ replica.close();
+ }
+
+ // Crash primary and then restart it, while a replica node is down, then bring replica node back up and make sure it properly "unforks" itself
+ public void testCrashPrimary3() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ replica.commit();
+
+ replica.close();
+ primary.crash();
+
+ // At this point replica is "in the future": it has 10 docs committed, but the primary crashed before committing so it has 0 docs
+
+ // Restart primary:
+ primary = startNode(-1, 0, path1, -1, true);
+
+ // Index 20 docs into primary:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<20;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Flush primary, but there are no replicas to sync to:
+ long primaryVersion2 = primary.flush(0);
+
+ // Now restart replica, which on init should detect on a "lost branch" because its 10 docs that were committed came from a different
+ // primary node:
+ replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ assertVersionAndHits(replica, primaryVersion2, 20);
+
+ primary.close();
+ replica.close();
+ }
+
+ public void testCrashPrimaryWhileCopying() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 100 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<100;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes (async) to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ Thread.sleep(TestUtil.nextInt(random(), 1, 30));
+
+ // Crash primary, likely/hopefully while replica is still copying
+ primary.crash();
+
+ // Could see either 100 docs (replica finished before crash) or 0 docs:
+ try (Connection c = new Connection(replica.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ c.flush();
+ long version = c.in.readVLong();
+ int hitCount = c.in.readVInt();
+ if (version == 0) {
+ assertEquals(0, hitCount);
+ } else {
+ assertEquals(primaryVersion1, version);
+ assertEquals(100, hitCount);
+ }
+ }
+
+ primary.close();
+ replica.close();
+ }
+
+ public void testCrashReplica() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Index 10 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replica:
+ long primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+
+ // Wait for replica to sync up:
+ waitForVersionAndHits(replica, primaryVersion1, 10);
+
+ // Crash replica
+ replica.crash();
+
+ sendReplicasToPrimary(primary);
+
+ // Lots of new flushes while replica is down:
+ long primaryVersion2 = 0;
+ for(int iter=0;iter<10;iter++) {
+ // Index 10 docs into primary:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+ primaryVersion2 = primary.flush(0);
+ }
+
+ // Start up replica again:
+ replica = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ sendReplicasToPrimary(primary, replica);
+
+ // Now ask replica to sync:
+ replica.newNRTPoint(primaryVersion2, primary.tcpPort);
+
+ // Make sure it sees all docs that were indexed while it was down:
+ assertVersionAndHits(primary, primaryVersion2, 110);
+
+ replica.close();
+ primary.close();
+ }
+
+ public void testFullClusterCrash() throws Exception {
+
+ Path path1 = createTempDir("1");
+ NodeProcess primary = startNode(-1, 0, path1, -1, true);
+
+ Path path2 = createTempDir("2");
+ NodeProcess replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
+
+ Path path3 = createTempDir("3");
+ NodeProcess replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
+
+ sendReplicasToPrimary(primary, replica1, replica2);
+
+ // Index 50 docs into primary:
+ LineFileDocs docs = new LineFileDocs(random());
+ long primaryVersion1 = 0;
+ for (int iter=0;iter<5;iter++) {
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Refresh primary, which also pushes to replicas:
+ primaryVersion1 = primary.flush(0);
+ assertTrue(primaryVersion1 > 0);
+ }
+
+ // Wait for replicas to sync up:
+ waitForVersionAndHits(replica1, primaryVersion1, 50);
+ waitForVersionAndHits(replica2, primaryVersion1, 50);
+
+ primary.commit();
+ replica1.commit();
+ replica2.commit();
+
+ // Index 10 more docs, but don't sync to replicas:
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ for(int i=0;i<10;i++) {
+ Document doc = docs.nextDoc();
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+ }
+
+ // Full cluster crash
+ primary.crash();
+ replica1.crash();
+ replica2.crash();
+
+ // Full cluster restart
+ primary = startNode(-1, 0, path1, -1, true);
+ replica1 = startNode(primary.tcpPort, 1, path2, -1, true);
+ replica2 = startNode(primary.tcpPort, 2, path3, -1, true);
+
+ // Only 50 because we didn't commit primary before the crash:
+
+ // It's -1 because it's unpredictable how IW changes segments version on init:
+ assertVersionAndHits(primary, -1, 50);
- assertVersionAndHits(replica1, primaryVersion1, 50);
- assertVersionAndHits(replica2, primaryVersion1, 50);
++ assertVersionAndHits(replica1, primary.initInfosVersion, 50);
++ assertVersionAndHits(replica2, primary.initInfosVersion, 50);
+
+ primary.close();
+ replica1.close();
+ replica2.close();
+ }
+
+ /** Tell primary current replicas. */
+ private void sendReplicasToPrimary(NodeProcess primary, NodeProcess... replicas) throws IOException {
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+ c.out.writeVInt(replicas.length);
+ for(int id=0;id<replicas.length;id++) {
+ NodeProcess replica = replicas[id];
+ c.out.writeVInt(replica.id);
+ c.out.writeVInt(replica.tcpPort);
+ }
+ c.flush();
+ c.in.readByte();
+ }
+ }
+
+ /** Verifies this node is currently searching the specified version with the specified total hit count, or that it eventually does when
+ * keepTrying is true. */
+ private void assertVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
+ try (Connection c = new Connection(node.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ c.flush();
+ long version = c.in.readVLong();
+ int hitCount = c.in.readVInt();
+ if (expectedVersion != -1) {
- assertEquals("hitCount=" + hitCount, expectedVersion, version);
++ assertEquals("wrong searcher version, with hitCount=" + hitCount, expectedVersion, version);
+ }
+ assertEquals(expectedHitCount, hitCount);
+ }
+ }
+
+ private void waitForVersionAndHits(NodeProcess node, long expectedVersion, int expectedHitCount) throws Exception {
+ try (Connection c = new Connection(node.tcpPort)) {
+ while (true) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH_ALL);
+ c.flush();
+ long version = c.in.readVLong();
+ int hitCount = c.in.readVInt();
+
+ if (version == expectedVersion) {
+ assertEquals(expectedHitCount, hitCount);
+ break;
+ }
+
+ assertTrue(version < expectedVersion);
+ Thread.sleep(10);
+ }
+ }
+ }
+
+ static void message(String message) {
+ long now = System.nanoTime();
+ System.out.println(String.format(Locale.ROOT,
+ "%5.3fs : parent [%11s] %s",
+ (now-Node.globalStartNS)/1000000000.,
+ Thread.currentThread().getName(),
+ message));
+ }
+}
[06/15] lucene-solr git commit: fix more nocommits;
add separate test that deleteAll can replicate
Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/022540e8/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
new file mode 100644
index 0000000..271c5d2
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java
@@ -0,0 +1,1160 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedReader;
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.UnsupportedEncodingException;
+import java.io.Writer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+import com.carrotsearch.randomizedtesting.SeedUtils;
+
+/*
+ TODO
+ - fangs
+ - sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc.
+ - why do we do the "rename temp to actual" all at the end...? what really does that buy us?
+ - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary?
+ - test should not print scary exceptions and then succeed!
+ - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp...
+ - are the pre-copied-completed-merged files not being cleared in primary?
+ - hmm the logic isn't right today? a replica may skip pulling a given copy state, that recorded the finished merged segments?
+ - beast & fix bugs
+ - graceful cluster restart
+ - better translog integration
+ - get "graceful primary shutdown" working
+ - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion
+ - clean up how version is persisted in commit data
+ - why am i not using hashes here? how does ES use them?
+ - get all other "single shard" functions working too: this cluster should "act like" a single shard
+ - SLM
+ - controlled nrt reopen thread / returning long gen on write
+ - live field values
+ - add indexes
+ - make cluster level APIs to search, index, that deal w/ primary failover, etc.
+ - must prune xlog
+ - refuse to start primary unless we have quorum
+ - later
+ - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback?
+ - back pressure on indexing if replicas can't keep up?
+ - get xlog working on top? needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit
+ quorum
+ - maybe fix IW to return "gen" or "seq id" or "segment name" or something?
+ - replica can copy files from other replicas too / use multicast / rsync / something
+ - each replica could also pre-open a SegmentReader after pre-copy when warming a merge
+ - we can pre-copy newly flushed files too, for cases where reopen rate is low vs IW's flushing because RAM buffer is full
+ - opto: pre-copy files as they are written; if they will become CFS, we can build CFS on the replica?
+ - what about multiple commit points?
+ - fix primary to init directly from an open replica, instead of having to commit/close the replica first
+*/
+
+// Tricky cases:
+// - we are pre-copying a merge, then replica starts up part way through, so it misses that pre-copy and must do it on next nrt point
+// - a down replica starts up, but it's "from the future" vs the current primary, and must copy over file names with different contents
+// but referenced by its latest commit point, so it must fully remove that commit ... which is a hazardous window
+// - replica comes up just as the primary is crashing / moving
+// - electing a new primary when a replica is just finishing its nrt sync: we need to wait for it so we are sure to get the "most up to
+// date" replica
+// - replica comes up after merged segment finished so it doesn't copy over the merged segment "promptly" (i.e. only sees it on NRT refresh)
+
+/**
+ * Test case showing how to implement NRT replication. This test spawns a sub-process per-node, running TestNRTReplicationChild.
+ *
+ * One node is primary, and segments are periodically flushed there, then concurrently the N replica nodes copy the new files over and open new readers, while
+ * primary also opens a new reader.
+ *
+ * Nodes randomly crash and are restarted. If the primary crashes, a replica is promoted.
+ *
+ * Merges are currently first finished on the primary and then pre-copied out to replicas with a merged segment warmer so they don't block
+ * ongoing NRT reopens. Probably replicas could do their own merging instead, but this is more complex and may not be better overall
+ * (merging takes a lot of IO resources).
+ *
+ * Slow network is simulated with a RateLimiter.
+ */
+
+// MockRandom's .sd file has no index header/footer:
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class TestStressNRTReplication extends LuceneTestCase {
+
+ // Test evilness controls:
+
+ /** Randomly crash the current primary (losing data!) and promote the "next best" replica. */
+ static final boolean DO_CRASH_PRIMARY = true;
+
+ /** Randomly crash (JVM core dumps) a replica; it will later randomly be restarted and sync itself. */
+ static final boolean DO_CRASH_REPLICA = true;
+
+ /** Randomly gracefully close a replica; it will later be restarted and sync itself. */
+ static final boolean DO_CLOSE_REPLICA = true;
+
+ /** If false, all child + parent output is interleaved into single stdout/err */
+ static final boolean SEPARATE_CHILD_OUTPUT = false;
+
+ // nocommit DO_CLOSE_PRIMARY?
+
+ /** Randomly crash whole cluster and then restart it */
+ static final boolean DO_FULL_CLUSTER_CRASH = true;
+
+ /** True if we randomly flip a bit while copying files out */
+ static final boolean DO_BIT_FLIPS_DURING_COPY = true;
+
+ /** Set to a non-null value to force exactly that many nodes; else, it's random. */
+ static final Integer NUM_NODES = null;
+
+ static final boolean DO_RANDOM_XLOG_REPLAY = false;
+
+ final AtomicBoolean failed = new AtomicBoolean();
+
+ final AtomicBoolean stop = new AtomicBoolean();
+
+ /** cwd where we start each child (server) node */
+ private Path childTempDir;
+
+ long primaryGen;
+
+ volatile long lastPrimaryVersion;
+
+ volatile NodeProcess primary;
+ volatile NodeProcess[] nodes;
+ volatile long[] nodeTimeStamps;
+ volatile boolean[] starting;
+
+ Path[] indexPaths;
+
+ Path transLogPath;
+ SimpleTransLog transLog;
+ final AtomicInteger markerUpto = new AtomicInteger();
+
+ /** Maps searcher version to how many hits the query body:the matched. */
+ final Map<Long,Integer> hitCounts = new ConcurrentHashMap<>();
+
+ /** Maps searcher version to how many marker documents matched. This should only ever grow (we never delete marker documents). */
+ final Map<Long,Integer> versionToMarker = new ConcurrentHashMap<>();
+
+ /** Maps searcher version to xlog location when refresh of this version started. */
+ final Map<Long,Long> versionToTransLogLocation = new ConcurrentHashMap<>();
+
+ final AtomicLong nodeStartCounter = new AtomicLong();
+
+ final Set<Integer> crashingNodes = Collections.synchronizedSet(new HashSet<>());
+
+ public void test() throws Exception {
+
+ Node.globalStartNS = System.nanoTime();
+
+ message("change thread name from " + Thread.currentThread().getName());
+ Thread.currentThread().setName("main");
+
+ childTempDir = createTempDir("child");
+
+ // We are parent process:
+
+ // Silly bootstrapping:
+ versionToTransLogLocation.put(0L, 0L);
+ versionToTransLogLocation.put(1L, 0L);
+
+ int numNodes;
+
+ if (NUM_NODES == null) {
+ numNodes = TestUtil.nextInt(random(), 2, 10);
+ } else {
+ numNodes = NUM_NODES.intValue();
+ }
+
+ System.out.println("TEST: using " + numNodes + " nodes");
+
+ transLogPath = createTempDir("NRTReplication").resolve("translog");
+ transLog = new SimpleTransLog(transLogPath);
+
+ //state.rateLimiters = new RateLimiter[numNodes];
+ indexPaths = new Path[numNodes];
+ nodes = new NodeProcess[numNodes];
+ nodeTimeStamps = new long[numNodes];
+ Arrays.fill(nodeTimeStamps, Node.globalStartNS);
+ starting = new boolean[numNodes];
+
+ for(int i=0;i<numNodes;i++) {
+ indexPaths[i] = createTempDir("index" + i);
+ }
+
+ Thread[] indexers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+ System.out.println("TEST: launch " + indexers.length + " indexer threads");
+ for(int i=0;i<indexers.length;i++) {
+ indexers[i] = new IndexThread();
+ indexers[i].setName("indexer" + i);
+ indexers[i].setDaemon(true);
+ indexers[i].start();
+ }
+
+ Thread[] searchers = new Thread[TestUtil.nextInt(random(), 1, 3)];
+ System.out.println("TEST: launch " + searchers.length + " searcher threads");
+ for(int i=0;i<searchers.length;i++) {
+ searchers[i] = new SearchThread();
+ searchers[i].setName("searcher" + i);
+ searchers[i].setDaemon(true);
+ searchers[i].start();
+ }
+
+ Thread restarter = new RestartThread();
+ restarter.setName("restarter");
+ restarter.setDaemon(true);
+ restarter.start();
+
+ int runTimeSec;
+ if (TEST_NIGHTLY) {
+ runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 120, 240);
+ } else {
+ runTimeSec = RANDOM_MULTIPLIER * TestUtil.nextInt(random(), 45, 120);
+ }
+
+ System.out.println("TEST: will run for " + runTimeSec + " sec");
+
+ long endTime = System.nanoTime() + runTimeSec*1000000000L;
+
+ sendReplicasToPrimary();
+
+ while (failed.get() == false && System.nanoTime() < endTime) {
+
+ // Wait a bit:
+ Thread.sleep(TestUtil.nextInt(random(), Math.min(runTimeSec*4, 200), runTimeSec*4));
+ if (primary != null && random().nextBoolean()) {
+ message("top: now flush primary");
+ NodeProcess curPrimary = primary;
+ if (curPrimary != null) {
+
+ // Save these before we start flush:
+ long nextTransLogLoc = transLog.getNextLocation();
+ int markerUptoSav = markerUpto.get();
+
+ long result;
+ try {
+ result = primary.flush();
+ } catch (Throwable t) {
+ message("top: flush failed; skipping: " + t.getMessage());
+ result = -1;
+ }
+ if (result > 0) {
+ // There were changes
+ lastPrimaryVersion = result;
+ addTransLogLoc(lastPrimaryVersion, nextTransLogLoc);
+ addVersionMarker(lastPrimaryVersion, markerUptoSav);
+ }
+ }
+ }
+
+ StringBuilder sb = new StringBuilder();
+ int liveCount = 0;
+ for(int i=0;i<nodes.length;i++) {
+ NodeProcess node = nodes[i];
+ if (node != null) {
+ if (sb.length() != 0) {
+ sb.append(" ");
+ }
+ liveCount++;
+ if (node.isPrimary) {
+ sb.append('P');
+ } else {
+ sb.append('R');
+ }
+ sb.append(i);
+ }
+ }
+
+ message("PG=" + (primary == null ? "X" : primaryGen) + " " + liveCount + " (of " + nodes.length + ") nodes running: " + sb);
+
+ // Commit a random node, primary or replica
+
+ {
+ NodeProcess node = nodes[random().nextInt(nodes.length)];
+ if (node != null) {
+ // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters.
+ // maybe we somehow allow IW to commit a specific sis (the one we just flushed)?
+ message("top: now commit node=" + node);
+ node.commitAsync();
+ }
+ }
+ }
+
+ message("TEST: top: test done, now close");
+ stop.set(true);
+ for(Thread thread : indexers) {
+ thread.join();
+ }
+ for(Thread thread : searchers) {
+ thread.join();
+ }
+ restarter.join();
+
+ // Close replicas before primary so we cancel any in-progres replications:
+ System.out.println("TEST: top: now close replicas");
+ List<Closeable> toClose = new ArrayList<>();
+ for(NodeProcess node : nodes) {
+ if (node != primary && node != null) {
+ toClose.add(node);
+ }
+ }
+ IOUtils.close(toClose);
+ IOUtils.close(primary);
+ IOUtils.close(transLog);
+
+ if (failed.get() == false) {
+ message("TEST: top: now checkIndex");
+ for(Path path : indexPaths) {
+ message("TEST: check " + path);
+ MockDirectoryWrapper dir = newMockFSDirectory(path);
+ // Just too slow otherwise
+ dir.setCrossCheckTermVectorsOnClose(false);
+ dir.close();
+ }
+ } else {
+ message("TEST: failed; skip checkIndex");
+ }
+ }
+
+ private boolean anyNodesStarting() {
+ for(int id=0;id<nodes.length;id++) {
+ if (starting[id]) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+ /** Picks a replica and promotes it as new primary. */
+ private void promoteReplica() throws IOException {
+ message("top: primary crashed; now pick replica to promote");
+ long maxSearchingVersion = -1;
+ NodeProcess replicaToPromote = null;
+
+ // We must promote the most current replica, because otherwise file name reuse can cause a replication to fail when it needs to copy
+ // over a file currently held open for searching. This also minimizes recovery work since the most current replica means less xlog
+ // replay to catch up:
+ for (NodeProcess node : nodes) {
+ if (node != null) {
+ message("ask " + node + " for its current searching version");
+ long searchingVersion = node.getSearchingVersion();
+ message(node + " has searchingVersion=" + searchingVersion);
+ if (searchingVersion > maxSearchingVersion) {
+ maxSearchingVersion = searchingVersion;
+ replicaToPromote = node;
+ }
+ }
+ }
+
+ if (replicaToPromote == null) {
+ message("top: no replicas running; skipping primary promotion");
+ return;
+ }
+
+ message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit");
+ if (replicaToPromote.commit() == false) {
+ message("top: commit failed; skipping primary promotion");
+ return;
+ }
+
+ message("top: now shutdown " + replicaToPromote);
+ if (replicaToPromote.shutdown() == false) {
+ message("top: shutdown failed for R" + replicaToPromote.id + "; skipping primary promotion");
+ return;
+ }
+
+ int id = replicaToPromote.id;
+ message("top: now startPrimary " + replicaToPromote);
+ startPrimary(replicaToPromote.id);
+ }
+
+ void startPrimary(int id) throws IOException {
+ message(id + ": top: startPrimary lastPrimaryVersion=" + lastPrimaryVersion);
+ assert nodes[id] == null;
+
+ // Force version of new primary to advance beyond where old primary was, so we never re-use versions. It may have
+ // already advanced beyond newVersion, e.g. if it flushed new segments while during xlog replay:
+
+ // First start node as primary (it opens an IndexWriter) but do not publish it for searching until we replay xlog:
+ NodeProcess newPrimary = startNode(id, indexPaths[id], true, lastPrimaryVersion+1);
+ if (newPrimary == null) {
+ message("top: newPrimary failed to start; abort");
+ return;
+ }
+
+ // Get xlog location that this node was guaranteed to already have indexed through; this may replay some ops already indexed but it's OK
+ // because the ops are idempotent: we updateDocument (by docid) on replay even for original addDocument:
+ Long startTransLogLoc;
+ Integer markerCount;
+ if (newPrimary.initCommitVersion == 0) {
+ startTransLogLoc = 0L;
+ markerCount = 0;
+ } else {
+ startTransLogLoc = versionToTransLogLocation.get(newPrimary.initCommitVersion);
+ markerCount = versionToMarker.get(newPrimary.initCommitVersion);
+ }
+ assert startTransLogLoc != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToTransLogLocation: keys=" + versionToTransLogLocation.keySet();
+ assert markerCount != null: "newPrimary.initCommitVersion=" + newPrimary.initCommitVersion + " is missing from versionToMarker: keys=" + versionToMarker.keySet();
+
+ // When the primary starts, the userData in its latest commit point tells us which version it had indexed up to, so we know where to
+ // replay from in the xlog. However, we forcefuly advance the version, and then IW on init (or maybe getReader) also adds 1 to it.
+ // Since we publish the primary in this state (before xlog replay is done), a replica can start up at this point and pull this version,
+ // and possibly later be chosen as a primary, causing problems if the version is known recorded in the translog map. So we record it
+ // here:
+
+ addTransLogLoc(newPrimary.initInfosVersion, startTransLogLoc);
+ addVersionMarker(newPrimary.initInfosVersion, markerCount);
+
+ assert newPrimary.initInfosVersion >= lastPrimaryVersion;
+ message("top: now change lastPrimaryVersion from " + lastPrimaryVersion + " to " + newPrimary.initInfosVersion);
+ lastPrimaryVersion = newPrimary.initInfosVersion;
+
+ // Publish new primary, before replaying xlog. This means other indexing ops can come in at the same time as we catch up indexing
+ // previous ops. Effectively, we have "forked" the indexing ops, by rolling back in time a bit, and replaying old indexing ops (from
+ // translog) concurrently with new incoming ops.
+ nodes[id] = newPrimary;
+ primary = newPrimary;
+
+ sendReplicasToPrimary();
+
+ long nextTransLogLoc = transLog.getNextLocation();
+ int nextMarkerUpto = markerUpto.get();
+ message("top: replay trans log " + startTransLogLoc + " (version=" + newPrimary.initCommitVersion + ") to " + nextTransLogLoc + " (translog end)");
+ try {
+ transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc);
+ } catch (IOException ioe) {
+ message("top: replay xlog failed; abort");
+ return;
+ }
+ message("top: done replay trans log");
+ }
+
+ /** Launches a child "server" (separate JVM), which is either primary or replica node */
+ NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException {
+ nodeTimeStamps[id] = System.nanoTime();
+ List<String> cmd = new ArrayList<>();
+
+ NodeProcess curPrimary = primary;
+
+ cmd.add(System.getProperty("java.home")
+ + System.getProperty("file.separator")
+ + "bin"
+ + System.getProperty("file.separator")
+ + "java");
+ cmd.add("-Xmx512m");
+
+ if (curPrimary != null) {
+ cmd.add("-Dtests.nrtreplication.primaryTCPPort=" + curPrimary.tcpPort);
+ } else if (isPrimary == false) {
+ // We cannot start a replica when there is no primary:
+ return null;
+ }
+
+ cmd.add("-Dtests.nrtreplication.node=true");
+ cmd.add("-Dtests.nrtreplication.nodeid=" + id);
+ cmd.add("-Dtests.nrtreplication.startNS=" + Node.globalStartNS);
+ cmd.add("-Dtests.nrtreplication.indexpath=" + indexPath);
+ if (isPrimary) {
+ cmd.add("-Dtests.nrtreplication.isPrimary=true");
+ cmd.add("-Dtests.nrtreplication.forcePrimaryVersion=" + forcePrimaryVersion);
+ if (DO_CRASH_PRIMARY) {
+ cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
+ }
+ } else {
+ if (DO_CRASH_REPLICA) {
+ cmd.add("-Dtests.nrtreplication.doRandomCrash=true");
+ }
+ if (DO_CLOSE_REPLICA) {
+ cmd.add("-Dtests.nrtreplication.doRandomClose=true");
+ }
+ }
+
+ if (DO_BIT_FLIPS_DURING_COPY) {
+ cmd.add("-Dtests.nrtreplication.doFlipBitsDuringCopy=true");
+ }
+
+ long myPrimaryGen = primaryGen;
+ cmd.add("-Dtests.nrtreplication.primaryGen=" + myPrimaryGen);
+
+ // Mixin our own counter because this is called from a fresh thread which means the seed otherwise isn't changing each time we spawn a
+ // new node:
+ long seed = random().nextLong() * nodeStartCounter.incrementAndGet();
+
+ cmd.add("-Dtests.seed=" + SeedUtils.formatSeed(seed));
+ cmd.add("-ea");
+ cmd.add("-cp");
+ cmd.add(System.getProperty("java.class.path"));
+ cmd.add("org.junit.runner.JUnitCore");
+ cmd.add(getClass().getName().replace(getClass().getSimpleName(), "SimpleServer"));
+
+ Writer childLog;
+
+ if (SEPARATE_CHILD_OUTPUT) {
+ Path childOut = childTempDir.resolve(id + ".log");
+ message("logging to " + childOut);
+ childLog = Files.newBufferedWriter(childOut, StandardCharsets.UTF_8, StandardOpenOption.APPEND, StandardOpenOption.CREATE);
+ childLog.write("\n\nSTART NEW CHILD:\n");
+ } else {
+ childLog = null;
+ }
+
+ message("child process command: " + cmd);
+ ProcessBuilder pb = new ProcessBuilder(cmd);
+ pb.redirectErrorStream(true);
+
+ // Important, so that the scary looking hs_err_<pid>.log appear under our test temp dir:
+ pb.directory(childTempDir.toFile());
+
+ Process p = pb.start();
+
+ BufferedReader r;
+ try {
+ r = new BufferedReader(new InputStreamReader(p.getInputStream(), IOUtils.UTF_8));
+ } catch (UnsupportedEncodingException uee) {
+ throw new RuntimeException(uee);
+ }
+
+ int tcpPort = -1;
+ long initCommitVersion = -1;
+ long initInfosVersion = -1;
+ Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*");
+ boolean willCrash = false;
+ boolean sawExistingSegmentsFile = false;
+
+ while (true) {
+ String l = r.readLine();
+ if (l == null) {
+ message("top: node=" + id + " failed to start");
+ try {
+ p.waitFor();
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ }
+ message("exit value=" + p.exitValue());
+
+ // Hackity hack, in case primary crashed/closed and we haven't noticed (reaped the process) yet:
+ if (isPrimary == false) {
+ if (sawExistingSegmentsFile) {
+ // This means MDW's virus checker blocked us from deleting segments_N that we must delete in order to start ... just return null
+ // and retry again later:
+ message("failed to remove segments_N; skipping");
+ return null;
+ }
+ for(int i=0;i<10;i++) {
+ if (primaryGen != myPrimaryGen || primary == null) {
+ // OK: primary crashed while we were trying to start, so it's expected/allowed that we could not start the replica:
+ message("primary crashed/closed while replica R" + id + " tried to start; skipping");
+ return null;
+ } else {
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
+ }
+ }
+ }
+ }
+
+ // Should fail the test:
+ message("top: now fail test replica R" + id + " failed to start");
+ failed.set(true);
+ throw new RuntimeException("replica R" + id + " failed to start");
+ }
+
+ if (childLog != null) {
+ childLog.write(l);
+ childLog.write("\n");
+ childLog.flush();
+ } else if (logTimeStart.matcher(l).matches()) {
+ // Already a well-formed log output:
+ System.out.println(l);
+ } else {
+ message(l);
+ }
+
+ if (l.startsWith("PORT: ")) {
+ tcpPort = Integer.parseInt(l.substring(6).trim());
+ } else if (l.startsWith("COMMIT VERSION: ")) {
+ initCommitVersion = Integer.parseInt(l.substring(16).trim());
+ } else if (l.startsWith("INFOS VERSION: ")) {
+ initInfosVersion = Integer.parseInt(l.substring(15).trim());
+ } else if (l.contains("will crash after")) {
+ willCrash = true;
+ } else if (l.startsWith("NODE STARTED")) {
+ break;
+ } else if (l.contains("replica cannot start: existing segments file=")) {
+ sawExistingSegmentsFile = true;
+ }
+ }
+
+ final boolean finalWillCrash = willCrash;
+
+ // Baby sits the child process, pulling its stdout and printing to our stdout, calling nodeClosed once it exits:
+ Thread pumper = ThreadPumper.start(
+ new Runnable() {
+ @Override
+ public void run() {
+ message("now wait for process " + p);
+ try {
+ p.waitFor();
+ } catch (Throwable t) {
+ throw new RuntimeException(t);
+ }
+
+ message("done wait for process " + p);
+ int exitValue = p.exitValue();
+ message("exit value=" + exitValue + " willCrash=" + finalWillCrash);
+ if (childLog != null) {
+ try {
+ childLog.write("process done; exitValue=" + exitValue + "\n");
+ childLog.close();
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ }
+ if (exitValue != 0 && finalWillCrash == false && crashingNodes.remove(id) == false) {
+ // should fail test
+ failed.set(true);
+ if (childLog != null) {
+ throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue + "; see " + childLog + " for details");
+ } else {
+ throw new RuntimeException("node " + id + " process had unexpected non-zero exit status=" + exitValue);
+ }
+ }
+ nodeClosed(id);
+ }
+ }, r, System.out, childLog);
+ pumper.setName("pump" + id);
+
+ message("top: node=" + id + " started at tcpPort=" + tcpPort + " initCommitVersion=" + initCommitVersion + " initInfosVersion=" + initInfosVersion);
+ return new NodeProcess(p, id, tcpPort, pumper, isPrimary, initCommitVersion, initInfosVersion);
+ }
+
+ private void nodeClosed(int id) {
+ NodeProcess oldNode = nodes[id];
+ if (primary != null && oldNode == primary) {
+ message("top: " + primary + ": primary process finished");
+ primary = null;
+ primaryGen++;
+ } else {
+ message("top: " + oldNode + ": replica process finished");
+ }
+ if (oldNode != null) {
+ oldNode.isOpen = false;
+ }
+ nodes[id] = null;
+ nodeTimeStamps[id] = System.nanoTime();
+
+ sendReplicasToPrimary();
+ }
+
+ /** Sends currently alive replicas to primary, which uses this to know who to notify when it does a refresh */
+ private void sendReplicasToPrimary() {
+ NodeProcess curPrimary = primary;
+ if (curPrimary != null) {
+ List<NodeProcess> replicas = new ArrayList<>();
+ for (NodeProcess node : nodes) {
+ if (node != null && node.isPrimary == false) {
+ replicas.add(node);
+ }
+ }
+
+ message("top: send " + replicas.size() + " replicas to primary");
+
+ try (Connection c = new Connection(curPrimary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_SET_REPLICAS);
+ c.out.writeVInt(replicas.size());
+ for(NodeProcess replica : replicas) {
+ c.out.writeVInt(replica.id);
+ c.out.writeVInt(replica.tcpPort);
+ }
+ c.flush();
+ c.in.readByte();
+ } catch (Throwable t) {
+ message("top: ignore exc sending replicas to primary: " + t);
+ }
+ }
+ }
+
+ void addVersionMarker(long version, int count) {
+ //System.out.println("ADD VERSION MARKER version=" + version + " count=" + count);
+ if (versionToMarker.containsKey(version)) {
+ int curCount = versionToMarker.get(version);
+ if (curCount != count) {
+ message("top: wrong marker count version=" + version + " count=" + count + " curCount=" + curCount);
+ throw new IllegalStateException("version=" + version + " count=" + count + " curCount=" + curCount);
+ }
+ } else {
+ message("top: record marker count: version=" + version + " count=" + count);
+ versionToMarker.put(version, count);
+ }
+ }
+
+ void addTransLogLoc(long version, long loc) {
+ message("top: record transLogLoc: version=" + version + " loc=" + loc);
+ versionToTransLogLocation.put(version, loc);
+ }
+
+ // Periodically wakes up and starts up any down nodes:
+ private class RestartThread extends Thread {
+ @Override
+ public void run() {
+
+ List<Thread> startupThreads = Collections.synchronizedList(new ArrayList<>());
+
+ try {
+ while (stop.get() == false) {
+ Thread.sleep(TestUtil.nextInt(random(), 50, 500));
+ message("top: restarter cycle");
+
+ // Randomly crash full cluster:
+ if (DO_FULL_CLUSTER_CRASH && random().nextInt(50) == 17) {
+ message("top: full cluster crash");
+ for(int i=0;i<nodes.length;i++) {
+ if (starting[i]) {
+ message("N" + i + ": top: wait for startup so we can crash...");
+ while (starting[i]) {
+ Thread.sleep(10);
+ }
+ message("N" + i + ": top: done wait for startup");
+ }
+ NodeProcess node = nodes[i];
+ if (node != null) {
+ crashingNodes.add(i);
+ message("top: N" + node.id + ": top: now crash node");
+ node.crash();
+ message("top: N" + node.id + ": top: done crash node");
+ }
+ }
+ }
+
+ List<Integer> downNodes = new ArrayList<>();
+ StringBuilder b = new StringBuilder();
+ long nowNS = System.nanoTime();
+ for(int i=0;i<nodes.length;i++) {
+ b.append(' ');
+ double sec = (nowNS - nodeTimeStamps[i])/1000000000.0;
+ String prefix;
+ if (nodes[i] == null) {
+ downNodes.add(i);
+ if (starting[i]) {
+ prefix = "s";
+ } else {
+ prefix = "x";
+ }
+ } else {
+ prefix = "";
+ }
+ if (primary != null && nodes[i] == primary) {
+ prefix += "p";
+ }
+ b.append(String.format(Locale.ROOT, "%s%d(%.1fs)", prefix, i, sec));
+ }
+ message("node status" + b.toString());
+ message("downNodes=" + downNodes);
+
+ // If primary is down, promote a replica:
+ if (primary == null) {
+ if (anyNodesStarting()) {
+ message("top: skip promote replica: nodes are still starting");
+ continue;
+ }
+ promoteReplica();
+ }
+
+ // Randomly start up a down a replica:
+
+ // Stop or start a replica
+ if (downNodes.isEmpty() == false) {
+ int idx = downNodes.get(random().nextInt(downNodes.size()));
+ if (starting[idx] == false) {
+ if (primary == null) {
+ if (downNodes.size() == nodes.length) {
+ // Cold start: entire cluster is down, start this node up as the new primary
+ message("N" + idx + ": top: cold start as primary");
+ startPrimary(idx);
+ }
+ } else if (random().nextDouble() < ((double) downNodes.size())/nodes.length) {
+ // Start up replica:
+ starting[idx] = true;
+ message("N" + idx + ": top: start up: launch thread");
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ try {
+ message("N" + idx + ": top: start up thread");
+ nodes[idx] = startNode(idx, indexPaths[idx], false, -1);
+ sendReplicasToPrimary();
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ } finally {
+ starting[idx] = false;
+ startupThreads.remove(Thread.currentThread());
+ }
+ }
+ };
+ t.setName("start R" + idx);
+ t.start();
+ startupThreads.add(t);
+ }
+ } else {
+ message("node " + idx + " still starting");
+ }
+ }
+ }
+
+ System.out.println("Restarter: now stop: join " + startupThreads.size() + " startup threads");
+
+ while (startupThreads.size() > 0) {
+ Thread.sleep(10);
+ }
+
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ }
+ }
+ }
+
+ /** Randomly picks a node and runs a search against it */
+ private class SearchThread extends Thread {
+
+ @Override
+ public void run() {
+ // Maps version to number of hits for silly 'the' TermQuery:
+ Query theQuery = new TermQuery(new Term("body", "the"));
+
+ // Persists connections
+ Map<Integer,Connection> connections = new HashMap<>();
+
+ while (stop.get() == false) {
+ NodeProcess node = nodes[random().nextInt(nodes.length)];
+ if (node == null || node.isOpen == false) {
+ continue;
+ }
+
+ if (node.lock.tryLock() == false) {
+ // Node is in the process of closing or crashing or something
+ continue;
+ }
+
+ try {
+
+ Thread.currentThread().setName("Searcher node=" + node);
+
+ //System.out.println("S: cycle; conns=" + connections);
+
+ Connection c = connections.get(node.id);
+
+ long version;
+ try {
+ if (c == null) {
+ //System.out.println("S: new connection " + node.id + " " + Thread.currentThread().getName());
+ c = new Connection(node.tcpPort);
+ connections.put(node.id, c);
+ } else {
+ //System.out.println("S: reuse connection " + node.id + " " + Thread.currentThread().getName());
+ }
+
+ c.out.writeByte(SimplePrimaryNode.CMD_SEARCH);
+ c.flush();
+
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node closed");
+ }
+ Thread.sleep(1);
+ }
+ version = c.in.readVLong();
+
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node closed");
+ }
+ Thread.sleep(1);
+ }
+ int hitCount = c.in.readVInt();
+
+ Integer oldHitCount = hitCounts.get(version);
+
+ // TODO: we never prune this map...
+ if (oldHitCount == null) {
+ hitCounts.put(version, hitCount);
+ message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node);
+ } else {
+ // Just ensure that all nodes show the same hit count for
+ // the same version, i.e. they really are replicas of one another:
+ if (oldHitCount.intValue() != hitCount) {
+ failed.set(true);
+ stop.set(true);
+ message("top: searcher: wrong version hitCount: version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+ fail("version=" + version + " oldHitCount=" + oldHitCount.intValue() + " hitCount=" + hitCount);
+ }
+ }
+ } catch (IOException ioe) {
+ //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+ //ioe.printStackTrace(System.out);
+ IOUtils.closeWhileHandlingException(c);
+ connections.remove(node.id);
+ continue;
+ }
+
+ // This can be null if we got the new primary after crash and that primary is still catching up (replaying xlog):
+ Integer expectedAtLeastHitCount = versionToMarker.get(version);
+
+ if (expectedAtLeastHitCount != null && expectedAtLeastHitCount > 0 && random().nextInt(10) == 7) {
+ try {
+ c.out.writeByte(SimplePrimaryNode.CMD_MARKER_SEARCH);
+ c.flush();
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node died");
+ }
+ Thread.sleep(1);
+ }
+
+ version = c.in.readVLong();
+
+ while (c.sockIn.available() == 0) {
+ if (stop.get()) {
+ break;
+ }
+ if (node.isOpen == false) {
+ throw new IOException("node died");
+ }
+ Thread.sleep(1);
+ }
+
+ int hitCount = c.in.readVInt();
+
+ // Look for data loss: make sure all marker docs are visible:
+
+ if (hitCount < expectedAtLeastHitCount) {
+
+ String failMessage = "node=" + node + ": documents were lost version=" + version + " hitCount=" + hitCount + " vs expectedAtLeastHitCount=" + expectedAtLeastHitCount;
+ message(failMessage);
+ failed.set(true);
+ stop.set(true);
+ fail(failMessage);
+ }
+ } catch (IOException ioe) {
+ //message("top: searcher: ignore exc talking to node " + node + ": " + ioe);
+ //throw new RuntimeException(ioe);
+ //ioe.printStackTrace(System.out);
+ IOUtils.closeWhileHandlingException(c);
+ connections.remove(node.id);
+ continue;
+ }
+ }
+
+ Thread.sleep(10);
+
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ } finally {
+ node.lock.unlock();
+ }
+ }
+ System.out.println("Searcher: now stop");
+ IOUtils.closeWhileHandlingException(connections.values());
+ }
+ }
+
+ private class IndexThread extends Thread {
+
+ @Override
+ public void run() {
+
+ try {
+ LineFileDocs docs = new LineFileDocs(random());
+ int docCount = 0;
+
+ // How often we do an update/delete vs add:
+ double updatePct = random().nextDouble();
+
+ // Varies how many docs/sec we index:
+ int sleepChance = TestUtil.nextInt(random(), 4, 100);
+
+ message("top: indexer: updatePct=" + updatePct + " sleepChance=" + sleepChance);
+
+ long lastTransLogLoc = transLog.getNextLocation();
+
+ NodeProcess curPrimary = null;
+ Connection c = null;
+
+ while (stop.get() == false) {
+
+ try {
+ while (stop.get() == false && curPrimary == null) {
+ Thread.sleep(10);
+ curPrimary = primary;
+ if (curPrimary != null) {
+ c = new Connection(curPrimary.tcpPort);
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ break;
+ }
+ }
+
+ if (stop.get()) {
+ break;
+ }
+
+ Thread.currentThread().setName("indexer p" + curPrimary.id);
+
+ if (random().nextInt(10) == 7) {
+ // We use the marker docs to check for data loss in search thread:
+ Document doc = new Document();
+ int id = markerUpto.getAndIncrement();
+ String idString = "m"+id;
+ doc.add(newStringField("docid", idString, Field.Store.YES));
+ doc.add(newStringField("marker", "marker", Field.Store.YES));
+ curPrimary.addOrUpdateDocument(c, doc, false);
+ transLog.addDocument(idString, doc);
+ message("index marker=" + idString + "; translog is " + Node.bytesToString(Files.size(transLogPath)));
+ }
+
+ if (docCount > 0 && random().nextDouble() < updatePct) {
+ int randomID = random().nextInt(docCount);
+ String randomIDString = Integer.toString(randomID);
+ if (random().nextBoolean()) {
+ // Replace previous doc
+ Document doc = docs.nextDoc();
+ ((Field) doc.getField("docid")).setStringValue(randomIDString);
+ curPrimary.addOrUpdateDocument(c, doc, true);
+ transLog.updateDocument(randomIDString, doc);
+ } else {
+ // Delete previous doc
+ curPrimary.deleteDocument(c, randomIDString);
+ transLog.deleteDocuments(randomIDString);
+ }
+ } else {
+ // Add new doc:
+ Document doc = docs.nextDoc();
+ String idString = Integer.toString(docCount++);
+ ((Field) doc.getField("docid")).setStringValue(idString);
+ curPrimary.addOrUpdateDocument(c, doc, false);
+ transLog.addDocument(idString, doc);
+
+ if (DO_RANDOM_XLOG_REPLAY && random().nextInt(10) == 7) {
+ long curLoc = transLog.getNextLocation();
+ // randomly replay chunks of translog just to test replay:
+ message("now randomly replay translog from " + lastTransLogLoc + " to " + curLoc);
+ transLog.replay(curPrimary, lastTransLogLoc, curLoc);
+ lastTransLogLoc = curLoc;
+ }
+ }
+ } catch (IOException se) {
+ // Assume primary crashed
+ message("top: indexer lost connection to primary");
+ try {
+ c.close();
+ } catch (Throwable t) {
+ }
+ curPrimary = null;
+ c = null;
+ }
+
+ if (random().nextInt(sleepChance) == 0) {
+ Thread.sleep(1);
+ }
+
+ if (random().nextInt(100) == 17) {
+ System.out.println("Indexer: now pause for a bit...");
+ Thread.sleep(TestUtil.nextInt(random(), 500, 2000));
+ System.out.println("Indexer: done pause for a bit...");
+ }
+ }
+ if (curPrimary != null) {
+ try {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
+ c.flush();
+ c.in.readByte();
+ } catch (IOException se) {
+ // Assume primary crashed
+ message("top: indexer lost connection to primary");
+ try {
+ c.close();
+ } catch (Throwable t) {
+ }
+ curPrimary = null;
+ c = null;
+ }
+ }
+ System.out.println("Indexer: now stop");
+ } catch (Throwable t) {
+ failed.set(true);
+ stop.set(true);
+ throw new RuntimeException(t);
+ }
+ }
+ }
+
+ static void message(String message) {
+ long now = System.nanoTime();
+ System.out.println(String.format(Locale.ROOT,
+ "%5.3fs : parent [%11s] %s",
+ (now-Node.globalStartNS)/1000000000.,
+ Thread.currentThread().getName(),
+ message));
+ }
+}
[02/15] lucene-solr git commit: current patch
Posted by mi...@apache.org.
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
new file mode 100644
index 0000000..4e29508
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java
@@ -0,0 +1,238 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.net.UnknownHostException;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+
+/** Parent JVM hold this "wrapper" to refer to each child JVM. This is roughly equivalent e.g. to a client-side "sugar" API. */
+class NodeProcess implements Closeable {
+ final Process p;
+
+ // Port sub-process is listening on
+ final int tcpPort;
+
+ final int id;
+
+ final Thread pumper;
+
+ // Acquired when searching or indexing wants to use this node:
+ final ReentrantLock lock;
+
+ final boolean isPrimary;
+
+ // Version in the commit point we opened on init:
+ final long initCommitVersion;
+
+ // SegmentInfos.version, which can be higher than the initCommitVersion
+ final long initInfosVersion;
+
+ volatile boolean isOpen = true;
+
+ public NodeProcess(Process p, int id, int tcpPort, Thread pumper, boolean isPrimary, long initCommitVersion, long initInfosVersion) {
+ this.p = p;
+ this.id = id;
+ this.tcpPort = tcpPort;
+ this.pumper = pumper;
+ this.isPrimary = isPrimary;
+ this.initCommitVersion = initCommitVersion;
+ this.initInfosVersion = initInfosVersion;
+ assert initInfosVersion >= initCommitVersion: "initInfosVersion=" + initInfosVersion + " initCommitVersion=" + initCommitVersion;
+ lock = new ReentrantLock();
+ }
+
+ @Override
+ public String toString() {
+ if (isPrimary) {
+ return "P" + id + " tcpPort=" + tcpPort;
+ } else {
+ return "R" + id + " tcpPort=" + tcpPort;
+ }
+ }
+
+ public synchronized void crash() {
+ if (isOpen) {
+ isOpen = false;
+ p.destroy();
+ try {
+ pumper.join();
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(ie);
+ }
+ }
+ }
+
+ public boolean commit() {
+ try (Connection c = new Connection(tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
+ c.flush();
+ c.s.shutdownOutput();
+ if (c.in.readByte() != 1) {
+ throw new RuntimeException("commit failed");
+ }
+ return true;
+ } catch (Throwable t) {
+ // Something wrong with this replica; skip it:
+ System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
+ return false;
+ }
+ }
+
+ public void commitAsync() {
+ try (Connection c = new Connection(tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_COMMIT);
+ c.flush();
+ } catch (Throwable t) {
+ // Something wrong with this replica; skip it:
+ System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping");
+ }
+ }
+
+ public long getSearchingVersion() {
+ try (Connection c = new Connection(tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_GET_SEARCHING_VERSION);
+ c.flush();
+ c.s.shutdownOutput();
+ return c.in.readVLong();
+ } catch (Throwable t) {
+ // Something wrong with this replica; skip it:
+ System.out.println("PARENT: top: hit SocketException during getSearchingVersion with R" + id + "; skipping");
+ return -1L;
+ }
+ }
+
+ /** Ask the primary node process to flush. We send it all currently up replicas so it can notify them about the new NRT point. Returns the newly
+ * flushed version, or a negative (current) version if there were no changes. */
+ public synchronized long flush() throws IOException {
+ assert isPrimary;
+ try (Connection c = new Connection(tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_FLUSH);
+ c.flush();
+ c.s.shutdownOutput();
+ return c.in.readLong();
+ }
+ }
+
+ @Override
+ public void close() {
+ shutdown();
+ }
+
+ public synchronized boolean shutdown() {
+ lock.lock();
+ try {
+ System.out.println("PARENT: now shutdown node=" + id + " isOpen=" + isOpen);
+ if (isOpen) {
+ // Ask the child process to shutdown gracefully:
+ isOpen = false;
+ System.out.println("PARENT: send CMD_CLOSE to node=" + id);
+ try (Connection c = new Connection(tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_CLOSE);
+ c.flush();
+ if (c.in.readByte() != 1) {
+ throw new RuntimeException("shutdown failed");
+ }
+ } catch (Throwable t) {
+ System.out.println("top: shutdown failed; skipping");
+ t.printStackTrace(System.out);
+ return false;
+ }
+ try {
+ p.waitFor();
+ pumper.join();
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(ie);
+ }
+ }
+
+ return true;
+ } finally {
+ lock.unlock();
+ }
+ }
+
+ public void addOrUpdateDocument(Connection c, Document doc, boolean isUpdate) throws IOException {
+ if (isPrimary == false) {
+ throw new IllegalStateException("only primary can index");
+ }
+ int fieldCount = 0;
+
+ String title = doc.get("title");
+ if (title != null) {
+ fieldCount++;
+ }
+
+ String docid = doc.get("docid");
+ assert docid != null;
+ fieldCount++;
+
+ String body = doc.get("body");
+ if (body != null) {
+ fieldCount++;
+ }
+
+ String marker = doc.get("marker");
+ if (marker != null) {
+ fieldCount++;
+ }
+
+ c.out.writeByte(isUpdate ? SimplePrimaryNode.CMD_UPDATE_DOC : SimplePrimaryNode.CMD_ADD_DOC);
+ c.out.writeVInt(fieldCount);
+ c.out.writeString("docid");
+ c.out.writeString(docid);
+ if (title != null) {
+ c.out.writeString("title");
+ c.out.writeString(title);
+ }
+ if (body != null) {
+ c.out.writeString("body");
+ c.out.writeString(body);
+ }
+ if (marker != null) {
+ c.out.writeString("marker");
+ c.out.writeString(marker);
+ }
+ c.flush();
+ c.in.readByte();
+ }
+
+ public void deleteDocument(Connection c, String docid) throws IOException {
+ if (isPrimary == false) {
+ throw new IllegalStateException("only primary can index");
+ }
+ c.out.writeByte(SimplePrimaryNode.CMD_DELETE_DOC);
+ c.out.writeString(docid);
+ c.flush();
+ c.in.readByte();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
new file mode 100644
index 0000000..1180967
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java
@@ -0,0 +1,294 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.nio.file.Files;
+import java.nio.file.NoSuchFileException;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.store.OutputStreamIndexOutput;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Handles one set of files that need copying, either because we have a
+ * new NRT point, or we are pre-copying merged files for merge warming. */
+class SimpleCopyJob extends CopyJob {
+ final Connection c;
+
+ final byte[] copyBuffer = new byte[65536];
+ final CopyState copyState;
+
+ private Iterator<Map.Entry<String,FileMetaData>> iter;
+
+ public SimpleCopyJob(String reason, Connection c, CopyState copyState, SimpleReplicaNode dest, Map<String,FileMetaData> files, boolean highPriority, OnceDone onceDone)
+ throws IOException {
+ super(reason, files, dest, highPriority, onceDone);
+ dest.message("create SimpleCopyJob o" + ord);
+ this.c = c;
+ this.copyState = copyState;
+ }
+
+ @Override
+ public synchronized void start() throws IOException {
+ if (iter == null) {
+ iter = toCopy.iterator();
+
+ // Send all file names / offsets up front to avoid ping-ping latency:
+ try {
+
+ // This means we resumed an already in-progress copy; we do this one first:
+ if (current != null) {
+ c.out.writeByte((byte) 0);
+ c.out.writeString(current.name);
+ c.out.writeVLong(current.getBytesCopied());
+ totBytes += current.metaData.length;
+ }
+
+ for (Map.Entry<String,FileMetaData> ent : toCopy) {
+ String fileName = ent.getKey();
+ FileMetaData metaData = ent.getValue();
+ totBytes += metaData.length;
+ c.out.writeByte((byte) 0);
+ c.out.writeString(fileName);
+ c.out.writeVLong(0);
+ }
+ c.out.writeByte((byte) 1);
+ c.flush();
+ c.s.shutdownOutput();
+
+ if (current != null) {
+ // Do this only at the end, after sending all requested files, so we don't deadlock due to socket buffering waiting for primary to
+ // send us this length:
+ long len = c.in.readVLong();
+ if (len != current.metaData.length) {
+ throw new IllegalStateException("file " + current.name + ": meta data says length=" + current.metaData.length + " but c.in says " + len);
+ }
+ }
+
+ dest.message("SimpleCopyJob.init: done start files count=" + toCopy.size() + " totBytes=" + totBytes);
+
+ } catch (Throwable t) {
+ cancel("exc during start", t);
+ throw new NodeCommunicationException("exc during start", t);
+ }
+ } else {
+ throw new IllegalStateException("already started");
+ }
+ }
+
+ @Override
+ public long getTotalBytesCopied() {
+ return totBytesCopied;
+ }
+
+ @Override
+ public Set<String> getFileNamesToCopy() {
+ Set<String> fileNames = new HashSet<>();
+ for(Map.Entry<String,FileMetaData> ent : toCopy) {
+ fileNames.add(ent.getKey());
+ }
+ return fileNames;
+ }
+
+ @Override
+ public Set<String> getFileNames() {
+ return files.keySet();
+ }
+
+ /** Higher priority and then "first come first serve" order. */
+ @Override
+ public int compareTo(CopyJob _other) {
+ SimpleCopyJob other = (SimpleCopyJob) _other;
+ if (highPriority != other.highPriority) {
+ return highPriority ? -1 : 1;
+ } else if (ord < other.ord) {
+ return -1;
+ } else {
+ return 0;
+ }
+ }
+
+ @Override
+ public void finish() throws IOException {
+ dest.message(String.format(Locale.ROOT,
+ "top: file copy done; took %.1f msec to copy %d bytes; now rename %d tmp files",
+ (System.nanoTime() - startNS)/1000000.0,
+ totBytesCopied,
+ copiedFiles.size()));
+
+ // NOTE: if any of the files we copied overwrote a file in the current commit point, we (ReplicaNode) removed the commit point up
+ // front so that the commit is not corrupt. This way if we hit exc here, or if we crash here, we won't leave a corrupt commit in
+ // the index:
+ for(Map.Entry<String,String> ent : copiedFiles.entrySet()) {
+ String tmpFileName = ent.getValue();
+ String fileName = ent.getKey();
+
+ // Tricky: if primary crashes while warming (pre-copying) a merged segment _X, the new primary can easily flush or merge to _X (since we don't
+ // have a distributed inflateGens for the new primary) and _X file names will be reused. In this case, our local deleter will be
+ // thinking it must remove _X's files (from the warmed merge that never went live), but this is dangerous when virus checker is active
+ // since deleter may finally succeed in deleting the file after we have copied the new _X flushed files. So at this point was ask the
+ // deleter to NOT delete the file anymore:
+ dest.deleter.clearPending(Collections.singleton(fileName));
+
+ if (Node.VERBOSE_FILES) {
+ dest.message("rename file " + tmpFileName + " to " + fileName);
+ }
+
+ // NOTE: if this throws exception, then some files have been moved to their true names, and others are leftover .tmp files. I don't
+ // think heroic exception handling is necessary (no harm will come, except some leftover files), nor warranted here (would make the
+ // code more complex, for the exceptional cases when something is wrong w/ your IO system):
+ dest.dir.renameFile(tmpFileName, fileName);
+ }
+
+ copiedFiles.clear();
+ }
+
+ /** Do an iota of work; returns true if all copying is done */
+ synchronized boolean visit() throws IOException {
+ if (exc != null) {
+ // We were externally cancelled:
+ return true;
+ }
+
+ if (current == null) {
+ if (iter.hasNext() == false) {
+ c.close();
+ return true;
+ }
+
+ Map.Entry<String,FileMetaData> next = iter.next();
+ FileMetaData metaData = next.getValue();
+ String fileName = next.getKey();
+ long len = c.in.readVLong();
+ if (len != metaData.length) {
+ throw new IllegalStateException("file " + fileName + ": meta data says length=" + metaData.length + " but c.in says " + len);
+ }
+ current = new CopyOneFile(c.in, dest, fileName, metaData, copyBuffer);
+ }
+
+ if (current.visit()) {
+ // This file is done copying
+ copiedFiles.put(current.name, current.tmpName);
+ totBytesCopied += current.getBytesCopied();
+ assert totBytesCopied <= totBytes: "totBytesCopied=" + totBytesCopied + " totBytes=" + totBytes;
+ current = null;
+ return false;
+ }
+
+ return false;
+ }
+
+ protected CopyOneFile newCopyOneFile(CopyOneFile prev) {
+ return new CopyOneFile(prev, c.in);
+ }
+
+ @Override
+ public synchronized void transferAndCancel(CopyJob prevJob) throws IOException {
+ try {
+ super.transferAndCancel(prevJob);
+ } finally {
+ IOUtils.closeWhileHandlingException(((SimpleCopyJob) prevJob).c);
+ }
+ }
+
+ public synchronized void cancel(String reason, Throwable exc) {
+ try {
+ super.cancel(reason, exc);
+ } finally {
+ IOUtils.closeWhileHandlingException(c);
+ }
+ }
+
+ @Override
+ public boolean getFailed() {
+ return exc != null;
+ }
+
+ @Override
+ public String toString() {
+ return "SimpleCopyJob(ord=" + ord + " " + reason + " highPriority=" + highPriority + " files count=" + files.size() + " bytesCopied=" + totBytesCopied + " (of " + totBytes + ") filesCopied=" + copiedFiles.size() + ")";
+ }
+
+ @Override
+ public void runBlocking() throws IOException {
+ while (visit() == false);
+
+ if (getFailed()) {
+ throw new RuntimeException("copy failed: " + cancelReason, exc);
+ }
+ }
+
+ @Override
+ public CopyState getCopyState() {
+ return copyState;
+ }
+
+ @Override
+ public synchronized boolean conflicts(CopyJob _other) {
+ Set<String> filesToCopy = new HashSet<>();
+ for(Map.Entry<String,FileMetaData> ent : toCopy) {
+ filesToCopy.add(ent.getKey());
+ }
+
+ SimpleCopyJob other = (SimpleCopyJob) _other;
+ synchronized (other) {
+ for(Map.Entry<String,FileMetaData> ent : other.toCopy) {
+ if (filesToCopy.contains(ent.getKey())) {
+ return true;
+ }
+ }
+ }
+
+ return false;
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
new file mode 100644
index 0000000..18e77ef
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java
@@ -0,0 +1,674 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LogMergePolicy;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.SegmentCommitInfo;
+import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.lucene.util.ThreadInterruptedException;
+
+/** A primary node that uses simple TCP connections to send commands and copy files */
+
+class SimplePrimaryNode extends PrimaryNode {
+
+ final int tcpPort;
+
+ // These are updated by parent test process whenever replicas change:
+ int[] replicaTCPPorts;
+ int[] replicaIDs;
+
+ // So we only flip a bit once per file name:
+ final Set<String> bitFlipped = Collections.synchronizedSet(new HashSet<>());
+
+ static class MergePreCopy {
+ final List<Connection> connections = Collections.synchronizedList(new ArrayList<>());
+ final Map<String,FileMetaData> files;
+ private boolean finished;
+
+ public MergePreCopy(Map<String,FileMetaData> files) {
+ this.files = files;
+ }
+
+ public synchronized boolean tryAddConnection(Connection c) {
+ if (finished == false) {
+ connections.add(c);
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ public synchronized boolean finished() {
+ if (connections.isEmpty()) {
+ finished = true;
+ return true;
+ } else {
+ return false;
+ }
+ }
+ }
+
+ final List<MergePreCopy> warmingSegments = Collections.synchronizedList(new ArrayList<>());
+
+ public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException {
+ super(initWriter(id, random, indexPath), id, primaryGen, forcePrimaryVersion, searcherFactory);
+ this.tcpPort = tcpPort;
+ }
+
+ /** Records currently alive replicas. */
+ public synchronized void setReplicas(int[] replicaIDs, int[] replicaTCPPorts) {
+ message("top: set replicasIDs=" + Arrays.toString(replicaIDs) + " tcpPorts=" + Arrays.toString(replicaTCPPorts));
+ this.replicaIDs = replicaIDs;
+ this.replicaTCPPorts = replicaTCPPorts;
+ }
+
+ private static IndexWriter initWriter(int id, Random random, Path indexPath) throws IOException {
+ Directory dir = SimpleReplicaNode.getDirectory(random, id, indexPath);
+
+ MockAnalyzer analyzer = new MockAnalyzer(random);
+ analyzer.setMaxTokenLength(TestUtil.nextInt(random, 1, IndexWriter.MAX_TERM_LENGTH));
+ IndexWriterConfig iwc = LuceneTestCase.newIndexWriterConfig(random, analyzer);
+
+ MergePolicy mp = iwc.getMergePolicy();
+ //iwc.setInfoStream(new PrintStreamInfoStream(System.out));
+
+ // Force more frequent merging so we stress merge warming:
+ if (mp instanceof TieredMergePolicy) {
+ TieredMergePolicy tmp = (TieredMergePolicy) mp;
+ tmp.setSegmentsPerTier(3);
+ tmp.setMaxMergeAtOnce(3);
+ } else if (mp instanceof LogMergePolicy) {
+ LogMergePolicy lmp = (LogMergePolicy) mp;
+ lmp.setMergeFactor(3);
+ }
+
+ IndexWriter writer = new IndexWriter(dir, iwc);
+
+ TestUtil.reduceOpenFiles(writer);
+ return writer;
+ }
+
+ @Override
+ protected void preCopyMergedSegmentFiles(SegmentCommitInfo info, Map<String,FileMetaData> files) throws IOException {
+ int[] replicaTCPPorts = this.replicaTCPPorts;
+ if (replicaTCPPorts == null) {
+ message("no replicas; skip warming " + info);
+ return;
+ }
+
+ message("top: warm merge " + info + " to " + replicaTCPPorts.length + " replicas: files=" + files.keySet());
+
+ MergePreCopy preCopy = new MergePreCopy(files);
+ warmingSegments.add(preCopy);
+
+ try {
+
+ Set<String> fileNames = files.keySet();
+
+ // Ask all currently known replicas to pre-copy this newly merged segment's files:
+ for (int replicaTCPPort : replicaTCPPorts) {
+ try {
+ Connection c = new Connection(replicaTCPPort);
+ c.out.writeByte(SimpleReplicaNode.CMD_PRE_COPY_MERGE);
+ c.out.writeVLong(primaryGen);
+ c.out.writeVInt(tcpPort);
+ SimpleServer.writeFilesMetaData(c.out, files);
+ c.flush();
+ c.s.shutdownOutput();
+ message("warm connection " + c.s);
+ preCopy.connections.add(c);
+ } catch (Throwable t) {
+ message("top: ignore exception trying to warm to replica port " + replicaTCPPort + ": " + t);
+ //t.printStackTrace(System.out);
+ }
+ }
+
+ long startNS = System.nanoTime();
+ long lastWarnNS = startNS;
+
+ // TODO: maybe ... place some sort of time limit on how long we are willing to wait for slow replicas to finish copying?
+ while (preCopy.finished() == false) {
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException ie) {
+ throw new ThreadInterruptedException(ie);
+ }
+
+ if (isClosed()) {
+ message("top: primary is closing: now cancel segment warming");
+ synchronized(preCopy.connections) {
+ IOUtils.closeWhileHandlingException(preCopy.connections);
+ }
+ return;
+ }
+
+ long ns = System.nanoTime();
+ if (ns - lastWarnNS > 1000000000L) {
+ message(String.format(Locale.ROOT, "top: warning: still warming merge " + info + " to " + preCopy.connections.size() + " replicas for %.1f sec...", (ns - startNS)/1000000000.0));
+ lastWarnNS = ns;
+ }
+
+ // Process keep-alives:
+ synchronized(preCopy.connections) {
+ Iterator<Connection> it = preCopy.connections.iterator();
+ while (it.hasNext()) {
+ Connection c = it.next();
+ try {
+ long nowNS = System.nanoTime();
+ boolean done = false;
+ while (c.sockIn.available() > 0) {
+ byte b = c.in.readByte();
+ if (b == 0) {
+ // keep-alive
+ c.lastKeepAliveNS = nowNS;
+ message("keep-alive for socket=" + c.s + " merge files=" + files.keySet());
+ } else {
+ // merge is done pre-copying to this node
+ if (b != 1) {
+ throw new IllegalArgumentException();
+ }
+ message("connection socket=" + c.s + " is done warming its merge " + info + " files=" + files.keySet());
+ IOUtils.closeWhileHandlingException(c);
+ it.remove();
+ done = true;
+ break;
+ }
+ }
+
+ // If > 2 sec since we saw a keep-alive, assume this replica is dead:
+ if (done == false && nowNS - c.lastKeepAliveNS > 2000000000L) {
+ message("top: warning: replica socket=" + c.s + " for segment=" + info + " seems to be dead; closing files=" + files.keySet());
+ IOUtils.closeWhileHandlingException(c);
+ it.remove();
+ }
+ } catch (Throwable t) {
+ message("top: ignore exception trying to read byte during warm for segment=" + info + " to replica socket=" + c.s + ": " + t + " files=" + files.keySet());
+ IOUtils.closeWhileHandlingException(c);
+ it.remove();
+ }
+ }
+ }
+ }
+ } finally {
+ warmingSegments.remove(preCopy);
+ }
+ }
+
+ /** Flushes all indexing ops to disk and notifies all replicas that they should now copy */
+ private void handleFlush(DataInput topIn, DataOutput topOut, BufferedOutputStream bos) throws IOException {
+ Thread.currentThread().setName("flush");
+
+ int[] replicaTCPPorts;
+ int[] replicaIDs;
+ synchronized (this) {
+ replicaTCPPorts = this.replicaTCPPorts;
+ replicaIDs = this.replicaIDs;
+ }
+
+ message("now flush; " + replicaIDs.length + " replicas");
+
+ if (flushAndRefresh()) {
+ // Something did get flushed (there were indexing ops since the last flush):
+
+ // Tell caller the version before pushing to replicas, so that even if we crash after this, caller will know what version we
+ // (possibly) pushed to some replicas. Alternatively we could make this 2 separate ops?
+ long version = getCopyStateVersion();
+ message("send flushed version=" + version);
+ topOut.writeLong(version);
+ bos.flush();
+
+ // Notify current replicas:
+ for(int i=0;i<replicaIDs.length;i++) {
+ int replicaID = replicaIDs[i];
+ try (Connection c = new Connection(replicaTCPPorts[i])) {
+ c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT);
+ c.out.writeVLong(version);
+ c.out.writeInt(tcpPort);
+ c.flush();
+ // TODO: we should use multicast to broadcast files out to replicas
+ // TODO: ... replicas could copy from one another instead of just primary
+ // TODO: we could also prioritize one replica at a time?
+ } catch (Throwable t) {
+ message("top: failed to connect R" + replicaID + " for newNRTPoint; skipping: " + t.getMessage());
+ }
+ }
+ } else {
+ // No changes flushed:
+ topOut.writeLong(-getCopyStateVersion());
+ }
+ }
+
+ /** Pushes CopyState on the wire */
+ private static void writeCopyState(CopyState state, DataOutput out) throws IOException {
+ // TODO (opto): we could encode to byte[] once when we created the copyState, and then just send same byts to all replicas...
+ out.writeVInt(state.infosBytes.length);
+ out.writeBytes(state.infosBytes, 0, state.infosBytes.length);
+ out.writeVLong(state.gen);
+ out.writeVLong(state.version);
+ SimpleServer.writeFilesMetaData(out, state.files);
+
+ out.writeVInt(state.completedMergeFiles.size());
+ for(String fileName : state.completedMergeFiles) {
+ out.writeString(fileName);
+ }
+ out.writeVLong(state.primaryGen);
+ }
+
+ /** Called when another node (replica) wants to copy files from us */
+ private boolean handleFetchFiles(Random random, Socket socket, DataInput destIn, DataOutput destOut, BufferedOutputStream bos) throws IOException {
+ Thread.currentThread().setName("send");
+
+ int replicaID = destIn.readVInt();
+ message("top: start fetch for R" + replicaID + " socket=" + socket);
+ byte b = destIn.readByte();
+ CopyState copyState;
+ if (b == 0) {
+ // Caller already has CopyState
+ copyState = null;
+ } else if (b == 1) {
+ // Caller does not have CopyState; we pull the latest one:
+ copyState = getCopyState();
+ Thread.currentThread().setName("send-R" + replicaID + "-" + copyState.version);
+ } else {
+ // Protocol error:
+ throw new IllegalArgumentException("invalid CopyState byte=" + b);
+ }
+
+ try {
+ if (copyState != null) {
+ // Serialize CopyState on the wire to the client:
+ writeCopyState(copyState, destOut);
+ bos.flush();
+ }
+
+ byte[] buffer = new byte[16384];
+ int fileCount = 0;
+ long totBytesSent = 0;
+ while (true) {
+ byte done = destIn.readByte();
+ if (done == 1) {
+ break;
+ } else if (done != 0) {
+ throw new IllegalArgumentException("expected 0 or 1 byte but got " + done);
+ }
+
+ // Name of the file the replica wants us to send:
+ String fileName = destIn.readString();
+
+ // Starting offset in the file we should start sending bytes from:
+ long fpStart = destIn.readVLong();
+
+ try (IndexInput in = dir.openInput(fileName, IOContext.DEFAULT)) {
+ long len = in.length();
+ //message("fetch " + fileName + ": send len=" + len);
+ destOut.writeVLong(len);
+ in.seek(fpStart);
+ long upto = fpStart;
+ while (upto < len) {
+ int chunk = (int) Math.min(buffer.length, (len-upto));
+ in.readBytes(buffer, 0, chunk);
+ if (TestNRTReplication.DO_BIT_FLIPS_DURING_COPY) {
+ if (random.nextInt(3000) == 17 && bitFlipped.contains(fileName) == false) {
+ bitFlipped.add(fileName);
+ message("file " + fileName + " to R" + replicaID + ": now randomly flipping a bit at byte=" + upto);
+ int x = random.nextInt(chunk);
+ int bit = random.nextInt(8);
+ buffer[x] ^= 1 << bit;
+ }
+ }
+ destOut.writeBytes(buffer, 0, chunk);
+ upto += chunk;
+ totBytesSent += chunk;
+ }
+ }
+
+ fileCount++;
+ }
+
+ message("top: done fetch files for R" + replicaID + ": sent " + fileCount + " files; sent " + totBytesSent + " bytes");
+ } catch (Throwable t) {
+ message("top: exception during fetch: " + t.getMessage() + "; now close socket");
+ socket.close();
+ return false;
+ } finally {
+ if (copyState != null) {
+ message("top: fetch: now release CopyState");
+ releaseCopyState(copyState);
+ }
+ }
+
+ return true;
+ }
+
+ static final FieldType tokenizedWithTermVectors;
+
+ static {
+ tokenizedWithTermVectors = new FieldType(TextField.TYPE_STORED);
+ tokenizedWithTermVectors.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+ tokenizedWithTermVectors.setStoreTermVectors(true);
+ tokenizedWithTermVectors.setStoreTermVectorOffsets(true);
+ tokenizedWithTermVectors.setStoreTermVectorPositions(true);
+ }
+
+ private void handleIndexing(Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException {
+ Thread.currentThread().setName("indexing");
+ message("start handling indexing socket=" + socket);
+ while (true) {
+ byte cmd;
+ try {
+ cmd = in.readByte();
+ } catch (EOFException eofe) {
+ // done
+ return;
+ }
+ //message("INDEXING OP " + cmd);
+ if (cmd == CMD_ADD_DOC) {
+ handleAddDocument(in, out);
+ out.writeByte((byte) 1);
+ bos.flush();
+ } else if (cmd == CMD_UPDATE_DOC) {
+ handleUpdateDocument(in, out);
+ out.writeByte((byte) 1);
+ bos.flush();
+ } else if (cmd == CMD_DELETE_DOC) {
+ handleDeleteDocument(in, out);
+ out.writeByte((byte) 1);
+ bos.flush();
+ } else if (cmd == CMD_INDEXING_DONE) {
+ out.writeByte((byte) 1);
+ bos.flush();
+ break;
+ } else {
+ throw new IllegalArgumentException("cmd must be add, update or delete; got " + cmd);
+ }
+ }
+ }
+
+ private void handleAddDocument(DataInput in, DataOutput out) throws IOException {
+ int fieldCount = in.readVInt();
+ Document doc = new Document();
+ for(int i=0;i<fieldCount;i++) {
+ String name = in.readString();
+ String value = in.readString();
+ // NOTE: clearly NOT general!
+ if (name.equals("docid") || name.equals("marker")) {
+ doc.add(new StringField(name, value, Field.Store.YES));
+ } else if (name.equals("title")) {
+ doc.add(new StringField("title", value, Field.Store.YES));
+ doc.add(new Field("titleTokenized", value, tokenizedWithTermVectors));
+ } else if (name.equals("body")) {
+ doc.add(new Field("body", value, tokenizedWithTermVectors));
+ } else {
+ throw new IllegalArgumentException("unhandled field name " + name);
+ }
+ }
+
+ writer.addDocument(doc);
+ }
+
+ private void handleUpdateDocument(DataInput in, DataOutput out) throws IOException {
+ int fieldCount = in.readVInt();
+ Document doc = new Document();
+ String docid = null;
+ for(int i=0;i<fieldCount;i++) {
+ String name = in.readString();
+ String value = in.readString();
+ // NOTE: clearly NOT general!
+ if (name.equals("docid")) {
+ docid = value;
+ doc.add(new StringField("docid", value, Field.Store.YES));
+ } else if (name.equals("marker")) {
+ doc.add(new StringField("marker", value, Field.Store.YES));
+ } else if (name.equals("title")) {
+ doc.add(new StringField("title", value, Field.Store.YES));
+ doc.add(new Field("titleTokenized", value, tokenizedWithTermVectors));
+ } else if (name.equals("body")) {
+ doc.add(new Field("body", value, tokenizedWithTermVectors));
+ } else {
+ throw new IllegalArgumentException("unhandled field name " + name);
+ }
+ }
+
+ writer.updateDocument(new Term("docid", docid), doc);
+ }
+
+ private void handleDeleteDocument(DataInput in, DataOutput out) throws IOException {
+ String docid = in.readString();
+ writer.deleteDocuments(new Term("docid", docid));
+ }
+
+ // Sent to primary to cutover new SIS:
+ static final byte CMD_FLUSH = 10;
+
+ // Sent by replica to primary asking to copy a set of files over:
+ static final byte CMD_FETCH_FILES = 1;
+ static final byte CMD_GET_SEARCHING_VERSION = 12;
+ static final byte CMD_SEARCH = 2;
+ static final byte CMD_MARKER_SEARCH = 3;
+ static final byte CMD_COMMIT = 4;
+ static final byte CMD_CLOSE = 5;
+
+ // Send (to primary) the list of currently running replicas:
+ static final byte CMD_SET_REPLICAS = 16;
+
+ // Multiple indexing ops
+ static final byte CMD_INDEXING = 18;
+ static final byte CMD_ADD_DOC = 6;
+ static final byte CMD_UPDATE_DOC = 7;
+ static final byte CMD_DELETE_DOC = 8;
+ static final byte CMD_INDEXING_DONE = 19;
+
+ // Sent by replica to primary when replica first starts up, so primary can add it to any warming merges:
+ static final byte CMD_NEW_REPLICA = 20;
+
+ /** Handles incoming request to the naive TCP server wrapping this node */
+ void handleOneConnection(Random random, ServerSocket ss, AtomicBoolean stop, InputStream is, Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
+
+ outer:
+ while (true) {
+ byte cmd;
+ while (true) {
+ if (is.available() > 0) {
+ break;
+ }
+ if (stop.get()) {
+ return;
+ }
+ Thread.sleep(10);
+ }
+
+ try {
+ cmd = in.readByte();
+ } catch (EOFException eofe) {
+ break;
+ }
+
+ switch (cmd) {
+
+ case CMD_FLUSH:
+ handleFlush(in, out, bos);
+ break;
+
+ case CMD_FETCH_FILES:
+ // Replica (other node) is asking us (primary node) for files to copy
+ handleFetchFiles(random, socket, in, out, bos);
+ break;
+
+ case CMD_INDEXING:
+ handleIndexing(socket, in, out, bos);
+ break;
+
+ case CMD_GET_SEARCHING_VERSION:
+ out.writeVLong(getCurrentSearchingVersion());
+ break;
+
+ case CMD_SEARCH:
+ {
+ Thread.currentThread().setName("search");
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.search(new TermQuery(new Term("body", "the")), 1).totalHits;
+ //message("version=" + version + " searcher=" + searcher);
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+ continue outer;
+
+ case CMD_MARKER_SEARCH:
+ {
+ Thread.currentThread().setName("msearch");
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+ continue outer;
+
+ case CMD_COMMIT:
+ Thread.currentThread().setName("commit");
+ commit();
+ out.writeByte((byte) 1);
+ break;
+
+ case CMD_CLOSE:
+ Thread.currentThread().setName("close");
+ message("top close: now close server socket");
+ ss.close();
+ out.writeByte((byte) 1);
+ message("top close: done close server socket");
+ break;
+
+ case CMD_SET_REPLICAS:
+ Thread.currentThread().setName("set repls");
+ int count = in.readVInt();
+ int[] replicaIDs = new int[count];
+ int[] replicaTCPPorts = new int[count];
+ for(int i=0;i<count;i++) {
+ replicaIDs[i] = in.readVInt();
+ replicaTCPPorts[i] = in.readVInt();
+ }
+ out.writeByte((byte) 1);
+ setReplicas(replicaIDs, replicaTCPPorts);
+ break;
+
+ case CMD_NEW_REPLICA:
+ Thread.currentThread().setName("new repl");
+ int replicaTCPPort = in.readVInt();
+ message("new replica: " + warmingSegments.size() + " current warming merges");
+ // Step through all currently warming segments and try to add this replica if it isn't there already:
+ for(MergePreCopy preCopy : warmingSegments) {
+ message("warming segment " + preCopy.files.keySet());
+ boolean found = false;
+ synchronized (preCopy.connections) {
+ for(Connection c : preCopy.connections) {
+ if (c.destTCPPort == replicaTCPPort) {
+ found = true;
+ break;
+ }
+ }
+ }
+
+ if (found) {
+ message("this replica is already warming this segment; skipping");
+ // It's possible (maybe) that the replica started up, then a merge kicked off, and it warmed to this new replica, all before the
+ // replica sent us this command:
+ continue;
+ }
+
+ // OK, this new replica is not already warming this segment, so attempt (could fail) to start warming now:
+
+ Connection c = new Connection(replicaTCPPort);
+ if (preCopy.tryAddConnection(c) == false) {
+ // This can happen, if all other replicas just now finished warming this segment, and so we were just a bit too late. In this
+ // case the segment will be copied over in the next nrt point sent to this replica
+ message("failed to add connection to segment warmer (too late); closing");
+ c.close();
+ }
+ c.out.writeByte(SimpleReplicaNode.CMD_PRE_COPY_MERGE);
+ c.out.writeVLong(primaryGen);
+ c.out.writeVInt(tcpPort);
+ SimpleServer.writeFilesMetaData(c.out, preCopy.files);
+ c.flush();
+ c.s.shutdownOutput();
+ message("successfully started warming");
+ }
+ break;
+
+ default:
+ throw new IllegalArgumentException("unrecognized cmd=" + cmd + " via socket=" + socket);
+ }
+ bos.flush();
+ break;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
new file mode 100644
index 0000000..8667df1
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java
@@ -0,0 +1,316 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.RateLimitedIndexOutput;
+import org.apache.lucene.store.RateLimiter;
+import org.apache.lucene.util.LuceneTestCase;
+
+class SimpleReplicaNode extends ReplicaNode {
+ final int tcpPort;
+ final Jobs jobs;
+
+ // Rate limits incoming bytes/sec when fetching files:
+ final RateLimiter fetchRateLimiter;
+ final AtomicLong bytesSinceLastRateLimiterCheck = new AtomicLong();
+ final Random random;
+
+ /** Changes over time, as primary node crashes and moves around */
+ int curPrimaryTCPPort;
+
+ public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory) throws IOException {
+ super(id, getDirectory(random, id, indexPath), searcherFactory);
+ this.tcpPort = tcpPort;
+ this.random = new Random(random.nextLong());
+
+ // Random IO throttling on file copies: 5 - 20 MB/sec:
+ double mbPerSec = 5 * (1.0 + 3*random.nextDouble());
+ message(String.format(Locale.ROOT, "top: will rate limit file fetch to %.2f MB/sec", mbPerSec));
+ fetchRateLimiter = new RateLimiter.SimpleRateLimiter(mbPerSec);
+ this.curPrimaryTCPPort = primaryTCPPort;
+
+ start(curPrimaryGen);
+
+ // Handles fetching files from primary:
+ jobs = new Jobs(this);
+ jobs.setName("R" + id + ".copyJobs");
+ jobs.setDaemon(true);
+ jobs.start();
+ }
+
+ @Override
+ protected void launch(CopyJob job) {
+ jobs.launch(job);
+ }
+
+ @Override
+ public void close() throws IOException {
+ // Can't be sync'd when calling jobs since it can lead to deadlock:
+ jobs.close();
+ message("top: jobs closed");
+ synchronized(mergeCopyJobs) {
+ for (CopyJob job : mergeCopyJobs) {
+ message("top: cancel merge copy job " + job);
+ job.cancel("jobs closing", null);
+ }
+ }
+ super.close();
+ }
+
+ @Override
+ protected CopyJob newCopyJob(String reason, Map<String,FileMetaData> files, Map<String,FileMetaData> prevFiles,
+ boolean highPriority, CopyJob.OnceDone onceDone) throws IOException {
+ Connection c;
+ CopyState copyState;
+
+ // Exceptions in here mean something went wrong talking over the socket, which are fine (e.g. primary node crashed):
+ try {
+ c = new Connection(curPrimaryTCPPort);
+ c.out.writeByte(SimplePrimaryNode.CMD_FETCH_FILES);
+ c.out.writeVInt(id);
+ if (files == null) {
+ // No incoming CopyState: ask primary for latest one now
+ c.out.writeByte((byte) 1);
+ c.flush();
+ copyState = SimpleServer.readCopyState(c.in);
+ files = copyState.files;
+ } else {
+ c.out.writeByte((byte) 0);
+ copyState = null;
+ }
+ } catch (Throwable t) {
+ throw new NodeCommunicationException("exc while reading files to copy", t);
+ }
+
+ return new SimpleCopyJob(reason, c, copyState, this, files, highPriority, onceDone);
+ }
+
+ static Directory getDirectory(Random random, int id, Path path) throws IOException {
+ MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path);
+
+ dir.setAssertNoUnrefencedFilesOnClose(true);
+ // This is very costly (takes more time to check than it did to index); we do this ourselves in the end instead of each time a replica
+ // is restarted:
+ dir.setCheckIndexOnClose(false);
+
+ // Corrupt any index files not referenced by current commit point; this is important (increases test evilness) because we may have done
+ // a hard crash of the previous JVM writing to this directory and so MDW's corrupt-unknown-files-on-close never ran:
+ Node.nodeMessage(id, "top: corrupt unknown files");
+ dir.corruptUnknownFiles();
+
+ return dir;
+ }
+
+ static final byte CMD_NEW_NRT_POINT = 0;
+
+ // Sent by primary to replica to pre-copy merge files:
+ static final byte CMD_PRE_COPY_MERGE = 17;
+
+ /** Handles incoming request to the naive TCP server wrapping this node */
+ void handleOneConnection(ServerSocket ss, AtomicBoolean stop, InputStream is, Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException {
+ outer:
+ while (true) {
+ byte cmd;
+ while (true) {
+ if (is.available() > 0) {
+ break;
+ }
+ if (stop.get()) {
+ return;
+ }
+ Thread.sleep(10);
+ }
+
+ try {
+ cmd = in.readByte();
+ } catch (EOFException eofe) {
+ break;
+ }
+
+ switch(cmd) {
+ case CMD_NEW_NRT_POINT:
+ {
+ long version = in.readVLong();
+ Thread.currentThread().setName("recv-" + version);
+ curPrimaryTCPPort = in.readInt();
+ newNRTPoint(version);
+ }
+ break;
+
+ case SimplePrimaryNode.CMD_GET_SEARCHING_VERSION:
+ // nocommit this is hacky:
+
+ // Tricky: if a sync is just finishing up, i.e. managed to finish copying all files just before we crashed primary, and is now
+ // in the process of opening a new reader, we need to wait for it, to be sure we really pick the most current replica:
+ if (isCopying()) {
+ message("top: getSearchingVersion: now wait for finish sync");
+ // TODO: use immediate concurrency instead of polling:
+ while (isCopying() && stop.get() == false) {
+ Thread.sleep(50);
+ message("top: curNRTCopy=" + curNRTCopy);
+ }
+ message("top: getSearchingVersion: done wait for finish sync");
+ }
+ if (stop.get() == false) {
+ out.writeVLong(getCurrentSearchingVersion());
+ } else {
+ message("top: getSearchingVersion: stop waiting for finish sync: stop is set");
+ }
+ break;
+
+ case SimplePrimaryNode.CMD_SEARCH:
+ {
+ Thread.currentThread().setName("search");
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.search(new TermQuery(new Term("body", "the")), 1).totalHits;
+ //node.message("version=" + version + " searcher=" + searcher);
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+ continue outer;
+
+ case SimplePrimaryNode.CMD_MARKER_SEARCH:
+ {
+ Thread.currentThread().setName("msearch");
+ IndexSearcher searcher = mgr.acquire();
+ try {
+ long version = ((DirectoryReader) searcher.getIndexReader()).getVersion();
+ int hitCount = searcher.search(new TermQuery(new Term("marker", "marker")), 1).totalHits;
+ out.writeVLong(version);
+ out.writeVInt(hitCount);
+ } finally {
+ mgr.release(searcher);
+ }
+ }
+ continue outer;
+
+ case SimplePrimaryNode.CMD_COMMIT:
+ Thread.currentThread().setName("commit");
+ commit();
+ out.writeByte((byte) 1);
+ break;
+
+ case SimplePrimaryNode.CMD_CLOSE:
+ Thread.currentThread().setName("close");
+ ss.close();
+ out.writeByte((byte) 1);
+ break outer;
+
+ case CMD_PRE_COPY_MERGE:
+ Thread.currentThread().setName("merge copy");
+
+ long newPrimaryGen = in.readVLong();
+ curPrimaryTCPPort = in.readVInt();
+ Map<String,FileMetaData> files = SimpleServer.readFilesMetaData(in);
+ message("done reading files to copy files=" + files.keySet());
+ AtomicBoolean finished = new AtomicBoolean();
+ CopyJob job = launchPreCopyMerge(finished, newPrimaryGen, files);
+ message("done launching copy job files=" + files.keySet());
+
+ // Silly keep alive mechanism, else if e.g. we (replica node) crash, the primary
+ // won't notice for a very long time:
+ boolean success = false;
+ try {
+ int count = 0;
+ while (true) {
+ if (finished.get() || stop.get()) {
+ break;
+ }
+ Thread.sleep(10);
+ count++;
+ if (count == 100) {
+ // Once per second or so, we send a keep alive
+ message("send merge pre copy keep alive... files=" + files.keySet());
+
+ // To be evil, we sometimes fail to keep-alive, e.g. simulating a long GC pausing us:
+ if (random.nextBoolean()) {
+ out.writeByte((byte) 0);
+ count = 0;
+ }
+ }
+ }
+
+ out.writeByte((byte) 1);
+ bos.flush();
+ success = true;
+ } finally {
+ message("done merge copy files=" + files.keySet() + " success=" + success);
+ }
+ break;
+
+ default:
+ throw new IllegalArgumentException("unrecognized cmd=" + cmd);
+ }
+ bos.flush();
+
+ break;
+ }
+ }
+
+ @Override
+ protected void sendNewReplica() throws IOException {
+ message("send new_replica to primary tcpPort=" + curPrimaryTCPPort);
+ try (Connection c = new Connection(curPrimaryTCPPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_NEW_REPLICA);
+ c.out.writeVInt(tcpPort);
+ c.flush();
+ c.s.shutdownOutput();
+ } catch (Throwable t) {
+ message("ignoring exc " + t + " sending new_replica to primary tcpPort=" + curPrimaryTCPPort);
+ }
+ }
+
+ @Override
+ public IndexOutput createTempOutput(String prefix, String suffix, IOContext ioContext) throws IOException {
+ return new RateLimitedIndexOutput(fetchRateLimiter, super.createTempOutput(prefix, suffix, ioContext));
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
new file mode 100644
index 0000000..f03a5c3
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java
@@ -0,0 +1,390 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.BufferedReader;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.PrintWriter;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.file.Path;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.store.NIOFSDirectory;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/** Child process with silly naive TCP socket server to handle
+ * between-node commands, launched for each node by TestNRTReplication. */
+@SuppressCodecs({"MockRandom", "Memory", "Direct", "SimpleText"})
+@SuppressSysoutChecks(bugUrl = "Stuff gets printed, important stuff for debugging a failure")
+public class SimpleServer extends LuceneTestCase {
+
+ final static Set<Thread> clientThreads = Collections.synchronizedSet(new HashSet<>());
+ final static AtomicBoolean stop = new AtomicBoolean();
+
+ /** Handles one client connection */
+ private static class ClientHandler extends Thread {
+
+ // We hold this just so we can close it to exit the process:
+ private final ServerSocket ss;
+ private final Socket socket;
+ private final Node node;
+ private final int bufferSize;
+
+ public ClientHandler(ServerSocket ss, Node node, Socket socket) {
+ this.ss = ss;
+ this.node = node;
+ this.socket = socket;
+ this.bufferSize = TestUtil.nextInt(random(), 128, 65536);
+ if (Node.VERBOSE_CONNECTIONS) {
+ node.message("new connection socket=" + socket);
+ }
+ }
+
+ @Override
+ public void run() {
+ boolean success = false;
+ try {
+ //node.message("using stream buffer size=" + bufferSize);
+ InputStream is = new BufferedInputStream(socket.getInputStream(), bufferSize);
+ DataInput in = new InputStreamDataInput(is);
+ BufferedOutputStream bos = new BufferedOutputStream(socket.getOutputStream(), bufferSize);
+ DataOutput out = new OutputStreamDataOutput(bos);
+
+ if (node instanceof SimplePrimaryNode) {
+ ((SimplePrimaryNode) node).handleOneConnection(random(), ss, stop, is, socket, in, out, bos);
+ } else {
+ ((SimpleReplicaNode) node).handleOneConnection(ss, stop, is, socket, in, out, bos);
+ }
+
+ bos.flush();
+ if (Node.VERBOSE_CONNECTIONS) {
+ node.message("bos.flush done");
+ }
+
+ success = true;
+ } catch (Throwable t) {
+ if (t instanceof SocketException == false) {
+ node.message("unexpected exception handling client connection:");
+ t.printStackTrace(System.out);
+ // Test should fail with this:
+ throw new RuntimeException(t);
+ } else {
+ node.message("SocketException " + t + " handling client connection; ignoring");
+ }
+ } finally {
+ if (success) {
+ try {
+ IOUtils.close(socket);
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ } else {
+ IOUtils.closeWhileHandlingException(socket);
+ }
+ }
+ if (Node.VERBOSE_CONNECTIONS) {
+ node.message("socket.close done");
+ }
+ }
+ }
+
+ /**
+ * currently, this only works/tested on Sun and IBM.
+ */
+
+ // poached from TestIndexWriterOnJRECrash ... should we factor out to TestUtil? seems dangerous to give it such "publicity"?
+ private static void crashJRE() {
+ final String vendor = Constants.JAVA_VENDOR;
+ final boolean supportsUnsafeNpeDereference =
+ vendor.startsWith("Oracle") ||
+ vendor.startsWith("Sun") ||
+ vendor.startsWith("Apple");
+
+ try {
+ if (supportsUnsafeNpeDereference) {
+ try {
+ Class<?> clazz = Class.forName("sun.misc.Unsafe");
+ java.lang.reflect.Field field = clazz.getDeclaredField("theUnsafe");
+ field.setAccessible(true);
+ Object o = field.get(null);
+ Method m = clazz.getMethod("putAddress", long.class, long.class);
+ m.invoke(o, 0L, 0L);
+ } catch (Throwable e) {
+ System.out.println("Couldn't kill the JVM via Unsafe.");
+ e.printStackTrace(System.out);
+ }
+ }
+
+ // Fallback attempt to Runtime.halt();
+ Runtime.getRuntime().halt(-1);
+ } catch (Exception e) {
+ System.out.println("Couldn't kill the JVM.");
+ e.printStackTrace(System.out);
+ }
+
+ // We couldn't get the JVM to crash for some reason.
+ throw new RuntimeException("JVM refuses to die!");
+ }
+
+ static void writeFilesMetaData(DataOutput out, Map<String,FileMetaData> files) throws IOException {
+ out.writeVInt(files.size());
+ for(Map.Entry<String,FileMetaData> ent : files.entrySet()) {
+ out.writeString(ent.getKey());
+
+ FileMetaData fmd = ent.getValue();
+ out.writeVLong(fmd.length);
+ out.writeVLong(fmd.checksum);
+ out.writeVInt(fmd.header.length);
+ out.writeBytes(fmd.header, 0, fmd.header.length);
+ out.writeVInt(fmd.footer.length);
+ out.writeBytes(fmd.footer, 0, fmd.footer.length);
+ }
+ }
+
+ static Map<String,FileMetaData> readFilesMetaData(DataInput in) throws IOException {
+ int fileCount = in.readVInt();
+ //System.out.println("readFilesMetaData: fileCount=" + fileCount);
+ Map<String,FileMetaData> files = new HashMap<>();
+ for(int i=0;i<fileCount;i++) {
+ String fileName = in.readString();
+ //System.out.println("readFilesMetaData: fileName=" + fileName);
+ long length = in.readVLong();
+ long checksum = in.readVLong();
+ byte[] header = new byte[in.readVInt()];
+ in.readBytes(header, 0, header.length);
+ byte[] footer = new byte[in.readVInt()];
+ in.readBytes(footer, 0, footer.length);
+ files.put(fileName, new FileMetaData(header, footer, length, checksum));
+ }
+ return files;
+ }
+
+ /** Pulls CopyState off the wire */
+ static CopyState readCopyState(DataInput in) throws IOException {
+
+ // Decode a new CopyState
+ byte[] infosBytes = new byte[in.readVInt()];
+ in.readBytes(infosBytes, 0, infosBytes.length);
+
+ long gen = in.readVLong();
+ long version = in.readVLong();
+ Map<String,FileMetaData> files = readFilesMetaData(in);
+
+ int count = in.readVInt();
+ Set<String> completedMergeFiles = new HashSet<>();
+ for(int i=0;i<count;i++) {
+ completedMergeFiles.add(in.readString());
+ }
+ long primaryGen = in.readVLong();
+
+ return new CopyState(files, version, gen, infosBytes, completedMergeFiles, primaryGen, null);
+ }
+
+ public void test() throws Exception {
+
+ int id = Integer.parseInt(System.getProperty("tests.nrtreplication.nodeid"));
+ Thread.currentThread().setName("init child " + id);
+ Path indexPath = Paths.get(System.getProperty("tests.nrtreplication.indexpath"));
+ boolean isPrimary = System.getProperty("tests.nrtreplication.isPrimary") != null;
+ int primaryTCPPort;
+ long forcePrimaryVersion;
+ if (isPrimary == false) {
+ forcePrimaryVersion = -1;
+ primaryTCPPort = Integer.parseInt(System.getProperty("tests.nrtreplication.primaryTCPPort"));
+ } else {
+ primaryTCPPort = -1;
+ forcePrimaryVersion = Long.parseLong(System.getProperty("tests.nrtreplication.forcePrimaryVersion"));
+ }
+ long primaryGen = Long.parseLong(System.getProperty("tests.nrtreplication.primaryGen"));
+ Node.globalStartNS = Long.parseLong(System.getProperty("tests.nrtreplication.startNS"));
+
+ boolean doRandomCrash = isPrimary ? TestNRTReplication.DO_CRASH_PRIMARY : TestNRTReplication.DO_CRASH_REPLICA;
+ boolean doRandomClose = isPrimary ? false : TestNRTReplication.DO_CLOSE_REPLICA;
+
+ // Create server socket that we listen for incoming requests on:
+ try (final ServerSocket ss = new ServerSocket(0)) {
+
+ int tcpPort = ((InetSocketAddress) ss.getLocalSocketAddress()).getPort();
+ System.out.println("\nPORT: " + tcpPort);
+ final Node node;
+ if (isPrimary) {
+ node = new SimplePrimaryNode(random(), indexPath, id, tcpPort, primaryGen, forcePrimaryVersion, null);
+ System.out.println("\nCOMMIT VERSION: " + ((PrimaryNode) node).getLastCommitVersion());
+ } else {
+ node = new SimpleReplicaNode(random(), id, tcpPort, indexPath, primaryGen, primaryTCPPort, null);
+ }
+ System.out.println("\nINFOS VERSION: " + node.getCurrentSearchingVersion());
+
+ if (doRandomClose || doRandomCrash) {
+ final int waitForMS;
+ if (isPrimary) {
+ waitForMS = TestUtil.nextInt(random(), 20000, 60000);
+ } else {
+ waitForMS = TestUtil.nextInt(random(), 5000, 60000);
+ }
+
+ boolean doClose;
+ if (doRandomCrash == false) {
+ doClose = true;
+ } else if (doRandomClose) {
+ doClose = random().nextBoolean();
+ } else {
+ doClose = false;
+ }
+
+ if (doClose) {
+ node.message("top: will close after " + (waitForMS/1000.0) + " seconds");
+ } else {
+ node.message("top: will crash after " + (waitForMS/1000.0) + " seconds");
+ }
+
+ Thread t = new Thread() {
+ @Override
+ public void run() {
+ long endTime = System.nanoTime() + waitForMS*1000000L;
+ while (System.nanoTime() < endTime) {
+ try {
+ Thread.sleep(10);
+ } catch (InterruptedException e) {
+ }
+ if (stop.get()) {
+ break;
+ }
+ }
+
+ if (stop.get() == false) {
+ if (doClose) {
+ try {
+ node.message("top: now force close server socket after " + (waitForMS/1000.0) + " seconds");
+ node.state = "top-closing";
+ ss.close();
+ } catch (IOException ioe) {
+ throw new RuntimeException(ioe);
+ }
+ } else {
+ node.message("top: now crash JVM after " + (waitForMS/1000.0) + " seconds");
+ crashJRE();
+ }
+ }
+ }
+ };
+
+ if (isPrimary) {
+ t.setName("crasher P" + id);
+ } else {
+ t.setName("crasher R" + id);
+ }
+
+ // So that if node exits naturally, this thread won't prevent process exit:
+ t.setDaemon(true);
+ t.start();
+ }
+ System.out.println("\nNODE STARTED");
+
+ //List<Thread> clientThreads = new ArrayList<>();
+
+ // Naive thread-per-connection server:
+ while (true) {
+ Socket socket;
+ try {
+ socket = ss.accept();
+ } catch (SocketException se) {
+ // when ClientHandler closes our ss we will hit this
+ node.message("top: server socket exc; now exit");
+ break;
+ }
+ Thread thread = new ClientHandler(ss, node, socket);
+ thread.setDaemon(true);
+ thread.start();
+
+ clientThreads.add(thread);
+
+ // Prune finished client threads:
+ Iterator<Thread> it = clientThreads.iterator();
+ while (it.hasNext()) {
+ Thread t = it.next();
+ if (t.isAlive() == false) {
+ it.remove();
+ }
+ }
+ //node.message(clientThreads.size() + " client threads are still alive");
+ }
+
+ stop.set(true);
+
+ // Make sure all client threads are done, else we get annoying (yet ultimately "harmless") messages about threads still running /
+ // lingering for them to finish from the child processes:
+ for(Thread clientThread : clientThreads) {
+ node.message("top: join clientThread=" + clientThread);
+ clientThread.join();
+ node.message("top: done join clientThread=" + clientThread);
+ }
+ node.message("done join all client threads; now close node");
+ node.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/1ae72914/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
new file mode 100644
index 0000000..d409ffc
--- /dev/null
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
@@ -0,0 +1,250 @@
+package org.apache.lucene.replicator.nrt;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.Closeable;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.RAMOutputStream;
+
+/** This is a stupid yet functional transaction log: it never fsync's, never prunes, it's over-synchronized, it hard-wires id field name to "docid", can
+ * only handle specific docs/fields used by this test, etc. It's just barely enough to show how a translog could work on top of NRT
+ * replication to guarantee no data loss when nodes crash */
+
+class SimpleTransLog implements Closeable {
+
+ final FileChannel channel;
+ final RAMOutputStream buffer = new RAMOutputStream();
+ final byte[] intBuffer = new byte[4];
+ final ByteBuffer intByteBuffer = ByteBuffer.wrap(intBuffer);
+
+ private final static byte OP_ADD_DOCUMENT = (byte) 0;
+ private final static byte OP_UPDATE_DOCUMENT = (byte) 1;
+ private final static byte OP_DELETE_DOCUMENTS = (byte) 2;
+
+ public SimpleTransLog(Path path) throws IOException {
+ channel = FileChannel.open(path, StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW);
+ }
+
+ public synchronized long getNextLocation() throws IOException {
+ return channel.position();
+ }
+
+ /** Appends an addDocument op */
+ public synchronized long addDocument(String id, Document doc) throws IOException {
+ assert buffer.getFilePointer() == 0;
+ buffer.writeByte(OP_ADD_DOCUMENT);
+ encode(id, doc);
+ return flushBuffer();
+ }
+
+ /** Appends an updateDocument op */
+ public synchronized long updateDocument(String id, Document doc) throws IOException {
+ assert buffer.getFilePointer() == 0;
+ buffer.writeByte(OP_UPDATE_DOCUMENT);
+ encode(id, doc);
+ return flushBuffer();
+ }
+
+ /** Appends a deleteDocuments op */
+ public synchronized long deleteDocuments(String id) throws IOException {
+ assert buffer.getFilePointer() == 0;
+ buffer.writeByte(OP_DELETE_DOCUMENTS);
+ buffer.writeString(id);
+ return flushBuffer();
+ }
+
+ /** Writes buffer to the file and returns the start position. */
+ private synchronized long flushBuffer() throws IOException {
+ long pos = channel.position();
+ int len = (int) buffer.getFilePointer();
+ byte[] bytes = new byte[len];
+ buffer.writeTo(bytes, 0);
+ buffer.reset();
+
+ intBuffer[0] = (byte) (len >> 24);
+ intBuffer[1] = (byte) (len >> 16);
+ intBuffer[2] = (byte) (len >> 8);
+ intBuffer[3] = (byte) len;
+ intByteBuffer.limit(4);
+ intByteBuffer.position(0);
+
+ writeBytesToChannel(intByteBuffer);
+ writeBytesToChannel(ByteBuffer.wrap(bytes));
+
+ return pos;
+ }
+
+ private void writeBytesToChannel(ByteBuffer src) throws IOException {
+ int left = src.limit();
+ while (left != 0) {
+ left -= channel.write(src);
+ }
+ }
+
+ private void readBytesFromChannel(long pos, ByteBuffer dest) throws IOException {
+ int left = dest.limit() - dest.position();
+ long end = pos + left;
+ while (pos < end) {
+ int inc = channel.read(dest, pos);
+ if (inc < 0) {
+ throw new EOFException();
+ }
+ pos += inc;
+ }
+ }
+
+ /** Replays ops between start and end location against the provided writer. Can run concurrently with ongoing operations. */
+ public void replay(NodeProcess primary, long start, long end) throws IOException {
+ try (Connection c = new Connection(primary.tcpPort)) {
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING);
+ byte[] intBuffer = new byte[4];
+ ByteBuffer intByteBuffer = ByteBuffer.wrap(intBuffer);
+ ByteArrayDataInput in = new ByteArrayDataInput();
+
+ long pos = start;
+ while (pos < end) {
+ intByteBuffer.position(0);
+ intByteBuffer.limit(4);
+ readBytesFromChannel(pos, intByteBuffer);
+ pos += 4;
+ int len = ((intBuffer[0] & 0xff) << 24) |
+ (intBuffer[1] & 0xff) << 16 |
+ (intBuffer[2] & 0xff) << 8 |
+ (intBuffer[3] & 0xff);
+
+ byte[] bytes = new byte[len];
+ readBytesFromChannel(pos, ByteBuffer.wrap(bytes));
+ pos += len;
+
+ in.reset(bytes);
+
+ byte op = in.readByte();
+ //System.out.println("xlog: replay op=" + op);
+ switch (op) {
+ case 0:
+ // We replay add as update:
+ replayAddDocument(c, primary, in);
+ break;
+
+ case 1:
+ // We replay add as update:
+ replayAddDocument(c, primary, in);
+ break;
+
+ case 2:
+ replayDeleteDocuments(c, primary, in);
+ break;
+
+ default:
+ throw new CorruptIndexException("invalid operation " + op, in);
+ }
+ }
+ assert pos == end;
+ //System.out.println("xlog: done replay");
+ c.out.writeByte(SimplePrimaryNode.CMD_INDEXING_DONE);
+ c.flush();
+ //System.out.println("xlog: done flush");
+ c.in.readByte();
+ //System.out.println("xlog: done readByte");
+ }
+ }
+
+ private void replayAddDocument(Connection c, NodeProcess primary, DataInput in) throws IOException {
+ String id = in.readString();
+
+ Document doc = new Document();
+ doc.add(new StringField("docid", id, Field.Store.YES));
+
+ String title = readNullableString(in);
+ if (title != null) {
+ doc.add(new StringField("title", title, Field.Store.NO));
+ doc.add(new TextField("titleTokenized", title, Field.Store.NO));
+ }
+ String body = readNullableString(in);
+ if (body != null) {
+ doc.add(new TextField("body", body, Field.Store.NO));
+ }
+ String marker = readNullableString(in);
+ if (marker != null) {
+ //System.out.println("xlog: replay marker=" + id);
+ doc.add(new StringField("marker", marker, Field.Store.YES));
+ }
+
+ // For both add and update originally, we use updateDocument to replay,
+ // because the doc could in fact already be in the index:
+ // nocomit what if this fails?
+ primary.addOrUpdateDocument(c, doc, false);
+ }
+
+
+ private void replayDeleteDocuments(Connection c, NodeProcess primary, DataInput in) throws IOException {
+ String id = in.readString();
+ // nocomit what if this fails?
+ primary.deleteDocument(c, id);
+ }
+
+ /** Encodes doc into buffer. NOTE: this is NOT general purpose! It only handles the fields used in this test! */
+ private synchronized void encode(String id, Document doc) throws IOException {
+ assert id.equals(doc.get("docid")): "id=" + id + " vs docid=" + doc.get("docid");
+ buffer.writeString(id);
+ writeNullableString(doc.get("title"));
+ writeNullableString(doc.get("body"));
+ writeNullableString(doc.get("marker"));
+ }
+
+ private synchronized void writeNullableString(String s) throws IOException {
+ if (s == null) {
+ buffer.writeByte((byte) 0);
+ } else {
+ buffer.writeByte((byte) 1);
+ buffer.writeString(s);
+ }
+ }
+
+ private String readNullableString(DataInput in) throws IOException {
+ byte b = in.readByte();
+ if (b == 0) {
+ return null;
+ } else if (b == 1) {
+ return in.readString();
+ } else {
+ throw new CorruptIndexException("invalid string lead byte " + b, in);
+ }
+ }
+
+ @Override
+ public synchronized void close() throws IOException {
+ channel.close();
+ }
+}