current patch

This commit is contained in:
Mike McCandless 2016-01-24 18:17:20 -05:00
parent b62c6715df
commit 1ae7291429
50 changed files with 6587 additions and 342 deletions

View File

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

View File

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

View File

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

View File

@ -45,7 +45,7 @@ public final class ReaderManager extends ReferenceManager<DirectoryReader> {
* @throws IOException If there is a low-level I/O error
*/
public ReaderManager(IndexWriter writer) throws IOException {
this(writer, true);
this(writer, true, false);
}
/**
@ -63,11 +63,13 @@ public final class ReaderManager extends ReferenceManager<DirectoryReader> {
* tolerate deleted documents being returned you might gain some
* performance by passing <code>false</code>. See
* {@link DirectoryReader#openIfChanged(DirectoryReader, IndexWriter, boolean)}.
* @param writeAllDeletes
* If <code>true</code>, new deletes will be forcefully written to index files.
*
* @throws IOException If there is a low-level I/O error
*/
public ReaderManager(IndexWriter writer, boolean applyAllDeletes) throws IOException {
current = DirectoryReader.open(writer, applyAllDeletes);
public ReaderManager(IndexWriter writer, boolean applyAllDeletes, boolean writeAllDeletes) throws IOException {
current = DirectoryReader.open(writer, applyAllDeletes, writeAllDeletes);
}
/**

View File

@ -279,126 +279,133 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
public static final SegmentInfos readCommit(Directory directory, String segmentFileName) throws IOException {
long generation = generationFromSegmentsFileName(segmentFileName);
//System.out.println(Thread.currentThread() + ": SegmentInfos.readCommit " + segmentFileName);
try (ChecksumIndexInput input = directory.openChecksumInput(segmentFileName, IOContext.READ)) {
// NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
// to read the magic ourselves.
int magic = input.readInt();
if (magic != CodecUtil.CODEC_MAGIC) {
throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
}
int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_50, VERSION_CURRENT);
byte id[] = new byte[StringHelper.ID_LENGTH];
input.readBytes(id, 0, id.length);
CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
SegmentInfos infos = new SegmentInfos();
infos.id = id;
infos.generation = generation;
infos.lastGeneration = generation;
if (format >= VERSION_53) {
// TODO: in the future (7.0? sigh) we can use this to throw IndexFormatTooOldException ... or just rely on the
// minSegmentLuceneVersion check instead:
infos.luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
} else {
// else compute the min version down below in the for loop
}
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<Integer,Set<String>> dvUpdateFiles;
final int numDVFields = input.readInt();
if (numDVFields == 0) {
dvUpdateFiles = Collections.emptyMap();
} else {
Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
for (int i = 0; i < numDVFields; i++) {
if (format >= VERSION_51) {
map.put(input.readInt(), input.readSetOfStrings());
} else {
map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
}
}
dvUpdateFiles = Collections.unmodifiableMap(map);
}
siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
infos.add(siPerCommit);
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<Integer,Set<String>> dvUpdateFiles;
final int numDVFields = input.readInt();
if (numDVFields == 0) {
dvUpdateFiles = Collections.emptyMap();
} else {
Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
for (int i = 0; i < numDVFields; i++) {
if (format >= VERSION_51) {
map.put(input.readInt(), input.readSetOfStrings());
} else {
map.put(input.readInt(), Collections.unmodifiableSet(input.readStringSet()));
}
}
dvUpdateFiles = Collections.unmodifiableMap(map);
}
siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
infos.add(siPerCommit);
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<String> unsupportedCodecs = Arrays.asList(
"Lucene3x", "Lucene40", "Lucene41", "Lucene42", "Lucene45", "Lucene46", "Lucene49", "Lucene410"
);
@ -454,68 +461,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
try {
segnOutput = directory.createOutput(segmentFileName, IOContext.DEFAULT);
CodecUtil.writeIndexHeader(segnOutput, "segments", VERSION_CURRENT,
StringHelper.randomId(), Long.toString(nextGeneration, Character.MAX_RADIX));
segnOutput.writeVInt(Version.LATEST.major);
segnOutput.writeVInt(Version.LATEST.minor);
segnOutput.writeVInt(Version.LATEST.bugfix);
segnOutput.writeLong(version);
segnOutput.writeInt(counter); // write counter
segnOutput.writeInt(size());
if (size() > 0) {
Version minSegmentVersion = null;
// We do a separate loop up front so we can write the minSegmentVersion before
// any SegmentInfo; this makes it cleaner to throw IndexFormatTooOldExc at read time:
for (SegmentCommitInfo siPerCommit : this) {
Version segmentVersion = siPerCommit.info.getVersion();
if (minSegmentVersion == null || segmentVersion.onOrAfter(minSegmentVersion) == false) {
minSegmentVersion = segmentVersion;
}
}
segnOutput.writeVInt(minSegmentVersion.major);
segnOutput.writeVInt(minSegmentVersion.minor);
segnOutput.writeVInt(minSegmentVersion.bugfix);
}
// write infos
for (SegmentCommitInfo siPerCommit : this) {
SegmentInfo si = siPerCommit.info;
segnOutput.writeString(si.name);
byte segmentID[] = si.getId();
// TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
if (segmentID == null) {
segnOutput.writeByte((byte)0);
} else {
if (segmentID.length != StringHelper.ID_LENGTH) {
throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
}
segnOutput.writeByte((byte)1);
segnOutput.writeBytes(segmentID, segmentID.length);
}
segnOutput.writeString(si.getCodec().getName());
segnOutput.writeLong(siPerCommit.getDelGen());
int delCount = siPerCommit.getDelCount();
if (delCount < 0 || delCount > si.maxDoc()) {
throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
}
segnOutput.writeInt(delCount);
segnOutput.writeLong(siPerCommit.getFieldInfosGen());
segnOutput.writeLong(siPerCommit.getDocValuesGen());
segnOutput.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
segnOutput.writeInt(dvUpdatesFiles.size());
for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
segnOutput.writeInt(e.getKey());
segnOutput.writeSetOfStrings(e.getValue());
}
}
segnOutput.writeMapOfStrings(userData);
CodecUtil.writeFooter(segnOutput);
write(directory, segnOutput);
segnOutput.close();
directory.sync(Collections.singleton(segmentFileName));
success = true;
@ -533,6 +479,72 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
}
}
public void write(Directory directory, IndexOutput out) throws IOException {
CodecUtil.writeIndexHeader(out, "segments", VERSION_CURRENT,
StringHelper.randomId(), Long.toString(generation, Character.MAX_RADIX));
out.writeVInt(Version.LATEST.major);
out.writeVInt(Version.LATEST.minor);
out.writeVInt(Version.LATEST.bugfix);
System.out.println(Thread.currentThread().getName() + ": now write " + out.getName() + " with version=" + version);
out.writeLong(version);
out.writeInt(counter); // write counter
out.writeInt(size());
if (size() > 0) {
Version minSegmentVersion = null;
// We do a separate loop up front so we can write the minSegmentVersion before
// any SegmentInfo; this makes it cleaner to throw IndexFormatTooOldExc at read time:
for (SegmentCommitInfo siPerCommit : this) {
Version segmentVersion = siPerCommit.info.getVersion();
if (minSegmentVersion == null || segmentVersion.onOrAfter(minSegmentVersion) == false) {
minSegmentVersion = segmentVersion;
}
}
out.writeVInt(minSegmentVersion.major);
out.writeVInt(minSegmentVersion.minor);
out.writeVInt(minSegmentVersion.bugfix);
}
// write infos
for (SegmentCommitInfo siPerCommit : this) {
SegmentInfo si = siPerCommit.info;
out.writeString(si.name);
byte segmentID[] = si.getId();
// TODO: remove this in lucene 6, we don't need to include 4.x segments in commits anymore
if (segmentID == null) {
out.writeByte((byte)0);
} else {
if (segmentID.length != StringHelper.ID_LENGTH) {
throw new IllegalStateException("cannot write segment: invalid id segment=" + si.name + "id=" + StringHelper.idToString(segmentID));
}
out.writeByte((byte)1);
out.writeBytes(segmentID, segmentID.length);
}
out.writeString(si.getCodec().getName());
out.writeLong(siPerCommit.getDelGen());
int delCount = siPerCommit.getDelCount();
if (delCount < 0 || delCount > si.maxDoc()) {
throw new IllegalStateException("cannot write segment: invalid maxDoc segment=" + si.name + " maxDoc=" + si.maxDoc() + " delCount=" + delCount);
}
out.writeInt(delCount);
out.writeLong(siPerCommit.getFieldInfosGen());
out.writeLong(siPerCommit.getDocValuesGen());
out.writeSetOfStrings(siPerCommit.getFieldInfosFiles());
final Map<Integer,Set<String>> dvUpdatesFiles = siPerCommit.getDocValuesUpdatesFiles();
out.writeInt(dvUpdatesFiles.size());
for (Entry<Integer,Set<String>> e : dvUpdatesFiles.entrySet()) {
out.writeInt(e.getKey());
out.writeSetOfStrings(e.getValue());
}
}
out.writeMapOfStrings(userData);
CodecUtil.writeFooter(out);
}
/**
* Returns a copy of this instance, also copying each
* SegmentInfo.
@ -698,8 +710,10 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
protected abstract T doBody(String segmentFileName) throws IOException;
}
// Carry over generation numbers from another SegmentInfos
void updateGeneration(SegmentInfos other) {
/** Carry over generation numbers from another SegmentInfos
*
* @lucene.internal */
public void updateGeneration(SegmentInfos other) {
lastGeneration = other.lastGeneration;
generation = other.generation;
}
@ -711,7 +725,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
this.counter = other.counter;
}
void setNextWriteGeneration(long generation) {
public void setNextWriteGeneration(long generation) {
assert generation >= this.generation;
this.generation = generation;
}
@ -777,7 +791,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
boolean success = false;
final String dest;
try {
final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
final String src = IndexFileNames.fileNameFromGeneration(IndexFileNames.PENDING_SEGMENTS, "", generation);
dest = IndexFileNames.fileNameFromGeneration(IndexFileNames.SEGMENTS, "", generation);
dir.renameFile(src, dest);
success = true;
@ -800,7 +814,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
* method if changes have been made to this {@link SegmentInfos} instance
* </p>
**/
final void commit(Directory dir) throws IOException {
public final void commit(Directory dir) throws IOException {
prepareCommit(dir);
finishCommit(dir);
}
@ -829,14 +843,15 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
return userData;
}
void setUserData(Map<String,String> data) {
public void setUserData(Map<String,String> data, boolean doIncrementVersion) {
if (data == null) {
userData = Collections.<String,String>emptyMap();
} else {
userData = data;
}
changed();
if (doIncrementVersion) {
changed();
}
}
/** Replaces all segments in this instance, but keeps
@ -864,8 +879,18 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
* segments. */
public void changed() {
version++;
System.out.println(Thread.currentThread().getName() + ": SIS.change to version=" + version);
//new Throwable().printStackTrace(System.out);
}
void setVersion(long newVersion) {
if (newVersion < version) {
throw new IllegalArgumentException("newVersion (=" + newVersion + ") cannot be less than current version (=" + version + ")");
}
System.out.println(Thread.currentThread().getName() + ": SIS.setVersion change from " + version + " to " + newVersion);
version = newVersion;
}
/** applies all changes caused by committing a merge to this SegmentInfos */
void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) {
final Set<SegmentCommitInfo> mergedAway = new HashSet<>(merge.segments);

View File

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

View File

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

View File

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

View File

@ -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);
}

View File

@ -107,6 +107,9 @@ public final class IOUtils {
* objects to call <tt>close()</tt> on
*/
public static void closeWhileHandlingException(Closeable... objects) {
if (objects.length == 0) {
throw new IllegalArgumentException("pass at least one Closeable");
}
closeWhileHandlingException(Arrays.asList(objects));
}

View File

@ -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);

View File

@ -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();

View File

@ -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);

View File

@ -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();
}
}

