[STORE]: Make use of Lucene build-in checksums
Since Lucene version 4.8 each file has a checksum written as it's footer. We used to calculate the checksums for all files transparently on the filesystem layer (Directory / Store) which is now not necessary anymore. This commit makes use of the new checksums in a backwards compatible way such that files written with the old checksum mechanism are still compared against the corresponding Alder32 checksum while newer files are compared against the Lucene build in CRC32 checksum. Since now every written file is checksummed by default this commit also verifies the checksum for files during recovery and restore if applicable. Closes #5924 This commit also has a fix for #6808 since the added tests in `CorruptedFileTest.java` exposed the issue. Closes #6808
This commit is contained in:
parent
9ca5e6e3e1
commit
72e6150bc1
|
@ -1,101 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.store;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.zip.Checksum;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class BufferedChecksumIndexOutput extends BufferedIndexOutput {
|
||||
|
||||
private final IndexOutput delegate;
|
||||
private final BufferedIndexOutput bufferedDelegate;
|
||||
private final Checksum digest;
|
||||
|
||||
public BufferedChecksumIndexOutput(IndexOutput delegate, Checksum digest) {
|
||||
super(delegate instanceof BufferedIndexOutput ? ((BufferedIndexOutput) delegate).getBufferSize() : BufferedIndexOutput.DEFAULT_BUFFER_SIZE);
|
||||
if (delegate instanceof BufferedIndexOutput) {
|
||||
bufferedDelegate = (BufferedIndexOutput) delegate;
|
||||
this.delegate = delegate;
|
||||
} else {
|
||||
this.delegate = delegate;
|
||||
bufferedDelegate = null;
|
||||
}
|
||||
this.digest = digest;
|
||||
}
|
||||
|
||||
public Checksum digest() {
|
||||
return digest;
|
||||
}
|
||||
|
||||
public IndexOutput underlying() {
|
||||
return this.delegate;
|
||||
}
|
||||
|
||||
// don't override it, base class method simple reads from input and writes to this output
|
||||
// @Override public void copyBytes(IndexInput input, long numBytes) throws IOException {
|
||||
// delegate.copyBytes(input, numBytes);
|
||||
// }
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
super.close();
|
||||
} finally {
|
||||
delegate.close();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void flushBuffer(byte[] b, int offset, int len) throws IOException {
|
||||
if (bufferedDelegate != null) {
|
||||
bufferedDelegate.flushBuffer(b, offset, len);
|
||||
} else {
|
||||
delegate.writeBytes(b, offset, len);
|
||||
}
|
||||
digest.update(b, offset, len);
|
||||
}
|
||||
|
||||
// don't override it, base class method simple reads from input and writes to this output
|
||||
// @Override public void copyBytes(IndexInput input, long numBytes) throws IOException {
|
||||
// delegate.copyBytes(input, numBytes);
|
||||
// }
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
try {
|
||||
super.flush();
|
||||
} finally {
|
||||
delegate.flush();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long length() throws IOException {
|
||||
return delegate.length();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return delegate.toString();
|
||||
}
|
||||
}
|
|
@ -1,155 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.lucene.store;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.zip.CRC32;
|
||||
|
||||
/** Base implementation class for buffered {@link IndexOutput}. */
|
||||
public abstract class BufferedIndexOutput extends IndexOutput {
|
||||
/** The default buffer size in bytes ({@value #DEFAULT_BUFFER_SIZE}). */
|
||||
public static final int DEFAULT_BUFFER_SIZE = 16384;
|
||||
|
||||
private final int bufferSize;
|
||||
private final byte[] buffer;
|
||||
private long bufferStart = 0; // position in file of buffer
|
||||
private int bufferPosition = 0; // position in buffer
|
||||
private final CRC32 crc = new CRC32();
|
||||
|
||||
/**
|
||||
* Creates a new {@link BufferedIndexOutput} with the default buffer size
|
||||
* ({@value #DEFAULT_BUFFER_SIZE} bytes see {@link #DEFAULT_BUFFER_SIZE})
|
||||
*/
|
||||
public BufferedIndexOutput() {
|
||||
this(DEFAULT_BUFFER_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link BufferedIndexOutput} with the given buffer size.
|
||||
* @param bufferSize the buffer size in bytes used to buffer writes internally.
|
||||
* @throws IllegalArgumentException if the given buffer size is less or equal to <tt>0</tt>
|
||||
*/
|
||||
public BufferedIndexOutput(int bufferSize) {
|
||||
if (bufferSize <= 0) {
|
||||
throw new IllegalArgumentException("bufferSize must be greater than 0 (got " + bufferSize + ")");
|
||||
}
|
||||
this.bufferSize = bufferSize;
|
||||
buffer = new byte[bufferSize];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeByte(byte b) throws IOException {
|
||||
if (bufferPosition >= bufferSize)
|
||||
flush();
|
||||
buffer[bufferPosition++] = b;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeBytes(byte[] b, int offset, int length) throws IOException {
|
||||
int bytesLeft = bufferSize - bufferPosition;
|
||||
// is there enough space in the buffer?
|
||||
if (bytesLeft >= length) {
|
||||
// we add the data to the end of the buffer
|
||||
System.arraycopy(b, offset, buffer, bufferPosition, length);
|
||||
bufferPosition += length;
|
||||
// if the buffer is full, flush it
|
||||
if (bufferSize - bufferPosition == 0)
|
||||
flush();
|
||||
} else {
|
||||
// is data larger then buffer?
|
||||
if (length > bufferSize) {
|
||||
// we flush the buffer
|
||||
if (bufferPosition > 0)
|
||||
flush();
|
||||
// and write data at once
|
||||
crc.update(b, offset, length);
|
||||
flushBuffer(b, offset, length);
|
||||
bufferStart += length;
|
||||
} else {
|
||||
// we fill/flush the buffer (until the input is written)
|
||||
int pos = 0; // position in the input data
|
||||
int pieceLength;
|
||||
while (pos < length) {
|
||||
pieceLength = (length - pos < bytesLeft) ? length - pos : bytesLeft;
|
||||
System.arraycopy(b, pos + offset, buffer, bufferPosition, pieceLength);
|
||||
pos += pieceLength;
|
||||
bufferPosition += pieceLength;
|
||||
// if the buffer is full, flush it
|
||||
bytesLeft = bufferSize - bufferPosition;
|
||||
if (bytesLeft == 0) {
|
||||
flush();
|
||||
bytesLeft = bufferSize;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
crc.update(buffer, 0, bufferPosition);
|
||||
flushBuffer(buffer, bufferPosition);
|
||||
bufferStart += bufferPosition;
|
||||
bufferPosition = 0;
|
||||
}
|
||||
|
||||
/** Expert: implements buffer write. Writes bytes at the current position in
|
||||
* the output.
|
||||
* @param b the bytes to write
|
||||
* @param len the number of bytes to write
|
||||
*/
|
||||
private void flushBuffer(byte[] b, int len) throws IOException {
|
||||
flushBuffer(b, 0, len);
|
||||
}
|
||||
|
||||
/** Expert: implements buffer write. Writes bytes at the current position in
|
||||
* the output.
|
||||
* @param b the bytes to write
|
||||
* @param offset the offset in the byte array
|
||||
* @param len the number of bytes to write
|
||||
*/
|
||||
protected abstract void flushBuffer(byte[] b, int offset, int len) throws IOException;
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
flush();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFilePointer() {
|
||||
return bufferStart + bufferPosition;
|
||||
}
|
||||
|
||||
@Override
|
||||
public abstract long length() throws IOException;
|
||||
|
||||
/**
|
||||
* Returns size of the used output buffer in bytes.
|
||||
* */
|
||||
public final int getBufferSize() {
|
||||
return bufferSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getChecksum() throws IOException {
|
||||
flush();
|
||||
return crc.getValue();
|
||||
}
|
||||
}
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.rest.RestStatus;
|
|||
|
||||
import java.io.PrintWriter;
|
||||
import java.io.StringWriter;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -120,4 +121,47 @@ public final class ExceptionsHelper {
|
|||
e.printStackTrace(printWriter);
|
||||
return stackTraceStringWriter.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Rethrows the first exception in the list and adds all remaining to the suppressed list.
|
||||
* If the given list is empty no exception is thrown
|
||||
*
|
||||
*/
|
||||
public static <T extends Throwable> void rethrowAndSuppress(List<T> exceptions) throws T {
|
||||
T main = null;
|
||||
for (T ex : exceptions) {
|
||||
if (main == null) {
|
||||
main = ex;
|
||||
} else {
|
||||
main.addSuppressed(ex);
|
||||
}
|
||||
}
|
||||
if (main != null) {
|
||||
throw main;
|
||||
}
|
||||
}
|
||||
|
||||
public static <T extends Throwable> T unwrap(Throwable t, Class<T> clazz) {
|
||||
if (t != null) {
|
||||
do {
|
||||
if (clazz.isInstance(t)) {
|
||||
return clazz.cast(t);
|
||||
}
|
||||
} while ((t = t.getCause()) != null);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the given throwable is and OutOfMemoryException, otherwise <code>false</code>
|
||||
*/
|
||||
public static boolean isOOM(Throwable t) {
|
||||
return t != null
|
||||
&& (t instanceof OutOfMemoryError
|
||||
|| (t instanceof IllegalStateException
|
||||
&& t.getMessage() != null
|
||||
&& t.getMessage().contains("OutOfMemoryError")
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -247,7 +247,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|||
* no primary is found or the primary is not active.
|
||||
*/
|
||||
public MutableShardRouting activePrimary(ShardRouting shard) {
|
||||
assert !shard.primary();
|
||||
for (MutableShardRouting shardRouting : assignedShards(shard.shardId())) {
|
||||
if (shardRouting.primary() && shardRouting.active()) {
|
||||
return shardRouting;
|
||||
|
|
|
@ -420,11 +420,28 @@ public class AllocationService extends AbstractComponent {
|
|||
}
|
||||
|
||||
RoutingNodes routingNodes = allocation.routingNodes();
|
||||
boolean dirty = false;
|
||||
if (failedShard.primary()) {
|
||||
// we have to fail the initializing replicas if the primary fails
|
||||
// since they might now yet have started the recovery and then they will
|
||||
// stick in the cluster-state forever since the replica has a retry logic that
|
||||
// retries infinitely in that case.
|
||||
List<MutableShardRouting> initializingReplicas = new ArrayList<>();
|
||||
for (MutableShardRouting shard : routingNodes.assignedShards(failedShard)){
|
||||
if (!shard.primary() && shard.initializing()) {
|
||||
initializingReplicas.add(shard);
|
||||
}
|
||||
}
|
||||
// we can't do this in the loop above since we modify the iterator and will get
|
||||
// concurrent modification exceptions
|
||||
for (MutableShardRouting shard : initializingReplicas) {
|
||||
dirty |= applyFailedShard(allocation, shard, addToIgnoreList);
|
||||
}
|
||||
}
|
||||
if (failedShard.relocatingNodeId() != null) {
|
||||
// the shard is relocating, either in initializing (recovery from another node) or relocating (moving to another node)
|
||||
if (failedShard.state() == INITIALIZING) {
|
||||
// the shard is initializing and recovering from another node
|
||||
boolean dirty = false;
|
||||
// first, we need to cancel the current node that is being initialized
|
||||
RoutingNodes.RoutingNodeIterator initializingNode = routingNodes.routingNodeIter(failedShard.currentNodeId());
|
||||
if (initializingNode != null) {
|
||||
|
@ -459,7 +476,6 @@ public class AllocationService extends AbstractComponent {
|
|||
}
|
||||
return dirty;
|
||||
} else if (failedShard.state() == RELOCATING) {
|
||||
boolean dirty = false;
|
||||
// the shard is relocating, meaning its the source the shard is relocating from
|
||||
// first, we need to cancel the current relocation from the current node
|
||||
// now, find the node that we are recovering from, cancel the relocation, remove it from the node
|
||||
|
@ -497,13 +513,11 @@ public class AllocationService extends AbstractComponent {
|
|||
} else {
|
||||
logger.debug("failed shard {} not found in routingNodes, ignoring it", failedShard);
|
||||
}
|
||||
return dirty;
|
||||
} else {
|
||||
throw new ElasticsearchIllegalStateException("illegal state for a failed shard, relocating node id is set, but state does not match: " + failedShard);
|
||||
}
|
||||
} else {
|
||||
// the shard is not relocating, its either started, or initializing, just cancel it and move on...
|
||||
boolean dirty = false;
|
||||
RoutingNodes.RoutingNodeIterator node = routingNodes.routingNodeIter(failedShard.currentNodeId());
|
||||
if (node != null) {
|
||||
while (node.hasNext()) {
|
||||
|
@ -541,7 +555,7 @@ public class AllocationService extends AbstractComponent {
|
|||
if (!dirty) {
|
||||
logger.debug("failed shard {} not found in routingNodes, ignoring it", failedShard);
|
||||
}
|
||||
return dirty;
|
||||
}
|
||||
return dirty;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,6 +54,8 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
|
|||
|
||||
public static final String CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES = "cluster.routing.allocation.node_initial_primaries_recoveries";
|
||||
public static final String CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES = "cluster.routing.allocation.node_concurrent_recoveries";
|
||||
public static final String CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES = "cluster.routing.allocation.concurrent_recoveries";
|
||||
|
||||
public static final int DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES = 2;
|
||||
public static final int DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES = 4;
|
||||
|
||||
|
@ -65,7 +67,7 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
|
|||
super(settings);
|
||||
|
||||
this.primariesInitialRecoveries = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES);
|
||||
this.concurrentRecoveries = settings.getAsInt("cluster.routing.allocation.concurrent_recoveries", settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES));
|
||||
this.concurrentRecoveries = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES));
|
||||
logger.debug("using node_concurrent_recoveries [{}], node_initial_primaries_recoveries [{}]", concurrentRecoveries, primariesInitialRecoveries);
|
||||
|
||||
nodeSettingsService.addListener(new ApplySettings());
|
||||
|
|
|
@ -21,15 +21,17 @@ package org.elasticsearch.common.lucene;
|
|||
|
||||
import org.apache.lucene.analysis.core.KeywordAnalyzer;
|
||||
import org.apache.lucene.analysis.standard.StandardAnalyzer;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.ElasticsearchIllegalArgumentException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -96,6 +98,28 @@ public class Lucene {
|
|||
return sis;
|
||||
}
|
||||
|
||||
public static void checkSegmentInfoIntegrity(final Directory directory) throws IOException {
|
||||
new SegmentInfos.FindSegmentsFile(directory) {
|
||||
|
||||
@Override
|
||||
protected Object doBody(String segmentFileName) throws IOException {
|
||||
try (IndexInput input = directory.openInput(segmentFileName, IOContext.READ)) {
|
||||
final int format = input.readInt();
|
||||
final int actualFormat;
|
||||
if (format == CodecUtil.CODEC_MAGIC) {
|
||||
// 4.0+
|
||||
actualFormat = CodecUtil.checkHeaderNoMagic(input, "segments", SegmentInfos.VERSION_40, Integer.MAX_VALUE);
|
||||
if (actualFormat >= SegmentInfos.VERSION_48) {
|
||||
CodecUtil.checksumEntireFile(input);
|
||||
}
|
||||
}
|
||||
// legacy....
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}.run();
|
||||
}
|
||||
|
||||
public static long count(IndexSearcher searcher, Query query) throws IOException {
|
||||
TotalHitCountCollector countCollector = new TotalHitCountCollector();
|
||||
// we don't need scores, so wrap it in a constant score query
|
||||
|
@ -371,4 +395,8 @@ public class Lucene {
|
|||
public static final boolean indexExists(final Directory directory) throws IOException {
|
||||
return DirectoryReader.indexExists(directory);
|
||||
}
|
||||
|
||||
public static boolean isCorruptionException(Throwable t) {
|
||||
return ExceptionsHelper.unwrap(t, CorruptIndexException.class) != null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -50,9 +50,7 @@ import org.elasticsearch.index.store.StoreFileMetaData;
|
|||
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
|
||||
import org.elasticsearch.transport.ConnectTransportException;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
/**
|
||||
|
|
|
@ -135,7 +135,7 @@ public interface Engine extends IndexShardComponent, CloseableComponent {
|
|||
void recover(RecoveryHandler recoveryHandler) throws EngineException;
|
||||
|
||||
/** fail engine due to some error. the engine will also be closed. */
|
||||
void failEngine(String reason, @Nullable Throwable failure);
|
||||
void failEngine(String reason, Throwable failure);
|
||||
|
||||
static interface FailedEngineListener {
|
||||
void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t);
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ElasticsearchIllegalStateException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.routing.operation.hash.djb.DjbHashFunction;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Preconditions;
|
||||
|
@ -93,6 +94,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
*/
|
||||
public class InternalEngine extends AbstractIndexShardComponent implements Engine {
|
||||
|
||||
private volatile boolean failEngineOnCorruption;
|
||||
private volatile ByteSizeValue indexingBufferSize;
|
||||
private volatile int indexConcurrency;
|
||||
private volatile boolean compoundOnFlush = true;
|
||||
|
@ -208,7 +210,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
this.optimizeAutoGenerateId = indexSettings.getAsBoolean("index.optimize_auto_generated_id", true);
|
||||
|
||||
this.indexSettingsService.addListener(applySettings);
|
||||
|
||||
this.failEngineOnCorruption = indexSettings.getAsBoolean(ENGINE_FAIL_ON_CORRUPTION, true);
|
||||
this.failOnMergeFailure = indexSettings.getAsBoolean(INDEX_FAIL_ON_MERGE_FAILURE, true);
|
||||
if (failOnMergeFailure) {
|
||||
this.mergeScheduler.addFailureListener(new FailEngineOnMergeFailure());
|
||||
|
@ -253,7 +255,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
|
||||
@Override
|
||||
public void start() throws EngineException {
|
||||
store.incRef();
|
||||
try (InternalLock _ = writeLock.acquire()) {
|
||||
|
||||
if (indexWriter != null) {
|
||||
throw new EngineAlreadyStartedException(shardId);
|
||||
}
|
||||
|
@ -269,6 +273,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
this.throttle = new IndexThrottle(mergeScheduler.getMaxMerges(), logger);
|
||||
mergeScheduler.addListener(throttle);
|
||||
} catch (IOException e) {
|
||||
maybeFailEngine(e, "start");
|
||||
throw new EngineCreationFailureException(shardId, "failed to create engine", e);
|
||||
}
|
||||
|
||||
|
@ -294,6 +299,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
versionMap.setManager(searcherManager);
|
||||
readLastCommittedSegmentsInfo();
|
||||
} catch (IOException e) {
|
||||
maybeFailEngine(e, "start");
|
||||
try {
|
||||
indexWriter.rollback();
|
||||
} catch (IOException e1) {
|
||||
|
@ -303,6 +309,8 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
}
|
||||
throw new EngineCreationFailureException(shardId, "failed to open reader on writer", e);
|
||||
}
|
||||
} finally {
|
||||
store.decRef();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -395,17 +403,12 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
possibleMergeNeeded = true;
|
||||
flushNeeded = true;
|
||||
} catch (OutOfMemoryError | IllegalStateException | IOException t) {
|
||||
maybeFailEngine(t);
|
||||
maybeFailEngine(t, "create");
|
||||
throw new CreateFailedEngineException(shardId, create, t);
|
||||
}
|
||||
checkVersionMapRefresh();
|
||||
}
|
||||
|
||||
private void maybeFailEngine(Throwable t) {
|
||||
if (t instanceof OutOfMemoryError || (t instanceof IllegalStateException && t.getMessage().contains("OutOfMemoryError"))) {
|
||||
failEngine("out of memory", t);
|
||||
}
|
||||
}
|
||||
|
||||
private void innerCreate(Create create, IndexWriter writer) throws IOException {
|
||||
if (optimizeAutoGenerateId && create.autoGeneratedId() && !create.canHaveDuplicates()) {
|
||||
|
@ -488,7 +491,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
possibleMergeNeeded = true;
|
||||
flushNeeded = true;
|
||||
} catch (OutOfMemoryError | IllegalStateException | IOException t) {
|
||||
maybeFailEngine(t);
|
||||
maybeFailEngine(t, "index");
|
||||
throw new IndexFailedEngineException(shardId, index, t);
|
||||
}
|
||||
checkVersionMapRefresh();
|
||||
|
@ -585,7 +588,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
possibleMergeNeeded = true;
|
||||
flushNeeded = true;
|
||||
} catch (OutOfMemoryError | IllegalStateException | IOException t) {
|
||||
maybeFailEngine(t);
|
||||
maybeFailEngine(t, "delete");
|
||||
throw new DeleteFailedEngineException(shardId, delete, t);
|
||||
}
|
||||
|
||||
|
@ -670,7 +673,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
possibleMergeNeeded = true;
|
||||
flushNeeded = true;
|
||||
} catch (Throwable t) {
|
||||
maybeFailEngine(t);
|
||||
maybeFailEngine(t, "delete_by_query");
|
||||
throw new DeleteByQueryFailedEngineException(shardId, delete, t);
|
||||
}
|
||||
|
||||
|
@ -908,7 +911,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
}
|
||||
|
||||
} catch (FlushFailedEngineException ex) {
|
||||
maybeFailEngine(ex.getCause());
|
||||
maybeFailEngine(ex.getCause(), "flush");
|
||||
throw ex;
|
||||
} finally {
|
||||
flushLock.unlock();
|
||||
|
@ -967,7 +970,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
try (InternalLock _ = readLock.acquire()) {
|
||||
currentIndexWriter().maybeMerge();
|
||||
} catch (Throwable t) {
|
||||
maybeFailEngine(t);
|
||||
maybeFailEngine(t, "maybe_merge");
|
||||
throw new OptimizeFailedEngineException(shardId, t);
|
||||
}
|
||||
}
|
||||
|
@ -1008,7 +1011,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
writer.forceMerge(optimize.maxNumSegments(), false);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
maybeFailEngine(t);
|
||||
maybeFailEngine(t, "optimize");
|
||||
throw new OptimizeFailedEngineException(shardId, t);
|
||||
} finally {
|
||||
if (elasticsearchMergePolicy != null) {
|
||||
|
@ -1054,6 +1057,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
try {
|
||||
phase1Snapshot = deletionPolicy.snapshot();
|
||||
} catch (Throwable e) {
|
||||
maybeFailEngine(e, "recovery");
|
||||
Releasables.closeWhileHandlingException(onGoingRecoveries);
|
||||
throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e);
|
||||
}
|
||||
|
@ -1061,6 +1065,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
try {
|
||||
recoveryHandler.phase1(phase1Snapshot);
|
||||
} catch (Throwable e) {
|
||||
maybeFailEngine(e, "recovery phase 1");
|
||||
Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot);
|
||||
throw new RecoveryEngineException(shardId, 1, "Execution failed", wrapIfClosed(e));
|
||||
}
|
||||
|
@ -1069,13 +1074,14 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
try {
|
||||
phase2Snapshot = translog.snapshot();
|
||||
} catch (Throwable e) {
|
||||
maybeFailEngine(e, "snapshot recovery");
|
||||
Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot);
|
||||
throw new RecoveryEngineException(shardId, 2, "Snapshot failed", wrapIfClosed(e));
|
||||
}
|
||||
|
||||
try {
|
||||
recoveryHandler.phase2(phase2Snapshot);
|
||||
} catch (Throwable e) {
|
||||
maybeFailEngine(e, "recovery phase 2");
|
||||
Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot, phase2Snapshot);
|
||||
throw new RecoveryEngineException(shardId, 2, "Execution failed", wrapIfClosed(e));
|
||||
}
|
||||
|
@ -1088,6 +1094,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
recoveryHandler.phase3(phase3Snapshot);
|
||||
success = true;
|
||||
} catch (Throwable e) {
|
||||
maybeFailEngine(e, "recovery phase 3");
|
||||
throw new RecoveryEngineException(shardId, 3, "Execution failed", wrapIfClosed(e));
|
||||
} finally {
|
||||
Releasables.close(success, onGoingRecoveries, writeLock, phase1Snapshot,
|
||||
|
@ -1095,6 +1102,18 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
}
|
||||
}
|
||||
|
||||
private void maybeFailEngine(Throwable t, String source) {
|
||||
if (Lucene.isCorruptionException(t)) {
|
||||
if (this.failEngineOnCorruption) {
|
||||
failEngine("corrupt file detected source: [" + source + "]", t);
|
||||
} else {
|
||||
logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, ENGINE_FAIL_ON_CORRUPTION, this.failEngineOnCorruption);
|
||||
}
|
||||
}else if (ExceptionsHelper.isOOM(t)) {
|
||||
failEngine("out of memory", t);
|
||||
}
|
||||
}
|
||||
|
||||
private Throwable wrapIfClosed(Throwable t) {
|
||||
if (closed) {
|
||||
return new EngineClosedException(shardId, t);
|
||||
|
@ -1240,13 +1259,23 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
class FailEngineOnMergeFailure implements MergeSchedulerProvider.FailureListener {
|
||||
@Override
|
||||
public void onFailedMerge(MergePolicy.MergeException e) {
|
||||
failEngine("merge exception", e);
|
||||
if (Lucene.isCorruptionException(e)) {
|
||||
if (failEngineOnCorruption) {
|
||||
failEngine("corrupt file detected source: [merge]", e);
|
||||
} else {
|
||||
logger.warn("corrupt file detected source: [merge] but [{}] is set to [{}]", e, ENGINE_FAIL_ON_CORRUPTION, failEngineOnCorruption);
|
||||
}
|
||||
} else {
|
||||
failEngine("merge exception", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failEngine(String reason, @Nullable Throwable failure) {
|
||||
public void failEngine(String reason, Throwable failure) {
|
||||
assert failure != null;
|
||||
if (failEngineLock.tryLock()) {
|
||||
|
||||
assert !readLock.assertLockIsHeld() : "readLock is held by a thread that tries to fail the engine";
|
||||
if (failedEngine != null) {
|
||||
logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure);
|
||||
|
@ -1255,17 +1284,23 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
try {
|
||||
logger.warn("failed engine [{}]", reason, failure);
|
||||
// we must set a failure exception, generate one if not supplied
|
||||
if (failure == null) {
|
||||
failedEngine = new EngineException(shardId(), reason);
|
||||
} else {
|
||||
failedEngine = failure;
|
||||
}
|
||||
failedEngine = failure;
|
||||
for (FailedEngineListener listener : failedEngineListeners) {
|
||||
listener.onFailedEngine(shardId, reason, failure);
|
||||
}
|
||||
} finally {
|
||||
// close the engine whatever happens...
|
||||
close();
|
||||
try {
|
||||
if (Lucene.isCorruptionException(failure)) {
|
||||
try {
|
||||
store.markStoreCorrupted(ExceptionsHelper.unwrap(failure, CorruptIndexException.class));
|
||||
} catch (IOException e) {
|
||||
logger.warn("Couldn't marks store corrupted", e);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
// close the engine whatever happens...
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
} else {
|
||||
|
@ -1367,6 +1402,8 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
public static final String INDEX_COMPOUND_ON_FLUSH = "index.compound_on_flush";
|
||||
public static final String INDEX_GC_DELETES = "index.gc_deletes";
|
||||
public static final String INDEX_FAIL_ON_MERGE_FAILURE = "index.fail_on_merge_failure";
|
||||
public static final String ENGINE_FAIL_ON_CORRUPTION = "index.fail_on_corruption";
|
||||
|
||||
|
||||
class ApplySettings implements IndexSettingsService.Listener {
|
||||
|
||||
|
@ -1385,6 +1422,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
|
|||
indexWriter.getConfig().setUseCompoundFile(compoundOnFlush);
|
||||
}
|
||||
|
||||
InternalEngine.this.failEngineOnCorruption = indexSettings.getAsBoolean(ENGINE_FAIL_ON_CORRUPTION, InternalEngine.this.failEngineOnCorruption);
|
||||
int indexConcurrency = settings.getAsInt(INDEX_INDEX_CONCURRENCY, InternalEngine.this.indexConcurrency);
|
||||
boolean failOnMergeFailure = settings.getAsBoolean(INDEX_FAIL_ON_MERGE_FAILURE, InternalEngine.this.failOnMergeFailure);
|
||||
String codecName = settings.get(INDEX_CODEC, InternalEngine.this.codecName);
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.lucene.index.SegmentInfos;
|
|||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
|
@ -114,187 +113,197 @@ public class LocalIndexShardGateway extends AbstractIndexShardComponent implemen
|
|||
recoveryState.setStage(RecoveryState.Stage.INDEX);
|
||||
long version = -1;
|
||||
long translogId = -1;
|
||||
indexShard.store().incRef();
|
||||
try {
|
||||
SegmentInfos si = null;
|
||||
try {
|
||||
si = Lucene.readSegmentInfos(indexShard.store().directory());
|
||||
indexShard.store().failIfCorrupted();
|
||||
SegmentInfos si = null;
|
||||
try {
|
||||
si = Lucene.readSegmentInfos(indexShard.store().directory());
|
||||
} catch (Throwable e) {
|
||||
String files = "_unknown_";
|
||||
try {
|
||||
files = Arrays.toString(indexShard.store().directory().listAll());
|
||||
} catch (Throwable e1) {
|
||||
files += " (failure=" + ExceptionsHelper.detailedMessage(e1) + ")";
|
||||
}
|
||||
if (indexShouldExists && indexShard.store().indexStore().persistent()) {
|
||||
throw new IndexShardGatewayRecoveryException(shardId(), "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e);
|
||||
}
|
||||
}
|
||||
if (si != null) {
|
||||
if (indexShouldExists) {
|
||||
version = si.getVersion();
|
||||
if (si.getUserData().containsKey(Translog.TRANSLOG_ID_KEY)) {
|
||||
translogId = Long.parseLong(si.getUserData().get(Translog.TRANSLOG_ID_KEY));
|
||||
} else {
|
||||
translogId = version;
|
||||
}
|
||||
logger.trace("using existing shard data, translog id [{}]", translogId);
|
||||
} else {
|
||||
// it exists on the directory, but shouldn't exist on the FS, its a leftover (possibly dangling)
|
||||
// its a "new index create" API, we have to do something, so better to clean it than use same data
|
||||
logger.trace("cleaning existing shard, shouldn't exists");
|
||||
IndexWriter writer = new IndexWriter(indexShard.store().directory(), new IndexWriterConfig(Lucene.VERSION, Lucene.STANDARD_ANALYZER).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
String files = "_unknown_";
|
||||
try {
|
||||
files = Arrays.toString(indexShard.store().directory().listAll());
|
||||
} catch (Throwable e1) {
|
||||
files += " (failure=" + ExceptionsHelper.detailedMessage(e1) + ")";
|
||||
}
|
||||
if (indexShouldExists && indexShard.store().indexStore().persistent()) {
|
||||
throw new IndexShardGatewayRecoveryException(shardId(), "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e);
|
||||
}
|
||||
throw new IndexShardGatewayRecoveryException(shardId(), "failed to fetch index version after copying it over", e);
|
||||
}
|
||||
if (si != null) {
|
||||
if (indexShouldExists) {
|
||||
version = si.getVersion();
|
||||
if (si.getUserData().containsKey(Translog.TRANSLOG_ID_KEY)) {
|
||||
translogId = Long.parseLong(si.getUserData().get(Translog.TRANSLOG_ID_KEY));
|
||||
} else {
|
||||
translogId = version;
|
||||
}
|
||||
logger.trace("using existing shard data, translog id [{}]", translogId);
|
||||
} else {
|
||||
// it exists on the directory, but shouldn't exist on the FS, its a leftover (possibly dangling)
|
||||
// its a "new index create" API, we have to do something, so better to clean it than use same data
|
||||
logger.trace("cleaning existing shard, shouldn't exists");
|
||||
IndexWriter writer = new IndexWriter(indexShard.store().directory(), new IndexWriterConfig(Lucene.VERSION, Lucene.STANDARD_ANALYZER).setOpenMode(IndexWriterConfig.OpenMode.CREATE));
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
throw new IndexShardGatewayRecoveryException(shardId(), "failed to fetch index version after copying it over", e);
|
||||
}
|
||||
recoveryState.getIndex().updateVersion(version);
|
||||
recoveryState.getIndex().time(System.currentTimeMillis() - recoveryState.getIndex().startTime());
|
||||
recoveryState.getIndex().updateVersion(version);
|
||||
recoveryState.getIndex().time(System.currentTimeMillis() - recoveryState.getIndex().startTime());
|
||||
|
||||
// since we recover from local, just fill the files and size
|
||||
try {
|
||||
int numberOfFiles = 0;
|
||||
long totalSizeInBytes = 0;
|
||||
for (String name : indexShard.store().directory().listAll()) {
|
||||
numberOfFiles++;
|
||||
long length = indexShard.store().directory().fileLength(name);
|
||||
totalSizeInBytes += length;
|
||||
recoveryState.getIndex().addFileDetail(name, length, length);
|
||||
}
|
||||
recoveryState.getIndex().files(numberOfFiles, totalSizeInBytes, numberOfFiles, totalSizeInBytes);
|
||||
recoveryState.getIndex().recoveredFileCount(numberOfFiles);
|
||||
recoveryState.getIndex().recoveredByteCount(totalSizeInBytes);
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
|
||||
recoveryState.getStart().startTime(System.currentTimeMillis());
|
||||
recoveryState.setStage(RecoveryState.Stage.START);
|
||||
if (translogId == -1) {
|
||||
// no translog files, bail
|
||||
indexShard.postRecovery("post recovery from gateway, no translog");
|
||||
// no index, just start the shard and bail
|
||||
recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
|
||||
recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
|
||||
return;
|
||||
}
|
||||
|
||||
// move an existing translog, if exists, to "recovering" state, and start reading from it
|
||||
FsTranslog translog = (FsTranslog) indexShard.translog();
|
||||
String translogName = "translog-" + translogId;
|
||||
String recoverTranslogName = translogName + ".recovering";
|
||||
|
||||
|
||||
File recoveringTranslogFile = null;
|
||||
for (File translogLocation : translog.locations()) {
|
||||
File tmpRecoveringFile = new File(translogLocation, recoverTranslogName);
|
||||
if (!tmpRecoveringFile.exists()) {
|
||||
File tmpTranslogFile = new File(translogLocation, translogName);
|
||||
if (tmpTranslogFile.exists()) {
|
||||
for (int i = 0; i < 3; i++) {
|
||||
if (tmpTranslogFile.renameTo(tmpRecoveringFile)) {
|
||||
recoveringTranslogFile = tmpRecoveringFile;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
recoveringTranslogFile = tmpRecoveringFile;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (recoveringTranslogFile == null || !recoveringTranslogFile.exists()) {
|
||||
// no translog to recovery from, start and bail
|
||||
// no translog files, bail
|
||||
indexShard.postRecovery("post recovery from gateway, no translog");
|
||||
// no index, just start the shard and bail
|
||||
recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
|
||||
recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
|
||||
return;
|
||||
}
|
||||
|
||||
// recover from the translog file
|
||||
indexShard.performRecoveryPrepareForTranslog();
|
||||
recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
|
||||
recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
|
||||
|
||||
recoveryState.getTranslog().startTime(System.currentTimeMillis());
|
||||
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
|
||||
FileInputStream fs = null;
|
||||
|
||||
final Set<String> typesToUpdate = Sets.newHashSet();
|
||||
try {
|
||||
fs = new FileInputStream(recoveringTranslogFile);
|
||||
InputStreamStreamInput si = new InputStreamStreamInput(fs);
|
||||
while (true) {
|
||||
Translog.Operation operation;
|
||||
try {
|
||||
int opSize = si.readInt();
|
||||
operation = TranslogStreams.readTranslogOperation(si);
|
||||
} catch (EOFException e) {
|
||||
// ignore, not properly written the last op
|
||||
break;
|
||||
} catch (IOException e) {
|
||||
// ignore, not properly written last op
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Engine.IndexingOperation potentialIndexOperation = indexShard.performRecoveryOperation(operation);
|
||||
if (potentialIndexOperation != null && potentialIndexOperation.parsedDoc().mappingsModified()) {
|
||||
if (!typesToUpdate.contains(potentialIndexOperation.docMapper().type())) {
|
||||
typesToUpdate.add(potentialIndexOperation.docMapper().type());
|
||||
}
|
||||
}
|
||||
recoveryState.getTranslog().addTranslogOperations(1);
|
||||
} catch (ElasticsearchException e) {
|
||||
if (e.status() == RestStatus.BAD_REQUEST) {
|
||||
// mainly for MapperParsingException and Failure to detect xcontent
|
||||
logger.info("ignoring recovery of a corrupt translog entry", e);
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
// we failed to recovery, make sure to delete the translog file (and keep the recovering one)
|
||||
indexShard.translog().closeWithDelete();
|
||||
throw new IndexShardGatewayRecoveryException(shardId, "failed to recover shard", e);
|
||||
} finally {
|
||||
// since we recover from local, just fill the files and size
|
||||
try {
|
||||
fs.close();
|
||||
} catch (IOException e) {
|
||||
int numberOfFiles = 0;
|
||||
long totalSizeInBytes = 0;
|
||||
for (String name : indexShard.store().directory().listAll()) {
|
||||
numberOfFiles++;
|
||||
long length = indexShard.store().directory().fileLength(name);
|
||||
totalSizeInBytes += length;
|
||||
recoveryState.getIndex().addFileDetail(name, length, length);
|
||||
}
|
||||
RecoveryState.Index index = recoveryState.getIndex();
|
||||
index.totalFileCount(numberOfFiles);
|
||||
index.totalByteCount(totalSizeInBytes);
|
||||
index.reusedFileCount(numberOfFiles);
|
||||
index.reusedByteCount(totalSizeInBytes);
|
||||
index.recoveredFileCount(numberOfFiles);
|
||||
index.recoveredByteCount(totalSizeInBytes);
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
indexShard.performRecoveryFinalization(true);
|
||||
|
||||
recoveringTranslogFile.delete();
|
||||
|
||||
for (final String type : typesToUpdate) {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() {
|
||||
@Override
|
||||
public void onMappingUpdate() {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
latch.countDown();
|
||||
logger.debug("failed to send mapping update post recovery to master for [{}]", t, type);
|
||||
}
|
||||
});
|
||||
|
||||
try {
|
||||
boolean waited = latch.await(waitForMappingUpdatePostRecovery.millis(), TimeUnit.MILLISECONDS);
|
||||
if (!waited) {
|
||||
logger.debug("waited for mapping update on master for [{}], yet timed out");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
logger.debug("interrupted while waiting for mapping update");
|
||||
recoveryState.getStart().startTime(System.currentTimeMillis());
|
||||
recoveryState.setStage(RecoveryState.Stage.START);
|
||||
if (translogId == -1) {
|
||||
// no translog files, bail
|
||||
indexShard.postRecovery("post recovery from gateway, no translog");
|
||||
// no index, just start the shard and bail
|
||||
recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
|
||||
recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
|
||||
return;
|
||||
}
|
||||
|
||||
// move an existing translog, if exists, to "recovering" state, and start reading from it
|
||||
FsTranslog translog = (FsTranslog) indexShard.translog();
|
||||
String translogName = "translog-" + translogId;
|
||||
String recoverTranslogName = translogName + ".recovering";
|
||||
|
||||
|
||||
File recoveringTranslogFile = null;
|
||||
for (File translogLocation : translog.locations()) {
|
||||
File tmpRecoveringFile = new File(translogLocation, recoverTranslogName);
|
||||
if (!tmpRecoveringFile.exists()) {
|
||||
File tmpTranslogFile = new File(translogLocation, translogName);
|
||||
if (tmpTranslogFile.exists()) {
|
||||
for (int i = 0; i < 3; i++) {
|
||||
if (tmpTranslogFile.renameTo(tmpRecoveringFile)) {
|
||||
recoveringTranslogFile = tmpRecoveringFile;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
recoveringTranslogFile = tmpRecoveringFile;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (recoveringTranslogFile == null || !recoveringTranslogFile.exists()) {
|
||||
// no translog to recovery from, start and bail
|
||||
// no translog files, bail
|
||||
indexShard.postRecovery("post recovery from gateway, no translog");
|
||||
// no index, just start the shard and bail
|
||||
recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
|
||||
recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
|
||||
return;
|
||||
}
|
||||
|
||||
// recover from the translog file
|
||||
indexShard.performRecoveryPrepareForTranslog();
|
||||
recoveryState.getStart().time(System.currentTimeMillis() - recoveryState.getStart().startTime());
|
||||
recoveryState.getStart().checkIndexTime(indexShard.checkIndexTook());
|
||||
|
||||
recoveryState.getTranslog().startTime(System.currentTimeMillis());
|
||||
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
|
||||
FileInputStream fs = null;
|
||||
|
||||
final Set<String> typesToUpdate = Sets.newHashSet();
|
||||
try {
|
||||
fs = new FileInputStream(recoveringTranslogFile);
|
||||
InputStreamStreamInput si = new InputStreamStreamInput(fs);
|
||||
while (true) {
|
||||
Translog.Operation operation;
|
||||
try {
|
||||
int opSize = si.readInt();
|
||||
operation = TranslogStreams.readTranslogOperation(si);
|
||||
} catch (EOFException e) {
|
||||
// ignore, not properly written the last op
|
||||
break;
|
||||
} catch (IOException e) {
|
||||
// ignore, not properly written last op
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Engine.IndexingOperation potentialIndexOperation = indexShard.performRecoveryOperation(operation);
|
||||
if (potentialIndexOperation != null && potentialIndexOperation.parsedDoc().mappingsModified()) {
|
||||
if (!typesToUpdate.contains(potentialIndexOperation.docMapper().type())) {
|
||||
typesToUpdate.add(potentialIndexOperation.docMapper().type());
|
||||
}
|
||||
}
|
||||
recoveryState.getTranslog().addTranslogOperations(1);
|
||||
} catch (ElasticsearchException e) {
|
||||
if (e.status() == RestStatus.BAD_REQUEST) {
|
||||
// mainly for MapperParsingException and Failure to detect xcontent
|
||||
logger.info("ignoring recovery of a corrupt translog entry", e);
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
// we failed to recovery, make sure to delete the translog file (and keep the recovering one)
|
||||
indexShard.translog().closeWithDelete();
|
||||
throw new IndexShardGatewayRecoveryException(shardId, "failed to recover shard", e);
|
||||
} finally {
|
||||
try {
|
||||
fs.close();
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
indexShard.performRecoveryFinalization(true);
|
||||
|
||||
recoveringTranslogFile.delete();
|
||||
|
||||
for (final String type : typesToUpdate) {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() {
|
||||
@Override
|
||||
public void onMappingUpdate() {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
latch.countDown();
|
||||
logger.debug("failed to send mapping update post recovery to master for [{}]", t, type);
|
||||
}
|
||||
});
|
||||
|
||||
try {
|
||||
boolean waited = latch.await(waitForMappingUpdatePostRecovery.millis(), TimeUnit.MILLISECONDS);
|
||||
if (!waited) {
|
||||
logger.debug("waited for mapping update on master for [{}], yet timed out");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
logger.debug("interrupted while waiting for mapping update");
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
indexShard.store().decRef();
|
||||
}
|
||||
|
||||
recoveryState.getTranslog().time(System.currentTimeMillis() - recoveryState.getTranslog().startTime());
|
||||
|
|
|
@ -53,7 +53,6 @@ public class LogDocMergePolicyProvider extends AbstractMergePolicyProvider<LogDo
|
|||
super(store);
|
||||
Preconditions.checkNotNull(store, "Store must be provided to merge policy");
|
||||
this.indexSettingsService = indexSettingsService;
|
||||
|
||||
this.minMergeDocs = componentSettings.getAsInt("min_merge_docs", LogDocMergePolicy.DEFAULT_MIN_MERGE_DOCS);
|
||||
this.maxMergeDocs = componentSettings.getAsInt("max_merge_docs", LogDocMergePolicy.DEFAULT_MAX_MERGE_DOCS);
|
||||
this.mergeFactor = componentSettings.getAsInt("merge_factor", LogDocMergePolicy.DEFAULT_MERGE_FACTOR);
|
||||
|
|
|
@ -24,10 +24,12 @@ import org.apache.lucene.codecs.PostingsFormat;
|
|||
import org.apache.lucene.index.CheckIndex;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.util.ThreadInterruptedException;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ElasticsearchIllegalArgumentException;
|
||||
import org.elasticsearch.ElasticsearchIllegalStateException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.common.Booleans;
|
||||
|
@ -522,6 +524,8 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
|
|||
return store.stats();
|
||||
} catch (IOException e) {
|
||||
throw new ElasticsearchException("io exception while building 'store stats'", e);
|
||||
} catch (AlreadyClosedException ex) {
|
||||
return null; // already closed
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -633,7 +637,7 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
|
|||
}
|
||||
|
||||
@Override
|
||||
public void failShard(String reason, @Nullable Throwable e) {
|
||||
public void failShard(String reason, Throwable e) {
|
||||
// fail the engine. This will cause this shard to also be removed from the node's index service.
|
||||
engine.failEngine(reason, e);
|
||||
}
|
||||
|
|
|
@ -39,13 +39,13 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.xcontent.*;
|
||||
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.snapshots.*;
|
||||
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo;
|
||||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.index.store.StoreFileMetaData;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.repositories.RepositoryName;
|
||||
|
||||
import java.io.FilterInputStream;
|
||||
|
@ -54,6 +54,7 @@ import java.io.InputStream;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
@ -423,19 +424,19 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
int indexNumberOfFiles = 0;
|
||||
long indexTotalFilesSize = 0;
|
||||
ArrayList<FileInfo> filesToSnapshot = newArrayList();
|
||||
final Store.MetadataSnapshot metadata;
|
||||
try {
|
||||
metadata = store.getMetadata();
|
||||
} catch (IOException e) {
|
||||
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
|
||||
}
|
||||
for (String fileName : snapshotIndexCommit.getFiles()) {
|
||||
if (snapshotStatus.aborted()) {
|
||||
logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName);
|
||||
throw new IndexShardSnapshotFailedException(shardId, "Aborted");
|
||||
}
|
||||
logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName);
|
||||
final StoreFileMetaData md;
|
||||
try {
|
||||
md = store.metaData(fileName);
|
||||
} catch (IOException e) {
|
||||
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
|
||||
}
|
||||
|
||||
final StoreFileMetaData md = metadata.get(fileName);
|
||||
boolean snapshotRequired = false;
|
||||
// TODO: For now segment files are copied on each commit because segment files don't have checksum
|
||||
// if (snapshot.indexChanged() && fileName.equals(snapshotIndexCommit.getSegmentsFileName())) {
|
||||
|
@ -453,7 +454,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
indexNumberOfFiles++;
|
||||
indexTotalFilesSize += md.length();
|
||||
// create a new FileInfo
|
||||
BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = new BlobStoreIndexShardSnapshot.FileInfo(fileNameFromGeneration(++generation), fileName, md.length(), chunkSize, md.checksum());
|
||||
BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = new BlobStoreIndexShardSnapshot.FileInfo(fileNameFromGeneration(++generation), md, chunkSize);
|
||||
indexCommitPointFiles.add(snapshotFileInfo);
|
||||
filesToSnapshot.add(snapshotFileInfo);
|
||||
} else {
|
||||
|
@ -535,7 +536,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
IndexInput indexInput = null;
|
||||
try {
|
||||
final String file = fileInfo.physicalName();
|
||||
indexInput = store.openInputRaw(file, IOContext.READONCE);
|
||||
indexInput = store.directory().openInput(file, IOContext.READONCE);
|
||||
indexInput.seek(i * fileInfo.partBytes());
|
||||
InputStreamIndexInput inputStreamIndexInput = new ThreadSafeInputStreamIndexInput(indexInput, fileInfo.partBytes());
|
||||
|
||||
|
@ -670,16 +671,18 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
long totalSize = 0;
|
||||
int numberOfReusedFiles = 0;
|
||||
long reusedTotalSize = 0;
|
||||
Map<String, StoreFileMetaData> metadata = Collections.emptyMap();
|
||||
try {
|
||||
metadata = store.getMetadata().asMap();
|
||||
} catch (Throwable e) {
|
||||
// if the index is broken we might not be able to read it
|
||||
logger.warn("{} Can't read metadata from store", e, shardId);
|
||||
}
|
||||
|
||||
List<FileInfo> filesToRecover = Lists.newArrayList();
|
||||
for (FileInfo fileInfo : snapshot.indexFiles()) {
|
||||
String fileName = fileInfo.physicalName();
|
||||
StoreFileMetaData md = null;
|
||||
try {
|
||||
md = store.metaData(fileName);
|
||||
} catch (IOException e) {
|
||||
// no file
|
||||
}
|
||||
final StoreFileMetaData md = metadata.get(fileName);
|
||||
numberOfFiles++;
|
||||
// we don't compute checksum for segments, so always recover them
|
||||
if (!fileName.startsWith("segments") && md != null && fileInfo.isSame(md)) {
|
||||
|
@ -704,7 +707,11 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
}
|
||||
}
|
||||
|
||||
recoveryState.getIndex().files(numberOfFiles, totalSize, numberOfReusedFiles, reusedTotalSize);
|
||||
final RecoveryState.Index index = recoveryState.getIndex();
|
||||
index.totalFileCount(numberOfFiles);
|
||||
index.totalByteCount(totalSize);
|
||||
index.reusedFileCount(numberOfReusedFiles);
|
||||
index.reusedByteCount(reusedTotalSize);
|
||||
if (filesToRecover.isEmpty()) {
|
||||
logger.trace("no files to recover, all exists within the local store");
|
||||
}
|
||||
|
@ -775,7 +782,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
try {
|
||||
// we create an output with no checksum, this is because the pure binary data of the file is not
|
||||
// the checksum (because of seek). We will create the checksum file once copying is done
|
||||
indexOutput = store.createOutputRaw(fileInfo.physicalName());
|
||||
indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), IOContext.DEFAULT, fileInfo.metadata());
|
||||
} catch (IOException e) {
|
||||
try {
|
||||
failures.add(e);
|
||||
|
@ -814,10 +821,14 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
} else {
|
||||
// we are done...
|
||||
try {
|
||||
Store.verify(indexOutput);
|
||||
indexOutput.close();
|
||||
// write the checksum
|
||||
if (fileInfo.checksum() != null) {
|
||||
store.writeChecksum(fileInfo.physicalName(), fileInfo.checksum());
|
||||
if (fileInfo.metadata().hasLegacyChecksum()) {
|
||||
Store.LegacyChecksums legacyChecksums = new Store.LegacyChecksums();
|
||||
legacyChecksums.add(fileInfo.metadata());
|
||||
legacyChecksums.write(store);
|
||||
|
||||
}
|
||||
store.directory().sync(Collections.singleton(fileInfo.physicalName()));
|
||||
recoveryState.getIndex().addRecoveredFileCount(1);
|
||||
|
@ -832,8 +843,9 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
@Override
|
||||
public void onFailure(Throwable t) {
|
||||
try {
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
failures.add(t);
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
store.deleteQuiet(fileInfo.physicalName());
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
|
@ -842,8 +854,13 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
latch.countDown();
|
||||
try {
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
store.deleteQuiet(fileInfo.physicalName());
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.index.snapshots.blobstore;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
|
@ -45,34 +46,28 @@ public class BlobStoreIndexShardSnapshot {
|
|||
*/
|
||||
public static class FileInfo {
|
||||
private final String name;
|
||||
private final String physicalName;
|
||||
private final long length;
|
||||
private final String checksum;
|
||||
private final ByteSizeValue partSize;
|
||||
private final long partBytes;
|
||||
private final long numberOfParts;
|
||||
private final StoreFileMetaData metadata;
|
||||
|
||||
/**
|
||||
* Constructs a new instance of file info
|
||||
*
|
||||
* @param name file name as stored in the blob store
|
||||
* @param physicalName original file name
|
||||
* @param length total length of the file
|
||||
* @param metaData the files meta data
|
||||
* @param partSize size of the single chunk
|
||||
* @param checksum checksum for the file
|
||||
*/
|
||||
public FileInfo(String name, String physicalName, long length, ByteSizeValue partSize, String checksum) {
|
||||
public FileInfo(String name, StoreFileMetaData metaData, ByteSizeValue partSize) {
|
||||
this.name = name;
|
||||
this.physicalName = physicalName;
|
||||
this.length = length;
|
||||
this.checksum = checksum;
|
||||
this.metadata = metaData;
|
||||
|
||||
long partBytes = Long.MAX_VALUE;
|
||||
if (partSize != null) {
|
||||
partBytes = partSize.bytes();
|
||||
}
|
||||
|
||||
long totalLength = length;
|
||||
long totalLength = metaData.length();
|
||||
long numberOfParts = totalLength / partBytes;
|
||||
if (totalLength % partBytes > 0) {
|
||||
numberOfParts++;
|
||||
|
@ -127,7 +122,7 @@ public class BlobStoreIndexShardSnapshot {
|
|||
* @return original file name
|
||||
*/
|
||||
public String physicalName() {
|
||||
return this.physicalName;
|
||||
return metadata.name();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -136,7 +131,7 @@ public class BlobStoreIndexShardSnapshot {
|
|||
* @return file length
|
||||
*/
|
||||
public long length() {
|
||||
return length;
|
||||
return metadata.length();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -173,7 +168,14 @@ public class BlobStoreIndexShardSnapshot {
|
|||
*/
|
||||
@Nullable
|
||||
public String checksum() {
|
||||
return checksum;
|
||||
return metadata.checksum();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the StoreFileMetaData for this file info.
|
||||
*/
|
||||
public StoreFileMetaData metadata() {
|
||||
return metadata;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -183,10 +185,7 @@ public class BlobStoreIndexShardSnapshot {
|
|||
* @return true if file in a store this this file have the same checksum and length
|
||||
*/
|
||||
public boolean isSame(StoreFileMetaData md) {
|
||||
if (checksum == null || md.checksum() == null) {
|
||||
return false;
|
||||
}
|
||||
return length == md.length() && checksum.equals(md.checksum());
|
||||
return metadata.isSame(md);
|
||||
}
|
||||
|
||||
static final class Fields {
|
||||
|
@ -195,6 +194,7 @@ public class BlobStoreIndexShardSnapshot {
|
|||
static final XContentBuilderString LENGTH = new XContentBuilderString("length");
|
||||
static final XContentBuilderString CHECKSUM = new XContentBuilderString("checksum");
|
||||
static final XContentBuilderString PART_SIZE = new XContentBuilderString("part_size");
|
||||
static final XContentBuilderString WRITTEN_BY = new XContentBuilderString("written_by");
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -208,14 +208,18 @@ public class BlobStoreIndexShardSnapshot {
|
|||
public static void toXContent(FileInfo file, XContentBuilder builder, ToXContent.Params params) throws IOException {
|
||||
builder.startObject();
|
||||
builder.field(Fields.NAME, file.name);
|
||||
builder.field(Fields.PHYSICAL_NAME, file.physicalName);
|
||||
builder.field(Fields.LENGTH, file.length);
|
||||
if (file.checksum != null) {
|
||||
builder.field(Fields.CHECKSUM, file.checksum);
|
||||
builder.field(Fields.PHYSICAL_NAME, file.metadata.name());
|
||||
builder.field(Fields.LENGTH, file.metadata.length());
|
||||
if (file.metadata.checksum() != null) {
|
||||
builder.field(Fields.CHECKSUM, file.metadata.checksum());
|
||||
}
|
||||
if (file.partSize != null) {
|
||||
builder.field(Fields.PART_SIZE, file.partSize.bytes());
|
||||
}
|
||||
|
||||
if (file.metadata.writtenBy() != null) {
|
||||
builder.field(Fields.WRITTEN_BY, file.metadata.writtenBy());
|
||||
}
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
|
@ -233,6 +237,7 @@ public class BlobStoreIndexShardSnapshot {
|
|||
long length = -1;
|
||||
String checksum = null;
|
||||
ByteSizeValue partSize = null;
|
||||
Version writtenBy = null;
|
||||
if (token == XContentParser.Token.START_OBJECT) {
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
|
@ -249,6 +254,8 @@ public class BlobStoreIndexShardSnapshot {
|
|||
checksum = parser.text();
|
||||
} else if ("part_size".equals(currentFieldName)) {
|
||||
partSize = new ByteSizeValue(parser.longValue());
|
||||
} else if ("written_by".equals(currentFieldName)) {
|
||||
writtenBy = Version.parseLeniently(parser.text());
|
||||
} else {
|
||||
throw new ElasticsearchParseException("unknown parameter [" + currentFieldName + "]");
|
||||
}
|
||||
|
@ -261,7 +268,7 @@ public class BlobStoreIndexShardSnapshot {
|
|||
}
|
||||
}
|
||||
// TODO: Verify???
|
||||
return new FileInfo(name, physicalName, length, partSize, checksum);
|
||||
return new FileInfo(name, new StoreFileMetaData(physicalName, length, checksum, writtenBy), partSize);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,236 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.index.store;
|
||||
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.common.math.MathUtils;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.index.store.distributor.Distributor;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
/**
|
||||
* A directory implementation that uses the Elasticsearch {@link Distributor} abstraction to distribute
|
||||
* files across multiple data directories.
|
||||
*/
|
||||
public final class DistributorDirectory extends BaseDirectory {
|
||||
|
||||
private final Distributor distributor;
|
||||
private final ConcurrentMap<String, Directory> nameDirMapping = ConcurrentCollections.newConcurrentMap();
|
||||
|
||||
/**
|
||||
* Creates a new DistributorDirectory from multiple directories. Note: The first directory in the given array
|
||||
* is used as the primary directory holding the file locks as well as the SEGMENTS_GEN file. All remaining
|
||||
* directories are used in a round robin fashion.
|
||||
*/
|
||||
public DistributorDirectory(final Directory... dirs) throws IOException {
|
||||
this(new Distributor() {
|
||||
final AtomicInteger count = new AtomicInteger();
|
||||
|
||||
@Override
|
||||
public Directory primary() {
|
||||
return dirs[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
public Directory[] all() {
|
||||
return dirs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Directory any() {
|
||||
return dirs[MathUtils.mod(count.incrementAndGet(), dirs.length)];
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new DistributorDirectory form the given Distributor.
|
||||
*/
|
||||
public DistributorDirectory(Distributor distributor) throws IOException {
|
||||
this.distributor = distributor;
|
||||
for (Directory dir : distributor.all()) {
|
||||
for (String file : dir.listAll()) {
|
||||
if (!usePrimary(file)) {
|
||||
nameDirMapping.put(file, dir);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final String[] listAll() throws IOException {
|
||||
final ArrayList<String> files = new ArrayList<>();
|
||||
for (Directory dir : distributor.all()) {
|
||||
for (String file : dir.listAll()) {
|
||||
files.add(file);
|
||||
}
|
||||
}
|
||||
return files.toArray(new String[files.size()]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean fileExists(String name) throws IOException {
|
||||
try {
|
||||
return getDirectory(name).fileExists(name);
|
||||
} catch (FileNotFoundException ex) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteFile(String name) throws IOException {
|
||||
getDirectory(name, true, true).deleteFile(name);
|
||||
Directory remove = nameDirMapping.remove(name);
|
||||
assert usePrimary(name) || remove != null : "Tried to delete file " + name + " but couldn't";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long fileLength(String name) throws IOException {
|
||||
return getDirectory(name).fileLength(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexOutput createOutput(String name, IOContext context) throws IOException {
|
||||
return getDirectory(name, false, false).createOutput(name, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync(Collection<String> names) throws IOException {
|
||||
for (Directory dir : distributor.all()) {
|
||||
dir.sync(names);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexInput openInput(String name, IOContext context) throws IOException {
|
||||
return getDirectory(name).openInput(name, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
IOUtils.close(distributor.all());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the directory that has previously been associated with this file name.
|
||||
*
|
||||
* @throws IOException if the name has not yet been associated with any directory ie. fi the file does not exists
|
||||
*/
|
||||
private Directory getDirectory(String name) throws IOException {
|
||||
return getDirectory(name, true, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the primary directory should be used for the given file.
|
||||
*/
|
||||
private boolean usePrimary(String name) {
|
||||
return IndexFileNames.SEGMENTS_GEN.equals(name) || Store.isChecksum(name);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the directory that has previously been associated with this file name or associates the name with a directory
|
||||
* if failIfNotAssociated is set to false.
|
||||
*/
|
||||
private Directory getDirectory(String name, boolean failIfNotAssociated, boolean iterate) throws IOException {
|
||||
if (usePrimary(name)) {
|
||||
return distributor.primary();
|
||||
}
|
||||
if (!nameDirMapping.containsKey(name)) {
|
||||
if (iterate) { // in order to get stuff like "write.lock" that might not be written though this directory
|
||||
for (Directory dir : distributor.all()) {
|
||||
if (dir.fileExists(name)) {
|
||||
final Directory directory = nameDirMapping.putIfAbsent(name, dir);
|
||||
return directory == null ? dir : directory;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (failIfNotAssociated) {
|
||||
throw new FileNotFoundException("No such file [" + name + "]");
|
||||
}
|
||||
}
|
||||
final Directory dir = distributor.any();
|
||||
final Directory directory = nameDirMapping.putIfAbsent(name, dir);
|
||||
return directory == null ? dir : directory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Lock makeLock(String name) {
|
||||
return distributor.primary().makeLock(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearLock(String name) throws IOException {
|
||||
distributor.primary().clearLock(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public LockFactory getLockFactory() {
|
||||
return distributor.primary().getLockFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setLockFactory(LockFactory lockFactory) throws IOException {
|
||||
distributor.primary().setLockFactory(lockFactory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getLockID() {
|
||||
return distributor.primary().getLockID();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return distributor.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Renames the given source file to the given target file unless the target already exists.
|
||||
*
|
||||
* @param directoryService the DirecotrySerivce to use.
|
||||
* @param from the source file name.
|
||||
* @param to the target file name
|
||||
* @throws IOException if the target file already exists.
|
||||
*/
|
||||
public void renameFile(DirectoryService directoryService, String from, String to) throws IOException {
|
||||
Directory directory = getDirectory(from);
|
||||
if (nameDirMapping.putIfAbsent(to, directory) != null) {
|
||||
throw new IOException("Can't rename file from " + from
|
||||
+ " to: " + to + "target file already exists");
|
||||
}
|
||||
boolean success = false;
|
||||
try {
|
||||
directoryService.renameFile(directory, from, to);
|
||||
nameDirMapping.remove(from);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
nameDirMapping.remove(to);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.index.store;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -39,25 +40,23 @@ public class StoreFileMetaData implements Streamable {
|
|||
|
||||
private String checksum;
|
||||
|
||||
private transient Directory directory;
|
||||
private Version writtenBy;
|
||||
|
||||
private StoreFileMetaData() {
|
||||
}
|
||||
|
||||
public StoreFileMetaData(String name, long length, String checksum) {
|
||||
this(name, length, checksum, null);
|
||||
public StoreFileMetaData(String name, long length) {
|
||||
this(name, length, null, null);
|
||||
|
||||
}
|
||||
|
||||
public StoreFileMetaData(String name, long length, String checksum, @Nullable Directory directory) {
|
||||
public StoreFileMetaData(String name, long length, String checksum, Version writtenBy) {
|
||||
this.name = name;
|
||||
this.length = length;
|
||||
this.checksum = checksum;
|
||||
this.directory = directory;
|
||||
this.writtenBy = writtenBy;
|
||||
}
|
||||
|
||||
public Directory directory() {
|
||||
return this.directory;
|
||||
}
|
||||
|
||||
public String name() {
|
||||
return name;
|
||||
|
@ -75,6 +74,9 @@ public class StoreFileMetaData implements Streamable {
|
|||
return this.checksum;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the length and the checksums are the same. otherwise <code>false</code>
|
||||
*/
|
||||
public boolean isSame(StoreFileMetaData other) {
|
||||
if (checksum == null || other.checksum == null) {
|
||||
return false;
|
||||
|
@ -90,15 +92,17 @@ public class StoreFileMetaData implements Streamable {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "name [" + name + "], length [" + length + "], checksum [" + checksum + "]";
|
||||
return "name [" + name + "], length [" + length + "], checksum [" + checksum + "], writtenBy [" + writtenBy + "]" ;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
name = in.readString();
|
||||
length = in.readVLong();
|
||||
if (in.readBoolean()) {
|
||||
checksum = in.readString();
|
||||
checksum = in.readOptionalString();
|
||||
if (in.getVersion().onOrAfter(org.elasticsearch.Version.V_1_3_0)) {
|
||||
String versionString = in.readOptionalString();
|
||||
writtenBy = versionString == null ? null : Version.parseLeniently(versionString);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -106,11 +110,24 @@ public class StoreFileMetaData implements Streamable {
|
|||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(name);
|
||||
out.writeVLong(length);
|
||||
if (checksum == null) {
|
||||
out.writeBoolean(false);
|
||||
} else {
|
||||
out.writeBoolean(true);
|
||||
out.writeString(checksum);
|
||||
out.writeOptionalString(checksum);
|
||||
if (out.getVersion().onOrAfter(org.elasticsearch.Version.V_1_3_0)) {
|
||||
out.writeOptionalString(writtenBy == null ? null : writtenBy.name());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the Lucene version this file has been written by or <code>null</code> if unknown
|
||||
*/
|
||||
public Version writtenBy() {
|
||||
return writtenBy;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the checksum is not <code>null</code> and if the file has NOT been written by
|
||||
* a Lucene version greater or equal to Lucene 4.8
|
||||
*/
|
||||
public boolean hasLegacyChecksum() {
|
||||
return checksum != null && ((writtenBy != null && writtenBy.onOrAfter(Version.LUCENE_4_8)) == false);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,10 @@ import org.elasticsearch.index.store.distributor.RandomWeightedDistributor;
|
|||
*/
|
||||
public class StoreModule extends AbstractModule {
|
||||
|
||||
public static final String DISTIBUTOR_KEY = "index.store.distributor";
|
||||
public static final String LEAST_USED_DISTRIBUTOR = "least_used";
|
||||
public static final String RANDOM_WEIGHT_DISTRIBUTOR = "random";
|
||||
|
||||
private final Settings settings;
|
||||
|
||||
private final IndexStore indexStore;
|
||||
|
@ -57,13 +61,13 @@ public class StoreModule extends AbstractModule {
|
|||
|
||||
private Class<? extends Distributor> loadDistributor(Settings settings) {
|
||||
final Class<? extends Distributor> distributor;
|
||||
final String type = settings.get("index.store.distributor");
|
||||
final String type = settings.get(DISTIBUTOR_KEY);
|
||||
if ("least_used".equals(type)) {
|
||||
distributor = LeastUsedDistributor.class;
|
||||
} else if ("random".equals(type)) {
|
||||
distributor = RandomWeightedDistributor.class;
|
||||
} else {
|
||||
distributor = settings.getAsClass("index.store.distributor", LeastUsedDistributor.class,
|
||||
distributor = settings.getAsClass(DISTIBUTOR_KEY, LeastUsedDistributor.class,
|
||||
"org.elasticsearch.index.store.distributor.", "Distributor");
|
||||
}
|
||||
return distributor;
|
||||
|
|
|
@ -71,10 +71,10 @@ public final class RamDirectoryService extends AbstractIndexShardComponent imple
|
|||
throw new FileNotFoundException(from);
|
||||
RAMFile toFile = fileMap.get(to);
|
||||
if (toFile != null) {
|
||||
sizeInBytes.addAndGet(-fileLength(from));
|
||||
fileMap.remove(from);
|
||||
sizeInBytes.addAndGet(-fileLength(to));
|
||||
}
|
||||
fileMap.put(to, fromFile);
|
||||
fileMap.remove(from);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -60,6 +60,7 @@ import org.elasticsearch.index.shard.IndexShardState;
|
|||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.service.IndexShard;
|
||||
import org.elasticsearch.index.shard.service.InternalIndexShard;
|
||||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.recovery.*;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
@ -560,12 +561,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
|
|||
private void cleanFailedShards(final ClusterChangedEvent event) {
|
||||
RoutingTable routingTable = event.state().routingTable();
|
||||
RoutingNodes.RoutingNodeIterator routingNode = event.state().readOnlyRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
|
||||
|
||||
if (routingNode == null) {
|
||||
failedShards.clear();
|
||||
return;
|
||||
}
|
||||
|
||||
DiscoveryNodes nodes = event.state().nodes();
|
||||
long now = System.currentTimeMillis();
|
||||
String localNodeId = nodes.localNodeId();
|
||||
|
@ -712,11 +711,20 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
|
|||
// For replicas: we are recovering a backup from a primary
|
||||
|
||||
RecoveryState.Type type = shardRouting.primary() ? RecoveryState.Type.RELOCATION : RecoveryState.Type.REPLICA;
|
||||
final StartRecoveryRequest request = new StartRecoveryRequest(indexShard.shardId(), sourceNode, nodes.localNode(),
|
||||
false, indexShard.store().list(), type, recoveryIdGenerator.incrementAndGet());
|
||||
final Store store = indexShard.store();
|
||||
final StartRecoveryRequest request;
|
||||
store.incRef();
|
||||
try {
|
||||
store.failIfCorrupted();
|
||||
request = new StartRecoveryRequest(indexShard.shardId(), sourceNode, nodes.localNode(),
|
||||
false, store.getMetadata().asMap(), type, recoveryIdGenerator.incrementAndGet());
|
||||
} finally {
|
||||
store.decRef();
|
||||
}
|
||||
recoveryTarget.startRecovery(request, indexShard, new PeerRecoveryListener(request, shardRouting, indexService, indexMetaData));
|
||||
|
||||
} catch (Throwable e) {
|
||||
indexShard.engine().failEngine("corrupted preexisting index", e);
|
||||
handleRecoveryFailure(indexService, indexMetaData, shardRouting, true, e);
|
||||
}
|
||||
} else {
|
||||
|
|
|
@ -19,12 +19,14 @@
|
|||
|
||||
package org.elasticsearch.indices.recovery;
|
||||
|
||||
import org.apache.lucene.util.Version;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.store.StoreFileMetaData;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -32,26 +34,22 @@ import java.io.IOException;
|
|||
/**
|
||||
*
|
||||
*/
|
||||
class RecoveryFileChunkRequest extends TransportRequest {
|
||||
public final class RecoveryFileChunkRequest extends TransportRequest { // public for testing
|
||||
|
||||
private long recoveryId;
|
||||
private ShardId shardId;
|
||||
private String name;
|
||||
private long position;
|
||||
private long length;
|
||||
private String checksum;
|
||||
private BytesReference content;
|
||||
private StoreFileMetaData metaData;
|
||||
|
||||
RecoveryFileChunkRequest() {
|
||||
}
|
||||
|
||||
RecoveryFileChunkRequest(long recoveryId, ShardId shardId, String name, long position, long length, String checksum, BytesArray content) {
|
||||
public RecoveryFileChunkRequest(long recoveryId, ShardId shardId, StoreFileMetaData metaData, long position, BytesReference content) {
|
||||
this.recoveryId = recoveryId;
|
||||
this.shardId = shardId;
|
||||
this.name = name;
|
||||
this.metaData = metaData;
|
||||
this.position = position;
|
||||
this.length = length;
|
||||
this.checksum = checksum;
|
||||
this.content = content;
|
||||
}
|
||||
|
||||
|
@ -64,7 +62,7 @@ class RecoveryFileChunkRequest extends TransportRequest {
|
|||
}
|
||||
|
||||
public String name() {
|
||||
return name;
|
||||
return metaData.name();
|
||||
}
|
||||
|
||||
public long position() {
|
||||
|
@ -73,11 +71,11 @@ class RecoveryFileChunkRequest extends TransportRequest {
|
|||
|
||||
@Nullable
|
||||
public String checksum() {
|
||||
return this.checksum;
|
||||
return metaData.checksum();
|
||||
}
|
||||
|
||||
public long length() {
|
||||
return length;
|
||||
return metaData.length();
|
||||
}
|
||||
|
||||
public BytesReference content() {
|
||||
|
@ -95,11 +93,17 @@ class RecoveryFileChunkRequest extends TransportRequest {
|
|||
super.readFrom(in);
|
||||
recoveryId = in.readLong();
|
||||
shardId = ShardId.readShardId(in);
|
||||
name = in.readString();
|
||||
String name = in.readString();
|
||||
position = in.readVLong();
|
||||
length = in.readVLong();
|
||||
checksum = in.readOptionalString();
|
||||
long length = in.readVLong();
|
||||
String checksum = in.readOptionalString();
|
||||
content = in.readBytesReference();
|
||||
Version writtenBy = null;
|
||||
if (in.getVersion().onOrAfter(org.elasticsearch.Version.V_1_3_0)) {
|
||||
String versionString = in.readOptionalString();
|
||||
writtenBy = versionString == null ? null : Version.parseLeniently(versionString);
|
||||
}
|
||||
metaData = new StoreFileMetaData(name, length, checksum, writtenBy);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -107,17 +111,24 @@ class RecoveryFileChunkRequest extends TransportRequest {
|
|||
super.writeTo(out);
|
||||
out.writeLong(recoveryId);
|
||||
shardId.writeTo(out);
|
||||
out.writeString(name);
|
||||
out.writeString(metaData.name());
|
||||
out.writeVLong(position);
|
||||
out.writeVLong(length);
|
||||
out.writeOptionalString(checksum);
|
||||
out.writeVLong(metaData.length());
|
||||
out.writeOptionalString(metaData.checksum());
|
||||
out.writeBytesReference(content);
|
||||
if (out.getVersion().onOrAfter(org.elasticsearch.Version.V_1_3_0)) {
|
||||
out.writeOptionalString(metaData.writtenBy() == null ? null : metaData.writtenBy().name());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return shardId + ": name='" + name + '\'' +
|
||||
return shardId + ": name='" + name() + '\'' +
|
||||
", position=" + position +
|
||||
", length=" + length;
|
||||
", length=" + length();
|
||||
}
|
||||
|
||||
public StoreFileMetaData metadata() {
|
||||
return metaData;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,10 +21,12 @@ package org.elasticsearch.indices.recovery;
|
|||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.ClusterService;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
|
@ -57,6 +59,7 @@ import org.elasticsearch.transport.*;
|
|||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -103,7 +106,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().index().name());
|
||||
final InternalIndexShard shard = (InternalIndexShard) indexService.shardSafe(request.shardId().id());
|
||||
|
||||
// verify that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise
|
||||
// starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise
|
||||
// the index operations will not be routed to it properly
|
||||
RoutingNode node = clusterService.state().readOnlyRoutingNodes().node(request.targetNode().id());
|
||||
if (node == null) {
|
||||
|
@ -138,13 +141,17 @@ public class RecoverySource extends AbstractComponent {
|
|||
store.incRef();
|
||||
try {
|
||||
StopWatch stopWatch = new StopWatch().start();
|
||||
|
||||
final Store.MetadataSnapshot metadata;
|
||||
metadata = store.getMetadata();
|
||||
for (String name : snapshot.getFiles()) {
|
||||
StoreFileMetaData md = store.metaData(name);
|
||||
final StoreFileMetaData md = metadata.get(name);
|
||||
if (md == null) {
|
||||
logger.info("Snapshot differs from actual index for file: {} meta: {}", name, metadata.asMap());
|
||||
throw new CorruptIndexException("Snapshot differs from actual index - maybe index was removed metadata has " + metadata.asMap().size() + " files");
|
||||
}
|
||||
boolean useExisting = false;
|
||||
if (request.existingFiles().containsKey(name)) {
|
||||
// we don't compute checksum for segments, so always recover them
|
||||
if (!name.startsWith("segments") && md.isSame(request.existingFiles().get(name))) {
|
||||
if (md.isSame(request.existingFiles().get(name))) {
|
||||
response.phase1ExistingFileNames.add(name);
|
||||
response.phase1ExistingFileSizes.add(md.length());
|
||||
existingTotalSize += md.length();
|
||||
|
@ -175,7 +182,8 @@ public class RecoverySource extends AbstractComponent {
|
|||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, TransportRequestOptions.options().withTimeout(internalActionTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(response.phase1FileNames.size());
|
||||
final AtomicReference<Throwable> lastException = new AtomicReference<>();
|
||||
final CopyOnWriteArrayList<Throwable> exceptions = new CopyOnWriteArrayList<>();
|
||||
final AtomicReference<CorruptIndexException> corruptedEngine = new AtomicReference<>();
|
||||
int fileIndex = 0;
|
||||
for (final String name : response.phase1FileNames) {
|
||||
ThreadPoolExecutor pool;
|
||||
|
@ -191,12 +199,11 @@ public class RecoverySource extends AbstractComponent {
|
|||
public void run() {
|
||||
IndexInput indexInput = null;
|
||||
store.incRef();
|
||||
final StoreFileMetaData md = metadata.get(name);
|
||||
try {
|
||||
final int BUFFER_SIZE = (int) recoverySettings.fileChunkSize().bytes();
|
||||
byte[] buf = new byte[BUFFER_SIZE];
|
||||
StoreFileMetaData md = store.metaData(name);
|
||||
// TODO: maybe use IOContext.READONCE?
|
||||
indexInput = store.openInputRaw(name, IOContext.READ);
|
||||
indexInput = store.directory().openInput(name, IOContext.READONCE);
|
||||
boolean shouldCompressRequest = recoverySettings.compress();
|
||||
if (CompressorFactory.isCompressed(indexInput)) {
|
||||
shouldCompressRequest = false;
|
||||
|
@ -217,12 +224,31 @@ public class RecoverySource extends AbstractComponent {
|
|||
|
||||
indexInput.readBytes(buf, 0, toRead, false);
|
||||
BytesArray content = new BytesArray(buf, 0, toRead);
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), name, position, len, md.checksum(), content),
|
||||
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILE_CHUNK, new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position, content),
|
||||
TransportRequestOptions.options().withCompress(shouldCompressRequest).withType(TransportRequestOptions.Type.RECOVERY).withTimeout(internalActionTimeout), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||
readCount += toRead;
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
lastException.set(e);
|
||||
final CorruptIndexException corruptIndexException;
|
||||
if ((corruptIndexException = ExceptionsHelper.unwrap(e, CorruptIndexException.class)) != null) {
|
||||
if (store.checkIntegrity(md) == false) { // we are corrupted on the primary -- fail!
|
||||
logger.warn("{} Corrupted file detected {} checksum mismatch", shard.shardId(), md);
|
||||
CorruptIndexException current = corruptedEngine.get();
|
||||
if (current != null || corruptedEngine.compareAndSet(null, corruptIndexException)) {
|
||||
current = corruptedEngine.get();
|
||||
assert current != null;
|
||||
current.addSuppressed(e);
|
||||
}
|
||||
|
||||
} else { // corruption has happened on the way to replica
|
||||
RemoteTransportException exception = new RemoteTransportException("File corruption occured on recovery but checksums are ok", null);
|
||||
exception.addSuppressed(e);
|
||||
exceptions.add(0, exception); // last exception first
|
||||
logger.warn("{} File corruption on recovery {} local checksum OK", corruptIndexException, shard.shardId(), md);
|
||||
}
|
||||
} else {
|
||||
exceptions.add(0, e); // last exceptions first
|
||||
}
|
||||
} finally {
|
||||
IOUtils.closeWhileHandlingException(indexInput);
|
||||
try {
|
||||
|
@ -237,9 +263,10 @@ public class RecoverySource extends AbstractComponent {
|
|||
}
|
||||
|
||||
latch.await();
|
||||
|
||||
if (lastException.get() != null) {
|
||||
throw lastException.get();
|
||||
if (corruptedEngine.get() != null) {
|
||||
throw corruptedEngine.get();
|
||||
} else {
|
||||
ExceptionsHelper.rethrowAndSuppress(exceptions);
|
||||
}
|
||||
|
||||
// now, set the clean files request
|
||||
|
@ -330,6 +357,7 @@ public class RecoverySource extends AbstractComponent {
|
|||
|
||||
@Override
|
||||
public void phase3(Translog.Snapshot snapshot) throws ElasticsearchException {
|
||||
|
||||
if (shard.state() == IndexShardState.CLOSED) {
|
||||
throw new IndexShardClosedException(request.shardId());
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
|
|||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.RestoreSource;
|
||||
|
@ -555,6 +556,7 @@ public class RecoveryState implements ToXContent, Streamable {
|
|||
public List<File> fileDetails() {
|
||||
return fileDetails;
|
||||
}
|
||||
|
||||
public List<File> reusedFileDetails() {
|
||||
return reusedFileDetails;
|
||||
}
|
||||
|
@ -618,13 +620,6 @@ public class RecoveryState implements ToXContent, Streamable {
|
|||
return this.version;
|
||||
}
|
||||
|
||||
public void files(int totalFileCount, long totalByteCount, int reusedFileCount, long reusedByteCount) {
|
||||
this.totalFileCount = totalFileCount;
|
||||
this.totalByteCount = totalByteCount;
|
||||
this.reusedFileCount = reusedFileCount;
|
||||
this.reusedByteCount = reusedByteCount;
|
||||
}
|
||||
|
||||
public int totalFileCount() {
|
||||
return totalFileCount;
|
||||
}
|
||||
|
@ -645,23 +640,19 @@ public class RecoveryState implements ToXContent, Streamable {
|
|||
this.recoveredFileCount.addAndGet(updatedCount);
|
||||
}
|
||||
|
||||
public float percentFilesRecovered(int numberRecovered) {
|
||||
public float percentFilesRecovered() {
|
||||
if (totalFileCount == 0) { // indicates we are still in init phase
|
||||
return 0.0f;
|
||||
}
|
||||
if ((totalFileCount - reusedFileCount) == 0) {
|
||||
final int filesRecovered = recoveredFileCount.get();
|
||||
if ((totalFileCount - filesRecovered) == 0) {
|
||||
return 100.0f;
|
||||
} else {
|
||||
int d = totalFileCount - reusedFileCount;
|
||||
float result = 100.0f * (numberRecovered / (float) d);
|
||||
float result = 100.0f * (filesRecovered / (float)totalFileCount);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
public float percentFilesRecovered() {
|
||||
return percentFilesRecovered(recoveredFileCount.get());
|
||||
}
|
||||
|
||||
public int numberOfRecoveredFiles() {
|
||||
return totalFileCount - reusedFileCount;
|
||||
}
|
||||
|
@ -690,27 +681,27 @@ public class RecoveryState implements ToXContent, Streamable {
|
|||
return recoveredByteCount.get() - reusedByteCount;
|
||||
}
|
||||
|
||||
public float percentBytesRecovered(long numberRecovered) {
|
||||
public float percentBytesRecovered() {
|
||||
if (totalByteCount == 0) { // indicates we are still in init phase
|
||||
return 0.0f;
|
||||
}
|
||||
if ((totalByteCount - reusedByteCount) == 0) {
|
||||
final long recByteCount = recoveredByteCount.get();
|
||||
if ((totalByteCount - recByteCount) == 0) {
|
||||
return 100.0f;
|
||||
} else {
|
||||
long d = totalByteCount - reusedByteCount;
|
||||
float result = 100.0f * (numberRecovered / (float) d);
|
||||
float result = 100.0f * (recByteCount / (float) totalByteCount);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
||||
public float percentBytesRecovered() {
|
||||
return percentBytesRecovered(recoveredByteCount.get());
|
||||
}
|
||||
|
||||
public int reusedFileCount() {
|
||||
return reusedFileCount;
|
||||
}
|
||||
|
||||
public void reusedFileCount(int reusedFileCount) {
|
||||
this.reusedFileCount = reusedFileCount;
|
||||
}
|
||||
|
||||
public long reusedByteCount() {
|
||||
return this.reusedByteCount;
|
||||
}
|
||||
|
@ -789,7 +780,7 @@ public class RecoveryState implements ToXContent, Streamable {
|
|||
builder.field(Fields.TOTAL, totalFileCount);
|
||||
builder.field(Fields.REUSED, reusedFileCount);
|
||||
builder.field(Fields.RECOVERED, filesRecovered);
|
||||
builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", percentFilesRecovered(filesRecovered)));
|
||||
builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", percentFilesRecovered()));
|
||||
if (detailed) {
|
||||
builder.startArray(Fields.DETAILS);
|
||||
for (File file : fileDetails) {
|
||||
|
@ -806,11 +797,25 @@ public class RecoveryState implements ToXContent, Streamable {
|
|||
builder.field(Fields.TOTAL, totalByteCount);
|
||||
builder.field(Fields.REUSED, reusedByteCount);
|
||||
builder.field(Fields.RECOVERED, bytesRecovered);
|
||||
builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", percentBytesRecovered(bytesRecovered)));
|
||||
builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", percentBytesRecovered()));
|
||||
builder.endObject();
|
||||
builder.timeValueField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, time);
|
||||
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
try {
|
||||
XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint();
|
||||
builder.startObject();
|
||||
toXContent(builder, EMPTY_PARAMS);
|
||||
builder.endObject();
|
||||
return builder.string();
|
||||
} catch (IOException e) {
|
||||
return "{ \"error\" : \"" + e.getMessage() + "\"}";
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,11 +19,13 @@
|
|||
|
||||
package org.elasticsearch.indices.recovery;
|
||||
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.service.InternalIndexShard;
|
||||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.index.store.StoreFileMetaData;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map.Entry;
|
||||
|
@ -53,7 +55,7 @@ public class RecoveryStatus {
|
|||
volatile boolean sentCanceledToSource;
|
||||
|
||||
private volatile ConcurrentMap<String, IndexOutput> openIndexOutputs = ConcurrentCollections.newConcurrentMap();
|
||||
ConcurrentMap<String, String> checksums = ConcurrentCollections.newConcurrentMap();
|
||||
public final Store.LegacyChecksums legacyChecksums = new Store.LegacyChecksums();
|
||||
|
||||
public RecoveryState recoveryState() {
|
||||
return recoveryState;
|
||||
|
@ -103,12 +105,12 @@ public class RecoveryStatus {
|
|||
return outputs.remove(name);
|
||||
}
|
||||
|
||||
public synchronized IndexOutput openAndPutIndexOutput(String key, String name, Store store) throws IOException {
|
||||
public synchronized IndexOutput openAndPutIndexOutput(String key, String fileName, StoreFileMetaData metaData, Store store) throws IOException {
|
||||
if (isCanceled()) {
|
||||
return null;
|
||||
}
|
||||
final ConcurrentMap<String, IndexOutput> outputs = openIndexOutputs;
|
||||
IndexOutput indexOutput = store.createOutputRaw(name);
|
||||
IndexOutput indexOutput = store.createVerifyingOutput(fileName, IOContext.DEFAULT, metaData);
|
||||
outputs.put(key, indexOutput);
|
||||
return indexOutput;
|
||||
}
|
||||
|
|
|
@ -162,16 +162,17 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
listener.onIgnoreRecovery(false, "already in recovering process, " + e.getMessage());
|
||||
return;
|
||||
}
|
||||
// create a new recovery status, and process...
|
||||
final RecoveryStatus recoveryStatus = new RecoveryStatus(request.recoveryId(), indexShard);
|
||||
recoveryStatus.recoveryState.setType(request.recoveryType());
|
||||
recoveryStatus.recoveryState.setSourceNode(request.sourceNode());
|
||||
recoveryStatus.recoveryState.setTargetNode(request.targetNode());
|
||||
recoveryStatus.recoveryState.setPrimary(indexShard.routingEntry().primary());
|
||||
onGoingRecoveries.put(recoveryStatus.recoveryId, recoveryStatus);
|
||||
|
||||
threadPool.generic().execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
// create a new recovery status, and process...
|
||||
RecoveryStatus recoveryStatus = new RecoveryStatus(request.recoveryId(), indexShard);
|
||||
recoveryStatus.recoveryState.setType(request.recoveryType());
|
||||
recoveryStatus.recoveryState.setSourceNode(request.sourceNode());
|
||||
recoveryStatus.recoveryState.setTargetNode(request.targetNode());
|
||||
recoveryStatus.recoveryState.setPrimary(indexShard.routingEntry().primary());
|
||||
onGoingRecoveries.put(recoveryStatus.recoveryId, recoveryStatus);
|
||||
doRecovery(request, recoveryStatus, listener);
|
||||
}
|
||||
});
|
||||
|
@ -187,7 +188,6 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
}
|
||||
|
||||
private void doRecovery(final StartRecoveryRequest request, final RecoveryStatus recoveryStatus, final RecoveryListener listener) {
|
||||
|
||||
assert request.sourceNode() != null : "can't do a recovery without a source node";
|
||||
|
||||
final InternalIndexShard shard = recoveryStatus.indexShard;
|
||||
|
@ -244,7 +244,9 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
removeAndCleanOnGoingRecovery(recoveryStatus);
|
||||
listener.onRecoveryDone();
|
||||
} catch (Throwable e) {
|
||||
// logger.trace("[{}][{}] Got exception on recovery", e, request.shardId().index().name(), request.shardId().id());
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("[{}][{}] Got exception on recovery", e, request.shardId().index().name(), request.shardId().id());
|
||||
}
|
||||
if (recoveryStatus.isCanceled()) {
|
||||
// don't remove it, the cancellation code will remove it...
|
||||
listener.onIgnoreRecovery(false, "canceled recovery");
|
||||
|
@ -360,7 +362,7 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
iterator.remove();
|
||||
|
||||
}
|
||||
status.checksums = null;
|
||||
status.legacyChecksums.clear();
|
||||
}
|
||||
|
||||
class PrepareForTranslogOperationsRequestHandler extends BaseTransportRequestHandler<RecoveryPrepareForTranslogOperationsRequest> {
|
||||
|
@ -456,12 +458,14 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
public void messageReceived(RecoveryFilesInfoRequest request, TransportChannel channel) throws Exception {
|
||||
RecoveryStatus onGoingRecovery = onGoingRecoveries.get(request.recoveryId());
|
||||
validateRecoveryStatus(onGoingRecovery, request.shardId());
|
||||
|
||||
onGoingRecovery.recoveryState().getIndex().addFileDetails(request.phase1FileNames, request.phase1FileSizes);
|
||||
onGoingRecovery.recoveryState().getIndex().addReusedFileDetails(request.phase1ExistingFileNames, request.phase1ExistingFileSizes);
|
||||
onGoingRecovery.recoveryState().getIndex().totalByteCount(request.phase1TotalSize);
|
||||
onGoingRecovery.recoveryState().getIndex().reusedByteCount(request.phase1ExistingTotalSize);
|
||||
onGoingRecovery.recoveryState().getIndex().totalFileCount(request.phase1FileNames.size());
|
||||
final RecoveryState.Index index = onGoingRecovery.recoveryState().getIndex();
|
||||
index.addFileDetails(request.phase1FileNames, request.phase1FileSizes);
|
||||
index.addReusedFileDetails(request.phase1ExistingFileNames, request.phase1ExistingFileSizes);
|
||||
index.totalByteCount(request.phase1TotalSize);
|
||||
index.totalFileCount(request.phase1FileNames.size() + request.phase1ExistingFileNames.size());
|
||||
index.reusedByteCount(request.phase1ExistingTotalSize);
|
||||
index.reusedFileCount(request.phase1ExistingFileNames.size());
|
||||
// recoveryBytesCount / recoveryFileCount will be set as we go...
|
||||
onGoingRecovery.stage(RecoveryState.Stage.INDEX);
|
||||
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||
}
|
||||
|
@ -514,7 +518,7 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
// now write checksums
|
||||
store.writeChecksums(onGoingRecovery.checksums);
|
||||
onGoingRecovery.legacyChecksums.write(store);
|
||||
|
||||
for (String existingFile : store.directory().listAll()) {
|
||||
// don't delete snapshot file, or the checksums file (note, this is extra protection since the Store won't delete checksum)
|
||||
|
@ -550,13 +554,13 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
RecoveryStatus onGoingRecovery = onGoingRecoveries.get(request.recoveryId());
|
||||
validateRecoveryStatus(onGoingRecovery, request.shardId());
|
||||
|
||||
Store store = onGoingRecovery.indexShard.store();
|
||||
final Store store = onGoingRecovery.indexShard.store();
|
||||
store.incRef();
|
||||
try {
|
||||
IndexOutput indexOutput;
|
||||
if (request.position() == 0) {
|
||||
// first request
|
||||
onGoingRecovery.checksums.remove(request.name());
|
||||
onGoingRecovery.legacyChecksums.remove(request.name());
|
||||
indexOutput = onGoingRecovery.removeOpenIndexOutputs(request.name());
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
// we create an output with no checksum, this is because the pure binary data of the file is not
|
||||
|
@ -571,7 +575,7 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
if (store.directory().fileExists(fileName)) {
|
||||
fileName = "recovery." + onGoingRecovery.recoveryState().getTimer().startTime() + "." + fileName;
|
||||
}
|
||||
indexOutput = onGoingRecovery.openAndPutIndexOutput(request.name(), fileName, store);
|
||||
indexOutput = onGoingRecovery.openAndPutIndexOutput(request.name(), fileName, request.metadata(), store);
|
||||
} else {
|
||||
indexOutput = onGoingRecovery.getOpenIndexOutput(request.name());
|
||||
}
|
||||
|
@ -596,12 +600,11 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
file.updateRecovered(request.length());
|
||||
}
|
||||
if (indexOutput.getFilePointer() == request.length()) {
|
||||
Store.verify(indexOutput);
|
||||
// we are done
|
||||
indexOutput.close();
|
||||
// write the checksum
|
||||
if (request.checksum() != null) {
|
||||
onGoingRecovery.checksums.put(request.name(), request.checksum());
|
||||
}
|
||||
onGoingRecovery.legacyChecksums.add(request.metadata());
|
||||
store.directory().sync(Collections.singleton(request.name()));
|
||||
IndexOutput remove = onGoingRecovery.removeOpenIndexOutputs(request.name());
|
||||
onGoingRecovery.recoveryState.getIndex().addRecoveredFileCount(1);
|
||||
|
@ -610,9 +613,14 @@ public class RecoveryTarget extends AbstractComponent {
|
|||
success = true;
|
||||
} finally {
|
||||
if (!success || onGoingRecovery.isCanceled()) {
|
||||
IndexOutput remove = onGoingRecovery.removeOpenIndexOutputs(request.name());
|
||||
assert remove == null || remove == indexOutput;
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
try {
|
||||
IndexOutput remove = onGoingRecovery.removeOpenIndexOutputs(request.name());
|
||||
assert remove == null || remove == indexOutput;
|
||||
IOUtils.closeWhileHandlingException(indexOutput);
|
||||
} finally {
|
||||
// trash the file - unsuccessful
|
||||
store.deleteQuiet(request.name(), "recovery." + onGoingRecovery.recoveryState().getTimer().startTime() + "." + request.name());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -151,7 +151,15 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio
|
|||
if (indexService != null) {
|
||||
InternalIndexShard indexShard = (InternalIndexShard) indexService.shard(shardId.id());
|
||||
if (indexShard != null) {
|
||||
return new StoreFilesMetaData(true, shardId, indexShard.store().list());
|
||||
Store store = indexShard.store();
|
||||
store.incRef();
|
||||
try {
|
||||
if (indexShard != null) {
|
||||
return new StoreFilesMetaData(true, shardId, indexShard.store().getMetadata().asMap());
|
||||
}
|
||||
} finally {
|
||||
store.decRef();
|
||||
}
|
||||
}
|
||||
}
|
||||
// try and see if we an list unallocated
|
||||
|
@ -178,31 +186,8 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio
|
|||
if (!exists) {
|
||||
return new StoreFilesMetaData(false, shardId, ImmutableMap.<String, StoreFileMetaData>of());
|
||||
}
|
||||
|
||||
Map<String, String> checksums = Store.readChecksums(shardIndexLocations);
|
||||
if (checksums == null) {
|
||||
checksums = ImmutableMap.of();
|
||||
}
|
||||
|
||||
Map<String, StoreFileMetaData> files = Maps.newHashMap();
|
||||
for (File shardIndexLocation : shardIndexLocations) {
|
||||
File[] listedFiles = shardIndexLocation.listFiles();
|
||||
if (listedFiles == null) {
|
||||
continue;
|
||||
}
|
||||
for (File file : listedFiles) {
|
||||
// BACKWARD CKS SUPPORT
|
||||
if (file.getName().endsWith(".cks")) {
|
||||
continue;
|
||||
}
|
||||
if (Store.isChecksum(file.getName())) {
|
||||
continue;
|
||||
}
|
||||
files.put(file.getName(), new StoreFileMetaData(file.getName(), file.length(), checksums.get(file.getName())));
|
||||
}
|
||||
}
|
||||
|
||||
return new StoreFilesMetaData(false, shardId, files);
|
||||
final Store.MetadataSnapshot storeFileMetaDatas = Store.readMetadataSnapshot(shardIndexLocations, logger);
|
||||
return new StoreFilesMetaData(false, shardId, storeFileMetaDatas.asMap());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -138,10 +138,6 @@ public class RestRecoveryAction extends AbstractCatAction {
|
|||
for (ShardRecoveryResponse shardResponse : shardRecoveryResponses) {
|
||||
|
||||
RecoveryState state = shardResponse.recoveryState();
|
||||
|
||||
int filesRecovered = state.getIndex().recoveredFileCount();
|
||||
long bytesRecovered = state.getIndex().recoveredByteCount();
|
||||
|
||||
t.startRow();
|
||||
t.addCell(index);
|
||||
t.addCell(shardResponse.getShardId());
|
||||
|
@ -153,9 +149,9 @@ public class RestRecoveryAction extends AbstractCatAction {
|
|||
t.addCell(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshotId().getRepository());
|
||||
t.addCell(state.getRestoreSource() == null ? "n/a" : state.getRestoreSource().snapshotId().getSnapshot());
|
||||
t.addCell(state.getIndex().totalFileCount());
|
||||
t.addCell(String.format(Locale.ROOT, "%1.1f%%", state.getIndex().percentFilesRecovered(filesRecovered)));
|
||||
t.addCell(String.format(Locale.ROOT, "%1.1f%%", state.getIndex().percentFilesRecovered()));
|
||||
t.addCell(state.getIndex().totalByteCount());
|
||||
t.addCell(String.format(Locale.ROOT, "%1.1f%%", state.getIndex().percentBytesRecovered(bytesRecovered)));
|
||||
t.addCell(String.format(Locale.ROOT, "%1.1f%%", state.getIndex().percentBytesRecovered()));
|
||||
t.endRow();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,14 +18,18 @@
|
|||
*/
|
||||
package org.elasticsearch.bwcompat;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.LifecycleScope;
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
import org.apache.lucene.util.English;
|
||||
import org.elasticsearch.ElasticsearchIllegalArgumentException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
|
||||
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
|
||||
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
|
||||
import org.elasticsearch.action.count.CountResponse;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
|
@ -43,6 +47,7 @@ import org.elasticsearch.index.VersionType;
|
|||
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||
import org.elasticsearch.index.mapper.internal.FieldNamesFieldMapper;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.snapshots.SnapshotState;
|
||||
import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -405,4 +410,95 @@ public class BasicBackwardsCompatibilityTest extends ElasticsearchBackwardsCompa
|
|||
public Version getMasterVersion() {
|
||||
return client().admin().cluster().prepareState().get().getState().nodes().masterNode().getVersion();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSnapshotAndRestore() throws ExecutionException, InterruptedException, IOException {
|
||||
logger.info("--> creating repository");
|
||||
assertAcked(client().admin().cluster().preparePutRepository("test-repo")
|
||||
.setType("fs").setSettings(ImmutableSettings.settingsBuilder()
|
||||
.put("location", newTempDir(LifecycleScope.SUITE).getAbsolutePath())
|
||||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000))));
|
||||
String[] indices = new String[randomIntBetween(1,5)];
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
indices[i] = "index_" + i;
|
||||
createIndex(indices[i]);
|
||||
}
|
||||
ensureYellow();
|
||||
logger.info("--> indexing some data");
|
||||
IndexRequestBuilder[] builders = new IndexRequestBuilder[randomIntBetween(10, 200)];
|
||||
for (int i = 0; i < builders.length; i++) {
|
||||
builders[i] = client().prepareIndex(RandomPicks.randomFrom(getRandom(), indices), "foo", Integer.toString(i)).setSource("{ \"foo\" : \"bar\" } ");
|
||||
}
|
||||
indexRandom(true, builders);
|
||||
assertThat(client().prepareCount(indices).get().getCount(), equalTo((long)builders.length));
|
||||
long[] counts = new long[indices.length];
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
counts[i] = client().prepareCount(indices[i]).get().getCount();
|
||||
}
|
||||
|
||||
logger.info("--> snapshot");
|
||||
CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices("index_*").get();
|
||||
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
|
||||
assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
|
||||
|
||||
assertThat(client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get().getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS));
|
||||
|
||||
logger.info("--> delete some data");
|
||||
int howMany = randomIntBetween(1, builders.length);
|
||||
|
||||
for (int i = 0; i < howMany; i++) {
|
||||
IndexRequestBuilder indexRequestBuilder = RandomPicks.randomFrom(getRandom(), builders);
|
||||
IndexRequest request = indexRequestBuilder.request();
|
||||
client().prepareDelete(request.index(), request.type(), request.id()).get();
|
||||
}
|
||||
refresh();
|
||||
final long numDocs = client().prepareCount(indices).get().getCount();
|
||||
assertThat(client().prepareCount(indices).get().getCount(), lessThan((long)builders.length));
|
||||
|
||||
|
||||
client().admin().indices().prepareUpdateSettings(indices).setSettings(ImmutableSettings.builder().put(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, "none")).get();
|
||||
backwardsCluster().allowOnAllNodes(indices);
|
||||
logClusterState();
|
||||
boolean upgraded;
|
||||
do {
|
||||
logClusterState();
|
||||
CountResponse countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
upgraded = backwardsCluster().upgradeOneNode();
|
||||
ensureYellow();
|
||||
countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
} while (upgraded);
|
||||
client().admin().indices().prepareUpdateSettings(indices).setSettings(ImmutableSettings.builder().put(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, "all")).get();
|
||||
|
||||
logger.info("--> close indices");
|
||||
|
||||
client().admin().indices().prepareClose(indices).get();
|
||||
|
||||
logger.info("--> restore all indices from the snapshot");
|
||||
RestoreSnapshotResponse restoreSnapshotResponse = client().admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute().actionGet();
|
||||
assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
|
||||
|
||||
ensureYellow();
|
||||
assertThat(client().prepareCount(indices).get().getCount(), equalTo((long)builders.length));
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
assertThat(counts[i], equalTo(client().prepareCount(indices[i]).get().getCount()));
|
||||
}
|
||||
|
||||
// Test restore after index deletion
|
||||
logger.info("--> delete indices");
|
||||
String index = RandomPicks.randomFrom(getRandom(), indices);
|
||||
cluster().wipeIndices(index);
|
||||
logger.info("--> restore one index after deletion");
|
||||
restoreSnapshotResponse = client().admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).setIndices(index).execute().actionGet();
|
||||
assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
|
||||
ensureYellow();
|
||||
assertThat(client().prepareCount(indices).get().getCount(), equalTo((long)builders.length));
|
||||
for (int i = 0; i < indices.length; i++) {
|
||||
assertThat(counts[i], equalTo(client().prepareCount(indices[i]).get().getCount()));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.gateway.local;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
|
||||
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
|
||||
import org.elasticsearch.action.count.CountResponse;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.test.ElasticsearchBackwardsCompatIntegrationTest;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.lessThan;
|
||||
|
||||
@ElasticsearchIntegrationTest.ClusterScope(numDataNodes = 0, scope = ElasticsearchIntegrationTest.Scope.TEST, numClientNodes = 0, transportClientRatio = 0.0)
|
||||
public class RecoveryBackwardsCompatibilityTests extends ElasticsearchBackwardsCompatIntegrationTest {
|
||||
|
||||
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return ImmutableSettings.builder()
|
||||
.put(super.nodeSettings(nodeOrdinal))
|
||||
.put("action.admin.cluster.node.shutdown.delay", "10ms")
|
||||
.put("gateway.recover_after_nodes", 3)
|
||||
.put(BalancedShardsAllocator.SETTING_THRESHOLD, 1.1f).build(); // use less agressive settings
|
||||
}
|
||||
|
||||
protected int minExternalNodes() {
|
||||
return 2;
|
||||
}
|
||||
|
||||
protected int maxExternalNodes() {
|
||||
return 3;
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
@LuceneTestCase.Slow
|
||||
public void testReusePeerRecovery() throws Exception {
|
||||
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder().put(indexSettings()).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)));
|
||||
logger.info("--> indexing docs");
|
||||
int numDocs = scaledRandomIntBetween(100, 1000);
|
||||
IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs];
|
||||
for (int i = 0; i < builders.length; i++) {
|
||||
builders[i] = client().prepareIndex("test", "type").setSource("field", "value");
|
||||
}
|
||||
indexRandom(true, builders);
|
||||
ensureGreen();
|
||||
|
||||
logger.info("--> bump number of replicas from 0 to 1");
|
||||
client().admin().indices().prepareFlush().execute().actionGet();
|
||||
client().admin().indices().prepareUpdateSettings("test").setSettings(ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "1").build()).get();
|
||||
ensureGreen();
|
||||
|
||||
assertAllShardsOnNodes("test", backwardsCluster().backwardsNodePattern());
|
||||
|
||||
logger.info("--> upgrade cluster");
|
||||
logClusterState();
|
||||
CountResponse countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
|
||||
client().admin().cluster().prepareUpdateSettings().setTransientSettings(ImmutableSettings.settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")).execute().actionGet();
|
||||
backwardsCluster().upgradeAllNodes();
|
||||
client().admin().cluster().prepareUpdateSettings().setTransientSettings(ImmutableSettings.settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "all")).execute().actionGet();
|
||||
ensureGreen();
|
||||
|
||||
countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
|
||||
RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").setDetailed(true).get();
|
||||
for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) {
|
||||
RecoveryState recoveryState = response.recoveryState();
|
||||
if (!recoveryState.getPrimary()) {
|
||||
RecoveryState.Index index = recoveryState.getIndex();
|
||||
if (compatibilityVersion().onOrAfter(Version.V_1_2_0)) {
|
||||
assertThat(index.toString(), index.recoveredByteCount(), equalTo(0l));
|
||||
assertThat(index.toString(), index.reusedByteCount(), greaterThan(0l));
|
||||
assertThat(index.toString(), index.reusedByteCount(), equalTo(index.totalByteCount()));
|
||||
assertThat(index.toString(), index.recoveredFileCount(), equalTo(0));
|
||||
assertThat(index.toString(), index.reusedFileCount(), equalTo(index.totalFileCount()));
|
||||
assertThat(index.toString(), index.reusedFileCount(), greaterThan(0));
|
||||
assertThat(index.toString(), index.percentBytesRecovered(), equalTo(0.f));
|
||||
assertThat(index.toString(), index.percentFilesRecovered(), equalTo(0.f));
|
||||
assertThat(index.toString(), index.reusedByteCount(), greaterThan(index.numberOfRecoveredBytes()));
|
||||
} else {
|
||||
/* We added checksums on 1.3 but they were available on 1.2 already since this uses Lucene 4.8.
|
||||
* yet in this test we upgrade the entire cluster and therefor the 1.3 nodes try to read the checksum
|
||||
* from the files even if they haven't been written with ES 1.3. Due to that we don't have to recover
|
||||
* the segments files if we are on 1.2 or above...*/
|
||||
assertThat(index.toString(), index.recoveredByteCount(), greaterThan(0l));
|
||||
assertThat(index.toString(), index.recoveredFileCount(), greaterThan(0));
|
||||
assertThat(index.toString(), index.reusedByteCount(), greaterThan(0l));
|
||||
assertThat(index.toString(), index.percentBytesRecovered(), greaterThan(0.0f));
|
||||
assertThat(index.toString(), index.percentBytesRecovered(), lessThan(100.0f));
|
||||
assertThat(index.toString(), index.percentFilesRecovered(), greaterThan(0.0f));
|
||||
assertThat(index.toString(), index.percentFilesRecovered(), lessThan(100.0f));
|
||||
assertThat(index.toString(), index.reusedByteCount(), greaterThan(index.numberOfRecoveredBytes()));
|
||||
}
|
||||
// TODO upgrade via optimize?
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -43,7 +43,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF
|
|||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.elasticsearch.test.ElasticsearchIntegrationTest.*;
|
||||
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
@ -379,13 +379,21 @@ public class SimpleRecoveryLocalGatewayTests extends ElasticsearchIntegrationTes
|
|||
ensureGreen();
|
||||
|
||||
RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get();
|
||||
|
||||
for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) {
|
||||
RecoveryState recoveryState = response.recoveryState();
|
||||
if (!recoveryState.getPrimary()) {
|
||||
logger.info("--> shard {}, recovered {}, reuse {}", response.getShardId(), recoveryState.getIndex().recoveredTotalSize(), recoveryState.getIndex().reusedByteCount());
|
||||
assertThat(recoveryState.getIndex().recoveredByteCount(), greaterThan(0l));
|
||||
assertThat(recoveryState.getIndex().recoveredByteCount(), equalTo(0l));
|
||||
assertThat(recoveryState.getIndex().reusedByteCount(), greaterThan(0l));
|
||||
assertThat(recoveryState.getIndex().reusedByteCount(), equalTo(recoveryState.getIndex().totalByteCount()));
|
||||
assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(0));
|
||||
assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount()));
|
||||
assertThat(recoveryState.getIndex().reusedFileCount(), greaterThan(0));
|
||||
assertThat(recoveryState.getIndex().reusedByteCount(), greaterThan(recoveryState.getIndex().numberOfRecoveredBytes()));
|
||||
} else {
|
||||
assertThat(recoveryState.getIndex().recoveredByteCount(), equalTo(recoveryState.getIndex().reusedByteCount()));
|
||||
assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(recoveryState.getIndex().reusedFileCount()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -36,6 +36,7 @@ public final class EngineSearcherTotalHitsMatcher extends TypeSafeMatcher<Engine
|
|||
private final Query query;
|
||||
|
||||
private final int totalHits;
|
||||
private int count;
|
||||
|
||||
public EngineSearcherTotalHitsMatcher(Query query, int totalHits) {
|
||||
this.query = query;
|
||||
|
@ -45,13 +46,18 @@ public final class EngineSearcherTotalHitsMatcher extends TypeSafeMatcher<Engine
|
|||
@Override
|
||||
public boolean matchesSafely(Engine.Searcher searcher) {
|
||||
try {
|
||||
long count = Lucene.count(searcher.searcher(), query);
|
||||
this.count = (int) Lucene.count(searcher.searcher(), query);
|
||||
return count == totalHits;
|
||||
} catch (IOException e) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void describeMismatchSafely(Engine.Searcher item, Description mismatchDescription) {
|
||||
mismatchDescription.appendText("was ").appendValue(count);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void describeTo(Description description) {
|
||||
description.appendText("total hits of size ").appendValue(totalHits).appendText(" with query ").appendValue(query);
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.lucene.analysis.Analyzer;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
|
@ -107,13 +108,13 @@ public class InternalEngineTests extends ElasticsearchTestCase {
|
|||
|
||||
private Settings defaultSettings;
|
||||
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
defaultSettings = ImmutableSettings.builder()
|
||||
.put(InternalEngine.INDEX_COMPOUND_ON_FLUSH, getRandom().nextBoolean())
|
||||
.put(InternalEngine.INDEX_GC_DELETES, "1h") // make sure this doesn't kick in on us
|
||||
.put(InternalEngine.ENGINE_FAIL_ON_CORRUPTION, randomBoolean())
|
||||
.build(); // TODO randomize more settings
|
||||
threadPool = new ThreadPool(getClass().getName());
|
||||
store = createStore();
|
||||
|
@ -600,6 +601,60 @@ public class InternalEngineTests extends ElasticsearchTestCase {
|
|||
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1));
|
||||
searchResult.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailEngineOnCorruption() {
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), Lucene.STANDARD_ANALYZER, B_1, false);
|
||||
engine.create(new Engine.Create(null, newUid("1"), doc));
|
||||
engine.flush(new Engine.Flush());
|
||||
final boolean failEngine = defaultSettings.getAsBoolean(InternalEngine.ENGINE_FAIL_ON_CORRUPTION, false);
|
||||
final int failInPhase = randomIntBetween(1,3);
|
||||
try {
|
||||
engine.recover(new Engine.RecoveryHandler() {
|
||||
@Override
|
||||
public void phase1(SnapshotIndexCommit snapshot) throws EngineException {
|
||||
if (failInPhase == 1) {
|
||||
throw new RuntimeException("bar", new CorruptIndexException("Foo"));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void phase2(Translog.Snapshot snapshot) throws EngineException {
|
||||
if (failInPhase == 2) {
|
||||
throw new RuntimeException("bar", new CorruptIndexException("Foo"));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void phase3(Translog.Snapshot snapshot) throws EngineException {
|
||||
if (failInPhase == 3) {
|
||||
throw new RuntimeException("bar", new CorruptIndexException("Foo"));
|
||||
}
|
||||
}
|
||||
});
|
||||
fail("exception expected");
|
||||
} catch (RuntimeException ex) {
|
||||
|
||||
}
|
||||
try {
|
||||
Engine.Searcher searchResult = engine.acquireSearcher("test");
|
||||
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1));
|
||||
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1));
|
||||
searchResult.close();
|
||||
|
||||
ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), Lucene.STANDARD_ANALYZER, B_2, false);
|
||||
engine.create(new Engine.Create(null, newUid("2"), doc2));
|
||||
engine.refresh(new Engine.Refresh("foo"));
|
||||
|
||||
searchResult = engine.acquireSearcher("test");
|
||||
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 2));
|
||||
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(2));
|
||||
searchResult.close();
|
||||
assertThat(failEngine, is(false));
|
||||
} catch (EngineClosedException ex) {
|
||||
assertThat(failEngine, is(true));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
|
|
|
@ -0,0 +1,485 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.index.store;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.index.CheckIndex;
|
||||
import org.apache.lucene.index.MergePolicy;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.count.CountResponse;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.*;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.index.engine.internal.InternalEngine;
|
||||
import org.elasticsearch.index.merge.policy.AbstractMergePolicyProvider;
|
||||
import org.elasticsearch.index.merge.policy.MergePolicyModule;
|
||||
import org.elasticsearch.index.shard.IndexShardException;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.service.IndexShard;
|
||||
import org.elasticsearch.index.shard.service.InternalIndexShard;
|
||||
import org.elasticsearch.indices.IndicesLifecycle;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
|
||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||
import org.elasticsearch.monitor.fs.FsStats;
|
||||
import org.elasticsearch.test.ElasticsearchIntegrationTest;
|
||||
import org.elasticsearch.test.store.MockFSDirectoryService;
|
||||
import org.elasticsearch.test.transport.MockTransportService;
|
||||
import org.elasticsearch.transport.*;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.SUITE)
|
||||
public class CorruptedFileTest extends ElasticsearchIntegrationTest {
|
||||
|
||||
@Override
|
||||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return ImmutableSettings.builder()
|
||||
// we really need local GW here since this also checks for corruption etc.
|
||||
// and we need to make sure primaries are not just trashed if we dont' have replicase
|
||||
.put(super.nodeSettings(nodeOrdinal)).put("gateway.type", "local")
|
||||
.put(TransportModule.TRANSPORT_SERVICE_TYPE_KEY, MockTransportService.class.getName()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we can actually recover from a corruption on the primary given that we have replica shards around.
|
||||
*/
|
||||
@Test
|
||||
public void testCorruptFileAndRecover() throws ExecutionException, InterruptedException, IOException {
|
||||
int numDocs = scaledRandomIntBetween(100, 1000);
|
||||
assertThat(cluster().numDataNodes(), greaterThanOrEqualTo(2));
|
||||
|
||||
while (cluster().numDataNodes() < 4) {
|
||||
/**
|
||||
* We need 4 nodes since if we have 2 replicas and only 3 nodes we can't get into green state since
|
||||
* the corrupted node will never be used reallocate a replica since it's marked as corrupted
|
||||
*/
|
||||
internalCluster().startNode(ImmutableSettings.builder().put("node.data", true).put("node.client", false).put("node.master", false));
|
||||
}
|
||||
assertThat(cluster().numDataNodes(), greaterThanOrEqualTo(3));
|
||||
|
||||
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "1")
|
||||
.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class)
|
||||
.put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) // no checkindex - we corrupt shards on purpose
|
||||
.put(InternalEngine.ENGINE_FAIL_ON_CORRUPTION, true)
|
||||
.put("indices.recovery.concurrent_streams", 10)
|
||||
));
|
||||
ensureGreen();
|
||||
disableAllocation("test");
|
||||
IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs];
|
||||
for (int i = 0; i < builders.length; i++) {
|
||||
builders[i] = client().prepareIndex("test", "type").setSource("field", "value");
|
||||
}
|
||||
indexRandom(true, builders);
|
||||
ensureGreen();
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).execute().actionGet());
|
||||
// we have to flush at least once here since we don't corrupt the translog
|
||||
CountResponse countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
|
||||
final int numShards = numShards("test");
|
||||
ShardRouting corruptedShardRouting = corruptRandomFile();
|
||||
enableAllocation("test");
|
||||
/*
|
||||
* we corrupted the primary shard - now lets make sure we never recover from it successfully
|
||||
*/
|
||||
Settings build = ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "2").build();
|
||||
client().admin().indices().prepareUpdateSettings("test").setSettings(build).get();
|
||||
ClusterHealthResponse health = client().admin().cluster()
|
||||
.health(Requests.clusterHealthRequest("test").waitForGreenStatus().waitForRelocatingShards(0)).actionGet();
|
||||
if (health.isTimedOut()) {
|
||||
logger.info("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint());
|
||||
assertThat("timed out waiting for green state", health.isTimedOut(), equalTo(false));
|
||||
}
|
||||
assertThat(health.getStatus(), equalTo(ClusterHealthStatus.GREEN));
|
||||
final int numIterations = scaledRandomIntBetween(5, 20);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
SearchResponse response = client().prepareSearch().setSize(numDocs).get();
|
||||
assertHitCount(response, numDocs);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/*
|
||||
* now hook into the IndicesService and register a close listener to
|
||||
* run the checkindex. if the corruption is still there we will catch it.
|
||||
*/
|
||||
final CountDownLatch latch = new CountDownLatch(numShards * 3); // primary + 2 replicas
|
||||
final CopyOnWriteArrayList<Throwable> exception = new CopyOnWriteArrayList<>();
|
||||
final IndicesLifecycle.Listener listener = new IndicesLifecycle.Listener() {
|
||||
@Override
|
||||
public void beforeIndexShardClosed(ShardId sid, @Nullable IndexShard indexShard) {
|
||||
if (indexShard != null) {
|
||||
Store store = ((InternalIndexShard) indexShard).store();
|
||||
store.incRef();
|
||||
try {
|
||||
if (!Lucene.indexExists(store.directory()) && indexShard.state() == IndexShardState.STARTED) {
|
||||
return;
|
||||
}
|
||||
CheckIndex checkIndex = new CheckIndex(store.directory());
|
||||
BytesStreamOutput os = new BytesStreamOutput();
|
||||
PrintStream out = new PrintStream(os, false, Charsets.UTF_8.name());
|
||||
checkIndex.setInfoStream(out);
|
||||
out.flush();
|
||||
CheckIndex.Status status = checkIndex.checkIndex();
|
||||
if (!status.clean) {
|
||||
logger.warn("check index [failure]\n{}", new String(os.bytes().toBytes(), Charsets.UTF_8));
|
||||
throw new IndexShardException(sid, "index check failure");
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
exception.add(t);
|
||||
} finally {
|
||||
store.decRef();
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
for (IndicesService service : internalCluster().getDataNodeInstances(IndicesService.class)) {
|
||||
service.indicesLifecycle().addListener(listener);
|
||||
}
|
||||
try {
|
||||
client().admin().indices().prepareDelete("test").get();
|
||||
latch.await();
|
||||
assertThat(exception, empty());
|
||||
} finally {
|
||||
for (IndicesService service : internalCluster().getDataNodeInstances(IndicesService.class)) {
|
||||
service.indicesLifecycle().removeListener(listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests corruption that happens on a single shard when no replicas are present. We make sure that the primary stays unassigned
|
||||
* and all other replicas for the healthy shards happens
|
||||
*/
|
||||
@Test
|
||||
public void testCorruptPrimaryNoReplica() throws ExecutionException, InterruptedException, IOException {
|
||||
int numDocs = scaledRandomIntBetween(100, 1000);
|
||||
assertThat(cluster().numDataNodes(), greaterThanOrEqualTo(2));
|
||||
|
||||
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "0")
|
||||
.put(MergePolicyModule.MERGE_POLICY_TYPE_KEY, NoMergePolicyProvider.class)
|
||||
.put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) // no checkindex - we corrupt shards on purpose
|
||||
.put(InternalEngine.ENGINE_FAIL_ON_CORRUPTION, true)
|
||||
.put("indices.recovery.concurrent_streams", 10)
|
||||
));
|
||||
ensureGreen();
|
||||
IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs];
|
||||
for (int i = 0; i < builders.length; i++) {
|
||||
builders[i] = client().prepareIndex("test", "type").setSource("field", "value");
|
||||
}
|
||||
indexRandom(true, builders);
|
||||
ensureGreen();
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).execute().actionGet());
|
||||
// we have to flush at least once here since we don't corrupt the translog
|
||||
CountResponse countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
|
||||
ShardRouting shardRouting = corruptRandomFile();
|
||||
/*
|
||||
* we corrupted the primary shard - now lets make sure we never recover from it successfully
|
||||
*/
|
||||
Settings build = ImmutableSettings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "1").build();
|
||||
client().admin().indices().prepareUpdateSettings("test").setSettings(build).get();
|
||||
client().admin().cluster().prepareReroute().get();
|
||||
|
||||
awaitBusy(new Predicate<Object>() {
|
||||
@Override
|
||||
public boolean apply(Object input) {
|
||||
ClusterHealthStatus test = client().admin().cluster()
|
||||
.health(Requests.clusterHealthRequest("test")).actionGet().getStatus();
|
||||
return test == ClusterHealthStatus.RED;
|
||||
}
|
||||
});
|
||||
final ClusterHealthResponse response = client().admin().cluster()
|
||||
.health(Requests.clusterHealthRequest("test")).get();
|
||||
if (response.getStatus() != ClusterHealthStatus.RED) {
|
||||
logger.info("cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint());
|
||||
}
|
||||
assertThat(response.getStatus(), is(ClusterHealthStatus.RED));
|
||||
ClusterState state = client().admin().cluster().prepareState().get().getState();
|
||||
GroupShardsIterator shardIterators = state.getRoutingNodes().getRoutingTable().activePrimaryShardsGrouped(new String[] {"test"}, false);
|
||||
for (ShardIterator iterator : shardIterators.iterators()) {
|
||||
ShardRouting routing;
|
||||
while ((routing = iterator.nextOrNull()) != null) {
|
||||
if (routing.getId() == shardRouting.getId()) {
|
||||
assertThat(routing.state(), equalTo(ShardRoutingState.UNASSIGNED));
|
||||
} else {
|
||||
assertThat(routing.state(), equalTo(ShardRoutingState.STARTED));
|
||||
}
|
||||
}
|
||||
}
|
||||
final List<File> files = listShardFiles(shardRouting);
|
||||
File corruptedFile = null;
|
||||
for (File file : files) {
|
||||
if (file.getName().startsWith("corrupted_")) {
|
||||
corruptedFile = file;
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertThat(corruptedFile, notNullValue());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests corruption that happens on the network layer and that the primary does not get affected by corruption that happens on the way
|
||||
* to the replica. The file on disk stays uncorrupted
|
||||
*/
|
||||
@Test
|
||||
public void testCorrupteOnNetworkLayer() throws ExecutionException, InterruptedException {
|
||||
int numDocs = scaledRandomIntBetween(100, 1000);
|
||||
assertThat(cluster().numDataNodes(), greaterThanOrEqualTo(2));
|
||||
if (cluster().numDataNodes() < 3) {
|
||||
internalCluster().startNode(ImmutableSettings.builder().put("node.data", true).put("node.client", false).put("node.master", false));
|
||||
}
|
||||
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().get();
|
||||
List<NodeStats> dataNodeStats = new ArrayList<>();
|
||||
for (NodeStats stat : nodeStats.getNodes()) {
|
||||
if (stat.getNode().isDataNode()) {
|
||||
dataNodeStats.add(stat);
|
||||
}
|
||||
}
|
||||
|
||||
assertThat(dataNodeStats.size(), greaterThanOrEqualTo(2));
|
||||
Collections.shuffle(dataNodeStats, getRandom());
|
||||
NodeStats primariesNode = dataNodeStats.get(0);
|
||||
NodeStats unluckyNode = dataNodeStats.get(1);
|
||||
|
||||
|
||||
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "0")
|
||||
.put(InternalEngine.ENGINE_FAIL_ON_CORRUPTION, true)
|
||||
.put("index.routing.allocation.include._name", primariesNode.getNode().name())
|
||||
.put("indices.recovery.concurrent_streams", 10)
|
||||
));
|
||||
ensureGreen();
|
||||
IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs];
|
||||
for (int i = 0; i < builders.length; i++) {
|
||||
builders[i] = client().prepareIndex("test", "type").setSource("field", "value");
|
||||
}
|
||||
indexRandom(true, builders);
|
||||
ensureGreen();
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).execute().actionGet());
|
||||
// we have to flush at least once here since we don't corrupt the translog
|
||||
CountResponse countResponse = client().prepareCount().get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
final boolean truncate = randomBoolean();
|
||||
for (NodeStats dataNode : dataNodeStats) {
|
||||
MockTransportService mockTransportService = ((MockTransportService) internalCluster().getInstance(TransportService.class, dataNode.getNode().name()));
|
||||
mockTransportService.addDelegate(internalCluster().getInstance(Discovery.class, unluckyNode.getNode().name()).localNode(), new MockTransportService.DelegateTransport(mockTransportService.original()) {
|
||||
|
||||
@Override
|
||||
public void sendRequest(DiscoveryNode node, long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException {
|
||||
if (action.equals(RecoveryTarget.Actions.FILE_CHUNK)) {
|
||||
RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request;
|
||||
if (truncate && req.length() > 1) {
|
||||
BytesArray array = new BytesArray(req.content().array(), req.content().arrayOffset(), (int)req.length()-1);
|
||||
request = new RecoveryFileChunkRequest(req.recoveryId(), req.shardId(), req.metadata(), req.position(), array);
|
||||
} else {
|
||||
byte[] array = req.content().array();
|
||||
int i = randomIntBetween(0, req.content().length() - 1);
|
||||
array[i] = (byte) ~array[i]; // flip one byte in the content
|
||||
}
|
||||
}
|
||||
super.sendRequest(node, requestId, action, request, options);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
Settings build = ImmutableSettings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, "1")
|
||||
.put("index.routing.allocation.include._name", "*").build();
|
||||
client().admin().indices().prepareUpdateSettings("test").setSettings(build).get();
|
||||
client().admin().cluster().prepareReroute().get();
|
||||
ClusterHealthResponse actionGet = client().admin().cluster()
|
||||
.health(Requests.clusterHealthRequest("test").waitForGreenStatus()).actionGet();
|
||||
if (actionGet.isTimedOut()) {
|
||||
logger.info("ensureGreen timed out, cluster state:\n{}\n{}", client().admin().cluster().prepareState().get().getState().prettyPrint(), client().admin().cluster().preparePendingClusterTasks().get().prettyPrint());
|
||||
assertThat("timed out waiting for green state", actionGet.isTimedOut(), equalTo(false));
|
||||
}
|
||||
// we are green so primaries got not corrupted.
|
||||
// ensure that no shard is actually allocated on the unlucky node
|
||||
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
|
||||
for (IndexShardRoutingTable table : clusterStateResponse.getState().routingNodes().getRoutingTable().index("test")) {
|
||||
for (ShardRouting routing : table) {
|
||||
if (unluckyNode.getNode().getId().equals(routing.currentNodeId())) {
|
||||
assertThat(routing.state(), not(equalTo(ShardRoutingState.STARTED)));
|
||||
assertThat(routing.state(), not(equalTo(ShardRoutingState.RELOCATING)));
|
||||
}
|
||||
}
|
||||
}
|
||||
final int numIterations = scaledRandomIntBetween(5, 20);
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
SearchResponse response = client().prepareSearch().setSize(numDocs).get();
|
||||
assertHitCount(response, numDocs);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private int numShards(String... index) {
|
||||
ClusterState state = client().admin().cluster().prepareState().get().getState();
|
||||
GroupShardsIterator shardIterators = state.getRoutingNodes().getRoutingTable().activePrimaryShardsGrouped(index, false);
|
||||
return shardIterators.size();
|
||||
}
|
||||
|
||||
|
||||
private ShardRouting corruptRandomFile() throws IOException {
|
||||
ClusterState state = client().admin().cluster().prepareState().get().getState();
|
||||
GroupShardsIterator shardIterators = state.getRoutingNodes().getRoutingTable().activePrimaryShardsGrouped(new String[]{"test"}, false);
|
||||
ShardIterator shardIterator = RandomPicks.randomFrom(getRandom(), shardIterators.iterators());
|
||||
ShardRouting shardRouting = shardIterator.nextOrNull();
|
||||
assertNotNull(shardRouting);
|
||||
assertTrue(shardRouting.primary());
|
||||
assertTrue(shardRouting.assignedToNode());
|
||||
String nodeId = shardRouting.currentNodeId();
|
||||
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(nodeId).setFs(true).get();
|
||||
File fileToCorrupt = null;
|
||||
for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) {
|
||||
String path = info.getPath();
|
||||
final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/index";
|
||||
File file = new File(path, relativeDataLocationPath);
|
||||
final File[] files = file.listFiles(new FileFilter() {
|
||||
@Override
|
||||
public boolean accept(File pathname) {
|
||||
return pathname.isFile() && !"write.lock".equals(pathname.getName());
|
||||
}
|
||||
});
|
||||
if (files.length > 1) {
|
||||
fileToCorrupt = RandomPicks.randomFrom(getRandom(), files);
|
||||
try (Directory dir = FSDirectory.open(file)) {
|
||||
long checksumBeforeCorruption;
|
||||
try (IndexInput input = dir.openInput(fileToCorrupt.getName(), IOContext.DEFAULT)) {
|
||||
checksumBeforeCorruption = CodecUtil.retrieveChecksum(input);
|
||||
}
|
||||
try (RandomAccessFile raf = new RandomAccessFile(fileToCorrupt, "rw")) {
|
||||
raf.seek(randomIntBetween(0, (int)Math.min(Integer.MAX_VALUE, raf.length()-1)));
|
||||
long filePointer = raf.getFilePointer();
|
||||
byte b = raf.readByte();
|
||||
raf.seek(filePointer);
|
||||
raf.writeByte(~b);
|
||||
raf.getFD().sync();
|
||||
logger.info("Corrupting file for shard {} -- flipping at position {} from {} to {} file: {}", shardRouting, filePointer, Integer.toHexString(b), Integer.toHexString(~b), fileToCorrupt.getName());
|
||||
}
|
||||
long checksumAfterCorruption;
|
||||
long actualChecksumAfterCorruption;
|
||||
try (ChecksumIndexInput input = dir.openChecksumInput(fileToCorrupt.getName(), IOContext.DEFAULT)) {
|
||||
assertThat(input.getFilePointer(), is(0l));
|
||||
input.seek(input.length() - 8); // one long is the checksum... 8 bytes
|
||||
checksumAfterCorruption = input.getChecksum();
|
||||
actualChecksumAfterCorruption = input.readLong();
|
||||
}
|
||||
// we need to add assumptions here that the checksums actually really don't match there is a small chance to get collisions
|
||||
// in the checksum which is ok though....
|
||||
StringBuilder msg = new StringBuilder();
|
||||
msg.append("Checksum before: [").append(checksumBeforeCorruption).append("]");
|
||||
msg.append(" after: [").append(checksumAfterCorruption).append("]");
|
||||
msg.append(" checksum value after corruption: ").append(actualChecksumAfterCorruption).append("]");
|
||||
msg.append(" file: ").append(fileToCorrupt.getName()).append(" length: ").append(dir.fileLength(fileToCorrupt.getName()));
|
||||
logger.info(msg.toString());
|
||||
assumeTrue("Checksum collision - " + msg.toString(),
|
||||
checksumAfterCorruption != checksumBeforeCorruption // collision
|
||||
|| actualChecksumAfterCorruption != checksumBeforeCorruption); // checksum corrupted
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
assertThat("no file corrupted", fileToCorrupt, notNullValue());
|
||||
return shardRouting;
|
||||
}
|
||||
|
||||
public List<File> listShardFiles(ShardRouting routing) {
|
||||
NodesStatsResponse nodeStatses = client().admin().cluster().prepareNodesStats(routing.currentNodeId()).setFs(true).get();
|
||||
|
||||
assertThat(routing.toString(), nodeStatses.getNodes().length, equalTo(1));
|
||||
List<File> files = new ArrayList<>();
|
||||
for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) {
|
||||
String path = info.getPath();
|
||||
File file = new File(path, "indices/test/" + Integer.toString(routing.getId()) + "/index");
|
||||
files.addAll(Arrays.asList(file.listFiles()));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
private void disableAllocation(String index) {
|
||||
client().admin().indices().prepareUpdateSettings(index).setSettings(ImmutableSettings.builder().put(
|
||||
"index.routing.allocation.enable", "none"
|
||||
)).get();
|
||||
}
|
||||
|
||||
private void enableAllocation(String index) {
|
||||
client().admin().indices().prepareUpdateSettings(index).setSettings(ImmutableSettings.builder().put(
|
||||
"index.routing.allocation.enable", "all"
|
||||
)).get();
|
||||
}
|
||||
|
||||
public static class NoMergePolicyProvider extends AbstractMergePolicyProvider<MergePolicy> {
|
||||
|
||||
@Inject
|
||||
public NoMergePolicyProvider(Store store) {
|
||||
super(store);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MergePolicy newMergePolicy() {
|
||||
return NoMergePolicy.INSTANCE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws ElasticsearchException {
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.index.store;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.Listeners;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
|
||||
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
|
||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||
import org.apache.lucene.store.BaseDirectoryTestCase;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
import org.elasticsearch.test.ElasticsearchThreadFilter;
|
||||
import org.elasticsearch.test.junit.listeners.LoggingListener;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
@ThreadLeakFilters(defaultFilters = true, filters = {ElasticsearchThreadFilter.class})
|
||||
@ThreadLeakScope(ThreadLeakScope.Scope.NONE)
|
||||
@TimeoutSuite(millis = 20 * TimeUnits.MINUTE) // timeout the suite after 20min and fail the test.
|
||||
@Listeners(LoggingListener.class)
|
||||
public class DistributorDirectoryTest extends BaseDirectoryTestCase {
|
||||
|
||||
@Override
|
||||
protected Directory getDirectory(File path) throws IOException {
|
||||
Directory[] directories = new Directory[1 + random().nextInt(5)];
|
||||
for (int i = 0; i < directories.length; i++) {
|
||||
directories[i] = newDirectory();
|
||||
}
|
||||
return new DistributorDirectory(directories);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,430 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.index.store;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.CodecUtil;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.SortedDocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.CorruptIndexException;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.store.distributor.Distributor;
|
||||
import org.elasticsearch.index.store.distributor.LeastUsedDistributor;
|
||||
import org.elasticsearch.index.store.distributor.RandomWeightedDistributor;
|
||||
import org.elasticsearch.test.ElasticsearchLuceneTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
public class StoreTest extends ElasticsearchLuceneTestCase {
|
||||
|
||||
@Test
|
||||
public void testVerifyingIndexOutput() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
IndexOutput output = dir.createOutput("foo.bar", IOContext.DEFAULT);
|
||||
int iters = scaledRandomIntBetween(10, 100);
|
||||
for (int i = 0; i < iters; i++) {
|
||||
BytesRef bytesRef = new BytesRef(TestUtil.randomRealisticUnicodeString(random(), 10, 1024));
|
||||
output.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
|
||||
}
|
||||
CodecUtil.writeFooter(output);
|
||||
output.close();
|
||||
IndexInput indexInput = dir.openInput("foo.bar", IOContext.DEFAULT);
|
||||
String checksum = Store.digestToString(CodecUtil.retrieveChecksum(indexInput));
|
||||
indexInput.seek(0);
|
||||
BytesRef ref = new BytesRef(scaledRandomIntBetween(1, 1024));
|
||||
long length = indexInput.length();
|
||||
IndexOutput verifyingOutput = new Store.VerifyingIndexOutput(new StoreFileMetaData("foo1.bar", length, checksum, TEST_VERSION_CURRENT), dir.createOutput("foo1.bar", IOContext.DEFAULT));
|
||||
while (length > 0) {
|
||||
if (random().nextInt(10) == 0) {
|
||||
verifyingOutput.writeByte(indexInput.readByte());
|
||||
length--;
|
||||
} else {
|
||||
int min = (int) Math.min(length, ref.bytes.length);
|
||||
indexInput.readBytes(ref.bytes, ref.offset, min);
|
||||
verifyingOutput.writeBytes(ref.bytes, ref.offset, min);
|
||||
length -= min;
|
||||
}
|
||||
}
|
||||
Store.verify(verifyingOutput);
|
||||
verifyingOutput.writeByte((byte) 0x0);
|
||||
try {
|
||||
Store.verify(verifyingOutput);
|
||||
fail("should be a corrupted index");
|
||||
} catch (CorruptIndexException ex) {
|
||||
// ok
|
||||
}
|
||||
IOUtils.close(indexInput, verifyingOutput, dir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testVerifyingIndexOutputWithBogusInput() throws IOException {
|
||||
Directory dir = newDirectory();
|
||||
int length = scaledRandomIntBetween(10, 1024);
|
||||
IndexOutput verifyingOutput = new Store.VerifyingIndexOutput(new StoreFileMetaData("foo1.bar", length, "", TEST_VERSION_CURRENT), dir.createOutput("foo1.bar", IOContext.DEFAULT));
|
||||
try {
|
||||
while (length > 0) {
|
||||
verifyingOutput.writeByte((byte) random().nextInt());
|
||||
length--;
|
||||
}
|
||||
fail("should be a corrupted index");
|
||||
} catch (CorruptIndexException ex) {
|
||||
// ok
|
||||
}
|
||||
IOUtils.close(verifyingOutput, dir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteLegacyChecksums() throws IOException {
|
||||
final ShardId shardId = new ShardId(new Index("index"), 1);
|
||||
DirectoryService directoryService = new LuceneManagedDirectoryService();
|
||||
Store store = new Store(shardId, ImmutableSettings.EMPTY, null, null, directoryService, randomDistributor(directoryService));
|
||||
// set default codec - all segments need checksums
|
||||
IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(actualDefaultCodec()));
|
||||
int docs = 1 + random().nextInt(100);
|
||||
|
||||
for (int i = 0; i < docs; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new TextField("body", TestUtil.randomRealisticUnicodeString(random()), random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new SortedDocValuesField("dv", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
for (int i = 0; i < docs; i++) {
|
||||
if (random().nextBoolean()) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new TextField("body", TestUtil.randomRealisticUnicodeString(random()), random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
writer.updateDocument(new Term("id", "" + i), doc);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
DirectoryReader.open(writer, random().nextBoolean()).close(); // flush
|
||||
}
|
||||
// check before we committed
|
||||
Store.MetadataSnapshot metadata = store.getMetadata();
|
||||
assertThat(metadata.asMap().isEmpty(), is(true)); // nothing committed
|
||||
|
||||
writer.commit();
|
||||
Store.LegacyChecksums checksums = new Store.LegacyChecksums();
|
||||
Map<String, StoreFileMetaData> legacyMeta = new HashMap<>();
|
||||
for (String file : store.directory().listAll()) {
|
||||
if (file.equals("write.lock")) {
|
||||
continue;
|
||||
}
|
||||
try (IndexInput input = store.directory().openInput(file, IOContext.READONCE)) {
|
||||
String checksum = Store.digestToString(CodecUtil.retrieveChecksum(input));
|
||||
StoreFileMetaData storeFileMetaData = new StoreFileMetaData(file, store.directory().fileLength(file), checksum, null);
|
||||
legacyMeta.put(file, storeFileMetaData);
|
||||
checksums.add(storeFileMetaData);
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
checksums.write(store);
|
||||
|
||||
metadata = store.getMetadata();
|
||||
Map<String, StoreFileMetaData> stringStoreFileMetaDataMap = metadata.asMap();
|
||||
assertThat(legacyMeta.size(), equalTo(stringStoreFileMetaDataMap.size()));
|
||||
for (StoreFileMetaData meta : legacyMeta.values()) {
|
||||
assertTrue(stringStoreFileMetaDataMap.containsKey(meta.name()));
|
||||
assertTrue(stringStoreFileMetaDataMap.get(meta.name()).isSame(meta));
|
||||
}
|
||||
writer.close();
|
||||
assertDeleteContent(store, directoryService);
|
||||
IOUtils.close(store);
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNewChecksums() throws IOException {
|
||||
final ShardId shardId = new ShardId(new Index("index"), 1);
|
||||
DirectoryService directoryService = new LuceneManagedDirectoryService();
|
||||
Store store = new Store(shardId, ImmutableSettings.EMPTY, null, null, directoryService, randomDistributor(directoryService));
|
||||
// set default codec - all segments need checksums
|
||||
IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(actualDefaultCodec()));
|
||||
int docs = 1 + random().nextInt(100);
|
||||
|
||||
for (int i = 0; i < docs; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new TextField("body", TestUtil.randomRealisticUnicodeString(random()), random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new SortedDocValuesField("dv", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
for (int i = 0; i < docs; i++) {
|
||||
if (random().nextBoolean()) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new TextField("body", TestUtil.randomRealisticUnicodeString(random()), random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
writer.updateDocument(new Term("id", "" + i), doc);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
DirectoryReader.open(writer, random().nextBoolean()).close(); // flush
|
||||
}
|
||||
// check before we committed
|
||||
Store.MetadataSnapshot metadata = store.getMetadata();
|
||||
assertThat(metadata.asMap().isEmpty(), is(true)); // nothing committed
|
||||
|
||||
writer.commit();
|
||||
writer.close();
|
||||
metadata = store.getMetadata();
|
||||
assertThat(metadata.asMap().isEmpty(), is(false));
|
||||
for (StoreFileMetaData meta : metadata) {
|
||||
try (IndexInput input = store.directory().openInput(meta.name(), IOContext.DEFAULT)) {
|
||||
String checksum = Store.digestToString(CodecUtil.retrieveChecksum(input));
|
||||
assertThat("File: " + meta.name() + " has a different checksum", meta.checksum(), equalTo(checksum));
|
||||
assertThat(meta.hasLegacyChecksum(), equalTo(false));
|
||||
assertThat(meta.writtenBy(), equalTo(TEST_VERSION_CURRENT));
|
||||
}
|
||||
}
|
||||
assertConsistent(store, metadata);
|
||||
|
||||
TestUtil.checkIndex(store.directory());
|
||||
assertDeleteContent(store, directoryService);
|
||||
IOUtils.close(store);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMixedChecksums() throws IOException {
|
||||
final ShardId shardId = new ShardId(new Index("index"), 1);
|
||||
DirectoryService directoryService = new LuceneManagedDirectoryService();
|
||||
Store store = new Store(shardId, ImmutableSettings.EMPTY, null, null, directoryService, randomDistributor(directoryService));
|
||||
// this time random codec....
|
||||
IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), TEST_VERSION_CURRENT, new MockAnalyzer(random())).setCodec(actualDefaultCodec()));
|
||||
int docs = 1 + random().nextInt(100);
|
||||
|
||||
for (int i = 0; i < docs; i++) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new TextField("body", TestUtil.randomRealisticUnicodeString(random()), random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new SortedDocValuesField("dv", new BytesRef(TestUtil.randomRealisticUnicodeString(random()))));
|
||||
writer.addDocument(doc);
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
for (int i = 0; i < docs; i++) {
|
||||
if (random().nextBoolean()) {
|
||||
Document doc = new Document();
|
||||
doc.add(new TextField("id", "" + i, random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
doc.add(new TextField("body", TestUtil.randomRealisticUnicodeString(random()), random().nextBoolean() ? Field.Store.YES : Field.Store.NO));
|
||||
writer.updateDocument(new Term("id", "" + i), doc);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (random().nextBoolean()) {
|
||||
DirectoryReader.open(writer, random().nextBoolean()).close(); // flush
|
||||
}
|
||||
// check before we committed
|
||||
Store.MetadataSnapshot metadata = store.getMetadata();
|
||||
assertThat(metadata.asMap().isEmpty(), is(true)); // nothing committed
|
||||
writer.commit();
|
||||
writer.close();
|
||||
Store.LegacyChecksums checksums = new Store.LegacyChecksums();
|
||||
metadata = store.getMetadata();
|
||||
assertThat(metadata.asMap().isEmpty(), is(false));
|
||||
for (StoreFileMetaData meta : metadata) {
|
||||
try (IndexInput input = store.directory().openInput(meta.name(), IOContext.DEFAULT)) {
|
||||
if (meta.checksum() == null) {
|
||||
String checksum = null;
|
||||
try {
|
||||
CodecUtil.retrieveChecksum(input);
|
||||
fail("expected a corrupt index - posting format has not checksums");
|
||||
} catch (CorruptIndexException ex) {
|
||||
try (ChecksumIndexInput checksumIndexInput = store.directory().openChecksumInput(meta.name(), IOContext.DEFAULT)) {
|
||||
checksumIndexInput.seek(meta.length());
|
||||
checksum = Store.digestToString(checksumIndexInput.getChecksum());
|
||||
}
|
||||
// fine - it's a postings format without checksums
|
||||
checksums.add(new StoreFileMetaData(meta.name(), meta.length(), checksum, null));
|
||||
}
|
||||
} else {
|
||||
String checksum = Store.digestToString(CodecUtil.retrieveChecksum(input));
|
||||
assertThat("File: " + meta.name() + " has a different checksum", meta.checksum(), equalTo(checksum));
|
||||
assertThat(meta.hasLegacyChecksum(), equalTo(false));
|
||||
assertThat(meta.writtenBy(), equalTo(TEST_VERSION_CURRENT));
|
||||
}
|
||||
}
|
||||
}
|
||||
assertConsistent(store, metadata);
|
||||
checksums.write(store);
|
||||
metadata = store.getMetadata();
|
||||
assertThat(metadata.asMap().isEmpty(), is(false));
|
||||
for (StoreFileMetaData meta : metadata) {
|
||||
assertThat("file: " + meta.name() + " has a null checksum", meta.checksum(), not(nullValue()));
|
||||
if (meta.hasLegacyChecksum()) {
|
||||
try (ChecksumIndexInput checksumIndexInput = store.directory().openChecksumInput(meta.name(), IOContext.DEFAULT)) {
|
||||
checksumIndexInput.seek(meta.length());
|
||||
assertThat(meta.checksum(), equalTo(Store.digestToString(checksumIndexInput.getChecksum())));
|
||||
}
|
||||
} else {
|
||||
try (IndexInput input = store.directory().openInput(meta.name(), IOContext.DEFAULT)) {
|
||||
String checksum = Store.digestToString(CodecUtil.retrieveChecksum(input));
|
||||
assertThat("File: " + meta.name() + " has a different checksum", meta.checksum(), equalTo(checksum));
|
||||
assertThat(meta.hasLegacyChecksum(), equalTo(false));
|
||||
assertThat(meta.writtenBy(), equalTo(TEST_VERSION_CURRENT));
|
||||
}
|
||||
}
|
||||
}
|
||||
assertConsistent(store, metadata);
|
||||
TestUtil.checkIndex(store.directory());
|
||||
assertDeleteContent(store, directoryService);
|
||||
IOUtils.close(store);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRenameFile() throws IOException {
|
||||
final ShardId shardId = new ShardId(new Index("index"), 1);
|
||||
DirectoryService directoryService = new LuceneManagedDirectoryService(false);
|
||||
Store store = new Store(shardId, ImmutableSettings.EMPTY, null, null, directoryService, randomDistributor(directoryService));
|
||||
{
|
||||
IndexOutput output = store.directory().createOutput("foo.bar", IOContext.DEFAULT);
|
||||
int iters = scaledRandomIntBetween(10, 100);
|
||||
for (int i = 0; i < iters; i++) {
|
||||
BytesRef bytesRef = new BytesRef(TestUtil.randomRealisticUnicodeString(random(), 10, 1024));
|
||||
output.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
|
||||
}
|
||||
CodecUtil.writeFooter(output);
|
||||
output.close();
|
||||
}
|
||||
store.renameFile("foo.bar", "bar.foo");
|
||||
assertThat(store.directory().listAll().length, is(1));
|
||||
final long lastChecksum;
|
||||
try (IndexInput input = store.directory().openInput("bar.foo", IOContext.DEFAULT)) {
|
||||
lastChecksum = CodecUtil.checksumEntireFile(input);
|
||||
}
|
||||
|
||||
try {
|
||||
store.directory().openInput("foo.bar", IOContext.DEFAULT);
|
||||
fail("file was renamed");
|
||||
} catch (FileNotFoundException | NoSuchFileException ex) {
|
||||
// expected
|
||||
}
|
||||
{
|
||||
IndexOutput output = store.directory().createOutput("foo.bar", IOContext.DEFAULT);
|
||||
int iters = scaledRandomIntBetween(10, 100);
|
||||
for (int i = 0; i < iters; i++) {
|
||||
BytesRef bytesRef = new BytesRef(TestUtil.randomRealisticUnicodeString(random(), 10, 1024));
|
||||
output.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
|
||||
}
|
||||
CodecUtil.writeFooter(output);
|
||||
output.close();
|
||||
}
|
||||
try {
|
||||
store.renameFile("foo.bar", "bar.foo");
|
||||
fail("targe file already exists");
|
||||
} catch (IOException ex) {
|
||||
// expected
|
||||
}
|
||||
|
||||
try (IndexInput input = store.directory().openInput("bar.foo", IOContext.DEFAULT)) {
|
||||
assertThat(lastChecksum, equalTo(CodecUtil.checksumEntireFile(input)));
|
||||
}
|
||||
assertThat(store.directory().listAll().length, is(2));
|
||||
assertDeleteContent(store, directoryService);
|
||||
IOUtils.close(store);
|
||||
}
|
||||
|
||||
public void assertDeleteContent(Store store,DirectoryService service) throws IOException {
|
||||
store.deleteContent();
|
||||
assertThat(Arrays.toString(store.directory().listAll()), store.directory().listAll().length, equalTo(0));
|
||||
assertThat(store.stats().sizeInBytes(), equalTo(0l));
|
||||
for (Directory dir : service.build()) {
|
||||
assertThat(dir.listAll().length, equalTo(0));
|
||||
}
|
||||
}
|
||||
|
||||
private final class LuceneManagedDirectoryService implements DirectoryService {
|
||||
private final Directory[] dirs;
|
||||
|
||||
public LuceneManagedDirectoryService() {
|
||||
this(true);
|
||||
}
|
||||
public LuceneManagedDirectoryService(boolean preventDoubleWrite) {
|
||||
this.dirs = new Directory[1 + random().nextInt(5)];
|
||||
for (int i = 0; i < dirs.length; i++) {
|
||||
dirs[i] = newDirectory();
|
||||
if (dirs[i] instanceof MockDirectoryWrapper) {
|
||||
((MockDirectoryWrapper)dirs[i]).setPreventDoubleWrite(preventDoubleWrite);
|
||||
}
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public Directory[] build() throws IOException {
|
||||
return dirs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long throttleTimeInNanos() {
|
||||
return random().nextInt(1000);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void renameFile(Directory dir, String from, String to) throws IOException {
|
||||
dir.copy(dir, from, to, IOContext.DEFAULT);
|
||||
dir.deleteFile(from);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void fullDelete(Directory dir) throws IOException {
|
||||
for (String file : dir.listAll()) {
|
||||
dir.deleteFile(file);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void assertConsistent(Store store, Store.MetadataSnapshot metadata) throws IOException {
|
||||
for (String file : store.directory().listAll()) {
|
||||
if (!"write.lock".equals(file) && !Store.isChecksum(file)) {
|
||||
assertTrue(file + " is not in the map: " + metadata.asMap().size() + " vs. " + store.directory().listAll().length, metadata.asMap().containsKey(file));
|
||||
} else {
|
||||
assertFalse(file + " is not in the map: " + metadata.asMap().size() + " vs. " + store.directory().listAll().length, metadata.asMap().containsKey(file));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Distributor randomDistributor(DirectoryService service) throws IOException {
|
||||
return random().nextBoolean() ? new LeastUsedDistributor(service) : new RandomWeightedDistributor(service);
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -0,0 +1,60 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.indices.recovery;
|
||||
|
||||
import org.elasticsearch.test.ElasticsearchTestCase;
|
||||
|
||||
import static org.hamcrest.Matchers.closeTo;
|
||||
|
||||
public class RecoveryStateTest extends ElasticsearchTestCase {
|
||||
|
||||
public void testPercentage() {
|
||||
RecoveryState state = new RecoveryState();
|
||||
RecoveryState.Index index = state.getIndex();
|
||||
index.totalByteCount(100);
|
||||
index.reusedByteCount(20);
|
||||
index.recoveredByteCount(80);
|
||||
assertThat((double)index.percentBytesRecovered(), closeTo(80.0d, 0.1d));
|
||||
|
||||
index.totalFileCount(100);
|
||||
index.reusedFileCount(80);
|
||||
index.recoveredFileCount(20);
|
||||
assertThat((double)index.percentFilesRecovered(), closeTo(20.0d, 0.1d));
|
||||
|
||||
index.totalByteCount(0);
|
||||
index.reusedByteCount(0);
|
||||
index.recoveredByteCount(0);
|
||||
assertThat((double)index.percentBytesRecovered(), closeTo(0d, 0.1d));
|
||||
|
||||
index.totalFileCount(0);
|
||||
index.reusedFileCount(0);
|
||||
index.recoveredFileCount(0);
|
||||
assertThat((double)index.percentFilesRecovered(), closeTo(00.0d, 0.1d));
|
||||
|
||||
index.totalByteCount(10);
|
||||
index.reusedByteCount(0);
|
||||
index.recoveredByteCount(10);
|
||||
assertThat((double)index.percentBytesRecovered(), closeTo(100d, 0.1d));
|
||||
|
||||
index.totalFileCount(20);
|
||||
index.reusedFileCount(0);
|
||||
index.recoveredFileCount(20);
|
||||
assertThat((double)index.percentFilesRecovered(), closeTo(100.0d, 0.1d));
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.snapshots;
|
|||
import com.carrotsearch.randomizedtesting.LifecycleScope;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ListenableActionFuture;
|
||||
|
@ -42,14 +43,12 @@ import org.elasticsearch.cluster.metadata.SnapshotMetaData;
|
|||
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.store.support.AbstractIndexStore;
|
||||
import org.elasticsearch.indices.InvalidIndexNameException;
|
||||
import org.elasticsearch.repositories.RepositoriesService;
|
||||
import org.elasticsearch.snapshots.mockstore.MockRepositoryModule;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.elasticsearch.test.store.MockDirectoryHelper;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
|
|
|
@ -29,10 +29,8 @@ import org.elasticsearch.common.settings.Settings;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.Random;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
/**
|
||||
* A test cluster implementation that holds a fixed set of external nodes as well as a InternalTestCluster
|
||||
|
@ -99,6 +97,38 @@ public class CompositeTestCluster extends TestCluster {
|
|||
return upgradeOneNode(ImmutableSettings.EMPTY);
|
||||
}
|
||||
|
||||
/**
|
||||
* Upgrades all external running nodes to a node from the version running the tests.
|
||||
* All nodes are shut down before the first upgrade happens.
|
||||
* @return <code>true</code> iff at least one node as upgraded.
|
||||
*/
|
||||
public synchronized boolean upgradeAllNodes() throws InterruptedException, ExecutionException {
|
||||
return upgradeAllNodes(ImmutableSettings.EMPTY);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Upgrades all external running nodes to a node from the version running the tests.
|
||||
* All nodes are shut down before the first upgrade happens.
|
||||
* @return <code>true</code> iff at least one node as upgraded.
|
||||
* @param nodeSettings settings for the upgrade nodes
|
||||
*/
|
||||
public synchronized boolean upgradeAllNodes(Settings nodeSettings) throws InterruptedException, ExecutionException {
|
||||
Collection<ExternalNode> runningNodes = runningNodes();
|
||||
final int numRunningNodes = runningNodes.size();
|
||||
for (ExternalNode node : runningNodes) {
|
||||
node.stop(true);
|
||||
}
|
||||
if (numRunningNodes > 0) {
|
||||
List<String> names = cluster.startNodesAsync(numRunningNodes, nodeSettings).get();
|
||||
for (String nodeName : names) {
|
||||
ExternalNode.waitForNode(cluster.client(), nodeName);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Upgrades one external running node to a node from the version running the tests. Commonly this is used
|
||||
* to move from a node with version N-1 to a node running version N. This works seamless since they will
|
||||
|
@ -203,6 +233,13 @@ public class CompositeTestCluster extends TestCluster {
|
|||
return Iterators.singletonIterator(client());
|
||||
}
|
||||
|
||||
/**
|
||||
* Delegates to {@link org.elasticsearch.test.InternalTestCluster#fullRestart()}
|
||||
*/
|
||||
public void fullRestartInternalCluster() throws Exception {
|
||||
cluster.fullRestart();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of current version data nodes in the cluster
|
||||
*/
|
||||
|
|
|
@ -20,6 +20,11 @@ package org.elasticsearch.test;
|
|||
|
||||
import org.apache.lucene.util.AbstractRandomizedTest;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.discovery.DiscoveryModule;
|
||||
|
@ -33,6 +38,8 @@ import org.junit.Ignore;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
/**
|
||||
* Abstract base class for backwards compatibility tests. Subclasses of this class
|
||||
* can run tests against a mixed version cluster. A subset of the nodes in the cluster
|
||||
|
@ -125,4 +132,18 @@ public abstract class ElasticsearchBackwardsCompatIntegrationTest extends Elasti
|
|||
.put(TransportModule.TRANSPORT_SERVICE_TYPE_KEY, TransportService.class.getName())
|
||||
.build();
|
||||
}
|
||||
|
||||
public void assertAllShardsOnNodes(String index, String pattern) {
|
||||
ClusterState clusterState = client().admin().cluster().prepareState().execute().actionGet().getState();
|
||||
for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) {
|
||||
for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) {
|
||||
for (ShardRouting shardRouting : indexShardRoutingTable) {
|
||||
if (shardRouting.currentNodeId() != null && index.equals(shardRouting.getIndex())) {
|
||||
String name = clusterState.nodes().get(shardRouting.currentNodeId()).name();
|
||||
assertThat("Allocated on new node: " + name, Regex.simpleMatch(pattern, name), is(true));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -85,6 +85,7 @@ import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider;
|
|||
import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule;
|
||||
import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
|
||||
import org.elasticsearch.index.service.IndexService;
|
||||
import org.elasticsearch.index.store.StoreModule;
|
||||
import org.elasticsearch.index.translog.TranslogService;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
|
@ -423,6 +424,10 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
|
|||
builder.put(IndicesStore.INDICES_STORE_THROTTLE_TYPE, RandomPicks.randomFrom(random, StoreRateLimiting.Type.values()));
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
builder.put(StoreModule.DISTIBUTOR_KEY, random.nextBoolean() ? StoreModule.LEAST_USED_DISTRIBUTOR : StoreModule.RANDOM_WEIGHT_DISTRIBUTOR);
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
if (random.nextInt(10) == 0) { // do something crazy slow here
|
||||
builder.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, new ByteSizeValue(RandomInts.randomIntBetween(random, 1, 10), ByteSizeUnit.MB));
|
||||
|
@ -963,8 +968,10 @@ public abstract class ElasticsearchIntegrationTest extends ElasticsearchTestCase
|
|||
}
|
||||
|
||||
void ensureClusterSizeConsistency() {
|
||||
logger.trace("Check consistency for [{}] nodes", cluster().size());
|
||||
assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(cluster().size())).get());
|
||||
if (cluster() != null) { // if static init fails the cluster can be null
|
||||
logger.trace("Check consistency for [{}] nodes", cluster().size());
|
||||
assertNoTimeout(client().admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(cluster().size())).get());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -44,6 +44,13 @@ public abstract class ElasticsearchLuceneTestCase extends LuceneTestCase {
|
|||
|
||||
private static final Codec DEFAULT_CODEC = Codec.getDefault();
|
||||
|
||||
/**
|
||||
* Returns the lucene default codec without any randomization
|
||||
*/
|
||||
public static Codec actualDefaultCodec() {
|
||||
return DEFAULT_CODEC;
|
||||
}
|
||||
|
||||
/**
|
||||
* Forcefully reset the default codec
|
||||
*/
|
||||
|
|
|
@ -181,11 +181,14 @@ final class ExternalNode implements Closeable {
|
|||
this.random.setSeed(seed);
|
||||
}
|
||||
|
||||
|
||||
synchronized void stop() {
|
||||
stop(false);
|
||||
}
|
||||
|
||||
synchronized void stop(boolean forceKill) {
|
||||
if (running()) {
|
||||
try {
|
||||
if (nodeInfo != null && random.nextBoolean()) {
|
||||
if (forceKill == false && nodeInfo != null && random.nextBoolean()) {
|
||||
// sometimes shut down gracefully
|
||||
getClient().admin().cluster().prepareNodesShutdown(this.nodeInfo.getNode().id()).setExit(random.nextBoolean()).setDelay("0s").get();
|
||||
}
|
||||
|
|
|
@ -554,12 +554,7 @@ public class ElasticsearchAssertions {
|
|||
try {
|
||||
for (final MockDirectoryHelper.ElasticsearchMockDirectoryWrapper w : MockDirectoryHelper.wrappers) {
|
||||
try {
|
||||
awaitBusy(new Predicate<Object>() {
|
||||
@Override
|
||||
public boolean apply(Object input) {
|
||||
return !w.isOpen();
|
||||
}
|
||||
});
|
||||
w.awaitClosed(5000);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.interrupted();
|
||||
}
|
||||
|
|
|
@ -48,7 +48,6 @@ public class MockDirectoryHelper {
|
|||
|
||||
public static final Set<ElasticsearchMockDirectoryWrapper> wrappers = ConcurrentCollections.newConcurrentSet();
|
||||
|
||||
|
||||
private final Random random;
|
||||
private final double randomIOExceptionRate;
|
||||
private final double randomIOExceptionRateOnOpen;
|
||||
|
@ -124,7 +123,8 @@ public class MockDirectoryHelper {
|
|||
|
||||
private final ESLogger logger;
|
||||
private final boolean crash;
|
||||
private RuntimeException closeException;
|
||||
private volatile RuntimeException closeException;
|
||||
private final Object lock = new Object();
|
||||
|
||||
public ElasticsearchMockDirectoryWrapper(Random random, Directory delegate, ESLogger logger, boolean crash) {
|
||||
super(random, delegate);
|
||||
|
@ -140,6 +140,20 @@ public class MockDirectoryHelper {
|
|||
logger.info("MockDirectoryWrapper#close() threw exception", ex);
|
||||
closeException = ex;
|
||||
throw ex;
|
||||
} finally {
|
||||
synchronized (lock) {
|
||||
lock.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
public void awaitClosed(long timeout) throws InterruptedException {
|
||||
synchronized (lock) {
|
||||
if(isOpen()) {
|
||||
lock.wait(timeout);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.store.LockFactory;
|
|||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
@ -86,7 +87,7 @@ public class MockFSDirectoryService extends FsDirectoryService {
|
|||
}
|
||||
|
||||
|
||||
private void checkIndex(Store store) throws IndexShardException {
|
||||
public void checkIndex(Store store) throws IndexShardException {
|
||||
try {
|
||||
if (!Lucene.indexExists(store.directory())) {
|
||||
return;
|
||||
|
|
|
@ -51,14 +51,21 @@ public class MockTransportService extends TransportService {
|
|||
* Clears all the registered rules.
|
||||
*/
|
||||
public void clearAllRules() {
|
||||
((LookupTestTransport) transport).transports.clear();
|
||||
transport().transports.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the rule associated with the provided node.
|
||||
*/
|
||||
public void clearRule(DiscoveryNode node) {
|
||||
((LookupTestTransport) transport).transports.remove(node);
|
||||
transport().transports.remove(node);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the original Transport service wrapped by this mock transport service.
|
||||
*/
|
||||
public Transport original() {
|
||||
return original;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -66,7 +73,7 @@ public class MockTransportService extends TransportService {
|
|||
* is added to fail as well.
|
||||
*/
|
||||
public void addFailToSendNoConnectRule(DiscoveryNode node) {
|
||||
((LookupTestTransport) transport).transports.put(node, new DelegateTransport(original) {
|
||||
addDelegate(node, new DelegateTransport(original) {
|
||||
@Override
|
||||
public void connectToNode(DiscoveryNode node) throws ConnectTransportException {
|
||||
throw new ConnectTransportException(node, "DISCONNECT: simulated");
|
||||
|
@ -90,7 +97,7 @@ public class MockTransportService extends TransportService {
|
|||
*/
|
||||
public void addUnresponsiveRule(DiscoveryNode node) {
|
||||
// TODO add a parameter to delay the connect timeout?
|
||||
((LookupTestTransport) transport).transports.put(node, new DelegateTransport(original) {
|
||||
addDelegate(node, new DelegateTransport(original) {
|
||||
@Override
|
||||
public void connectToNode(DiscoveryNode node) throws ConnectTransportException {
|
||||
throw new ConnectTransportException(node, "UNRESPONSIVE: simulated");
|
||||
|
@ -108,6 +115,18 @@ public class MockTransportService extends TransportService {
|
|||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new delegate transport that is used for communication with the given node.
|
||||
* @return <tt>true</tt> iff no other delegate was registered for this node before, otherwise <tt>false</tt>
|
||||
*/
|
||||
public boolean addDelegate(DiscoveryNode node, DelegateTransport transport) {
|
||||
return transport().transports.put(node, transport) == null;
|
||||
}
|
||||
|
||||
private LookupTestTransport transport() {
|
||||
return (LookupTestTransport) transport;
|
||||
}
|
||||
|
||||
/**
|
||||
* A lookup transport that has a list of potential Transport implementations to delegate to for node operations,
|
||||
* if none is registered, then the default one is used.
|
||||
|
@ -159,14 +178,16 @@ public class MockTransportService extends TransportService {
|
|||
* A pure delegate transport.
|
||||
* Can be extracted to a common class if needed in other places in the codebase.
|
||||
*/
|
||||
private static class DelegateTransport implements Transport {
|
||||
public static class DelegateTransport implements Transport {
|
||||
|
||||
protected final Transport transport;
|
||||
|
||||
DelegateTransport(Transport transport) {
|
||||
public DelegateTransport(Transport transport) {
|
||||
this.transport = transport;
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Override
|
||||
public void transportServiceAdapter(TransportServiceAdapter service) {
|
||||
transport.transportServiceAdapter(service);
|
||||
|
|
Loading…
Reference in New Issue