From 1ae7291429bad742715344f86cfa5200229b3698 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Sun, 24 Jan 2016 18:17:20 -0500 Subject: [PATCH 01/19] current patch --- .../org/apache/lucene/codecs/CodecUtil.java | 30 + .../apache/lucene/index/DirectoryReader.java | 9 +- .../org/apache/lucene/index/IndexWriter.java | 43 +- .../apache/lucene/index/ReaderManager.java | 8 +- .../org/apache/lucene/index/SegmentInfos.java | 397 +++--- .../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 +- .../index/TestIndexWriterOnJRECrash.java | 42 +- .../index/TestIndexWriterOnVMError.java | 2 +- .../lucene/index/TestRollingUpdates.java | 2 +- .../lucene/index/TestStressDeletes.java | 2 +- .../TestControlledRealTimeReopenThread.java | 6 +- .../lucene/search/TestLRUQueryCache.java | 2 +- .../lucene/search/TestSearcherManager.java | 10 +- .../taxonomy/SearcherTaxonomyManager.java | 2 +- .../directory/DirectoryTaxonomyWriter.java | 2 +- .../apache/lucene/replicator/nrt/CopyJob.java | 237 ++++ .../lucene/replicator/nrt/CopyOneFile.java | 132 ++ .../lucene/replicator/nrt/CopyState.java | 56 + .../lucene/replicator/nrt/FileMetaData.java | 40 + .../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 + .../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 +- .../lucene/index/RandomIndexWriter.java | 8 +- .../lucene/store/MockDirectoryWrapper.java | 296 +++-- .../apache/lucene/util/LuceneTestCase.java | 13 +- .../java/org/apache/lucene/util/TestUtil.java | 3 +- 50 files changed, 6587 insertions(+), 342 deletions(-) create mode 100644 lucene/core/src/java/org/apache/lucene/store/ByteArrayIndexInput.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyState.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/NodeCommunicationException.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PreCopyMergedSegmentWarmer.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Connection.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java create mode 100644 lucene/replicator/test.cmd 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 1bc2f40a9d1..4ddad22ab0b 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 { 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 3df0b70b510..a9267261778 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 { * @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 { * 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 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 b05e15a0c70..96dfa0bf3fc 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 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. NOTE: 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 commitUserData) { - segmentInfos.setUserData(new HashMap<>(commitUserData)); + setCommitData(commitUserData, true); + } + + public final synchronized void setCommitData(Map 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")) { 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 79d9a948d5a..8044dc1eb8d 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 { * @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 { * tolerate deleted documents being returned you might gain some * performance by passing false. See * {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}. + * @param writeAllDeletes + * If true, 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); } /** 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 de5dbff1904..2f8d914cc97 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -279,126 +279,133 @@ public final class SegmentInfos implements Cloneable, Iterable= 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(); - 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 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> dvUpdateFiles; - final int numDVFields = input.readInt(); - if (numDVFields == 0) { - dvUpdateFiles = Collections.emptyMap(); - } else { - Map> 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); - - 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()); - } - - 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; + return readCommit(directory, input, generation); } } + public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput input, long generation) throws IOException { + + // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need + // 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 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> dvUpdateFiles; + final int numDVFields = input.readInt(); + if (numDVFields == 0) { + dvUpdateFiles = Collections.emptyMap(); + } else { + Map> 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); + + 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()); + } + + 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; + } + private static final List unsupportedCodecs = Arrays.asList( "Lucene3x", "Lucene40", "Lucene41", "Lucene42", "Lucene45", "Lucene46", "Lucene49", "Lucene410" ); @@ -454,68 +461,7 @@ public final class SegmentInfos implements Cloneable, Iterable 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> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles(); - segnOutput.writeInt(dvUpdatesFiles.size()); - for (Entry> 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 **/ - 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 data) { + public void setUserData(Map data, boolean doIncrementVersion) { if (data == null) { userData = Collections.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 mergedAway = new HashSet<>(merge.segments); 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 8d51532aa22..28dd55f15c7 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 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 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(); 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 3d3b0648eaa..a57f26082d3 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 { * @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 { * tolerate deleted documents being returned you might gain some * performance by passing false. See * {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}. + * @param writeAllDeletes + * If true, new deletes will be forcefully written to index files. * @param searcherFactory * An optional {@link SearcherFactory}. Pass null 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 { * * @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); } /** 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 00000000000..6363ed1ce63 --- /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. + * WARNING: 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(); + } +} 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 fd5e3d7b92e..538e7c0907d 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); } 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 510545f8250..1c5aabd568f 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 close() on */ public static void closeWhileHandlingException(Closeable... objects) { + if (objects.length == 0) { + throw new IllegalArgumentException("pass at least one Closeable"); + } closeWhileHandlingException(Arrays.asList(objects)); } 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 baa24849e7f..ab75fb82a63 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); 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 9213c25f696..b97051992ce 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(); 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 58e3ac78c94..2013ab3d090 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); 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 777ef46d4a1..3fecf718bda 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(); } } 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 90371fe08d5..1c8a43eb74c 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); 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 f36702426e5..51c0eb512c2 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 { 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 1df019c1121..cc4b80c9d5b 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(); 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 2e1f385bbc9..d90eaba31a0 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 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 db632dec758..8eb7b44fa9b 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 error = new AtomicReference<>(); final int numDocs = atLeast(10000); 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 76b56a56ba8..32185696f59 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); 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 cdd0a15cbc6..5a3a551991f 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 { + private final static AtomicLong counter = new AtomicLong(); + protected final ReplicaNode dest; + + protected final Map 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> 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 copiedFiles = new ConcurrentHashMap<>(); + + protected CopyJob(String reason, Map 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> it = toCopy.iterator(); + long bytesAlreadyCopied = 0; + + // Iterate over all files we think we need to copy: + while (it.hasNext()) { + Map.Entry 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 getFileNamesToCopy(); + + /** Returns all file names referenced in this copy job */ + public abstract Set getFileNames(); + + public abstract CopyState getCopyState(); + + public abstract long getTotalBytesCopied(); +} 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 00000000000..e3f0f7d6381 --- /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; + } +} 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 00000000000..c19fabc1e33 --- /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 files; + public final long version; + public final long gen; + public final byte[] infosBytes; + public final Set completedMergeFiles; + public final long primaryGen; + + // only non-null on the primary node + public final SegmentInfos infos; + + public CopyState(Map files, long version, long gen, byte[] infosBytes, + Set 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 + ")"; + } +} 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 00000000000..aca408c60d6 --- /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; + } +} + 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 00000000000..742b19fdd29 --- /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 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 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}. + * + *

This returns null, logging a message, if there are any problems (the file does not exist, is corrupt, truncated, etc.).

*/ + public FileMetaData readLocalFileMetaData(String fileName) throws IOException { + + Map 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; + } +} 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 00000000000..67a9d0a1fb5 --- /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); + } +} 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 00000000000..1918ede3a24 --- /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 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()); + } +} 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 00000000000..183f16ff81b --- /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 finishedMergedFiles = Collections.synchronizedSet(new HashSet()); + + 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 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.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 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 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 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 filesMetaData = new HashMap(); + 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 files) throws IOException; +} 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 00000000000..005f93875c5 --- /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 refCounts = new HashMap(); + private final Set pending = new HashSet(); + 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 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 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 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 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 copy = new HashSet(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 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 getPending() { + return new HashSet(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(); + } +} 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 00000000000..af142d57a85 --- /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 lastCommitFiles = new HashSet<>(); + + /** IncRef'd files in the current NRT point: */ + protected final Collection lastNRTFiles = new HashSet<>(); + + /** Currently running merge pre-copy jobs */ + protected final Set 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 pendingMergeFiles = Collections.synchronizedSet(new HashSet()); + + /** 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 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 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 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 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 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 files, Map 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 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 files) throws IOException { + + CopyJob job; + + maybeNewPrimary(newPrimaryGen); + final long primaryGenStart = lastPrimaryGen; + Set 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> getFilesToCopy(Map files) throws IOException { + + boolean doCopyCommitFiles = false; + List> toCopy = new ArrayList<>(); + for (Map.Entry 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 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?"); + } + } +} 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 00000000000..72ed9213d3c --- /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 { + 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 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(); + } + } +} 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 00000000000..7db7bc185bc --- /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(); + } +} 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 00000000000..369414fa448 --- /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 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); + } + } +} 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 00000000000..4e29508670e --- /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(); + } +} + 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 00000000000..1180967a0e1 --- /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> iter; + + public SimpleCopyJob(String reason, Connection c, CopyState copyState, SimpleReplicaNode dest, Map 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 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 getFileNamesToCopy() { + Set fileNames = new HashSet<>(); + for(Map.Entry ent : toCopy) { + fileNames.add(ent.getKey()); + } + return fileNames; + } + + @Override + public Set 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 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 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 filesToCopy = new HashSet<>(); + for(Map.Entry ent : toCopy) { + filesToCopy.add(ent.getKey()); + } + + SimpleCopyJob other = (SimpleCopyJob) _other; + synchronized (other) { + for(Map.Entry ent : other.toCopy) { + if (filesToCopy.contains(ent.getKey())) { + return true; + } + } + } + + return false; + } +} 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 00000000000..18e77ef4ea7 --- /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 bitFlipped = Collections.synchronizedSet(new HashSet<>()); + + static class MergePreCopy { + final List connections = Collections.synchronizedList(new ArrayList<>()); + final Map files; + private boolean finished; + + public MergePreCopy(Map 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 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 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 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 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 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 files, Map 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 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)); + } +} 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 00000000000..f03a5c3361c --- /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 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 files) throws IOException { + out.writeVInt(files.size()); + for(Map.Entry 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 readFilesMetaData(DataInput in) throws IOException { + int fileCount = in.readVInt(); + //System.out.println("readFilesMetaData: fileCount=" + fileCount); + Map files = new HashMap<>(); + for(int i=0;i 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 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(); + } + } +} 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 00000000000..d409ffc1515 --- /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(); + } +} 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 00000000000..5a073ffca19 --- /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 hitCounts = new ConcurrentHashMap<>(); + + /** Maps searcher version to how many marker documents matched. This should only ever grow (we never delete marker documents). */ + final Map versionToMarker = new ConcurrentHashMap<>(); + + /** Maps searcher version to xlog location when refresh of this version started. */ + final Map 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 0) { + // There were changes + lastPrimaryVersion = result; + addTransLogLoc(lastPrimaryVersion, nextTransLogLoc); + addVersionMarker(lastPrimaryVersion, markerUptoSav); + } + } + } + + StringBuilder sb = new StringBuilder(); + int liveCount = 0; + for(int i=0;i 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 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 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_.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 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 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 downNodes = new ArrayList<>(); + StringBuilder b = new StringBuilder(); + long nowNS = System.nanoTime(); + for(int i=0;i 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 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)); + } +} 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 00000000000..6ddb777519e --- /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; + } +} + diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd new file mode 100644 index 00000000000..14e3bd2055b --- /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 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 4236e88be6b..9f876ef6cb0 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 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)); + } + } + + Set 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 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; + + switch(damage) { + + case 0: + action = "deleted"; + try { + deleteFile(name, true); + } catch (IOException ioe) { + // ignore + } + break; + + case 1: + action = "zeroed"; + // Zero out file entirely + long length; + try { + length = fileLength(name); + } catch (IOException ioe) { + // Ignore + continue; + } + byte[] zeroes = new byte[256]; + long upto = 0; + 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 + } + 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 + } + } + break; + + case 3: + // The file survived intact: + action = "didn't change"; + 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 + 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 (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() throws IOException { + public synchronized void crash() { crashed = true; openFiles = new HashMap<>(); openFilesForWrite = new HashSet<>(); openFilesDeleted = new HashSet<>(); - Iterator 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 m = new IdentityHashMap<>(openFileHandles); @@ -295,70 +499,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { f.close(); } catch (Exception ignored) {} } - - while(it.hasNext()) { - String name = it.next(); - int damage = randomState.nextInt(5); - String action = null; - - if (damage == 0) { - action = "deleted"; - deleteFile(name, true); - } else if (damage == 1) { - action = "zeroed"; - // Zero out file entirely - long length = fileLength(name); - 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; - } - } - 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) { - // The file survived intact: - action = "didn't change"; - } else { - action = "fully truncated"; - // Totally truncate the file to zero bytes - deleteFile(name, true); - IndexOutput out = in.createOutput(name, LuceneTestCase.newIOContext(randomState)); - out.close(); - } - if (LuceneTestCase.VERBOSE) { - System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name); - } - } + corruptFiles(unSyncedFiles); + unSyncedFiles = new HashSet<>(); } public synchronized void clearCrash() { @@ -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 files = new HashSet<>(Arrays.asList(listAll())); + // Disregard what happens with the pendingDeletions files: + files.removeAll(pendingDeletions); + String[] endFiles = files.toArray(new String[0]); Set startSet = new TreeSet<>(Arrays.asList(startFiles)); Set 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: 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 aaab0308903..e6536f35348 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; } 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 99d4be3ebe2..de2cf57f901 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) { From 33890681a0efd840874c5ec79c8862a7e94e59ab Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Sun, 24 Jan 2016 19:50:59 -0500 Subject: [PATCH 02/19] exempt NodeCommunicationException as well --- .../test/org/apache/lucene/replicator/nrt/SimpleServer.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 f03a5c3361c..7a257de89ab 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) { From 022540e8c23e57046f1b54a9cd485dab5ff4b563 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 27 Jan 2016 06:00:12 -0500 Subject: [PATCH 03/19] fix more nocommits; add separate test that deleteAll can replicate --- .../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 | 1098 ++-------------- .../nrt/TestStressNRTReplication.java | 1160 +++++++++++++++++ 9 files changed, 1363 insertions(+), 1095 deletions(-) create mode 100644 lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.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 1918ede3a24..77f23ab00d6 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()); 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 183f16ff81b..ccd88480c61 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(); } 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 af142d57a85..713c6f19626 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); } 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 4e29508670e..9de2c0447e0 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(); + } } 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 18e77ef4ea7..0afd1b459e1 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 bitFlipped = Collections.synchronizedSet(new HashSet<>()); + final List warmingSegments = Collections.synchronizedList(new ArrayList<>()); + + final boolean doFlipBitsDuringCopy; + static class MergePreCopy { final List connections = Collections.synchronizedList(new ArrayList<>()); final Map files; @@ -109,11 +112,12 @@ class SimplePrimaryNode extends PrimaryNode { } } - final List 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"); 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 8667df1e44a..27a55477559 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; 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 7a257de89ab..5b0472111a5 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); 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 5a073ffca19..755b234463a 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 hitCounts = new ConcurrentHashMap<>(); - - /** Maps searcher version to how many marker documents matched. This should only ever grow (we never delete marker documents). */ - final Map versionToMarker = new ConcurrentHashMap<>(); - - /** Maps searcher version to xlog location when refresh of this version started. */ - final Map 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 0) { - // There were changes - lastPrimaryVersion = result; - addTransLogLoc(lastPrimaryVersion, nextTransLogLoc); - addVersionMarker(lastPrimaryVersion, markerUptoSav); - } - } - } - - StringBuilder sb = new StringBuilder(); - int liveCount = 0; - for(int i=0;i 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 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 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"); - } - if (oldNode != null) { - oldNode.isOpen = false; - } - nodes[id] = null; - nodeTimeStamps[id] = System.nanoTime(); + public void testReplicateDeleteAllDocuments() throws Exception { - sendReplicasToPrimary(); - } + Node.globalStartNS = System.nanoTime(); + childTempDir = createTempDir("child"); - /** 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 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); + 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); + } + + // 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); + + // Refresh primary, which also pushes to replica: + long primaryVersion1 = primary.flush(); + 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 (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); - } + // 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); - // Periodically wakes up and starts up any down nodes: - private class RestartThread extends Thread { - @Override - public void run() { - - List 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 downNodes = new ArrayList<>(); - StringBuilder b = new StringBuilder(); - long nowNS = System.nanoTime(); - for(int i=0;i 0) { - Thread.sleep(10); - } - - } 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 { + // Index 10 docs again: + for(int i=0;i<10;i++) { + Document doc = docs.nextDoc(); + primary.addOrUpdateDocument(primaryC, doc, false); + } - @Override - public void run() { - // Maps version to number of hits for silly 'the' TermQuery: - Query theQuery = new TermQuery(new Term("body", "the")); + // Refresh primary, which also pushes to replica: + long primaryVersion3 = primary.flush(); + assertTrue(primaryVersion3 > primaryVersion2); - // Persists connections - Map 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(); - } + // 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 { + replicaC.close(); + primaryC.close(); - @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); - } - } + replica.close(); + primary.close(); } static void message(String message) { 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 00000000000..271c5d277fe --- /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 hitCounts = new ConcurrentHashMap<>(); + + /** Maps searcher version to how many marker documents matched. This should only ever grow (we never delete marker documents). */ + final Map versionToMarker = new ConcurrentHashMap<>(); + + /** Maps searcher version to xlog location when refresh of this version started. */ + final Map versionToTransLogLocation = new ConcurrentHashMap<>(); + + final AtomicLong nodeStartCounter = new AtomicLong(); + + final Set 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 0) { + // There were changes + lastPrimaryVersion = result; + addTransLogLoc(lastPrimaryVersion, nextTransLogLoc); + addVersionMarker(lastPrimaryVersion, markerUptoSav); + } + } + } + + StringBuilder sb = new StringBuilder(); + int liveCount = 0; + for(int i=0;i 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 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_.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 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 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 downNodes = new ArrayList<>(); + StringBuilder b = new StringBuilder(); + long nowNS = System.nanoTime(); + for(int i=0;i 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 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)); + } +} From 8889469b8bb1445133c39069adfdb5db028dbad1 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Tue, 2 Feb 2016 06:09:57 -0500 Subject: [PATCH 04/19] 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 --- .../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 | 111 +++++++++++------- .../lucene/replicator/nrt/ThreadPumper.java | 6 +- lucene/replicator/test.cmd | 2 +- 12 files changed, 287 insertions(+), 79 deletions(-) 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 2f8d914cc97..046f3ea5099 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= 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(); + } } 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 0afd1b459e1..b9ecced157c 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 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); + } + } } 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 27a55477559..bc8bb037409 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 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; 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 5b0472111a5..3fdc45f9be6 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()); 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 d409ffc1515..4c11e52a62f 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)); } 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 755b234463a..b2240ebc955 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, 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 271c5d277fe..63ff12a6a41 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 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. + long nextTransLogLoc = transLog.getNextLocation(); + 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; + } + + 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(); - 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 */ @@ -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 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..."); } } 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 6ddb777519e..d74e1703676 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) { diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd index 14e3bd2055b..c38fc963cf8 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 From d9f20f53edd9dc5895ad18e1844fb4ae9652189e Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Tue, 2 Feb 2016 06:16:34 -0500 Subject: [PATCH 05/19] fix compilation errors --- .../replicator/nrt/TestNRTReplication.java | 36 ++++++++++--------- 1 file changed, 19 insertions(+), 17 deletions(-) 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 b2240ebc955..e2824afe22a 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); From 7af83c486b50ceb13d58e79dd65c158bc96130dc Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 3 Feb 2016 16:45:46 -0500 Subject: [PATCH 06/19] add nocommit; try to fix false test failure due to concurrent check index running while primary is still closing --- .../test/org/apache/lucene/replicator/nrt/NodeProcess.java | 4 +--- .../org/apache/lucene/replicator/nrt/TestNRTReplication.java | 2 ++ 2 files changed, 3 insertions(+), 3 deletions(-) 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 dcbef87aa1f..be0b3df3336 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(); 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 b2240ebc955..510a0759e40 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") From b1b2c799aa1d6598daa4ea8c63a5fa5484b5052d Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Sat, 6 Feb 2016 09:04:15 -0500 Subject: [PATCH 07/19] add more non-stress test cases --- .../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 | 721 +++++++++++++++--- .../nrt/TestStressNRTReplication.java | 13 +- 8 files changed, 642 insertions(+), 151 deletions(-) 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 742b19fdd29..e54c01eee7c 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: 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 c7af42976b6..133992f739b 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); 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 be0b3df3336..9d8b7641294 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"); 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 b9ecced157c..7f5634cde63 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 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; 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 bc8bb037409..2510c40e9a1 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: 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 3fdc45f9be6..72e33d7af8e 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: 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 7ba3bc23a0c..15e9c8ca1b7 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 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); + } + + @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 { - 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()); @@ -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; - } - Thread.sleep(10); - } + waitForVersionAndHits(replica, primaryVersion3, 20); - replicaC.close(); 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); + + 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(); openFilesForWrite = new HashSet<>(); openFilesDeleted = new HashSet<>(); @@ -451,6 +453,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { } catch (Exception ignored) {} } corruptFiles(unSyncedFiles); + crashed = true; unSyncedFiles = new HashSet<>(); } @@ -569,6 +572,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { unSyncedFiles.remove(name); in.deleteFile(name); + createdFiles.remove(name); } // sets the cause of the incoming ioe to be the stack @@ -829,7 +833,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { } throw new RuntimeException("MockDirectoryWrapper: cannot close: there are still open locks: " + openLocks, cause); } - + if (getCheckIndexOnClose()) { randomIOExceptionRate = 0.0; randomIOExceptionRateOnOpen = 0.0; @@ -846,6 +850,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { TestUtil.checkIndex(this, getCrossCheckTermVectorsOnClose(), true); // TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles + // nocommit pull this outside of "getCheckIndexOnClose" if (assertNoUnreferencedFilesOnClose) { // now look for unreferenced files: discount ones that we tried to delete but could not From f0f42780a91852ed05b3793922f73d85d8b3092a Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Sun, 7 Feb 2016 14:46:08 -0500 Subject: [PATCH 09/19] cutover to 'Directory owns retrying deletes on buggy filesystems, cleaning up a lot of crazy retry logic --- .../apache/lucene/replicator/nrt/CopyJob.java | 2 +- .../apache/lucene/replicator/nrt/Node.java | 3 +- .../replicator/nrt/ReplicaFileDeleter.java | 102 +++++------------- .../lucene/replicator/nrt/ReplicaNode.java | 21 +--- .../apache/lucene/replicator/nrt/Jobs.java | 3 +- .../lucene/replicator/nrt/SimpleCopyJob.java | 9 +- .../replicator/nrt/SimpleReplicaNode.java | 4 +- lucene/replicator/test.cmd | 4 +- 8 files changed, 41 insertions(+), 107 deletions(-) diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java index 5baf1d3af67..217c2c0500e 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java @@ -186,7 +186,7 @@ public abstract class CopyJob implements Comparable { /** Use current thread (blocking) to do all copying and then return once done, or throw exception on failure */ public abstract void runBlocking() throws Exception; - public void cancel(String reason, Throwable exc) { + public void cancel(String reason, Throwable exc) throws IOException { if (this.exc != null) { // Already cancelled return; 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 e54c01eee7c..742b19fdd29 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java @@ -49,8 +49,7 @@ import org.apache.lucene.util.StringHelper; abstract class Node implements Closeable { static boolean VERBOSE_FILES = true; - // nocommit - static boolean VERBOSE_CONNECTIONS = true; + static boolean VERBOSE_CONNECTIONS = false; // Keys we store into IndexWriter's commit user data: diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java index b15fc05ff2c..652b3af4e5e 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaFileDeleter.java @@ -38,7 +38,6 @@ import org.apache.lucene.store.IOContext; class ReplicaFileDeleter { private final Map refCounts = new HashMap(); - private final Set pending = new HashSet(); private final Directory dir; private final Node node; @@ -63,10 +62,6 @@ class ReplicaFileDeleter { public synchronized void incRef(Collection fileNames) throws IOException { for(String fileName : fileNames) { - if (pending.contains(fileName)) { - throw new IllegalStateException("cannot incRef file \"" + fileName + "\": it is pending delete"); - } - assert slowFileExists(dir, fileName): "file " + fileName + " does not exist!"; Integer curCount = refCounts.get(fileName); @@ -78,24 +73,23 @@ class ReplicaFileDeleter { } } - public synchronized void decRef(Collection fileNames) { - // We don't delete the files immediately when their RC drops to 0; instead, we add to the pending set, and then call deletePending in - // the end: + public synchronized void decRef(Collection fileNames) throws IOException { + Set toDelete = new HashSet<>(); for(String fileName : fileNames) { Integer curCount = refCounts.get(fileName); assert curCount != null: "fileName=" + fileName; assert curCount.intValue() > 0; if (curCount.intValue() == 1) { refCounts.remove(fileName); - pending.add(fileName); + toDelete.add(fileName); } else { refCounts.put(fileName, curCount.intValue() - 1); } } - deletePending(); + delete(toDelete); - // TODO: this local IR could incRef files here, like we do now with IW ... then we can assert this again: + // TODO: this local IR could incRef files here, like we do now with IW's NRT readers ... then we can assert this again: // we can't assert this, e.g a search can be running when we switch to a new NRT point, holding a previous IndexReader still open for // a bit: @@ -109,111 +103,69 @@ class ReplicaFileDeleter { */ } - private synchronized boolean delete(String fileName) { - try { - if (Node.VERBOSE_FILES) { - node.message("file " + fileName + ": now delete"); - } - dir.deleteFile(fileName); - pending.remove(fileName); - return true; - } catch (FileNotFoundException|NoSuchFileException missing) { - // This should never happen: we should only be asked to track files that do exist - node.message("file " + fileName + ": delete failed: " + missing); - throw new IllegalStateException("file " + fileName + ": we attempted delete but the file does not exist?", missing); - } catch (IOException ioe) { - // nocommit remove this retry logic! it's Directory's job now... - if (Node.VERBOSE_FILES) { - node.message("file " + fileName + ": delete failed: " + ioe + "; will retry later"); - } - pending.add(fileName); - return false; - } - } - - public synchronized Integer getRefCount(String fileName) { - return refCounts.get(fileName); - } - - public synchronized boolean isPending(String fileName) { - return pending.contains(fileName); - } - - public synchronized void deletePending() { + private synchronized void delete(Collection toDelete) throws IOException { if (Node.VERBOSE_FILES) { - node.message("now deletePending: " + pending.size() + " files to try: " + pending); + node.message("now delete " + toDelete.size() + " files: " + toDelete); } - // Clone the set because it will change as we iterate: - List toDelete = new ArrayList<>(pending); - // First pass: delete any segments_N files. We do these first to be certain stale commit points are removed - // before we remove any files they reference. If any delete of segments_N fails, we leave all other files - // undeleted so index is never in a corrupt state: + // before we remove any files they reference, in case we crash right now: for (String fileName : toDelete) { - Integer rc = refCounts.get(fileName); - if (rc != null && rc > 0) { - // Should never happen! This means we are about to pending-delete a referenced index file - throw new IllegalStateException("file \"" + fileName + "\" is in pending delete set but has non-zero refCount=" + rc); - } else if (fileName.startsWith(IndexFileNames.SEGMENTS)) { - if (delete(fileName) == false) { - if (Node.VERBOSE_FILES) { - node.message("failed to remove commit point \"" + fileName + "\"; skipping deletion of all other pending files"); - } - return; - } + assert refCounts.containsKey(fileName) == false; + if (fileName.startsWith(IndexFileNames.SEGMENTS)) { + delete(fileName); } } // Only delete other files if we were able to remove the segments_N files; this way we never // leave a corrupt commit in the index even in the presense of virus checkers: for(String fileName : toDelete) { + assert refCounts.containsKey(fileName) == false; if (fileName.startsWith(IndexFileNames.SEGMENTS) == false) { delete(fileName); } } - Set copy = new HashSet(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 fileNames) { - for(String fileName : fileNames) { - if (pending.remove(fileName)) { - node.message("file " + fileName + ": deleter.clearPending now clear from pending"); - } + private synchronized void delete(String fileName) throws IOException { + if (Node.VERBOSE_FILES) { + node.message("file " + fileName + ": now delete"); } + dir.deleteFile(fileName); } - public synchronized void deleteIfNoRef(String fileName) { + public synchronized Integer getRefCount(String fileName) { + return refCounts.get(fileName); + } + + public synchronized void deleteIfNoRef(String fileName) throws IOException { if (refCounts.containsKey(fileName) == false) { deleteNewFile(fileName); } } - public synchronized void deleteNewFile(String fileName) { + public synchronized void deleteNewFile(String fileName) throws IOException { delete(fileName); } + /* public synchronized Set getPending() { return new HashSet(pending); } + */ public synchronized void deleteUnknownFiles(String segmentsFileName) throws IOException { + Set toDelete = new HashSet<>(); for(String fileName : dir.listAll()) { if (refCounts.containsKey(fileName) == false && fileName.equals("write.lock") == false && fileName.equals(segmentsFileName) == false) { node.message("will delete unknown file \"" + fileName + "\""); - pending.add(fileName); + toDelete.add(fileName); } } - deletePending(); + delete(toDelete); } } 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 54083b4e682..e191caf4964 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java @@ -195,10 +195,13 @@ abstract class ReplicaNode extends Node { // If this throws exc (e.g. due to virus checker), we cannot start this replica: assert deleter.getRefCount(segmentsFileName) == 1; deleter.decRef(Collections.singleton(segmentsFileName)); - if (deleter.isPending(segmentsFileName)) { - // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else we can cause corruption: + + if (dir instanceof FSDirectory && ((FSDirectory) dir).checkPendingDeletions()) { + // If e.g. virus checker blocks us from deleting, we absolutely cannot start this node else there is a definite window during + // which if we carsh, we cause corruption: throw new RuntimeException("replica cannot start: existing segments file=" + segmentsFileName + " must be removed in order to start, but the file delete failed"); } + // So we don't later try to decRef it (illegally) again: boolean didRemove = lastCommitFiles.remove(segmentsFileName); assert didRemove; @@ -427,9 +430,6 @@ abstract class ReplicaNode extends Node { } lastFileMetaData = copyState.files; - - // It's a good time to delete pending files, since we just refreshed and some previously open files are now closed: - deleter.deletePending(); } int markerCount; @@ -720,17 +720,6 @@ abstract class ReplicaNode extends Node { * (inclues the segment id), length, footer (including checksum) differ, then this returns false, else true. */ private boolean fileIsIdentical(String fileName, FileMetaData srcMetaData) throws IOException { - if (deleter.isPending(fileName)) { - // This was a file we had wanted to delete yet a virus checker prevented us, and now we need to overwrite it. - // Such files are in an unknown state, and even if their header and footer and length all - // match, since they may not have been fsync'd by the previous node instance on this directory, - // they could in theory have corruption internally. So we always force ourselves to copy them here: - if (Node.VERBOSE_FILES) { - message("file " + fileName + ": will copy [we had wanted to delete this file on init, but failed]"); - } - return false; - } - FileMetaData destMetaData = readLocalFileMetaData(fileName); if (destMetaData == null) { // Something went wrong in reading the file (it's corrupt, truncated, does not exist, etc.): diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java index 3cb2fbb77a4..a0b9535acd4 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java @@ -18,6 +18,7 @@ package org.apache.lucene.replicator.nrt; */ import java.io.Closeable; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.PriorityQueue; @@ -131,7 +132,7 @@ class Jobs extends Thread implements Closeable { } /** Cancels any existing jobs that are copying the same file names as this one */ - public synchronized void cancelConflictingJobs(CopyJob newJob) { + public synchronized void cancelConflictingJobs(CopyJob newJob) throws IOException { for (CopyJob job : queue) { if (job.conflicts(newJob)) { node.message("top: now cancel existing conflicting job=" + job + " due to newJob=" + newJob); diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java index 1180967a0e1..c8663e22863 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleCopyJob.java @@ -173,13 +173,6 @@ class SimpleCopyJob extends CopyJob { String tmpFileName = ent.getValue(); String fileName = ent.getKey(); - // Tricky: if primary crashes while warming (pre-copying) a merged segment _X, the new primary can easily flush or merge to _X (since we don't - // have a distributed inflateGens for the new primary) and _X file names will be reused. In this case, our local deleter will be - // thinking it must remove _X's files (from the warmed merge that never went live), but this is dangerous when virus checker is active - // since deleter may finally succeed in deleting the file after we have copied the new _X flushed files. So at this point was ask the - // deleter to NOT delete the file anymore: - dest.deleter.clearPending(Collections.singleton(fileName)); - if (Node.VERBOSE_FILES) { dest.message("rename file " + tmpFileName + " to " + fileName); } @@ -241,7 +234,7 @@ class SimpleCopyJob extends CopyJob { } } - public synchronized void cancel(String reason, Throwable exc) { + public synchronized void cancel(String reason, Throwable exc) throws IOException { try { super.cancel(reason, exc); } finally { 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 83ce6cb6cc6..4868338476b 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java @@ -135,9 +135,7 @@ class SimpleReplicaNode extends ReplicaNode { MockDirectoryWrapper dir = LuceneTestCase.newMockFSDirectory(path); dir.setAssertNoUnrefencedFilesOnClose(true); - // nocommit - //dir.setCheckIndexOnClose(doCheckIndexOnClose); - dir.setCheckIndexOnClose(true); + dir.setCheckIndexOnClose(doCheckIndexOnClose); // Corrupt any index files not referenced by current commit point; this is important (increases test evilness) because we may have done // a hard crash of the previous JVM writing to this directory and so MDW's corrupt-unknown-files-on-close never ran: diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd index c38fc963cf8..f636a61cf43 100644 --- a/lucene/replicator/test.cmd +++ b/lucene/replicator/test.cmd @@ -1 +1,3 @@ -python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 1 -mult 4 -nightly +python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 3 + +# -mult 4 -nightly From 9afa560054aad72b8cc623b130745bc8778d1d5c Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Mon, 8 Feb 2016 14:07:04 -0500 Subject: [PATCH 10/19] fix some nocommits; fix one stress test failure --- .../lucene/replicator/nrt/ReplicaNode.java | 21 ++++++-- .../apache/lucene/replicator/nrt/Jobs.java | 18 +++++-- .../lucene/replicator/nrt/NodeProcess.java | 24 +++------ .../replicator/nrt/SimplePrimaryNode.java | 1 + .../replicator/nrt/SimpleReplicaNode.java | 5 +- .../replicator/nrt/TestNRTReplication.java | 6 +-- .../nrt/TestStressNRTReplication.java | 53 ++++++++++++------- .../lucene/replicator/nrt/ThreadPumper.java | 4 +- lucene/replicator/test.cmd | 4 +- .../lucene/store/MockDirectoryWrapper.java | 1 + 10 files changed, 84 insertions(+), 53 deletions(-) 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 e191caf4964..62827e857ec 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java @@ -462,12 +462,19 @@ abstract class ReplicaNode extends Node { /** Call this to notify this replica node that a new NRT infos is available on the primary. * We kick off a job (runs in the background) to copy files across, and open a new reader once that's done. */ - public synchronized CopyJob newNRTPoint(long version) throws IOException { + public synchronized CopyJob newNRTPoint(long newPrimaryGen, long version) throws IOException { if (isClosed()) { throw new AlreadyClosedException("this replica is closed: state=" + state); } + // Cutover (possibly) to new primary first, so we discard any pre-copied merged segments up front, before checking for which files need + // copying. While it's possible the pre-copied merged segments could still be useful to us, in the case that the new primary is either + // the same primary (just e.g. rebooted), or a promoted replica that had a newer NRT point than we did that included the pre-copied + // merged segments, it's still a bit risky to rely solely on checksum/file length to catch the difference, so we defensively discard + // here and re-copy in that case: + maybeNewPrimary(newPrimaryGen); + // Caller should not "publish" us until we have finished .start(): assert mgr != null; @@ -520,9 +527,9 @@ abstract class ReplicaNode extends Node { return null; } + assert newPrimaryGen == job.getCopyState().primaryGen; + Collection newNRTFiles = job.getFileNames(); - long newPrimaryGen = job.getCopyState().primaryGen; - maybeNewPrimary(newPrimaryGen); message("top: newNRTPoint: job files=" + newNRTFiles); @@ -608,9 +615,15 @@ abstract class ReplicaNode extends Node { } /** Called when the primary changed */ - protected synchronized void maybeNewPrimary(long newPrimaryGen) { + protected synchronized void maybeNewPrimary(long newPrimaryGen) throws IOException { if (newPrimaryGen != lastPrimaryGen) { message("top: now change lastPrimaryGen from " + lastPrimaryGen + " to " + newPrimaryGen + " pendingMergeFiles=" + pendingMergeFiles); + + message("top: delete if no ref pendingMergeFiles=" + pendingMergeFiles); + for(String fileName : pendingMergeFiles) { + deleter.deleteIfNoRef(fileName); + } + assert newPrimaryGen > lastPrimaryGen: "newPrimaryGen=" + newPrimaryGen + " vs lastPrimaryGen=" + lastPrimaryGen; lastPrimaryGen = newPrimaryGen; pendingMergeFiles.clear(); diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java index a0b9535acd4..f75a027c68d 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java @@ -80,7 +80,12 @@ class Jobs extends Thread implements Closeable { } else { node.message("AlreadyClosedException during job.visit job=" + topJob + "; now cancel"); } - topJob.cancel("unexpected exception in visit", t); + try { + topJob.cancel("unexpected exception in visit", t); + } catch (Throwable t2) { + node.message("ignore exception calling cancel: " + t2); + t2.printStackTrace(System.out); + } try { topJob.onceDone.run(topJob); } catch (Throwable t2) { @@ -101,6 +106,7 @@ class Jobs extends Thread implements Closeable { topJob.onceDone.run(topJob); } catch (Throwable t) { node.message("ignore exception calling OnceDone: " + t); + t.printStackTrace(System.out); } } } @@ -112,11 +118,17 @@ class Jobs extends Thread implements Closeable { while (queue.isEmpty() == false) { SimpleCopyJob job = (SimpleCopyJob) queue.poll(); node.message("top: Jobs: now cancel job=" + job); - job.cancel("jobs closing", null); + try { + job.cancel("jobs closing", null); + } catch (Throwable t) { + node.message("ignore exception calling cancel"); + t.printStackTrace(System.out); + } try { job.onceDone.run(job); } catch (Throwable t) { - node.message("ignore exception calling OnceDone: " + t); + node.message("ignore exception calling OnceDone"); + t.printStackTrace(System.out); } } } 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 9d8b7641294..a0bfb78f399 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/NodeProcess.java @@ -86,7 +86,7 @@ class NodeProcess implements Closeable { } } - public boolean commit() { + public boolean commit() throws IOException { try (Connection c = new Connection(tcpPort)) { c.out.writeByte(SimplePrimaryNode.CMD_COMMIT); c.flush(); @@ -95,36 +95,22 @@ class NodeProcess implements Closeable { throw new RuntimeException("commit failed"); } return true; - } catch (Throwable t) { - // nocommit throw this - // Something wrong with this replica; skip it: - System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping"); - return false; } } - public void commitAsync() { + public void commitAsync() throws IOException { try (Connection c = new Connection(tcpPort)) { c.out.writeByte(SimplePrimaryNode.CMD_COMMIT); c.flush(); - } catch (Throwable t) { - // nocommit throw this - // Something wrong with this replica; skip it: - System.out.println("PARENT: top: hit SocketException during commit with R" + id + ": " + t + "; skipping"); } } - public long getSearchingVersion() { + public long getSearchingVersion() throws IOException { try (Connection c = new Connection(tcpPort)) { c.out.writeByte(SimplePrimaryNode.CMD_GET_SEARCHING_VERSION); c.flush(); c.s.shutdownOutput(); return c.in.readVLong(); - } catch (Throwable t) { - // nocommit throw this - // Something wrong with this replica; skip it: - System.out.println("PARENT: top: hit SocketException during getSearchingVersion with R" + id + "; skipping"); - return -1L; } } @@ -162,6 +148,7 @@ class NodeProcess implements Closeable { } } catch (Throwable t) { System.out.println("top: shutdown failed; ignoring"); + t.printStackTrace(System.out); } try { p.waitFor(); @@ -178,10 +165,11 @@ class NodeProcess implements Closeable { } } - public void newNRTPoint(long version, int primaryTCPPort) throws IOException { + public void newNRTPoint(long version, long primaryGen, int primaryTCPPort) throws IOException { try (Connection c = new Connection(tcpPort)) { c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT); c.out.writeVLong(version); + c.out.writeVLong(primaryGen); c.out.writeInt(primaryTCPPort); c.flush(); } 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 7f5634cde63..fe142344569 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java @@ -304,6 +304,7 @@ class SimplePrimaryNode extends PrimaryNode { message("send NEW_NRT_POINT to R" + replicaID + " at tcpPort=" + replicaTCPPorts[i]); c.out.writeByte(SimpleReplicaNode.CMD_NEW_NRT_POINT); c.out.writeVLong(version); + c.out.writeVLong(primaryGen); c.out.writeInt(tcpPort); c.flush(); // TODO: we should use multicast to broadcast files out to replicas 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 4868338476b..9658ad17775 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java @@ -176,10 +176,11 @@ class SimpleReplicaNode extends ReplicaNode { case CMD_NEW_NRT_POINT: { long version = in.readVLong(); + long newPrimaryGen = in.readVLong(); Thread.currentThread().setName("recv-" + version); curPrimaryTCPPort = in.readInt(); - message("newNRTPoint primaryTCPPort=" + curPrimaryTCPPort); - newNRTPoint(version); + message("newNRTPoint primaryTCPPort=" + curPrimaryTCPPort + " version=" + version + " newPrimaryGen=" + newPrimaryGen); + newNRTPoint(newPrimaryGen, version); } break; 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 2c66994c27d..262e68e01d6 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java @@ -356,7 +356,7 @@ public class TestNRTReplication extends LuceneTestCase { assertVersionAndHits(replica, 0, 0); // Ask replica to sync: - replica.newNRTPoint(primaryVersion1, primary.tcpPort); + replica.newNRTPoint(primaryVersion1, 0, primary.tcpPort); waitForVersionAndHits(replica, primaryVersion1, 10); replica.close(); @@ -461,7 +461,7 @@ public class TestNRTReplication extends LuceneTestCase { assertVersionAndHits(replica, primaryVersion1, 10); // Now ask replica to sync: - replica.newNRTPoint(primaryVersion2, primary.tcpPort); + replica.newNRTPoint(primaryVersion2, 0, primary.tcpPort); waitForVersionAndHits(replica, primaryVersion2, 20); @@ -736,7 +736,7 @@ public class TestNRTReplication extends LuceneTestCase { sendReplicasToPrimary(primary, replica); // Now ask replica to sync: - replica.newNRTPoint(primaryVersion2, primary.tcpPort); + replica.newNRTPoint(primaryVersion2, 0, primary.tcpPort); // Make sure it sees all docs that were indexed while it was down: assertVersionAndHits(primary, primaryVersion2, 110); 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 a765f11b436..28b15f88922 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -60,14 +60,8 @@ import com.carrotsearch.randomizedtesting.SeedUtils; // nocommit why so many "hit SocketException during commit with R0"? -// nocommit why so much time when so many nodes are down - -// nocommit indexing is too fast? (xlog replay fails to finish before primary crashes itself) - // nocommit why all these NodeCommunicationExcs? -// nocommit the sockets are a pita on jvm crashing ... - /* TODO - fangs @@ -161,8 +155,7 @@ public class TestStressNRTReplication extends LuceneTestCase { static final boolean DO_BIT_FLIPS_DURING_COPY = true; /** Set to a non-null value to force exactly that many nodes; else, it's random. */ - // nocommit - static final Integer NUM_NODES = 2; + static final Integer NUM_NODES = null; final AtomicBoolean failed = new AtomicBoolean(); @@ -214,9 +207,6 @@ public class TestStressNRTReplication extends LuceneTestCase { // Silly bootstrapping: versionToTransLogLocation.put(0L, 0L); - // nocommit why also 1? - //versionToTransLogLocation.put(1L, 0L); - versionToMarker.put(0L, 0); int numNodes; @@ -334,10 +324,15 @@ public class TestStressNRTReplication extends LuceneTestCase { { NodeProcess node = nodes[random().nextInt(nodes.length)]; if (node != null && node.nodeIsClosing.get() == false) { - // TODO: if this node is primary, it means we committed a "partial" version (not exposed as an NRT point)... not sure it matters. + // TODO: if this node is primary, it means we committed an unpublished version (not exposed as an NRT point)... not sure it matters. // maybe we somehow allow IW to commit a specific sis (the one we just flushed)? message("top: now commit node=" + node); - node.commitAsync(); + try { + node.commitAsync(); + } catch (Throwable t) { + message("top: hit exception during commit with R" + node.id + "; skipping"); + t.printStackTrace(System.out); + } } } } @@ -400,7 +395,14 @@ public class TestStressNRTReplication extends LuceneTestCase { for (NodeProcess node : nodes) { if (node != null) { message("ask " + node + " for its current searching version"); - long searchingVersion = node.getSearchingVersion(); + long searchingVersion; + try { + searchingVersion = node.getSearchingVersion(); + } catch (Throwable t) { + message("top: hit SocketException during getSearchingVersion with R" + node.id + "; skipping"); + t.printStackTrace(System.out); + continue; + } message(node + " has searchingVersion=" + searchingVersion); if (searchingVersion > maxSearchingVersion) { maxSearchingVersion = searchingVersion; @@ -415,8 +417,12 @@ public class TestStressNRTReplication extends LuceneTestCase { } message("top: promote " + replicaToPromote + " version=" + maxSearchingVersion + "; now commit"); - if (replicaToPromote.commit() == false) { - message("top: commit failed; skipping primary promotion"); + try { + replicaToPromote.commit(); + } catch (Throwable t) { + // Something wrong with this replica; skip it: + message("top: hit exception during commit with R" + replicaToPromote.id + "; skipping"); + t.printStackTrace(System.out); return; } @@ -478,8 +484,9 @@ public class TestStressNRTReplication extends LuceneTestCase { try { transLog.replay(newPrimary, startTransLogLoc, nextTransLogLoc); } catch (IOException ioe) { - // nocommit what if primary node is still running here, and we failed for some other reason? - message("top: replay xlog failed; abort"); + message("top: replay xlog failed; shutdown new primary"); + ioe.printStackTrace(System.out); + newPrimary.shutdown(); return; } @@ -1182,4 +1189,14 @@ public class TestStressNRTReplication extends LuceneTestCase { Thread.currentThread().getName(), message)); } + + static void message(String message, long localStartNS) { + long now = System.nanoTime(); + System.out.println(String.format(Locale.ROOT, + "%5.3fs %5.1fs: parent [%11s] %s", + (now-Node.globalStartNS)/1000000000., + (now-localStartNS)/1000000000., + Thread.currentThread().getName(), + message)); + } } 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 d74e1703676..73f3908eee1 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/ThreadPumper.java @@ -31,6 +31,7 @@ class ThreadPumper { @Override public void run() { try { + long startTimeNS = System.nanoTime(); Pattern logTimeStart = Pattern.compile("^[0-9\\.]+s .*"); String line; while ((line = from.readLine()) != null) { @@ -42,7 +43,7 @@ class ThreadPumper { // Already a well-formed log output: System.out.println(line); } else { - TestNRTReplication.message(line); + TestStressNRTReplication.message(line, startTimeNS); } if (line.contains("now force close server socket after")) { nodeClosing.set(true); @@ -60,4 +61,3 @@ class ThreadPumper { return t; } } - diff --git a/lucene/replicator/test.cmd b/lucene/replicator/test.cmd index f636a61cf43..18045ce0579 100644 --- a/lucene/replicator/test.cmd +++ b/lucene/replicator/test.cmd @@ -1,3 +1 @@ -python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 3 - -# -mult 4 -nightly +python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestStressNRTReplication -jvms 3 -verbose -mult 4 -nightly 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 a36d6d4a733..aa892096a92 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java +++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java @@ -852,6 +852,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { // TODO: factor this out / share w/ TestIW.assertNoUnreferencedFiles if (assertNoUnreferencedFilesOnClose) { + System.out.println("MDW: now assert no unref'd files at close"); // now look for unreferenced files: discount ones that we tried to delete but could not Set allFiles = new HashSet<>(Arrays.asList(listAll())); From c14fa123da435a34c4f40e6685c3184ecd9a60e0 Mon Sep 17 00:00:00 2001 From: Michael McCandless Date: Mon, 8 Feb 2016 14:07:14 -0500 Subject: [PATCH 11/19] fix compilation errors --- .../test/org/apache/lucene/replicator/nrt/Jobs.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java index a0b9535acd4..de8c0c38e6d 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/Jobs.java @@ -80,7 +80,12 @@ class Jobs extends Thread implements Closeable { } else { node.message("AlreadyClosedException during job.visit job=" + topJob + "; now cancel"); } - topJob.cancel("unexpected exception in visit", t); + try { + topJob.cancel("unexpected exception in visit", t); + } catch (Throwable t2) { + node.message("ignore exception calling cancel: " + t2); + t2.printStackTrace(System.out); + } try { topJob.onceDone.run(topJob); } catch (Throwable t2) { @@ -112,7 +117,11 @@ class Jobs extends Thread implements Closeable { while (queue.isEmpty() == false) { SimpleCopyJob job = (SimpleCopyJob) queue.poll(); node.message("top: Jobs: now cancel job=" + job); - job.cancel("jobs closing", null); + try { + job.cancel("jobs closing", null); + } catch (Throwable t) { + node.message("ignore exception calling cancel: " + t); + } try { job.onceDone.run(job); } catch (Throwable t) { From ae2b58c03c82b6faedc35698542f4898892683b2 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Mon, 8 Feb 2016 19:15:17 -0500 Subject: [PATCH 12/19] indexer threads should stop on shutdown; don't suppress exceptions; decrease random commit frequency --- .../replicator/nrt/SimplePrimaryNode.java | 19 ++++++++++++++----- .../lucene/replicator/nrt/SimpleServer.java | 3 ++- .../nrt/TestStressNRTReplication.java | 7 ++++--- 3 files changed, 20 insertions(+), 9 deletions(-) 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 fe142344569..d3415ad4441 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java @@ -163,7 +163,7 @@ class SimplePrimaryNode extends PrimaryNode { return; } - message("top: warm merge " + info + " to " + replicaTCPPorts.length + " replicas: files=" + files.keySet()); + message("top: warm merge " + info + " to " + replicaTCPPorts.length + " replicas; tcpPort=" + tcpPort + ": files=" + files.keySet()); MergePreCopy preCopy = new MergePreCopy(files); warmingSegments.add(preCopy); @@ -289,8 +289,8 @@ class SimplePrimaryNode extends PrimaryNode { // Something did get flushed (there were indexing ops since the last flush): verifyAtLeastMarkerCount(atLeastMarkerCount, null); - - // Tell caller the version before pushing to replicas, so that even if we crash after this, caller will know what version we + + // Tell caller the version before pushing to replicas, so that even if we crash after this, caller will know what version we // (possibly) pushed to some replicas. Alternatively we could make this 2 separate ops? long version = getCopyStateVersion(); message("send flushed version=" + version); @@ -432,10 +432,19 @@ class SimplePrimaryNode extends PrimaryNode { tokenizedWithTermVectors.setStoreTermVectorPositions(true); } - private void handleIndexing(Socket socket, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException { + private void handleIndexing(Socket socket, AtomicBoolean stop, InputStream is, DataInput in, DataOutput out, BufferedOutputStream bos) throws IOException, InterruptedException { Thread.currentThread().setName("indexing"); message("start handling indexing socket=" + socket); while (true) { + while (true) { + if (is.available() > 0) { + break; + } + if (stop.get()) { + return; + } + Thread.sleep(10); + } byte cmd; try { cmd = in.readByte(); @@ -587,7 +596,7 @@ class SimplePrimaryNode extends PrimaryNode { break; case CMD_INDEXING: - handleIndexing(socket, in, out, bos); + handleIndexing(socket, stop, is, in, out, bos); break; case CMD_GET_SEARCHING_VERSION: 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 72e33d7af8e..3bad39b621b 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleServer.java @@ -106,7 +106,8 @@ public class SimpleServer extends LuceneTestCase { // Test should fail with this: throw new RuntimeException(t); } else { - node.message("exception " + t + " handling client connection; ignoring"); + node.message("exception handling client connection; ignoring:"); + t.printStackTrace(System.out); } } finally { if (success) { 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 28b15f88922..86550c56bf2 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -155,7 +155,7 @@ public class TestStressNRTReplication extends LuceneTestCase { static final boolean DO_BIT_FLIPS_DURING_COPY = true; /** Set to a non-null value to force exactly that many nodes; else, it's random. */ - static final Integer NUM_NODES = null; + static final Integer NUM_NODES; final AtomicBoolean failed = new AtomicBoolean(); @@ -321,7 +321,7 @@ public class TestStressNRTReplication extends LuceneTestCase { // Commit a random node, primary or replica - { + if (random().nextInt(10) == 1) { NodeProcess node = nodes[random().nextInt(nodes.length)]; if (node != null && node.nodeIsClosing.get() == false) { // TODO: if this node is primary, it means we committed an unpublished version (not exposed as an NRT point)... not sure it matters. @@ -751,7 +751,8 @@ public class TestStressNRTReplication extends LuceneTestCase { c.flush(); c.in.readByte(); } catch (Throwable t) { - message("top: ignore exc sending replicas to primary: " + t); + message("top: ignore exc sending replicas to primary P" + curPrimary.id + " at tcpPort=" + curPrimary.tcpPort); + t.printStackTrace(System.out); } } } From f7a56ed6bca6d47de1b14e1aa6c13941076eca2d Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Mon, 8 Feb 2016 19:16:33 -0500 Subject: [PATCH 13/19] fix compilation --- .../apache/lucene/replicator/nrt/TestStressNRTReplication.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 86550c56bf2..f31376b5909 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -155,7 +155,7 @@ public class TestStressNRTReplication extends LuceneTestCase { static final boolean DO_BIT_FLIPS_DURING_COPY = true; /** Set to a non-null value to force exactly that many nodes; else, it's random. */ - static final Integer NUM_NODES; + static final Integer NUM_NODES = null; final AtomicBoolean failed = new AtomicBoolean(); From e7351d34050b4b5cdaf4b85ec9260ddfd3af9dda Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Mon, 8 Feb 2016 19:18:22 -0500 Subject: [PATCH 14/19] fix test failure when primary is flushing just before crashing --- .../replicator/nrt/TestStressNRTReplication.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) 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 28b15f88922..f2595802bb4 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -785,7 +785,7 @@ public class TestStressNRTReplication extends LuceneTestCase { try { while (stop.get() == false) { Thread.sleep(TestUtil.nextInt(random(), 50, 500)); - message("top: restarter cycle"); + //message("top: restarter cycle"); // Randomly crash full cluster: if (DO_FULL_CLUSTER_CRASH && random().nextInt(500) == 17) { @@ -921,6 +921,8 @@ public class TestStressNRTReplication extends LuceneTestCase { continue; } + boolean nodeIsPrimary = node == primary; + try { Thread.currentThread().setName("Searcher node=" + node); @@ -970,6 +972,12 @@ public class TestStressNRTReplication extends LuceneTestCase { if (oldHitCount == null) { hitCounts.put(version, hitCount); message("top: searcher: record search hitCount version=" + version + " hitCount=" + hitCount + " node=" + node); + if (nodeIsPrimary && version > lastPrimaryVersion) { + // It's possible a search request sees a new primary version because it's in the process of flushing, but then the primary + // crashes. In this case we need to ensure new primary forces its version beyond this: + message("top: searcher: set lastPrimaryVersion=" + lastPrimaryVersion + " vs " + version); + lastPrimaryVersion = version; + } } else { // Just ensure that all nodes show the same hit count for // the same version, i.e. they really are replicas of one another: From 88fea2317629d545a1c8ccee18384d38b305215e Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Tue, 9 Feb 2016 05:22:16 -0500 Subject: [PATCH 15/19] turn off debug prints; add lucene.experimental to javadocs --- .../src/java/org/apache/lucene/index/SegmentInfos.java | 2 +- .../java/org/apache/lucene/replicator/nrt/CopyJob.java | 5 +++-- .../java/org/apache/lucene/replicator/nrt/CopyState.java | 4 +++- .../org/apache/lucene/replicator/nrt/FileMetaData.java | 4 +++- .../src/java/org/apache/lucene/replicator/nrt/Node.java | 4 +++- .../lucene/replicator/nrt/NodeCommunicationException.java | 6 ++++++ .../org/apache/lucene/replicator/nrt/PrimaryNode.java | 5 ++++- .../apache/lucene/replicator/nrt/ReplicaFileDeleter.java | 4 +--- .../org/apache/lucene/replicator/nrt/ReplicaNode.java | 4 +++- .../replicator/nrt/SegmentInfosSearcherManager.java | 4 +++- .../lucene/replicator/nrt/TestStressNRTReplication.java | 8 +++----- .../org/apache/lucene/store/MockDirectoryWrapper.java | 2 +- .../org/apache/solr/core/StandardIndexReaderFactory.java | 1 - 13 files changed, 34 insertions(+), 19 deletions(-) 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 40f03e8601d..3b95f25d90d 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -485,7 +485,7 @@ public final class SegmentInfos implements Cloneable, Iterable refCounts = new HashMap(); private final Directory dir; @@ -50,7 +48,7 @@ class ReplicaFileDeleter { * (can be opened), false if it cannot be opened, and * (unlike Java's File.exists) throws IOException if * there's some unexpected error. */ - static boolean slowFileExists(Directory dir, String fileName) throws IOException { + private static boolean slowFileExists(Directory dir, String fileName) throws IOException { try { dir.openInput(fileName, IOContext.DEFAULT).close(); return true; 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 62827e857ec..4caf5cfa639 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java @@ -50,7 +50,9 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; import org.apache.lucene.util.IOUtils; -/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files */ +/** Replica node, that pulls index changes from the primary node by copying newly flushed or merged index files. + * + * @lucene.experimental */ abstract class ReplicaNode extends Node { diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java index 72ed9213d3c..bae260615fe 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java @@ -36,7 +36,9 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.ThreadInterruptedException; /** A SearcherManager that refreshes via an externally provided (NRT) SegmentInfos, either from {@link IndexWriter} or via - * nrt replication to another index. */ + * nrt replication to another index. + * + * @lucene.experimental */ class SegmentInfosSearcherManager extends ReferenceManager { private volatile SegmentInfos currentInfos; private final Directory dir; 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 fff59fa68d1..d9222c0330c 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -58,10 +58,6 @@ import org.apache.lucene.util.ThreadInterruptedException; import com.carrotsearch.randomizedtesting.SeedUtils; -// nocommit why so many "hit SocketException during commit with R0"? - -// nocommit why all these NodeCommunicationExcs? - /* TODO - fangs @@ -1145,7 +1141,9 @@ public class TestStressNRTReplication extends LuceneTestCase { } } catch (IOException se) { // Assume primary crashed - message("top: indexer lost connection to primary"); + if (c != null) { + message("top: indexer lost connection to primary"); + } try { c.close(); } catch (Throwable t) { 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 aa892096a92..ef4397f40af 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java +++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java @@ -432,7 +432,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper { throw new AssertionError(); } - if (true || LuceneTestCase.VERBOSE) { + if (LuceneTestCase.VERBOSE) { System.out.println("MockDirectoryWrapper: " + action + " unsynced file: " + name); } } diff --git a/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java b/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java index 71cb012ae97..9ad40034e96 100644 --- a/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java +++ b/solr/core/src/java/org/apache/solr/core/StandardIndexReaderFactory.java @@ -26,7 +26,6 @@ import org.apache.lucene.store.Directory; * {@link DirectoryReader}. * * @see DirectoryReader#open(Directory) - * @see DirectoryReader#open(IndexWriter, boolean) */ public class StandardIndexReaderFactory extends IndexReaderFactory { From 564047059333ba391595f0dfb080ef2c3157d3c0 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Tue, 9 Feb 2016 10:43:36 -0500 Subject: [PATCH 16/19] cleanup some precommit failures --- .../java/org/apache/lucene/index/DirectoryReader.java | 2 +- .../src/java/org/apache/lucene/index/IndexWriter.java | 11 +++++++++-- .../java/org/apache/lucene/index/SegmentInfos.java | 2 +- .../apache/lucene/replicator/nrt/SimpleServer.java | 2 ++ .../lucene/replicator/nrt/TestNRTReplication.java | 2 ++ .../replicator/nrt/TestStressNRTReplication.java | 4 +++- 6 files changed, 18 insertions(+), 5 deletions(-) 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 c60e6c3b536..7f65d15c275 100644 --- a/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java @@ -93,7 +93,7 @@ public abstract class DirectoryReader extends BaseCompositeReader { * gain some performance by passing false. * @param writeAllDeletes If true, new deletes will be written * down to index files instead of carried over from writer to - * reader in heap + * reader directly in heap * * @see #open(IndexWriter) * 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 bde56867e60..eee17fc5ac1 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -438,7 +438,6 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { anyChanges |= maybeApplyDeletes(applyAllDeletes); if (writeAllDeletes) { // Must move the deletes to disk: - System.out.println("IW: now readerPool.commit"); readerPool.commit(segmentInfos); } @@ -1174,7 +1173,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { return docWriter.getNumDocs() + segmentInfos.totalMaxDoc(); } - /** If {@link SegmentInfos#getVersion} is below {@code newVersion} then update it to this value. */ + /** If {@link SegmentInfos#getVersion} is below {@code newVersion} then update it to this value. + * + * @lucene.internal */ public synchronized void advanceSegmentInfosVersion(long newVersion) { ensureOpen(); if (segmentInfos.getVersion() < newVersion) { @@ -2897,6 +2898,12 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { setCommitData(commitUserData, true); } + /** + * Sets the commit user data map, controlling whether to advance the {@link SegmentInfos#getVersion}. + * + * @see #setCommitData(Map) + * + * @lucene.internal */ public final synchronized void setCommitData(Map commitUserData, boolean doIncrementVersion) { segmentInfos.setUserData(new HashMap<>(commitUserData), doIncrementVersion); changeCount.incrementAndGet(); 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 3b95f25d90d..324f8ddace1 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -887,7 +887,7 @@ public final class SegmentInfos implements Cloneable, Iterable clientThreads = Collections.synchronizedSet(new HashSet<>()); 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 262e68e01d6..773390e3ece 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java @@ -35,6 +35,7 @@ import org.apache.lucene.util.LineFileDocs; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.SuppressForbidden; import org.apache.lucene.util.TestUtil; import com.carrotsearch.randomizedtesting.SeedUtils; @@ -53,6 +54,7 @@ public class TestNRTReplication extends LuceneTestCase { LineFileDocs docs; /** Launches a child "server" (separate JVM), which is either primary or replica node */ + @SuppressForbidden(reason = "ProcessBuilder requires java.io.File for CWD") private NodeProcess startNode(int primaryTCPPort, final int id, Path indexPath, long forcePrimaryVersion, boolean willCrash) throws IOException { List cmd = new ArrayList<>(); 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 d9222c0330c..630058925f9 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -50,9 +50,10 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LineFileDocs; -import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.SuppressForbidden; import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.ThreadInterruptedException; @@ -502,6 +503,7 @@ public class TestStressNRTReplication extends LuceneTestCase { } /** Launches a child "server" (separate JVM), which is either primary or replica node */ + @SuppressForbidden(reason = "ProcessBuilder requires java.io.File for CWD") NodeProcess startNode(final int id, Path indexPath, boolean isPrimary, long forcePrimaryVersion) throws IOException { nodeTimeStamps[id] = System.nanoTime(); List cmd = new ArrayList<>(); From 027bc0e4d65fb5089478134ae3b208a08cf5f90f Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Tue, 9 Feb 2016 13:59:05 -0500 Subject: [PATCH 17/19] cleanup stale TODOs; fix precommit --- .../org/apache/lucene/index/IndexWriter.java | 9 ++- .../org/apache/lucene/index/SegmentInfos.java | 8 ++- .../lucene/index/StandardDirectoryReader.java | 5 +- .../index/TestNumericDocValuesUpdates.java | 1 - .../lucene/replicator/nrt/CopyOneFile.java | 2 +- .../lucene/replicator/nrt/FileMetaData.java | 2 +- .../apache/lucene/replicator/nrt/Node.java | 63 +++++++++++-------- .../lucene/replicator/nrt/PrimaryNode.java | 8 ++- .../lucene/replicator/nrt/ReplicaNode.java | 15 ++--- .../nrt/SegmentInfosSearcherManager.java | 1 + .../lucene/replicator/nrt/package-info.java | 21 +++++++ .../replicator/nrt/SimplePrimaryNode.java | 2 +- .../replicator/nrt/SimpleReplicaNode.java | 7 ++- .../nrt/TestStressNRTReplication.java | 28 +++------ 14 files changed, 105 insertions(+), 67 deletions(-) create mode 100644 lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.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 eee17fc5ac1..8886ab12730 100644 --- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java +++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java @@ -4738,7 +4738,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - /** @lucene.internal */ + /** Record that the files referenced by this {@link SegmentInfos} are still in use. + * + * @lucene.internal */ public synchronized void incRefDeleter(SegmentInfos segmentInfos) throws IOException { ensureOpen(); deleter.incRef(segmentInfos, false); @@ -4747,7 +4749,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable { } } - /** @lucene.internal */ + /** Record that the files referenced by this {@link SegmentInfos} are no longer in use. Only call this if you are sure you previously + * called {@link #incRefDeleter}. + * + * @lucene.internal */ public synchronized void decRefDeleter(SegmentInfos segmentInfos) throws IOException { ensureOpen(); deleter.decRef(segmentInfos); 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 324f8ddace1..642b320cf37 100644 --- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java +++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java @@ -285,6 +285,7 @@ public final class SegmentInfos implements Cloneable, Iterable= this.generation; + if (generation < this.generation) { + throw new IllegalStateException("cannot decrease generation to " + generation + " from current generation " + this.generation); + } this.generation = generation; } @@ -843,6 +848,7 @@ public final class SegmentInfos implements Cloneable, Iterable data, boolean doIncrementVersion) { if (data == null) { userData = Collections.emptyMap(); 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 1261f9d0a3a..c0425c6c3b1 100644 --- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java +++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java @@ -30,6 +30,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.util.IOUtils; +/** Default implementation of {@link DirectoryReader}. */ public final class StandardDirectoryReader extends DirectoryReader { final IndexWriter writer; @@ -336,7 +337,9 @@ public final class StandardDirectoryReader extends DirectoryReader { return segmentInfos.getVersion(); } - /** @lucene.internal */ + /** Return the {@link SegmentInfos} for this reader. + * + * @lucene.internal */ public SegmentInfos getSegmentInfos() { return segmentInfos; } diff --git a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java index 26f1cbb91e1..33666463da1 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java @@ -54,7 +54,6 @@ import org.junit.Test; import com.carrotsearch.randomizedtesting.generators.RandomPicks; - @SuppressWarnings("resource") public class TestNumericDocValuesUpdates extends LuceneTestCase { diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java index e3f0f7d6381..faec77867e6 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java @@ -29,7 +29,7 @@ import org.apache.lucene.util.IOUtils; /** Copies one file from an incoming DataInput to a dest filename in a local Directory */ -class CopyOneFile implements Closeable { +public class CopyOneFile implements Closeable { private final DataInput in; private final IndexOutput out; private final ReplicaNode dest; diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java index 427b1d1a39e..897d5ca5af6 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/FileMetaData.java @@ -21,7 +21,7 @@ package org.apache.lucene.replicator.nrt; * * @lucene.experimental */ -class FileMetaData { +public class FileMetaData { // Header and footer of the file must be identical between primary and replica to consider the files equal: public final byte[] header; 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 34175723758..f5853cebcb4 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/Node.java @@ -21,6 +21,7 @@ import java.io.Closeable; import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.PrintStream; import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.util.ArrayList; @@ -78,6 +79,9 @@ abstract class Node implements Closeable { /** When this node was started */ public static final long localStartNS = System.nanoTime(); + /** For debug logging */ + protected final PrintStream printStream; + // public static final long globalStartNS; // For debugging: @@ -86,10 +90,11 @@ abstract class Node implements Closeable { /** File metadata for last sync that succeeded; we use this as a cache */ protected volatile Map lastFileMetaData; - public Node(int id, Directory dir, SearcherFactory searcherFactory) { + public Node(int id, Directory dir, SearcherFactory searcherFactory, PrintStream printStream) { this.id = id; this.dir = dir; this.searcherFactory = searcherFactory; + this.printStream = printStream; } @Override @@ -99,37 +104,41 @@ abstract class Node implements Closeable { public abstract void commit() throws IOException; - public static void nodeMessage(String message) { - long now = System.nanoTime(); - System.out.println(String.format(Locale.ROOT, - "%5.3fs %5.1fs: [%11s] %s", - (now-globalStartNS)/1000000000., - (now-localStartNS)/1000000000., - Thread.currentThread().getName(), - message)); - + public static void nodeMessage(PrintStream printStream, String message) { + if (printStream != null) { + long now = System.nanoTime(); + printStream.println(String.format(Locale.ROOT, + "%5.3fs %5.1fs: [%11s] %s", + (now-globalStartNS)/1000000000., + (now-localStartNS)/1000000000., + Thread.currentThread().getName(), + message)); + } } - public static void nodeMessage(int id, String message) { - long now = System.nanoTime(); - System.out.println(String.format(Locale.ROOT, - "%5.3fs %5.1fs: N%d [%11s] %s", - (now-globalStartNS)/1000000000., - (now-localStartNS)/1000000000., - id, - Thread.currentThread().getName(), - message)); - + public static void nodeMessage(PrintStream printStream, int id, String message) { + if (printStream != null) { + long now = System.nanoTime(); + printStream.println(String.format(Locale.ROOT, + "%5.3fs %5.1fs: N%d [%11s] %s", + (now-globalStartNS)/1000000000., + (now-localStartNS)/1000000000., + id, + Thread.currentThread().getName(), + message)); + } } protected void message(String message) { - long now = System.nanoTime(); - System.out.println(String.format(Locale.ROOT, - "%5.3fs %5.1fs: %7s %2s [%11s] %s", - (now-globalStartNS)/1000000000., - (now-localStartNS)/1000000000., - state, name(), - Thread.currentThread().getName(), message)); + if (printStream != null) { + long now = System.nanoTime(); + printStream.println(String.format(Locale.ROOT, + "%5.3fs %5.1fs: %7s %2s [%11s] %s", + (now-globalStartNS)/1000000000., + (now-localStartNS)/1000000000., + state, name(), + Thread.currentThread().getName(), message)); + } } public String name() { 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 7343b69e634..a045cde4cb5 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java @@ -18,6 +18,7 @@ package org.apache.lucene.replicator.nrt; */ import java.io.IOException; +import java.io.PrintStream; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -67,8 +68,9 @@ public abstract class PrimaryNode extends Node { private final AtomicInteger copyingCount = new AtomicInteger(); - public PrimaryNode(IndexWriter writer, int id, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory) throws IOException { - super(id, writer.getDirectory(), searcherFactory); + public PrimaryNode(IndexWriter writer, int id, long primaryGen, long forcePrimaryVersion, + SearcherFactory searcherFactory, PrintStream printStream) throws IOException { + super(id, writer.getDirectory(), searcherFactory, printStream); message("top: now init primary"); this.writer = writer; this.primaryGen = primaryGen; @@ -115,7 +117,7 @@ public abstract class PrimaryNode extends Node { } catch (Throwable t) { message("init: exception"); - t.printStackTrace(System.out); + t.printStackTrace(printStream); throw new RuntimeException(t); } } 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 4caf5cfa639..e7d40ac7a60 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java @@ -18,6 +18,7 @@ package org.apache.lucene.replicator.nrt; */ import java.io.IOException; +import java.io.PrintStream; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -54,7 +55,7 @@ import org.apache.lucene.util.IOUtils; * * @lucene.experimental */ -abstract class ReplicaNode extends Node { +public abstract class ReplicaNode extends Node { ReplicaFileDeleter deleter; @@ -79,8 +80,8 @@ abstract class ReplicaNode extends Node { /** Primary gen last time we successfully replicated: */ protected long lastPrimaryGen; - public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory) throws IOException { - super(id, dir, searcherFactory); + public ReplicaNode(int id, Directory dir, SearcherFactory searcherFactory, PrintStream printStream) throws IOException { + super(id, dir, searcherFactory, printStream); if (dir instanceof FSDirectory && ((FSDirectory) dir).checkPendingDeletions()) { throw new IllegalArgumentException("Directory " + dir + " still has pending deleted files; cannot initialize IndexWriter"); @@ -98,7 +99,7 @@ abstract class ReplicaNode extends Node { deleter = new ReplicaFileDeleter(this, dir); } catch (Throwable t) { message("exc on init:"); - t.printStackTrace(System.out); + t.printStackTrace(printStream); throw t; } finally { if (success == false) { @@ -307,7 +308,7 @@ abstract class ReplicaNode extends Node { } catch (Throwable t) { if (t.getMessage().startsWith("replica cannot start") == false) { message("exc on start:"); - t.printStackTrace(System.out); + t.printStackTrace(printStream); } else { dir.close(); } @@ -522,7 +523,7 @@ abstract class ReplicaNode extends Node { } catch (NodeCommunicationException nce) { // E.g. primary could crash/close when we are asking it for the copy state: message("top: ignoring communication exception creating CopyJob: " + nce); - //nce.printStackTrace(System.out); + //nce.printStackTrace(printStream); if (state.equals("syncing")) { state = "idle"; } @@ -560,7 +561,7 @@ abstract class ReplicaNode extends Node { } catch (NodeCommunicationException nce) { // E.g. primary could crash/close when we are asking it for the copy state: message("top: ignoring exception starting CopyJob: " + nce); - nce.printStackTrace(System.out); + nce.printStackTrace(printStream); if (state.equals("syncing")) { state = "idle"; } diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java index bae260615fe..bdb95101d9b 100644 --- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/SegmentInfosSearcherManager.java @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SegmentInfos; diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java new file mode 100644 index 00000000000..bbbae55dce9 --- /dev/null +++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/package-info.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + *

Near-real-time replication framework

+ */ +package org.apache.lucene.replicator.nrt; 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 d3415ad4441..93d20f7c749 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java @@ -116,7 +116,7 @@ class SimplePrimaryNode extends PrimaryNode { public SimplePrimaryNode(Random random, Path indexPath, int id, int tcpPort, long primaryGen, long forcePrimaryVersion, SearcherFactory searcherFactory, boolean doFlipBitsDuringCopy, boolean doCheckIndexOnClose) throws IOException { - super(initWriter(id, random, indexPath, doCheckIndexOnClose), id, primaryGen, forcePrimaryVersion, searcherFactory); + super(initWriter(id, random, indexPath, doCheckIndexOnClose), id, primaryGen, forcePrimaryVersion, searcherFactory, System.out); this.tcpPort = tcpPort; this.random = new Random(random.nextLong()); this.doFlipBitsDuringCopy = doFlipBitsDuringCopy; 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 9658ad17775..6948b78d2ee 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleReplicaNode.java @@ -64,8 +64,9 @@ class SimpleReplicaNode extends ReplicaNode { /** Changes over time, as primary node crashes and moves around */ int curPrimaryTCPPort; - public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, SearcherFactory searcherFactory, boolean doCheckIndexOnClose) throws IOException { - super(id, getDirectory(random, id, indexPath, doCheckIndexOnClose), searcherFactory); + public SimpleReplicaNode(Random random, int id, int tcpPort, Path indexPath, long curPrimaryGen, int primaryTCPPort, + SearcherFactory searcherFactory, boolean doCheckIndexOnClose) throws IOException { + super(id, getDirectory(random, id, indexPath, doCheckIndexOnClose), searcherFactory, System.out); this.tcpPort = tcpPort; this.random = new Random(random.nextLong()); @@ -139,7 +140,7 @@ class SimpleReplicaNode extends ReplicaNode { // Corrupt any index files not referenced by current commit point; this is important (increases test evilness) because we may have done // a hard crash of the previous JVM writing to this directory and so MDW's corrupt-unknown-files-on-close never ran: - Node.nodeMessage(id, "top: corrupt unknown files"); + Node.nodeMessage(System.out, id, "top: corrupt unknown files"); dir.corruptUnknownFiles(); return dir; 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 630058925f9..f2d0bf893d9 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestStressNRTReplication.java @@ -50,6 +50,7 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.MockDirectoryWrapper; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LineFileDocs; +import org.apache.lucene.util.LuceneTestCase.Nightly; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import org.apache.lucene.util.LuceneTestCase; @@ -63,29 +64,17 @@ import com.carrotsearch.randomizedtesting.SeedUtils; TODO - fangs - sometimes have one replica be really slow at copying / have random pauses (fake GC) / etc. - - graceful primary close - - why do we do the "rename temp to actual" all at the end...? what really does that buy us? - - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary? - test should not print scary exceptions and then succeed! - - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp... - are the pre-copied-completed-merged files not being cleared in primary? - hmm the logic isn't right today? a replica may skip pulling a given copy state, that recorded the finished merged segments? - - beast & fix bugs - - graceful cluster restart - - better translog integration - - get "graceful primary shutdown" working - - there is still some global state we rely on for "correctness", e.g. lastPrimaryVersion - - clean up how version is persisted in commit data - - why am i not using hashes here? how does ES use them? - - get all other "single shard" functions working too: this cluster should "act like" a single shard - - SLM - - controlled nrt reopen thread / returning long gen on write - - live field values - - add indexes - - make cluster level APIs to search, index, that deal w/ primary failover, etc. - - must prune xlog - - refuse to start primary unless we have quorum - later + - since all nodes are local, we could have a different test only impl that just does local file copies instead of via tcp... + - get all other "single shard" functions working too: this cluster should "act like" a single shard + - SLM + - controlled nrt reopen thread / returning long gen on write + - live field values + - add indexes + - replica should also track maxSegmentName its seen, and tap into inflateGens if it's later promoted to primary? - if we named index files using segment's ID we wouldn't have file name conflicts after primary crash / rollback? - back pressure on indexing if replicas can't keep up? - get xlog working on top? needs to be checkpointed, so we can correlate IW ops to NRT reader version and prune xlog based on commit @@ -190,6 +179,7 @@ public class TestStressNRTReplication extends LuceneTestCase { final Set crashingNodes = Collections.synchronizedSet(new HashSet<>()); + @Nightly public void test() throws Exception { Node.globalStartNS = System.nanoTime(); From 3a47dd29bcf5d3a277a16ced687cf296c85e8996 Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Wed, 10 Feb 2016 09:09:49 -0500 Subject: [PATCH 18/19] remove some core changes; add missing sync that caused stress test failure --- .../lucene/store/NRTCachingDirectory.java | 6 +- .../java/org/apache/lucene/util/IOUtils.java | 3 - .../replicator/nrt/SimplePrimaryNode.java | 60 ++++++++++--------- 3 files changed, 33 insertions(+), 36 deletions(-) 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 908722f49ba..22a9571b76b 100644 --- a/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java +++ b/lucene/core/src/java/org/apache/lucene/store/NRTCachingDirectory.java @@ -175,10 +175,8 @@ public class NRTCachingDirectory extends FilterDirectory implements Accountable @Override public void renameFile(String source, String dest) throws IOException { unCache(source); - try { - cache.deleteFile(dest); - } catch (FileNotFoundException fnfe) { - // OK -- it may not exist + if (cache.fileNameExists(dest)) { + throw new IllegalArgumentException("target file " + dest + " already exists"); } in.renameFile(source, dest); } 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 1a265428dcc..ce8884cf47f 100644 --- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java +++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java @@ -107,9 +107,6 @@ public final class IOUtils { * objects to call close() on */ public static void closeWhileHandlingException(Closeable... objects) { - if (objects.length == 0) { - throw new IllegalArgumentException("pass at least one Closeable"); - } closeWhileHandlingException(Arrays.asList(objects)); } 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 93d20f7c749..3d41b32870c 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimplePrimaryNode.java @@ -679,41 +679,43 @@ class SimplePrimaryNode extends PrimaryNode { int replicaTCPPort = in.readVInt(); message("new replica: " + warmingSegments.size() + " current warming merges"); // Step through all currently warming segments and try to add this replica if it isn't there already: - for(MergePreCopy preCopy : warmingSegments) { - message("warming segment " + preCopy.files.keySet()); - boolean found = false; - synchronized (preCopy.connections) { - for(Connection c : preCopy.connections) { - if (c.destTCPPort == replicaTCPPort) { - found = true; - break; + synchronized(warmingSegments) { + for(MergePreCopy preCopy : warmingSegments) { + message("warming segment " + preCopy.files.keySet()); + boolean found = false; + synchronized (preCopy.connections) { + for(Connection c : preCopy.connections) { + if (c.destTCPPort == replicaTCPPort) { + found = true; + break; + } } } - } - if (found) { - message("this replica is already warming this segment; skipping"); - // It's possible (maybe) that the replica started up, then a merge kicked off, and it warmed to this new replica, all before the - // replica sent us this command: - continue; - } + if (found) { + message("this replica is already warming this segment; skipping"); + // It's possible (maybe) that the replica started up, then a merge kicked off, and it warmed to this new replica, all before the + // replica sent us this command: + continue; + } - // OK, this new replica is not already warming this segment, so attempt (could fail) to start warming now: + // OK, this new replica is not already warming this segment, so attempt (could fail) to start warming now: - Connection c = new Connection(replicaTCPPort); - if (preCopy.tryAddConnection(c) == false) { - // This can happen, if all other replicas just now finished warming this segment, and so we were just a bit too late. In this - // case the segment will be copied over in the next nrt point sent to this replica - message("failed to add connection to segment warmer (too late); closing"); - c.close(); + Connection c = new Connection(replicaTCPPort); + if (preCopy.tryAddConnection(c) == false) { + // This can happen, if all other replicas just now finished warming this segment, and so we were just a bit too late. In this + // case the segment will be copied over in the next nrt point sent to this replica + message("failed to add connection to segment warmer (too late); closing"); + c.close(); + } + c.out.writeByte(SimpleReplicaNode.CMD_PRE_COPY_MERGE); + c.out.writeVLong(primaryGen); + c.out.writeVInt(tcpPort); + SimpleServer.writeFilesMetaData(c.out, preCopy.files); + c.flush(); + c.s.shutdownOutput(); + message("successfully started warming"); } - c.out.writeByte(SimpleReplicaNode.CMD_PRE_COPY_MERGE); - c.out.writeVLong(primaryGen); - c.out.writeVInt(tcpPort); - SimpleServer.writeFilesMetaData(c.out, preCopy.files); - c.flush(); - c.s.shutdownOutput(); - message("successfully started warming"); } break; From 20c38e7b875afd51f6f567e6bb90c6b16daa044c Mon Sep 17 00:00:00 2001 From: Mike McCandless Date: Thu, 11 Feb 2016 04:25:38 -0500 Subject: [PATCH 19/19] mark tests as Nightly --- .../lucene/replicator/nrt/TestNRTReplication.java | 11 +++++++++++ 1 file changed, 11 insertions(+) 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 773390e3ece..955d84eb257 100644 --- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java +++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/TestNRTReplication.java @@ -32,6 +32,7 @@ import java.util.regex.Pattern; import org.apache.lucene.document.Document; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LineFileDocs; +import org.apache.lucene.util.LuceneTestCase.Nightly; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.LuceneTestCase.SuppressSysoutChecks; import org.apache.lucene.util.LuceneTestCase; @@ -202,6 +203,7 @@ public class TestNRTReplication extends LuceneTestCase { docs.close(); } + @Nightly public void testReplicateDeleteAllDocuments() throws Exception { Path primaryPath = createTempDir("primary"); @@ -272,6 +274,7 @@ public class TestNRTReplication extends LuceneTestCase { primary.close(); } + @Nightly public void testReplicateForceMerge() throws Exception { Path primaryPath = createTempDir("primary"); @@ -321,6 +324,7 @@ public class TestNRTReplication extends LuceneTestCase { } // Start up, index 10 docs, replicate, but crash and restart the replica without committing it: + @Nightly public void testReplicaCrashNoCommit() throws Exception { Path primaryPath = createTempDir("primary"); @@ -366,6 +370,7 @@ public class TestNRTReplication extends LuceneTestCase { } // Start up, index 10 docs, replicate, commit, crash and restart the replica + @Nightly public void testReplicaCrashWithCommit() throws Exception { Path primaryPath = createTempDir("primary"); @@ -408,6 +413,7 @@ public class TestNRTReplication extends LuceneTestCase { } // Start up, index 10 docs, replicate, commit, crash, index more docs, replicate, then restart the replica + @Nightly public void testIndexingWhileReplicaIsDown() throws Exception { Path primaryPath = createTempDir("primary"); @@ -472,6 +478,7 @@ public class TestNRTReplication extends LuceneTestCase { } // Crash primary and promote a replica + @Nightly public void testCrashPrimary1() throws Exception { Path path1 = createTempDir("1"); @@ -515,6 +522,7 @@ public class TestNRTReplication extends LuceneTestCase { } // Crash primary and then restart it + @Nightly public void testCrashPrimary2() throws Exception { Path path1 = createTempDir("1"); @@ -581,6 +589,7 @@ public class TestNRTReplication extends LuceneTestCase { } // Crash primary and then restart it, while a replica node is down, then bring replica node back up and make sure it properly "unforks" itself + @Nightly public void testCrashPrimary3() throws Exception { Path path1 = createTempDir("1"); @@ -639,6 +648,7 @@ public class TestNRTReplication extends LuceneTestCase { replica.close(); } + @Nightly public void testCrashPrimaryWhileCopying() throws Exception { Path path1 = createTempDir("1"); @@ -747,6 +757,7 @@ public class TestNRTReplication extends LuceneTestCase { primary.close(); } + @Nightly public void testFullClusterCrash() throws Exception { Path path1 = createTempDir("1");