View File

@ -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);

View File

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

View File

@ -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();

View File

@ -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

View File

@ -94,7 +94,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
}
};
final boolean applyDeletes = random().nextBoolean();
final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, searcherFactory);
final SearcherManager mgr = new SearcherManager(w.w, applyDeletes, false, searcherFactory);
final AtomicBoolean indexing = new AtomicBoolean(true);
final AtomicReference<Throwable> error = new AtomicReference<>();
final int numDocs = atLeast(10000);

View File

@ -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);

View File

@ -76,7 +76,7 @@ public class SearcherTaxonomyManager extends ReferenceManager<SearcherTaxonomyMa
this.searcherFactory = searcherFactory;
this.taxoWriter = taxoWriter;
DirectoryTaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoWriter);
current = new SearcherAndTaxonomy(SearcherManager.getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes), null), taxoReader);
current = new SearcherAndTaxonomy(SearcherManager.getSearcher(searcherFactory, DirectoryReader.open(writer, applyAllDeletes, false), null), taxoReader);
this.taxoEpoch = taxoWriter.getTaxonomyEpoch();
}

View File

@ -271,7 +271,7 @@ public class DirectoryTaxonomyWriter implements TaxonomyWriter {
// verify that the taxo-writer hasn't been closed on us.
ensureOpen();
if (!initializedReaderManager) {
readerManager = new ReaderManager(indexWriter, false);
readerManager = new ReaderManager(indexWriter, false, false);
shouldRefreshReaderManager = false;
initializedReaderManager = true;
}

View File

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

View File

@ -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;
}
}

