mirror of https://github.com/apache/lucene.git
current patch
This commit is contained in:
parent
b62c6715df
commit
1ae7291429
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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")) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 + ")";
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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?");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1 @@
|
|||
python -u /l/util/src/python/repeatLuceneTest.py -tmpDir /b/tmp -logDir /l/logs TestNRTReplication -jvms 1 -mult 4 -nightly
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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:
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
Loading…
Reference in New Issue