View File

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

View File

@ -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;
}
}

View File

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

View File

@ -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);
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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();
}
}

View File

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

View File

@ -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();
}
}

View File

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

View File

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

View File

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

View File

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

View File

@ -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();
}
}

View File

@ -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;
}
}

View File

@ -0,0 +1 @@
python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestNRTReplication -jvms 1 -mult 4 -nightly

View File

@ -637,7 +637,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
for (int iter=0;iter<iters && failed.get() == false;iter++) {
if (VERBOSE) {
System.out.println("\nTEST: iter=" + iter + " s=" + s);
System.out.println("\n" + Thread.currentThread().getName() + ": TEST: iter=" + iter + " s=" + s);
}
Query query;
VerifyHits verifyHits;

View File

@ -457,7 +457,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
if (random().nextBoolean()) {
DirectoryReader ir = null;
try {
ir = DirectoryReader.open(iw, random().nextBoolean());
ir = DirectoryReader.open(iw, random().nextBoolean(), false);
dir.setRandomIOExceptionRateOnOpen(0.0); // disable exceptions on openInput until next iteration
TestUtil.checkReader(ir);
} finally {

View File

@ -303,7 +303,7 @@ public class RandomIndexWriter implements Closeable {
public DirectoryReader getReader() throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
return getReader(true);
return getReader(true, false);
}
private boolean doRandomForceMerge = true;
@ -353,7 +353,7 @@ public class RandomIndexWriter implements Closeable {
}
}
public DirectoryReader getReader(boolean applyDeletions) throws IOException {
public DirectoryReader getReader(boolean applyDeletions, boolean writeAllDeletes) throws IOException {
LuceneTestCase.maybeChangeLiveIndexWriterConfig(r, w.getConfig());
getReaderCalled = true;
if (r.nextInt(20) == 2) {
@ -366,7 +366,7 @@ public class RandomIndexWriter implements Closeable {
if (r.nextInt(5) == 1) {
w.commit();
}
return w.getReader(applyDeletions);
return w.getReader(applyDeletions, writeAllDeletes);
} else {
if (LuceneTestCase.VERBOSE) {
System.out.println("RIW.getReader: open new reader");
@ -375,7 +375,7 @@ public class RandomIndexWriter implements Closeable {
if (r.nextBoolean()) {
return DirectoryReader.open(w.getDirectory());
} else {
return w.getReader(applyDeletions);
return w.getReader(applyDeletions, writeAllDeletes);
}
}
}

View File

@ -38,8 +38,10 @@ import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Matcher;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.NoDeletionPolicy;
@ -239,12 +241,24 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
if (openFiles.containsKey(source)) {
if (assertNoDeleteOpenFile) {
throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
} else if (noDeleteOpenFile) {
throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: file \"" + source + "\" is still open: cannot rename"), source, true);
throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: source file \"" + source + "\" is still open: cannot rename"), source, true);
}
}
if (openFiles.containsKey(dest)) {
if (assertNoDeleteOpenFile) {
throw (AssertionError) fillOpenTrace(new AssertionError("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
} else if (noDeleteOpenFile) {
throw (IOException) fillOpenTrace(new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" is still open: cannot rename"), dest, true);
}
}
if (createdFiles.contains(dest)) {
throw new IOException("MockDirectoryWrapper: dest file \"" + dest + "\" already exists: cannot rename");
}
boolean success = false;
try {
in.renameFile(source, dest);
@ -257,6 +271,8 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
unSyncedFiles.add(dest);
}
openFilesDeleted.remove(source);
triedToDelete.remove(dest);
createdFiles.add(dest);
}
}
}
@ -278,15 +294,203 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
}
}
public synchronized void corruptUnknownFiles() throws IOException {
System.out.println("MDW: corrupt unknown files");
Set<String> knownFiles = new HashSet<>();
for(String fileName : listAll()) {
if (fileName.startsWith(IndexFileNames.SEGMENTS)) {
System.out.println("MDW: read " + fileName + " to gather files it references");
knownFiles.addAll(SegmentInfos.readCommit(this, fileName).files(true));
}
}
Set<String> toCorrupt = new HashSet<>();
Matcher m = IndexFileNames.CODEC_FILE_PATTERN.matcher("");
for(String fileName : listAll()) {
m.reset(fileName);
if (knownFiles.contains(fileName) == false &&
fileName.endsWith("write.lock") == false &&
(m.matches() || fileName.startsWith(IndexFileNames.PENDING_SEGMENTS))) {
toCorrupt.add(fileName);
}
}
corruptFiles(toCorrupt);
}
public synchronized void corruptFiles(Collection<String> files) {
// Must make a copy because we change the incoming unsyncedFiles
// when we create temp files, delete, etc., below:
for(String name : new ArrayList<>(files)) {
int damage = randomState.nextInt(6);
String action = null;
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<String> it = unSyncedFiles.iterator();
unSyncedFiles = new HashSet<>();
// first force-close all files, so we can corrupt on windows etc.
// clone the file map, as these guys want to remove themselves on close.
Map<Closeable,Exception> m = new IdentityHashMap<>(openFileHandles);
@ -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<String> files = new HashSet<>(Arrays.asList(listAll()));
// Disregard what happens with the pendingDeletions files:
files.removeAll(pendingDeletions);
String[] endFiles = files.toArray(new String[0]);
Set<String> startSet = new TreeSet<>(Arrays.asList(startFiles));
Set<String> endSet = new TreeSet<>(Arrays.asList(endFiles));
@ -839,7 +986,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
assert pendingDeletions.contains(s);
if (LuceneTestCase.VERBOSE) {
System.out.println("MDW: Unreferenced check: Ignoring referenced file: " + s + " " +
"from " + file + " that we could not delete.");
"from " + file + " that we could not delete.");
}
startSet.add(s);
}
@ -884,7 +1031,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
extras += "\n\nThese files we had previously tried to delete, but couldn't: " + pendingDeletions;
}
throw new RuntimeException("unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
throw new RuntimeException(this + ": unreferenced files: before delete:\n " + Arrays.toString(startFiles) + "\n after delete:\n " + Arrays.toString(endFiles) + extras);
}
DirectoryReader ir1 = DirectoryReader.open(this);
@ -1036,7 +1183,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
}
}
// don't override optional methods like copyFrom: we need the default impl for things like disk
// full checks. we randomly exercise "raw" directories anyway. We ensure default impls are used:

View File

@ -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;
}

View File

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