Decouple recoveries from engine flush

In order to safely complete recoveries / relocations we have to keep all operation done since the recovery start at available for replay. At the moment we do so by preventing the engine from flushing and thus making sure that the operations are kept in the translog. A side effect of this is that the translog keeps on growing until the recovery is done. This is not a problem as we do need these operations but if the another recovery starts concurrently it may have an unneededly long translog to replay. Also, if we shutdown the engine for some reason at this point (like when a node is restarted)  we have to recover a long translog when we come back.

To void this, the translog is changed to be based on multiple files instead of a single one. This allows recoveries to keep hold to the files they need while allowing the engine to flush and do a lucene commit (which will create a new translog files bellow the hood).

Change highlights:
- Refactor Translog file management to allow for multiple files.
- Translog maintains a list of referenced files, both by outstanding recoveries and files containing operations not yet committed to Lucene.
- A new Translog.View concept is introduced, allowing recoveries to get a reference to all currently uncommitted translog files plus all future translog files created until the view is closed. They can use this view to iterate over operations.
- Recovery phase3 is removed. That phase was replaying operations while preventing new writes to the engine. This is unneeded as standard indexing also send all operations from the start of the recovery  to the recovering shard. Replay all ops in the view acquired in recovery start is enough to guarantee no operation is lost.
- IndexShard now creates the translog together with the engine. The translog is closed by the engine on close. ShadowIndexShards do not open the translog.
- Moved the ownership of translog fsyncing to the translog it self, changing the responsible setting to `index.translog.sync_interval` (was `index.gateway.local.sync`)

Closes #10624
This commit is contained in:
Boaz Leskes 2015-03-27 10:18:09 +01:00
parent 12c19508fb
commit d596f5cc45
54 changed files with 2289 additions and 1822 deletions

View File

@ -24,7 +24,7 @@ The period with no flush happening to force a flush. Defaults to `30m`.
How often to check if a flush is needed, randomized
between the interval value and 2x the interval value. Defaults to `5s`.
`index.gateway.local.sync`::
`index.translog.sync_interval`::
How often the translog is ++fsync++ed to disk. Defaults to `5s`.

View File

@ -87,6 +87,22 @@ public final class Channels {
return readFromFileChannel(channel, channelPosition, buffer);
}
/**
* read from a file channel into a byte buffer, starting at a certain position. An EOFException will be thrown if you
* attempt to read beyond the end of file.
*
* @param channel channel to read from
* @param channelPosition position to read from
* @param dest destination {@link java.nio.ByteBuffer} to put data in
*/
public static void readFromFileChannelWithEofException(FileChannel channel, long channelPosition, ByteBuffer dest) throws IOException {
int read = readFromFileChannel(channel, channelPosition, dest);
if (read < 0) {
throw new EOFException("read past EOF. pos [" + channelPosition + "] length: [" + dest.limit() + "] end: [" + channel.size() + "]");
}
}
/**
* read from a file channel into a byte buffer, starting at a certain position.
*

View File

@ -73,5 +73,11 @@ public abstract class AbstractRefCounted implements RefCounted {
return this.refCount.get();
}
/** gets the name of this instance */
public String getName() {
return name;
}
protected abstract void closeInternal();
}

View File

@ -30,17 +30,48 @@ import java.util.concurrent.locks.Lock;
public class ReleasableLock implements Releasable {
private final Lock lock;
/* a per thread boolean indicating the lock is held by it. only works when assertions are enabled */
private final ThreadLocal<Boolean> holdingThreads;
public ReleasableLock(Lock lock) {
this.lock = lock;
boolean useHoldingThreads = false;
assert (useHoldingThreads = true);
if (useHoldingThreads) {
holdingThreads = new ThreadLocal<>();
} else {
holdingThreads = null;
}
}
@Override
public void close() {
lock.unlock();
assert removeCurrentThread();
}
public ReleasableLock acquire() throws EngineException {
lock.lock();
assert addCurrentThread();
return this;
}
private boolean addCurrentThread() {
holdingThreads.set(true);
return true;
}
private boolean removeCurrentThread() {
holdingThreads.remove();
return true;
}
public Boolean isHeldByCurrentThread() {
if (holdingThreads == null) {
throw new UnsupportedOperationException("asserts must be enabled");
}
Boolean b = holdingThreads.get();
return b != null && b.booleanValue();
}
}

View File

@ -65,8 +65,6 @@ import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.StoreModule;
import org.elasticsearch.index.suggest.SuggestShardModule;
import org.elasticsearch.index.termvectors.ShardTermVectorsModule;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogModule;
import org.elasticsearch.index.translog.TranslogService;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService;
@ -187,6 +185,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
}
return null;
}
/**
* Return the shard with the provided id, or throw an exception if it doesn't exist.
*/
@ -320,7 +319,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
modules.add(new ShardQueryCacheModule());
modules.add(new ShardBitsetFilterCacheModule());
modules.add(new ShardFieldDataModule());
modules.add(new TranslogModule(indexSettings));
modules.add(new IndexShardGatewayModule());
modules.add(new PercolatorShardModule());
modules.add(new ShardTermVectorsModule());
@ -386,7 +384,8 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
}
}
// now we can close the translog service, we need to close it before the we close the shard
closeInjectorResource(sId, shardInjector, TranslogService.class);
// note the that the translog service is not there for shadow replicas
closeInjectorOptionalResource(sId, shardInjector, TranslogService.class);
// this logic is tricky, we want to close the engine so we rollback the changes done to it
// and close the shard so no operations are allowed to it
if (indexShard != null) {
@ -402,7 +401,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
MergeSchedulerProvider.class,
MergePolicyProvider.class,
IndexShardGatewayService.class,
Translog.class,
PercolatorQueriesRegistry.class);
// call this before we close the store, so we can release resources for it
@ -423,18 +421,30 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
*/
private void closeInjectorResource(ShardId shardId, Injector shardInjector, Class<? extends Closeable>... toClose) {
for (Class<? extends Closeable> closeable : toClose) {
try {
final Closeable instance = shardInjector.getInstance(closeable);
if (instance == null) {
throw new NullPointerException("No instance available for " + closeable.getName());
}
IOUtils.close(instance);
} catch (Throwable t) {
logger.debug("{} failed to close {}", t, shardId, Strings.toUnderscoreCase(closeable.getSimpleName()));
if (closeInjectorOptionalResource(shardId, shardInjector, closeable) == false) {
logger.warn("[{}] no instance available for [{}], ignoring... ", shardId, closeable.getSimpleName());
}
}
}
/**
* Closes an optional resource. Returns true if the resource was found;
* NOTE: this method swallows all exceptions thrown from the close method of the injector and logs them as debug log
*/
private boolean closeInjectorOptionalResource(ShardId shardId, Injector shardInjector, Class<? extends Closeable> toClose) {
try {
final Closeable instance = shardInjector.getInstance(toClose);
if (instance == null) {
return false;
}
IOUtils.close(instance);
} catch (Throwable t) {
logger.debug("{} failed to close {}", t, shardId, Strings.toUnderscoreCase(toClose.getSimpleName()));
}
return true;
}
private void onShardClose(ShardLock lock, boolean ownsShard) {
if (deleted.get()) { // we remove that shards content if this index has been deleted
try {
@ -464,7 +474,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override
public void handle(ShardLock lock) {
assert lock.getShardId().equals(shardId) : "shard Id mismatch, expected: " + shardId + " but got: " + lock.getShardId();
assert lock.getShardId().equals(shardId) : "shard id mismatch, expected: " + shardId + " but got: " + lock.getShardId();
onShardClose(lock, ownsShard);
}
}

View File

@ -82,7 +82,6 @@ public abstract class Engine implements Closeable {
protected Engine(EngineConfig engineConfig) {
Preconditions.checkNotNull(engineConfig.getStore(), "Store must be provided to the engine");
Preconditions.checkNotNull(engineConfig.getDeletionPolicy(), "Snapshot deletion policy must be provided to the engine");
Preconditions.checkNotNull(engineConfig.getTranslog(), "Translog must be provided to the engine");
this.engineConfig = engineConfig;
this.shardId = engineConfig.getShardId();
@ -278,6 +277,9 @@ public abstract class Engine implements Closeable {
}
}
/** returns the translog for this engine */
public abstract Translog translog();
protected void ensureOpen() {
if (isClosed.get()) {
throw new EngineClosedException(shardId, failedEngine);
@ -449,12 +451,12 @@ public abstract class Engine implements Closeable {
public abstract void forceMerge(boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade, boolean upgradeOnlyAncientSegments) throws EngineException;
/**
* Snapshots the index and returns a handle to it. Will always try and "commit" the
* Snapshots the index and returns a handle to it. If needed will try and "commit" the
* lucene index to make sure we have a "fresh" copy of the files to snapshot.
*
* @param flushFirst indicates whether the engine should flush before returning the snapshot
*/
public abstract SnapshotIndexCommit snapshotIndex() throws EngineException;
public abstract void recover(RecoveryHandler recoveryHandler) throws EngineException;
public abstract SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineException;
/** fail engine due to some error. the engine will also be closed. */
public void failEngine(String reason, Throwable failure) {
@ -1048,7 +1050,7 @@ public abstract class Engine implements Closeable {
public void flushAndClose() throws IOException {
if (isClosed.get() == false) {
logger.trace("flushAndClose now acquire writeLock");
try (ReleasableLock _ = writeLock.acquire()) {
try (ReleasableLock lock = writeLock.acquire()) {
logger.trace("flushAndClose now acquired writeLock");
try {
logger.debug("flushing shard on close - this might take some time to sync files to disk");
@ -1070,7 +1072,7 @@ public abstract class Engine implements Closeable {
public void close() throws IOException {
if (isClosed.get() == false) { // don't acquire the write lock if we are already closed
logger.debug("close now acquiring writeLock");
try (ReleasableLock _ = writeLock.acquire()) {
try (ReleasableLock lock = writeLock.acquire()) {
logger.debug("close acquired writeLock");
closeNoLock("api");
}

View File

@ -37,7 +37,6 @@ import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.threadpool.ThreadPool;
@ -67,13 +66,13 @@ public final class EngineConfig {
private final IndicesWarmer warmer;
private final Store store;
private final SnapshotDeletionPolicy deletionPolicy;
private final Translog translog;
private final MergePolicyProvider mergePolicyProvider;
private final MergeSchedulerProvider mergeScheduler;
private final Analyzer analyzer;
private final Similarity similarity;
private final CodecService codecService;
private final Engine.FailedEngineListener failedEngineListener;
private final boolean ignoreUnknownTranslog;
/**
* Index setting for index concurrency / number of threadstates in the indexwriter.
@ -117,6 +116,11 @@ public final class EngineConfig {
*/
public static final String INDEX_VERSION_MAP_SIZE = "index.version_map_size";
/** if set to true the engine will start even if the translog id in the commit point can not be found */
public static final String INDEX_IGNORE_UNKNOWN_TRANSLOG = "index.engine.ignore_unknown_translog";
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
@ -130,7 +134,7 @@ public final class EngineConfig {
/**
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
*/
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService, IndexSettingsService indexSettingsService, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy, Translog translog, MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler, Analyzer analyzer, Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener, TranslogRecoveryPerformer translogRecoveryPerformer) {
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService, IndexSettingsService indexSettingsService, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy, MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler, Analyzer analyzer, Similarity similarity, CodecService codecService, Engine.FailedEngineListener failedEngineListener, TranslogRecoveryPerformer translogRecoveryPerformer) {
this.shardId = shardId;
this.threadPool = threadPool;
this.indexingService = indexingService;
@ -138,7 +142,6 @@ public final class EngineConfig {
this.warmer = warmer;
this.store = store;
this.deletionPolicy = deletionPolicy;
this.translog = translog;
this.mergePolicyProvider = mergePolicyProvider;
this.mergeScheduler = mergeScheduler;
this.analyzer = analyzer;
@ -155,6 +158,7 @@ public final class EngineConfig {
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize();
this.translogRecoveryPerformer = translogRecoveryPerformer;
this.ignoreUnknownTranslog = indexSettings.getAsBoolean(INDEX_IGNORE_UNKNOWN_TRANSLOG, false);
}
/** updates {@link #versionMapSize} based on current setting and {@link #indexingBufferSize} */
@ -182,6 +186,10 @@ public final class EngineConfig {
return versionMapSizeSetting;
}
/** if true the engine will start even if the translog id in the commit point can not be found */
public boolean getIgnoreUnknownTranslog() {
return ignoreUnknownTranslog;
}
/**
* returns the size of the version map that should trigger a refresh
@ -318,13 +326,6 @@ public final class EngineConfig {
return deletionPolicy;
}
/**
* Returns a {@link Translog instance}
*/
public Translog getTranslog() {
return translog;
}
/**
* Returns the {@link org.elasticsearch.index.merge.policy.MergePolicyProvider} used to obtain
* a {@link org.apache.lucene.index.MergePolicy} for the engines {@link org.apache.lucene.index.IndexWriter}

View File

@ -18,12 +18,14 @@
*/
package org.elasticsearch.index.engine;
import org.elasticsearch.index.translog.fs.FsTranslog;
/**
* Simple Engine Factory
*/
public interface EngineFactory {
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery);
public Engine newReadWriteEngine(EngineConfig config, FsTranslog translog, boolean skipTranslogRecovery);
public Engine newReadOnlyEngine(EngineConfig config);
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.engine;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
@ -31,9 +32,7 @@ import org.apache.lucene.util.InfoStream;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.routing.DjbHashFunction;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.LoggerInfoStream;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
@ -51,7 +50,7 @@ import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
import org.elasticsearch.index.search.nested.IncludeNestedDocsQuery;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TruncatedTranslogException;
import org.elasticsearch.index.translog.fs.FsTranslog;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
@ -61,7 +60,6 @@ import java.io.IOException;
import java.util.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -79,7 +77,7 @@ public class InternalEngine extends Engine {
private final ShardIndexingService indexingService;
@Nullable
private final IndicesWarmer warmer;
private final Translog translog;
private final FsTranslog translog;
private final MergePolicyProvider mergePolicyProvider;
private final MergeSchedulerProvider mergeScheduler;
@ -94,33 +92,31 @@ public class InternalEngine extends Engine {
private final Lock flushLock = new ReentrantLock();
private final ReentrantLock optimizeLock = new ReentrantLock();
protected final FlushingRecoveryCounter onGoingRecoveries;
// A uid (in the form of BytesRef) to the version map
// we use the hashed variant since we iterate over it and check removal and additions on existing keys
private final LiveVersionMap versionMap;
private final Object[] dirtyLocks;
private final AtomicLong translogIdGenerator = new AtomicLong();
private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean();
private volatile SegmentInfos lastCommittedSegmentInfos;
private final IndexThrottle throttle;
public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogRecovery) throws EngineException {
public InternalEngine(EngineConfig engineConfig, FsTranslog translog, boolean skipInitialTranslogRecovery) throws EngineException {
super(engineConfig);
Preconditions.checkNotNull(translog, "Translog must be provided to the engine");
this.versionMap = new LiveVersionMap();
store.incRef();
IndexWriter writer = null;
SearcherManager manager = null;
boolean success = false;
try {
this.onGoingRecoveries = new FlushingRecoveryCounter(this, store, logger);
this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis();
this.indexingService = engineConfig.getIndexingService();
this.warmer = engineConfig.getWarmer();
this.translog = engineConfig.getTranslog();
this.translog = translog;
this.mergePolicyProvider = engineConfig.getMergePolicyProvider();
this.mergeScheduler = engineConfig.getMergeScheduler();
this.dirtyLocks = new Object[engineConfig.getIndexConcurrency() * 50]; // we multiply it to have enough...
@ -130,11 +126,11 @@ public class InternalEngine extends Engine {
throttle = new IndexThrottle();
this.searcherFactory = new SearchFactory(engineConfig);
final Tuple<Long, Long> translogId; // nextTranslogId, currentTranslogId
final Long committedTranslogId;
try {
writer = createWriter();
indexWriter = writer;
translogId = loadTranslogIds(writer, translog);
committedTranslogId = loadCommittedTranslogId(writer, translog);
} catch (IOException e) {
throw new EngineCreationFailureException(shardId, "failed to create engine", e);
}
@ -145,15 +141,12 @@ public class InternalEngine extends Engine {
this.mergeSchedulerListener = new MergeSchedulerListener();
this.mergeScheduler.addListener(mergeSchedulerListener);
this.mergeScheduler.addFailureListener(mergeSchedulerFailureListener);
final TranslogRecoveryPerformer transformer = engineConfig.getTranslogRecoveryPerformer();
try {
long nextTranslogID = translogId.v2();
translog.newTranslog(nextTranslogID);
translogIdGenerator.set(nextTranslogID);
if (translogId.v1() != null && skipInitialTranslogRecovery == false) {
recoverFromTranslog(translogId.v1(), transformer);
if (skipInitialTranslogRecovery) {
// make sure we point at the latest translog from now on..
commitIndexWriter(writer, translog.currentId());
} else {
flush(true, true);
recoverFromTranslog(engineConfig, committedTranslogId);
}
} catch (IOException | EngineException ex) {
throw new EngineCreationFailureException(shardId, "failed to recover from translog", ex);
@ -172,24 +165,71 @@ public class InternalEngine extends Engine {
logger.trace("created new InternalEngine");
}
@Override
public Translog translog() {
ensureOpen();
return translog;
}
protected void recoverFromTranslog(EngineConfig engineConfig, Long committedTranslogId) throws IOException {
if (committedTranslogId != null) {
try {
// trim unneeded files
translog.markCommitted(committedTranslogId);
} catch (FileNotFoundException ex) {
if (engineConfig.getIgnoreUnknownTranslog()) {
logger.warn("ignoring committed translog id [{}] ([{}] set to true)", committedTranslogId,
EngineConfig.INDEX_IGNORE_UNKNOWN_TRANSLOG);
} else {
throw ex;
}
}
}
int opsRecovered = 0;
final TranslogRecoveryPerformer handler = engineConfig.getTranslogRecoveryPerformer();
try (Translog.Snapshot snapshot = translog.newSnapshot()) {
Translog.Operation operation;
while ((operation = snapshot.next()) != null) {
try {
handler.performRecoveryOperation(this, operation);
opsRecovered++;
} 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) {
throw new EngineException(shardId, "failed to recover from translog", e);
}
// flush if we recovered something or if we have references to older translogs
// note: if opsRecovered == 0 and we have older translogs it means they are corrupted or 0 length.
if (opsRecovered > 0 ||
(committedTranslogId != null && translog.currentId() != committedTranslogId)) {
logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]",
opsRecovered, committedTranslogId, translog.currentId());
flush(true, true);
}
}
/**
* Reads the current stored translog ID (v1) from the IW commit data and generates a new/next translog ID (v2)
* from the largest present translog ID. If there is no stored translog ID v1 is <code>null</code>
* Reads the current stored translog ID from the IW commit data. If the id is not found, recommits the current
* translog id into lucene and returns null.
*/
private Tuple<Long, Long> loadTranslogIds(IndexWriter writer, Translog translog) throws IOException {
@Nullable
private Long loadCommittedTranslogId(IndexWriter writer, Translog translog) throws IOException {
// commit on a just opened writer will commit even if there are no changes done to it
// we rely on that for the commit data translog id key
final long nextTranslogId = Math.max(0, translog.findLargestPresentTranslogId()) + 1;
final Map<String, String> commitUserData = writer.getCommitData();
if (commitUserData.containsKey(Translog.TRANSLOG_ID_KEY)) {
final long currentTranslogId = Long.parseLong(commitUserData.get(Translog.TRANSLOG_ID_KEY));
return new Tuple<>(currentTranslogId, nextTranslogId);
return Long.parseLong(commitUserData.get(Translog.TRANSLOG_ID_KEY));
}
// translog id is not in the metadata - fix this inconsistency some code relies on this and old indices might not have it.
writer.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(nextTranslogId)));
commitIndexWriter(writer);
logger.debug("no translog ID present in the current commit - creating one");
return new Tuple<>(null, nextTranslogId);
commitIndexWriter(writer, translog.currentId());
return null;
}
private SearcherManager createSearcherManager() throws EngineException {
@ -617,12 +657,6 @@ public class InternalEngine extends Engine {
private void flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException {
ensureOpen();
if (commitTranslog) {
// check outside the lock as well so we can check without blocking on the write lock
if (onGoingRecoveries.get() > 0) {
throw new FlushNotAllowedEngineException(shardId, "recovery is in progress, flush with committing translog is not allowed");
}
}
/*
* Unfortunately the lock order is important here. We have to acquire the readlock first otherwise
* if we are flushing at the end of the recovery while holding the write lock we can deadlock if:
@ -646,32 +680,19 @@ public class InternalEngine extends Engine {
}
try {
if (commitTranslog) {
if (onGoingRecoveries.get() > 0) {
throw new FlushNotAllowedEngineException(shardId, "Recovery is in progress, flush is not allowed");
}
if (flushNeeded || force) {
flushNeeded = false;
final long translogId;
try {
long translogId = translogIdGenerator.incrementAndGet();
translog.newTransientTranslog(translogId);
indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)));
translogId = translog.newTranslog();
logger.trace("starting commit for flush; commitTranslog=true");
commitIndexWriter(indexWriter);
commitIndexWriter(indexWriter, translogId);
logger.trace("finished commit for flush");
// we need to refresh in order to clear older version values
refresh("version_table_flush");
// we need to move transient to current only after we refresh
// so items added to current will still be around for realtime get
// when tans overrides it
translog.makeTransientCurrent();
translog.markCommitted(translogId);
} catch (Throwable e) {
try {
translog.revertTransient();
} catch (IOException ex) {
e.addSuppressed(ex);
}
throw new FlushFailedEngineException(shardId, e);
}
}
@ -683,10 +704,8 @@ public class InternalEngine extends Engine {
// its ok to use this, only a flush will cause a new translogId, and we are locked here from
// other flushes use flushLock
try {
long translogId = translog.currentId();
indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)));
logger.trace("starting commit for flush; commitTranslog=false");
commitIndexWriter(indexWriter);
commitIndexWriter(indexWriter, translog.currentId());
logger.trace("finished commit for flush");
} catch (Throwable e) {
throw new FlushFailedEngineException(shardId, e);
@ -807,12 +826,14 @@ public class InternalEngine extends Engine {
}
@Override
public SnapshotIndexCommit snapshotIndex() throws EngineException {
public SnapshotIndexCommit snapshotIndex(final boolean flushFirst) throws EngineException {
// we have to flush outside of the readlock otherwise we might have a problem upgrading
// the to a write lock when we fail the engine in this operation
logger.trace("start flush for snapshot");
flush(false, false, true);
logger.trace("finish flush for snapshot");
if (flushFirst) {
logger.trace("start flush for snapshot");
flush(false, false, true);
logger.trace("finish flush for snapshot");
}
try (ReleasableLock lock = readLock.acquire()) {
ensureOpen();
logger.trace("pulling snapshot");
@ -822,65 +843,6 @@ public class InternalEngine extends Engine {
}
}
@Override
public void recover(RecoveryHandler recoveryHandler) throws EngineException {
// take a write lock here so it won't happen while a flush is in progress
// this means that next commits will not be allowed once the lock is released
try (ReleasableLock lock = writeLock.acquire()) {
ensureOpen();
onGoingRecoveries.startRecovery();
}
SnapshotIndexCommit phase1Snapshot;
try {
phase1Snapshot = deletionPolicy.snapshot();
} catch (Throwable e) {
maybeFailEngine("recovery", e);
Releasables.closeWhileHandlingException(onGoingRecoveries);
throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e);
}
try {
recoveryHandler.phase1(phase1Snapshot);
} catch (Throwable e) {
maybeFailEngine("recovery phase 1", e);
Releasables.closeWhileHandlingException(phase1Snapshot, onGoingRecoveries);
throw new RecoveryEngineException(shardId, 1, "Execution failed", wrapIfClosed(e));
}
Translog.Snapshot phase2Snapshot;
try {
phase2Snapshot = translog.snapshot();
} catch (Throwable e) {
maybeFailEngine("snapshot recovery", e);
Releasables.closeWhileHandlingException(phase1Snapshot, onGoingRecoveries);
throw new RecoveryEngineException(shardId, 2, "Snapshot failed", wrapIfClosed(e));
}
try {
recoveryHandler.phase2(phase2Snapshot);
} catch (Throwable e) {
maybeFailEngine("recovery phase 2", e);
Releasables.closeWhileHandlingException(phase1Snapshot, phase2Snapshot, onGoingRecoveries);
throw new RecoveryEngineException(shardId, 2, "Execution failed", wrapIfClosed(e));
}
writeLock.acquire();
Translog.Snapshot phase3Snapshot = null;
boolean success = false;
try {
ensureOpen();
phase3Snapshot = translog.snapshot(phase2Snapshot);
recoveryHandler.phase3(phase3Snapshot);
success = true;
} catch (Throwable e) {
maybeFailEngine("recovery phase 3", e);
throw new RecoveryEngineException(shardId, 3, "Execution failed", wrapIfClosed(e));
} finally {
Releasables.close(success, phase1Snapshot, phase2Snapshot, phase3Snapshot,
onGoingRecoveries, writeLock); // hmm why can't we use try-with here?
}
}
@Override
protected boolean maybeFailEngine(String source, Throwable t) {
boolean shouldFail = super.maybeFailEngine(source, t);
@ -950,11 +912,6 @@ public class InternalEngine extends Engine {
if (isClosed.compareAndSet(false, true)) {
assert rwl.isWriteLockedByCurrentThread() || failEngineLock.isHeldByCurrentThread() : "Either the write lock must be held or the engine must be currently be failing itself";
try {
try {
IOUtils.close(this.translog);
} catch (IOException ex) {
logger.warn("failed to close translog", ex);
}
this.versionMap.clear();
logger.trace("close searcherManager");
try {
@ -962,6 +919,11 @@ public class InternalEngine extends Engine {
} catch (Throwable t) {
logger.warn("Failed to close SearcherManager", t);
}
try {
IOUtils.close(translog);
} catch (Throwable t) {
logger.warn("Failed to close translog", t);
}
// no need to commit in this case!, we snapshot before we close the shard, so translog and all sync'ed
logger.trace("rollback indexWriter");
try {
@ -1194,47 +1156,14 @@ public class InternalEngine extends Engine {
}
private void commitIndexWriter(IndexWriter writer) throws IOException {
private void commitIndexWriter(IndexWriter writer, long translogId) throws IOException {
try {
logger.trace("committing writer with translog id [{}] ", translogId);
indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)));
writer.commit();
} catch (Throwable ex) {
failEngine("lucene commit failed", ex);
throw ex;
}
}
protected void recoverFromTranslog(long translogId, TranslogRecoveryPerformer handler) throws IOException {
final Translog translog = engineConfig.getTranslog();
int operationsRecovered = 0;
try (Translog.OperationIterator in = translog.openIterator(translogId)) {
Translog.Operation operation;
while ((operation = in.next()) != null) {
try {
handler.performRecoveryOperation(this, operation);
operationsRecovered++;
} 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 (FileNotFoundException ex) {
logger.debug("no translog file found for ID: " + translogId);
} catch (TruncatedTranslogException e) {
// file is empty or header has been half-written and should be ignored
logger.trace("ignoring truncation exception, the translog is either empty or half-written", e);
} catch (Throwable e) {
IOUtils.closeWhileHandlingException(translog);
throw new EngineException(shardId, "failed to recover from translog", e);
}
flush(true, true);
if (operationsRecovered > 0) {
refresh("translog recovery");
}
translog.clearUnreferenced();
}
}

View File

@ -18,10 +18,12 @@
*/
package org.elasticsearch.index.engine;
import org.elasticsearch.index.translog.fs.FsTranslog;
public class InternalEngineFactory implements EngineFactory {
@Override
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery) {
return new InternalEngine(config, skipTranslogRecovery);
public Engine newReadWriteEngine(EngineConfig config, FsTranslog translog, boolean skipTranslogRecovery) {
return new InternalEngine(config, translog, skipTranslogRecovery);
}
@Override

View File

@ -25,13 +25,13 @@ import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.shard.IndexShardException;
import org.elasticsearch.index.translog.Translog;
import java.io.IOException;
import java.util.Arrays;
@ -168,6 +168,11 @@ public class ShadowEngine extends Engine {
return getFromSearcher(get);
}
@Override
public Translog translog() {
throw new UnsupportedOperationException("shard engines don't have translogs");
}
@Override
public List<Segment> segments(boolean verbose) {
try (ReleasableLock lock = readLock.acquire()) {
@ -199,15 +204,10 @@ public class ShadowEngine extends Engine {
}
@Override
public SnapshotIndexCommit snapshotIndex() throws EngineException {
public SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineException {
throw new UnsupportedOperationException("Can not take snapshot from a shadow engine");
}
@Override
public void recover(RecoveryHandler recoveryHandler) throws EngineException {
throw new UnsupportedOperationException("Can not recover from a shadow engine");
}
@Override
protected SearcherManager getSearcherManager() {
return searcherManager;

View File

@ -30,14 +30,12 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CancellableThreads;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.threadpool.ThreadPool;
@ -47,7 +45,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
@ -61,9 +58,7 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl
private final IndexService indexService;
private final IndexShard indexShard;
private final TimeValue waitForMappingUpdatePostRecovery;
private final TimeValue syncInterval;
private volatile ScheduledFuture<?> flushScheduler;
private final CancellableThreads cancellableThreads = new CancellableThreads();
@ -76,17 +71,7 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl
this.indexService = indexService;
this.indexShard = indexShard;
this.waitForMappingUpdatePostRecovery = indexSettings.getAsTime("index.gateway.wait_for_mapping_update_post_recovery", TimeValue.timeValueMinutes(15));
syncInterval = indexSettings.getAsTime("index.gateway.sync", TimeValue.timeValueSeconds(5));
if (syncInterval.millis() > 0) {
this.indexShard.translog().syncOnEachOperation(false);
flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, new Sync());
} else if (syncInterval.millis() == 0) {
flushScheduler = null;
this.indexShard.translog().syncOnEachOperation(true);
} else {
flushScheduler = null;
}
this.waitForMappingUpdatePostRecovery = indexSettings.getAsTime("index.gateway.wait_for_mapping_update_post_recovery", TimeValue.timeValueSeconds(15));
}
/**
@ -198,39 +183,9 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl
@Override
public void close() {
FutureUtils.cancel(flushScheduler);
cancellableThreads.cancel("closed");
}
class Sync implements Runnable {
@Override
public void run() {
// don't re-schedule if its closed..., we are done
if (indexShard.state() == IndexShardState.CLOSED) {
return;
}
if (indexShard.state() == IndexShardState.STARTED && indexShard.translog().syncNeeded()) {
threadPool.executor(ThreadPool.Names.FLUSH).execute(new Runnable() {
@Override
public void run() {
try {
indexShard.translog().sync();
} catch (Exception e) {
if (indexShard.state() == IndexShardState.STARTED) {
logger.warn("failed to sync translog", e);
}
}
if (indexShard.state() != IndexShardState.CLOSED) {
flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
}
}
});
} else {
flushScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
}
}
}
@Override
public String toString() {
return "shard_gateway";

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.shard;
import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex;
@ -54,6 +53,7 @@ import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.MetaDataStateFormat;
@ -69,14 +69,7 @@ import org.elasticsearch.index.cache.query.ShardQueryCache;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.engine.CommitStats;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineClosedException;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.RefreshFailedEngineException;
import org.elasticsearch.index.engine.SegmentsStats;
import org.elasticsearch.index.engine.*;
import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.fielddata.ShardFieldData;
@ -85,12 +78,7 @@ import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.get.ShardGetService;
import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperAnalyzer;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.merge.policy.MergePolicyProvider;
@ -113,6 +101,7 @@ import org.elasticsearch.index.suggest.stats.SuggestStats;
import org.elasticsearch.index.termvectors.ShardTermVectorsService;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.index.translog.fs.FsTranslog;
import org.elasticsearch.index.warmer.ShardIndexWarmerService;
import org.elasticsearch.index.warmer.WarmerStats;
import org.elasticsearch.indices.IndicesLifecycle;
@ -145,7 +134,6 @@ public class IndexShard extends AbstractIndexShardComponent {
private final InternalIndicesLifecycle indicesLifecycle;
private final Store store;
private final MergeSchedulerProvider mergeScheduler;
private final Translog translog;
private final IndexAliasesService indexAliasesService;
private final ShardIndexingService indexingService;
private final ShardSearchService searchService;
@ -171,6 +159,7 @@ public class IndexShard extends AbstractIndexShardComponent {
private final SnapshotDeletionPolicy deletionPolicy;
private final SimilarityService similarityService;
private final MergePolicyProvider mergePolicyProvider;
private final BigArrays bigArrays;
private final EngineConfig engineConfig;
private TimeValue refreshInterval;
@ -205,27 +194,26 @@ public class IndexShard extends AbstractIndexShardComponent {
private final ShardPath path;
@Inject
public IndexShard(ShardId shardId, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog,
public IndexShard(ShardId shardId, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler,
ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService,
ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache,
@Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory,
ClusterService clusterService, NodeEnvironment nodeEnv, ShardPath path) {
ClusterService clusterService, NodeEnvironment nodeEnv, ShardPath path, BigArrays bigArrays) {
super(shardId, indexSettingsService.getSettings());
this.codecService = codecService;
this.warmer = warmer;
this.deletionPolicy = deletionPolicy;
this.similarityService = similarityService;
this.mergePolicyProvider = mergePolicyProvider;
this.bigArrays = bigArrays;
Preconditions.checkNotNull(store, "Store must be provided to the index shard");
Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard");
Preconditions.checkNotNull(translog, "Translog must be provided to the index shard");
this.engineFactory = factory;
this.indicesLifecycle = (InternalIndicesLifecycle) indicesLifecycle;
this.indexSettingsService = indexSettingsService;
this.store = store;
this.mergeScheduler = mergeScheduler;
this.translog = translog;
this.threadPool = threadPool;
this.mapperService = mapperService;
this.queryParserService = queryParserService;
@ -266,8 +254,13 @@ public class IndexShard extends AbstractIndexShardComponent {
return this.store;
}
/** returns true if this shard supports indexing (i.e., write) operations. */
public boolean canIndex() {
return true;
}
public Translog translog() {
return translog;
return engine().translog();
}
public ShardIndexingService indexingService() {
@ -662,7 +655,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}
public TranslogStats translogStats() {
return translog.stats();
return engine().translog().stats();
}
public SuggestStats suggestStats() {
@ -703,24 +696,19 @@ public class IndexShard extends AbstractIndexShardComponent {
logger.trace("optimize with {}", optimize);
}
engine().forceMerge(optimize.flush(), optimize.maxNumSegments(), optimize.onlyExpungeDeletes(),
optimize.upgrade(), optimize.upgradeOnlyAncientSegments());
optimize.upgrade(), optimize.upgradeOnlyAncientSegments());
}
public SnapshotIndexCommit snapshotIndex() throws EngineException {
public SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineException {
IndexShardState state = this.state; // one time volatile read
// we allow snapshot on closed index shard, since we want to do one after we close the shard and before we close the engine
if (state == IndexShardState.STARTED || state == IndexShardState.RELOCATED || state == IndexShardState.CLOSED) {
return engine().snapshotIndex();
return engine().snapshotIndex(flushFirst);
} else {
throw new IllegalIndexShardStateException(shardId, state, "snapshot is not allowed");
}
}
public void recover(Engine.RecoveryHandler recoveryHandler) throws EngineException {
verifyStarted();
engine().recover(recoveryHandler);
}
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);
@ -817,7 +805,6 @@ public class IndexShard extends AbstractIndexShardComponent {
if (Booleans.parseBoolean(checkIndexOnStartup, false)) {
checkIndex();
}
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
// we disable deletes since we allow for operations to be executed against the shard while recovering
// but we need to make sure we don't loose deletes until we are done recovering
@ -832,10 +819,10 @@ public class IndexShard extends AbstractIndexShardComponent {
* a remote peer.
*/
public void skipTranslogRecovery() {
assert engineUnsafe() == null : "engine was already created";
Map<String, Mapping> recoveredTypes = internalPerformTranslogRecovery(true);
assert recoveredTypes.isEmpty();
assert recoveryState.getTranslog().recoveredOperations() == 0;
assert engineUnsafe() == null : "engine was already created";
Map<String, Mapping> recoveredTypes = internalPerformTranslogRecovery(true);
assert recoveredTypes.isEmpty();
assert recoveryState.getTranslog().recoveredOperations() == 0;
}
/** called if recovery has to be restarted after network error / delay ** */
@ -869,8 +856,6 @@ public class IndexShard extends AbstractIndexShardComponent {
*/
public void finalizeRecovery() {
recoveryState().setStage(RecoveryState.Stage.FINALIZE);
// clear unreferenced files
translog.clearUnreferenced();
engine().refresh("recovery_finalization");
startScheduledTasksIfNeeded();
engineConfig.setEnableGcDeletes(true);
@ -977,7 +962,10 @@ public class IndexShard extends AbstractIndexShardComponent {
logger.debug("updating index_buffer_size from [{}] to [{}]", preValue, shardIndexingBufferSize);
}
}
translog().updateBuffer(shardTranslogBufferSize);
Engine engine = engineUnsafe();
if (engine != null) {
engine.translog().updateBuffer(shardTranslogBufferSize);
}
}
public void markAsInactive() {
@ -999,6 +987,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}
MetaDataStateFormat.deleteMetaState(shardPath().getDataPath());
}
public ShardPath shardPath() {
return path;
}
@ -1229,10 +1218,23 @@ public class IndexShard extends AbstractIndexShardComponent {
}
protected Engine newEngine(boolean skipTranslogRecovery, EngineConfig config) {
return engineFactory.newReadWriteEngine(config, skipTranslogRecovery);
final FsTranslog translog;
try {
translog = new FsTranslog(shardId, indexSettingsService, bigArrays, path, threadPool);
} catch (IOException e) {
throw new EngineCreationFailureException(shardId, "failed to create translog", e);
}
Engine engine = null;
try {
engine = engineFactory.newReadWriteEngine(config, translog, skipTranslogRecovery);
} finally {
if (engine == null) {
IOUtils.closeWhileHandlingException(translog);
}
}
return engine;
}
/**
* Returns <code>true</code> iff this shard allows primary promotion, otherwise <code>false</code>
*/
@ -1290,7 +1292,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}
};
return new EngineConfig(shardId,
threadPool, indexingService, indexSettingsService, warmer, store, deletionPolicy, translog, mergePolicyProvider, mergeScheduler,
threadPool, indexingService, indexSettingsService, warmer, store, deletionPolicy, mergePolicyProvider, mergeScheduler,
mapperAnalyzer, similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer);
}
}

View File

@ -46,7 +46,7 @@ public class IndexShardException extends IndexException {
@Override
public String toString() {
return (shardId == null ? "_na" : shardId) + getMessage();
return (shardId == null ? "_na" : shardId) + " " + getMessage();
}
@Override

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.translog.TranslogService;
import org.elasticsearch.index.warmer.ShardIndexWarmerService;
/**
@ -47,6 +48,9 @@ public class IndexShardModule extends AbstractModule {
this.settings = settings;
this.shardId = shardId;
this.primary = primary;
if (settings.get("index.translog.type") != null) {
throw new IllegalStateException("a custom translog type is no longer supported. got [" + settings.get("index.translog.type") + "]");
}
}
/** Return true if a shadow engine should be used */
@ -61,6 +65,7 @@ public class IndexShardModule extends AbstractModule {
bind(IndexShard.class).to(ShadowIndexShard.class).asEagerSingleton();
} else {
bind(IndexShard.class).asEagerSingleton();
bind(TranslogService.class).asEagerSingleton();
}
bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, ENGINE_PREFIX, ENGINE_SUFFIX));

View File

@ -22,6 +22,7 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.aliases.IndexAliasesService;
@ -56,6 +57,8 @@ import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
/**
* ShadowIndexShard extends {@link IndexShard} to add file synchronization
* from the primary when a flush happens. It also ensures that a replica being
@ -67,7 +70,7 @@ public final class ShadowIndexShard extends IndexShard {
@Inject
public ShadowIndexShard(ShardId shardId, IndexSettingsService indexSettingsService,
IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler,
Translog translog, ThreadPool threadPool, MapperService mapperService,
ThreadPool threadPool, MapperService mapperService,
IndexQueryParserService queryParserService, IndexCache indexCache,
IndexAliasesService indexAliasesService, ShardIndexingService indexingService,
ShardGetService getService, ShardSearchService searchService,
@ -78,14 +81,15 @@ public final class ShadowIndexShard extends IndexShard {
IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache,
ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer,
SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService,
MergePolicyProvider mergePolicyProvider, EngineFactory factory, ClusterService clusterService, NodeEnvironment nodeEnv, ShardPath path) {
MergePolicyProvider mergePolicyProvider, EngineFactory factory, ClusterService clusterService,
NodeEnvironment nodeEnv, ShardPath path, BigArrays bigArrays) throws IOException {
super(shardId, indexSettingsService, indicesLifecycle, store, mergeScheduler,
translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indexingService, getService, searchService, shardWarmerService, shardFilterCache,
shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService,
termVectorsService, indexFieldDataService, indexService, shardSuggestService,
shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService,
mergePolicyProvider, factory, clusterService, nodeEnv, path);
mergePolicyProvider, factory, clusterService, nodeEnv, path, bigArrays);
}
/**
@ -102,6 +106,11 @@ public final class ShadowIndexShard extends IndexShard {
super.updateRoutingEntry(newRouting, persistState);
}
@Override
public boolean canIndex() {
return false;
}
@Override
protected Engine newEngine(boolean skipInitialTranslogRecovery, EngineConfig config) {
assert this.shardRouting.primary() == false;
@ -112,4 +121,10 @@ public final class ShadowIndexShard extends IndexShard {
public boolean allowsPrimaryPromotion() {
return false;
}
@Override
@Nullable
public Translog translog() {
throw new UnsupportedOperationException("shadow shards don't have a translog");
}
}

View File

@ -80,7 +80,8 @@ public class IndexShardSnapshotAndRestoreService extends AbstractIndexShardCompo
}
try {
SnapshotIndexCommit snapshotIndexCommit = indexShard.snapshotIndex();
// we flush first to make sure we get the latest writes snapshotted
SnapshotIndexCommit snapshotIndexCommit = indexShard.snapshotIndex(true);
try {
indexShardRepository.snapshot(snapshotId, shardId, snapshotIndexCommit, snapshotStatus);
if (logger.isDebugEnabled()) {

View File

@ -23,9 +23,14 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.io.stream.*;
import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.NoopStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.*;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.channels.Channels;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
@ -65,12 +70,12 @@ public class ChecksummedTranslogStream implements TranslogStream {
Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte());
operation = TranslogStreams.newOperationFromType(type);
operation.readFrom(in);
verifyChecksum(in);
} catch (EOFException e) {
throw new TruncatedTranslogException("reached premature end of file, translog is truncated", e);
} catch (AssertionError|Exception e) {
throw new TranslogCorruptedException("translog corruption while reading from stream", e);
}
verifyChecksum(in);
return operation;
}
@ -103,6 +108,11 @@ public class ChecksummedTranslogStream implements TranslogStream {
// closing it will close the FileChannel
OutputStreamDataOutput out = new OutputStreamDataOutput(Channels.newOutputStream(channel));
CodecUtil.writeHeader(out, TranslogStreams.TRANSLOG_CODEC, VERSION);
return headerLength();
}
@Override
public int headerLength() {
return CodecUtil.headerLength(TranslogStreams.TRANSLOG_CODEC);
}

View File

@ -23,9 +23,6 @@ import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
@ -41,6 +38,10 @@ public class LegacyTranslogStream implements TranslogStream {
@Override
public Translog.Operation read(StreamInput in) throws IOException {
// read the opsize before an operation.
// Note that this was written & read out side of the stream when this class was used, but it makes things more consistent
// to read this here
in.readInt();
Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte());
Translog.Operation operation = TranslogStreams.newOperationFromType(type);
operation.readFrom(in);
@ -49,8 +50,7 @@ public class LegacyTranslogStream implements TranslogStream {
@Override
public void write(StreamOutput out, Translog.Operation op) throws IOException {
out.writeByte(op.opType().id());
op.writeTo(out);
throw new UnsupportedOperationException("LegacyTranslogStream is depracated. Use TranslogStreams.LATEST");
}
@Override
@ -59,6 +59,11 @@ public class LegacyTranslogStream implements TranslogStream {
return 0;
}
@Override
public int headerLength() {
return 0;
}
@Override
public StreamInput openInput(Path translogFile) throws IOException {
// nothing to do, legacy translogs have no header

View File

@ -37,17 +37,17 @@ import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShardComponent;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
/**
*
*/
public interface Translog extends IndexShardComponent, Closeable, Accountable {
public interface Translog extends IndexShardComponent {
static ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb");
@ -61,42 +61,21 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
long currentId();
/**
* Returns the number of operations in the transaction log.
* Returns the number of operations in the transaction files that aren't committed to lucene..
* Note: may return -1 if unknown
*/
int estimatedNumberOfOperations();
int totalOperations();
/**
* Returns the size in bytes of the translog.
* Returns the size in bytes of the translog files that aren't committed to lucene.
*/
long translogSizeInBytes();
long sizeInBytes();
/**
* Creates a new transaction log internally.
* <p/>
* <p>Can only be called by one thread.
* @param id the translog id for the new translog
* Creates a new transaction log file internally. That new file will be visible to all outstanding views.
* The id of the new translog file is returned.
*/
void newTranslog(long id) throws TranslogException, IOException;
/**
* Creates a new transient translog, where added ops will be added to the current one, and to
* it.
* <p/>
* <p>Can only be called by one thread.
*/
void newTransientTranslog(long id) throws TranslogException;
/**
* Swaps the transient translog to be the current one.
* <p/>
* <p>Can only be called by one thread.
*/
void makeTransientCurrent() throws IOException;
/**
* Reverts back to not have a transient translog.
*/
void revertTransient() throws IOException;
long newTranslog() throws TranslogException, IOException;
/**
* Adds a create operation to the transaction log.
@ -107,22 +86,15 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
/**
* Snapshots the current transaction log allowing to safely iterate over the snapshot.
* Snapshots are fixed in time and will not be updated with future operations.
*/
Snapshot snapshot() throws TranslogException;
Snapshot newSnapshot() throws TranslogException;
/**
* Snapshots the delta between the current state of the translog, and the state defined
* by the provided snapshot. If a new translog has been created after the provided snapshot
* has been take, will return a snapshot on the current trasnlog.
* Returns a view into the current translog that is guaranteed to retain all current operations
* while receiving future ones as well
*/
Snapshot snapshot(Snapshot snapshot);
/**
* Clears unreferenced transaction logs.
*
* @return the number of clean up files
*/
int clearUnreferenced();
View newView();
/**
* Sync's the translog.
@ -140,36 +112,19 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
*/
public Path location();
/**
* Returns the translog filename for the given id.
*/
String getFilename(long translogId);
/**
* return stats
*/
TranslogStats stats();
/**
* Returns the largest translog id present in all locations or <tt>-1</tt> if no translog is present.
* notifies the translog that translogId was committed as part of the commit data in lucene, together
* with all operations from previous translogs. This allows releasing all previous translogs.
*
* @throws FileNotFoundException if the given translog id can not be found.
*/
long findLargestPresentTranslogId() throws IOException;
void markCommitted(long translogId) throws FileNotFoundException;
/**
* Returns an OperationIterator to iterate over all translog entries in the given translog ID.
* @throws java.io.FileNotFoundException if the file for the translog ID can not be found
*/
OperationIterator openIterator(long translogId) throws IOException;
/**
* Iterator for translog operations.
*/
public static interface OperationIterator extends Releasable {
/**
* Returns the next operation in the translog or <code>null</code> if we reached the end of the stream.
*/
public Translog.Operation next() throws IOException;
}
static class Location implements Accountable {
@ -185,7 +140,7 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
@Override
public long ramBytesUsed() {
return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2*RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2 * RamUsageEstimator.NUM_BYTES_LONG + RamUsageEstimator.NUM_BYTES_INT;
}
@Override
@ -202,22 +157,7 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
/**
* A snapshot of the transaction log, allows to iterate over all the transaction log operations.
*/
static interface Snapshot extends OperationIterator {
/**
* The id of the translog the snapshot was taken with.
*/
long translogId();
/**
* Returns the current position in the translog stream
*/
long position();
/**
* Returns the internal length (*not* number of operations) of this snapshot.
*/
long length();
static interface Snapshot extends Releasable {
/**
* The total number of operations in the translog.
@ -225,14 +165,31 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
int estimatedTotalOperations();
/**
* Seek to the specified position in the translog stream
* Returns the next operation in the snapshot or <code>null</code> if we reached the end.
*/
void seekTo(long position);
public Translog.Operation next() throws IOException;
}
/** a view into the current translog that receives all operations from the moment created */
interface View extends Releasable {
/**
* The length in bytes of this stream.
* The total number of operations in the view.
*/
long lengthInBytes();
int totalOperations();
/**
* Returns the size in bytes of the files behind the view.
*/
long sizeInBytes();
/** create a snapshot from this view */
Snapshot snapshot();
/** this smallest translog id in this view */
long minTranslogId();
}
/**
@ -277,6 +234,7 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
long estimateSize();
Source getSource();
}
static class Source {
@ -435,6 +393,57 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeLong(ttl);
out.writeByte(versionType.getValue());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Create create = (Create) o;
if (timestamp != create.timestamp ||
ttl != create.ttl ||
version != create.version ||
id.equals(create.id) == false ||
type.equals(create.type) == false ||
source.equals(create.source) == false) {
return false;
}
if (routing != null ? !routing.equals(create.routing) : create.routing != null) {
return false;
}
if (parent != null ? !parent.equals(create.parent) : create.parent != null) {
return false;
}
return versionType == create.versionType;
}
@Override
public int hashCode() {
int result = id.hashCode();
result = 31 * result + type.hashCode();
result = 31 * result + source.hashCode();
result = 31 * result + (routing != null ? routing.hashCode() : 0);
result = 31 * result + (parent != null ? parent.hashCode() : 0);
result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
result = 31 * result + (int) (ttl ^ (ttl >>> 32));
result = 31 * result + (int) (version ^ (version >>> 32));
result = 31 * result + versionType.hashCode();
return result;
}
@Override
public String toString() {
return "Create{" +
"id='" + id + '\'' +
", type='" + type + '\'' +
'}';
}
}
static class Index implements Operation {
@ -581,6 +590,55 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeLong(ttl);
out.writeByte(versionType.getValue());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Index index = (Index) o;
if (version != index.version ||
timestamp != index.timestamp ||
ttl != index.ttl ||
id.equals(index.id) == false ||
type.equals(index.type) == false ||
versionType != index.versionType ||
source.equals(index.source) == false) {
return false;
}
if (routing != null ? !routing.equals(index.routing) : index.routing != null) {
return false;
}
return !(parent != null ? !parent.equals(index.parent) : index.parent != null);
}
@Override
public int hashCode() {
int result = id.hashCode();
result = 31 * result + type.hashCode();
result = 31 * result + (int) (version ^ (version >>> 32));
result = 31 * result + versionType.hashCode();
result = 31 * result + source.hashCode();
result = 31 * result + (routing != null ? routing.hashCode() : 0);
result = 31 * result + (parent != null ? parent.hashCode() : 0);
result = 31 * result + (int) (timestamp ^ (timestamp >>> 32));
result = 31 * result + (int) (ttl ^ (ttl >>> 32));
return result;
}
@Override
public String toString() {
return "Index{" +
"id='" + id + '\'' +
", type='" + type + '\'' +
'}';
}
}
static class Delete implements Operation {
@ -658,6 +716,37 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeLong(version);
out.writeByte(versionType.getValue());
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Delete delete = (Delete) o;
return version == delete.version &&
uid.equals(delete.uid) &&
versionType == delete.versionType;
}
@Override
public int hashCode() {
int result = uid.hashCode();
result = 31 * result + (int) (version ^ (version >>> 32));
result = 31 * result + versionType.hashCode();
return result;
}
@Override
public String toString() {
return "Delete{" +
"uid=" + uid +
'}';
}
}
/** @deprecated Delete-by-query is removed in 2.0, but we keep this so translog can replay on upgrade. */
@ -755,5 +844,40 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeVInt(0);
}
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
DeleteByQuery that = (DeleteByQuery) o;
if (!Arrays.equals(filteringAliases, that.filteringAliases)) {
return false;
}
if (!Arrays.equals(types, that.types)) {
return false;
}
return source.equals(that.source);
}
@Override
public int hashCode() {
int result = source.hashCode();
result = 31 * result + (filteringAliases != null ? Arrays.hashCode(filteringAliases) : 0);
result = 31 * result + Arrays.hashCode(types);
return result;
}
@Override
public String toString() {
return "DeleteByQuery{" +
"types=" + Arrays.toString(types) +
'}';
}
}
}

View File

@ -1,49 +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.elasticsearch.index.translog;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Scopes;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.translog.fs.FsTranslog;
/**
*
*/
public class TranslogModule extends AbstractModule {
public static class TranslogSettings {
public static final String TYPE = "index.translog.type";
}
private final Settings settings;
public TranslogModule(Settings settings) {
this.settings = settings;
}
@Override
protected void configure() {
bind(Translog.class)
.to(settings.getAsClass(TranslogSettings.TYPE, FsTranslog.class))
.in(Scopes.SINGLETON);
bind(TranslogService.class).asEagerSingleton();
}
}

View File

@ -29,11 +29,7 @@ import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
@ -57,7 +53,7 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
private final ThreadPool threadPool;
private final IndexSettingsService indexSettingsService;
private final IndexShard indexShard;
private final Translog translog;
private volatile Translog translog;
private volatile TimeValue interval;
private volatile int flushThresholdOperations;
@ -69,12 +65,11 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
private final ApplySettings applySettings = new ApplySettings();
@Inject
public TranslogService(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, ThreadPool threadPool, IndexShard indexShard, Translog translog) {
public TranslogService(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, ThreadPool threadPool, IndexShard indexShard) {
super(shardId, indexSettings);
this.threadPool = threadPool;
this.indexSettingsService = indexSettingsService;
this.indexShard = indexShard;
this.translog = translog;
this.flushThresholdOperations = indexSettings.getAsInt(INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS, indexSettings.getAsInt("index.translog.flush_threshold", Integer.MAX_VALUE));
this.flushThresholdSize = indexSettings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB));
this.flushThresholdPeriod = indexSettings.getAsTime(INDEX_TRANSLOG_FLUSH_THRESHOLD_PERIOD, TimeValue.timeValueMinutes(30));
@ -88,7 +83,6 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
indexSettingsService.addListener(applySettings);
}
@Override
public void close() {
indexSettingsService.removeListener(applySettings);
@ -148,12 +142,12 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
return;
}
if (indexShard.state() == IndexShardState.CREATED) {
if (indexShard.translog() == null) {
reschedule();
return;
}
int currentNumberOfOperations = translog.estimatedNumberOfOperations();
int currentNumberOfOperations = translog.totalOperations();
if (currentNumberOfOperations == 0) {
reschedule();
return;
@ -168,7 +162,7 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
}
if (flushThresholdSize.bytes() > 0) {
long sizeInBytes = translog.translogSizeInBytes();
long sizeInBytes = translog.sizeInBytes();
if (sizeInBytes > flushThresholdSize.bytes()) {
logger.trace("flushing translog, size [{}], breached [{}]", new ByteSizeValue(sizeInBytes), flushThresholdSize);
asyncFlushAndReschedule();

View File

@ -34,14 +34,13 @@ import java.io.IOException;
public class TranslogStats implements ToXContent, Streamable {
private long translogSizeInBytes = 0;
private int estimatedNumberOfOperations = 0;
private int estimatedNumberOfOperations = -1;
public TranslogStats() {
}
public TranslogStats(int estimatedNumberOfOperations, long translogSizeInBytes) {
assert estimatedNumberOfOperations >= 0 : "estimatedNumberOfOperations must be >=0, got [" + estimatedNumberOfOperations + "]";
assert translogSizeInBytes >= 0 : "translogSizeInBytes must be >=0, got [" + translogSizeInBytes + "]";
assert translogSizeInBytes >= 0 : "translogSizeInBytes must be >= 0, got [" + translogSizeInBytes + "]";
this.estimatedNumberOfOperations = estimatedNumberOfOperations;
this.translogSizeInBytes = translogSizeInBytes;
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index.translog;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.File;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Path;
@ -49,6 +48,11 @@ public interface TranslogStream {
*/
public int writeHeader(FileChannel channel) throws IOException;
/**
* returns the site of the header in bytes
*/
public int headerLength();
/**
* Seek past the header, if any header is present
*/

View File

@ -28,8 +28,6 @@ import org.elasticsearch.common.io.stream.InputStreamStreamInput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
@ -91,7 +89,6 @@ public class TranslogStreams {
* has zero length, returns the latest version. If the header does not
* exist, assumes Version 0 of the translog file format.
* <p/>
* The caller is responsible for closing the TranslogStream.
*
* @throws IOException
*/

View File

@ -21,34 +21,21 @@ package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.TranslogStream;
import org.elasticsearch.index.translog.TranslogStreams;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogException;
import org.elasticsearch.index.translog.TranslogStream;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.file.Path;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.nio.ByteBuffer;
/**
*/
public class BufferingFsTranslogFile implements FsTranslogFile {
private final long id;
private final ShardId shardId;
private final ChannelReference channelReference;
private final TranslogStream translogStream;
private final int headerSize;
private final ReadWriteLock rwl = new ReentrantReadWriteLock();
private final AtomicBoolean closed = new AtomicBoolean();
public final class BufferingFsTranslogFile extends FsTranslogFile {
private volatile int operationCounter;
private volatile long lastPosition;
private volatile long lastWrittenPosition;
@ -59,36 +46,28 @@ public class BufferingFsTranslogFile implements FsTranslogFile {
private WrapperOutputStream bufferOs = new WrapperOutputStream();
public BufferingFsTranslogFile(ShardId shardId, long id, ChannelReference channelReference, int bufferSize) throws IOException {
this.shardId = shardId;
this.id = id;
this.channelReference = channelReference;
super(shardId, id, channelReference);
this.buffer = new byte[bufferSize];
this.translogStream = TranslogStreams.translogStreamFor(this.channelReference.file());
this.headerSize = this.translogStream.writeHeader(channelReference.channel());
final TranslogStream stream = this.channelReference.stream();
int headerSize = stream.writeHeader(channelReference.channel());
this.lastPosition += headerSize;
this.lastWrittenPosition += headerSize;
this.lastSyncPosition += headerSize;
}
@Override
public long id() {
return this.id;
}
@Override
public int estimatedNumberOfOperations() {
public int totalOperations() {
return operationCounter;
}
@Override
public long translogSizeInBytes() {
public long sizeInBytes() {
return lastWrittenPosition;
}
@Override
public Translog.Location add(BytesReference data) throws IOException {
rwl.writeLock().lock();
try {
try (ReleasableLock lock = writeLock.acquire()) {
operationCounter++;
long position = lastPosition;
if (data.length() >= buffer.length) {
@ -106,64 +85,49 @@ public class BufferingFsTranslogFile implements FsTranslogFile {
data.writeTo(bufferOs);
lastPosition += data.length();
return new Translog.Location(id, position, data.length());
} finally {
rwl.writeLock().unlock();
}
}
private void flushBuffer() throws IOException {
assert (((ReentrantReadWriteLock.WriteLock) rwl.writeLock()).isHeldByCurrentThread());
assert writeLock.isHeldByCurrentThread();
if (bufferCount > 0) {
// we use the channel to write, since on windows, writing to the RAF might not be reflected
// when reading through the channel
Channels.writeToChannel(buffer, 0, bufferCount, channelReference.channel());
lastWrittenPosition += bufferCount;
bufferCount = 0;
}
}
@Override
public byte[] read(Translog.Location location) throws IOException {
rwl.readLock().lock();
try {
if (location.translogLocation >= lastWrittenPosition) {
byte[] data = new byte[location.size];
System.arraycopy(buffer, (int) (location.translogLocation - lastWrittenPosition), data, 0, location.size);
return data;
protected void readBytes(ByteBuffer targetBuffer, long position) throws IOException {
try (ReleasableLock lock = readLock.acquire()) {
if (position >= lastWrittenPosition) {
System.arraycopy(buffer, (int) (position - lastWrittenPosition),
targetBuffer.array(), targetBuffer.position(), targetBuffer.limit());
return;
}
} finally {
rwl.readLock().unlock();
}
// we don't have to have a read lock here because we only write ahead to the file, so all writes has been complete
// for the requested location.
return Channels.readFromFileChannel(channelReference.channel(), location.translogLocation, location.size);
Channels.readFromFileChannelWithEofException(channelReference.channel(), position, targetBuffer);
}
@Override
public FsChannelSnapshot snapshot() throws TranslogException {
public FsChannelImmutableReader immutableReader() throws TranslogException {
if (channelReference.tryIncRef()) {
boolean success = false;
try {
rwl.writeLock().lock();
try {
flushBuffer();
FsChannelSnapshot snapshot = new FsChannelSnapshot(this.id, channelReference, lastWrittenPosition, operationCounter);
snapshot.seekTo(this.headerSize);
success = true;
return snapshot;
} catch (Exception e) {
throw new TranslogException(shardId, "exception while creating snapshot", e);
} finally {
rwl.writeLock().unlock();
}
try (ReleasableLock lock = writeLock.acquire()) {
flushBuffer();
FsChannelImmutableReader reader = new FsChannelImmutableReader(this.id, channelReference, lastWrittenPosition, operationCounter);
channelReference.incRef(); // for new reader
return reader;
} catch (Exception e) {
throw new TranslogException(shardId, "exception while creating an immutable reader", e);
} finally {
if (!success) {
channelReference.decRef();
}
channelReference.decRef();
}
} else {
throw new TranslogException(shardId, "can't increment channel [" + channelReference + "] ref count");
}
return null;
}
@Override
@ -171,34 +135,24 @@ public class BufferingFsTranslogFile implements FsTranslogFile {
return lastPosition != lastSyncPosition;
}
@Override
public TranslogStream getStream() {
return this.translogStream;
}
@Override
public void sync() throws IOException {
if (!syncNeeded()) {
return;
}
rwl.writeLock().lock();
try {
try (ReleasableLock lock = writeLock.acquire()) {
flushBuffer();
lastSyncPosition = lastPosition;
} finally {
rwl.writeLock().unlock();
}
channelReference.channel().force(false);
}
@Override
public void close() throws IOException {
if (closed.compareAndSet(false, true)) {
try {
sync();
} finally {
channelReference.decRef();
}
protected void doClose() throws IOException {
try {
sync();
} finally {
super.doClose();
}
}
@ -207,21 +161,18 @@ public class BufferingFsTranslogFile implements FsTranslogFile {
if (!(other instanceof BufferingFsTranslogFile)) {
return;
}
rwl.writeLock().lock();
try {
flushBuffer();
this.buffer = ((BufferingFsTranslogFile) other).buffer;
} catch (IOException e) {
throw new TranslogException(shardId, "failed to flush", e);
} finally {
rwl.writeLock().unlock();
try (ReleasableLock lock = writeLock.acquire()) {
try {
flushBuffer();
this.buffer = ((BufferingFsTranslogFile) other).buffer;
} catch (IOException e) {
throw new TranslogException(shardId, "failed to flush", e);
}
}
}
@Override
public void updateBufferSize(int bufferSize) {
rwl.writeLock().lock();
try {
try (ReleasableLock lock = writeLock.acquire()) {
if (this.buffer.length == bufferSize) {
return;
}
@ -229,21 +180,9 @@ public class BufferingFsTranslogFile implements FsTranslogFile {
this.buffer = new byte[bufferSize];
} catch (IOException e) {
throw new TranslogException(shardId, "failed to flush", e);
} finally {
rwl.writeLock().unlock();
}
}
@Override
public Path getPath() {
return channelReference.file();
}
@Override
public boolean closed() {
return this.closed.get();
}
class WrapperOutputStream extends OutputStream {
@Override
@ -258,4 +197,5 @@ public class BufferingFsTranslogFile implements FsTranslogFile {
bufferCount += len;
}
}
}

View File

@ -19,33 +19,40 @@
package org.elasticsearch.index.translog.fs;
import com.google.common.collect.Iterables;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.translog.TranslogStream;
import org.elasticsearch.index.translog.TranslogStreams;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.OpenOption;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.Map;
/**
*
*/
abstract class ChannelReference extends AbstractRefCounted {
class ChannelReference extends AbstractRefCounted {
private final Path file;
private final FileChannel channel;
private final TranslogStream stream;
public ChannelReference(Path file, OpenOption... openOptions) throws IOException {
super(file.toString());
this.file = file;
this.channel = FileChannel.open(file, openOptions);
try {
this.stream = TranslogStreams.translogStreamFor(file);
final Map<FsChannelReader, RuntimeException> existing = openedFiles.put(file().toString(), ConcurrentCollections.<FsChannelReader, RuntimeException>newConcurrentMap());
assert existing == null || existing.size() == 0 : "a channel for the file[" + file + "] was previously opened from " + ExceptionsHelper.stackTrace(Iterables.getFirst(existing.values(), null));
} catch (Throwable t) {
IOUtils.closeWhileHandlingException(channel);
throw t;
}
}
public Path file() {
@ -56,8 +63,54 @@ abstract class ChannelReference extends AbstractRefCounted {
return this.channel;
}
public TranslogStream stream() {
return this.stream;
}
/**
* called to add this owner to the list of reference holders (used for leakage detection).
* also asserts that there is no double "attachment"
*/
boolean assertAttach(FsChannelReader owner) {
Map<FsChannelReader, RuntimeException> ownerMap = openedFiles.get(file().toString());
Throwable previous = ownerMap.put(owner, new RuntimeException(file.toString() + " attached", null));
assert previous == null : "double attachment by the same owner";
return true;
}
/** removes an owner to the least of list holders (used for leakage detection).
* also asserts that this owner did attach before.
*/
boolean assertDetach(FsChannelReader owner) {
Map<FsChannelReader, RuntimeException> ownerMap = openedFiles.get(file().toString());
Throwable previous = ownerMap.remove(owner);
assert previous != null : "reader detaches, but was never attached";
return true;
}
@Override
public String toString() {
return "channel: file [" + file + "], ref count [" + refCount() + "]";
}
@Override
protected void closeInternal() {
IOUtils.closeWhileHandlingException(channel);
assert openedFiles.remove(file().toString()) != null;
}
// per file, which objects refer to it and a throwable of the allocation code
static final Map<String, Map<FsChannelReader, RuntimeException>> openedFiles;
static {
boolean assertsEnabled = false;
assert (assertsEnabled = true);
if (assertsEnabled) {
openedFiles = ConcurrentCollections.newConcurrentMap();
} else {
openedFiles = null;
}
}
}

View File

@ -0,0 +1,86 @@
/*
* 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.translog.fs;
import org.elasticsearch.common.io.Channels;
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
/**
* a channel reader which is fixed in length
*/
public final class FsChannelImmutableReader extends FsChannelReader {
private final int totalOperations;
private final long length;
/**
* Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point
* at the end of the last operation in this snapshot.
*/
public FsChannelImmutableReader(long id, ChannelReference channelReference, long length, int totalOperations) {
super(id, channelReference);
this.length = length;
this.totalOperations = totalOperations;
}
public FsChannelImmutableReader clone() {
if (channelReference.tryIncRef()) {
try {
FsChannelImmutableReader reader = new FsChannelImmutableReader(id, channelReference, length, totalOperations);
channelReference.incRef(); // for the new object
return reader;
} finally {
channelReference.decRef();
}
} else {
throw new IllegalStateException("can't increment translog [" + id + "] channel ref count");
}
}
public long sizeInBytes() {
return length;
}
public int totalOperations() {
return totalOperations;
}
/**
* reads an operation at the given position into the given buffer.
*/
protected void readBytes(ByteBuffer buffer, long position) throws IOException {
if (position >= length) {
throw new EOFException("read requested past EOF. pos [" + position + "] end: [" + length + "]");
}
if (position < firstPosition()) {
throw new IOException("read requested before position of first ops. pos [" + position + "] first op on: [" + firstPosition() + "]");
}
Channels.readFromFileChannelWithEofException(channel, position, buffer);
}
@Override
public FsChannelSnapshot newSnapshot() {
return new FsChannelSnapshot(clone());
}
}

View File

@ -0,0 +1,133 @@
/*
* 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.translog.fs;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.index.translog.Translog;
import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* A base class for all classes that allows reading ops from translog files
*/
public abstract class FsChannelReader implements Closeable, Comparable<FsChannelReader> {
public static final int UNKNOWN_OP_COUNT = -1;
protected final long id;
protected final ChannelReference channelReference;
protected final FileChannel channel;
protected final AtomicBoolean closed = new AtomicBoolean(false);
public FsChannelReader(long id, ChannelReference channelReference) {
this.id = id;
this.channelReference = channelReference;
this.channel = channelReference.channel();
assert channelReference.assertAttach(this);
}
public long translogId() {
return this.id;
}
abstract public long sizeInBytes();
/** the position the first operation is written at */
public long firstPosition() {
return channelReference.stream().headerLength();
}
abstract public int totalOperations();
public Translog.Operation read(Translog.Location location) throws IOException {
assert location.translogId == id : "read location's translog id [" + location.translogId + "] is not [" + id + "]";
ByteBuffer buffer = ByteBuffer.allocate(location.size);
return read(buffer, location.translogLocation, location.size);
}
/** read the size of the op (i.e., number of bytes, including the op size) written at the given position */
public int readSize(ByteBuffer reusableBuffer, long position) {
// read op size from disk
assert reusableBuffer.capacity() >= 4 : "reusable buffer must have capacity >=4 when reading opSize. got [" + reusableBuffer.capacity() + "]";
try {
reusableBuffer.clear();
reusableBuffer.limit(4);
readBytes(reusableBuffer, position);
reusableBuffer.flip();
// Add an extra 4 to account for the operation size integer itself
return reusableBuffer.getInt() + 4;
} catch (IOException e) {
throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.channelReference.file(), e);
}
}
/**
* reads an operation at the given position and returns it. The buffer length is equal to the number
* of bytes reads.
*/
public Translog.Operation read(ByteBuffer reusableBuffer, long position, int opSize) throws IOException {
final ByteBuffer buffer;
if (reusableBuffer.capacity() >= opSize) {
buffer = reusableBuffer;
} else {
buffer = ByteBuffer.allocate(opSize);
}
buffer.clear();
buffer.limit(opSize);
readBytes(buffer, position);
BytesArray bytesArray = new BytesArray(buffer.array(), 0, buffer.limit());
return channelReference.stream().read(bytesArray.streamInput());
}
/**
* reads bytes at position into the given buffer, filling it.
*/
abstract protected void readBytes(ByteBuffer buffer, long position) throws IOException;
/** create snapshot for this channel */
abstract public FsChannelSnapshot newSnapshot();
@Override
public void close() throws IOException {
if (closed.compareAndSet(false, true)) {
doClose();
}
}
protected void doClose() throws IOException {
assert channelReference.assertDetach(this);
channelReference.decRef();
}
@Override
public String toString() {
return "translog [" + id + "][" + channelReference.file() + "]";
}
@Override
public int compareTo(FsChannelReader o) {
return Long.compare(translogId(), o.translogId());
}
}

View File

@ -16,138 +16,60 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.translog.fs;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.io.stream.BytesStreamInput;
import org.elasticsearch.index.translog.TranslogStreams;
import org.elasticsearch.index.translog.Translog;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.concurrent.atomic.AtomicBoolean;
/**
*
* an implementation of {@link org.elasticsearch.index.translog.Translog.Snapshot}, wrapping
* a {@link FsChannelReader}. This class is NOT thread-safe.
*/
public class FsChannelSnapshot implements Translog.Snapshot {
public class FsChannelSnapshot implements Closeable {
private final long id;
protected final FsChannelReader reader;
protected final AtomicBoolean closed = new AtomicBoolean(false);
private final int totalOperations;
// we use an atomic long to allow passing it by reference :(
protected long position;
private final ChannelReference channelReference;
private final FileChannel channel;
private final long length;
private Translog.Operation lastOperationRead = null;
private long position = 0;
private ByteBuffer cacheBuffer;
private AtomicBoolean closed = new AtomicBoolean(false);
/**
* Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point
* at the end of the last operation in this snapshot.
*/
public FsChannelSnapshot(long id, ChannelReference channelReference, long length, int totalOperations) throws FileNotFoundException {
this.id = id;
this.channelReference = channelReference;
this.channel = channelReference.channel();
this.length = length;
this.totalOperations = totalOperations;
public FsChannelSnapshot(FsChannelReader reader) {
this.reader = reader;
this.position = reader.firstPosition();
}
@Override
public long translogId() {
return this.id;
return reader.translogId();
}
@Override
public long position() {
return this.position;
}
@Override
public long length() {
return this.length;
}
@Override
public int estimatedTotalOperations() {
return this.totalOperations;
return reader.totalOperations();
}
@Override
public long lengthInBytes() {
return length - position;
}
@Override
public Translog.Operation next() {
try {
if (position >= length) {
return null;
}
if (cacheBuffer == null) {
cacheBuffer = ByteBuffer.allocate(1024);
}
cacheBuffer.limit(4);
int bytesRead = Channels.readFromFileChannel(channel, position, cacheBuffer);
if (bytesRead < 0) {
// the snapshot is acquired under a write lock. we should never
// read beyond the EOF, must be an abrupt EOF
throw new EOFException("read past EOF. pos [" + position + "] length: [" + cacheBuffer.limit() + "] end: [" + channel.size() + "]");
}
assert bytesRead == 4;
cacheBuffer.flip();
// Add an extra 4 to account for the operation size integer itself
int opSize = cacheBuffer.getInt() + 4;
if ((position + opSize) > length) {
// the snapshot is acquired under a write lock. we should never
// read beyond the EOF, must be an abrupt EOF
throw new EOFException("opSize of [" + opSize + "] pointed beyond EOF. position [" + position + "] length [" + length + "]");
}
if (cacheBuffer.capacity() < opSize) {
cacheBuffer = ByteBuffer.allocate(opSize);
}
cacheBuffer.clear();
cacheBuffer.limit(opSize);
bytesRead = Channels.readFromFileChannel(channel, position, cacheBuffer);
if (bytesRead < 0) {
// the snapshot is acquired under a write lock. we should never
// read beyond the EOF, must be an abrupt EOF
throw new EOFException("tried to read past EOF. opSize [" + opSize + "] position [" + position + "] length [" + length + "]");
}
cacheBuffer.flip();
position += opSize;
BytesArray bytesArray = new BytesArray(cacheBuffer.array(), 0, opSize);
return TranslogStreams.readTranslogOperation(new BytesStreamInput(bytesArray.copyBytesArray()));
} catch (IOException e) {
throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.channelReference.file(), e);
public Translog.Operation next(ByteBuffer reusableBuffer) throws IOException {
if (position >= reader.sizeInBytes()) {
return null;
}
}
@Override
public void seekTo(long position) {
this.position = position;
final int opSize = reader.readSize(reusableBuffer, position);
Translog.Operation op = reader.read(reusableBuffer, position, opSize);
position += opSize;
return op;
}
@Override
public void close() {
if (closed.compareAndSet(false, true)) {
channelReference.decRef();
try {
IOUtils.close(reader);
} catch (IOException e) {
throw new ElasticsearchException("failed to close translogs", e);
}
}
}
}

View File

@ -19,37 +19,39 @@
package org.elasticsearch.index.translog.fs;
import org.apache.lucene.util.Accountable;
import com.google.common.collect.Iterables;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.ReleasablePagedBytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.stream.BytesStreamInput;
import org.elasticsearch.common.io.stream.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.translog.*;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogException;
import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.index.translog.TranslogStreams;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.EOFException;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.channels.ClosedChannelException;
import java.nio.file.*;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.*;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -59,11 +61,21 @@ import java.util.regex.Pattern;
/**
*
*/
public class FsTranslog extends AbstractIndexShardComponent implements Translog {
public class FsTranslog extends AbstractIndexShardComponent implements Translog, Closeable {
public static final String INDEX_TRANSLOG_FS_TYPE = "index.translog.fs.type";
private static final String TRANSLOG_FILE_PREFIX = "translog-";
public static final String INDEX_TRANSLOG_BUFFER_SIZE = "index.translog.fs.buffer_size";
public static final String INDEX_TRANSLOG_SYNC_INTERVAL = "index.translog.sync_interval";
public static final String TRANSLOG_FILE_PREFIX = "translog-";
private static final Pattern PARSE_ID_PATTERN = Pattern.compile(TRANSLOG_FILE_PREFIX + "(\\d+).*");
private final TimeValue syncInterval;
private volatile ScheduledFuture<?> syncScheduler;
// this is a concurrent set and is not protected by any of the locks. The main reason
// is that is being accessed by two separate classes (additions & reading are done by FsTranslog, remove by FsView when closed)
private final Set<FsView> outstandingViews = ConcurrentCollections.newConcurrentSet();
class ApplySettings implements IndexSettingsService.Listener {
@Override
@ -78,75 +90,142 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
private final IndexSettingsService indexSettingsService;
private final BigArrays bigArrays;
private final ThreadPool threadPool;
protected final ReleasableLock readLock;
protected final ReleasableLock writeLock;
private final ReadWriteLock rwl = new ReentrantReadWriteLock();
private final Path location;
private volatile FsTranslogFile current;
private volatile FsTranslogFile trans;
// protected by the write lock
private long idGenerator = 1;
private FsTranslogFile current;
// ordered by age
private final List<FsChannelImmutableReader> uncommittedTranslogs = new ArrayList<>();
private long lastCommittedTranslogId = -1; // -1 is safe as it will not cause an translog deletion.
private FsTranslogFile.Type type;
private boolean syncOnEachOperation = false;
private volatile int bufferSize;
private volatile int transientBufferSize;
private final ApplySettings applySettings = new ApplySettings();
@Inject
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService,
BigArrays bigArrays, ShardPath shardPath) throws IOException {
super(shardId, indexSettings);
this.indexSettingsService = indexSettingsService;
this.bigArrays = bigArrays;
this.location = shardPath.resolveTranslog();
Files.createDirectories(location);
this.type = FsTranslogFile.Type.fromString(indexSettings.get("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name()));
this.bufferSize = (int) indexSettings.getAsBytesSize("index.translog.fs.buffer_size", ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController...
this.transientBufferSize = (int) indexSettings.getAsBytesSize("index.translog.fs.transient_buffer_size", ByteSizeValue.parseBytesSizeValue("8k")).bytes();
indexSettingsService.addListener(applySettings);
private final AtomicBoolean closed = new AtomicBoolean();
public FsTranslog(ShardId shardId, IndexSettingsService indexSettingsService,
BigArrays bigArrays, ShardPath shardPath, ThreadPool threadPool) throws IOException {
this(shardId, indexSettingsService.getSettings(), indexSettingsService, bigArrays, shardPath.resolveTranslog(), threadPool);
}
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, Path location) throws IOException {
super(shardId, indexSettings);
this.indexSettingsService = null;
this.location = location;
Files.createDirectories(location);
this.bigArrays = BigArrays.NON_RECYCLING_INSTANCE;
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings,
BigArrays bigArrays, Path location) throws IOException {
this(shardId, indexSettings, null, bigArrays, location, null);
}
this.type = FsTranslogFile.Type.fromString(indexSettings.get("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name()));
this.bufferSize = (int) indexSettings.getAsBytesSize("index.translog.fs.buffer_size", ByteSizeValue.parseBytesSizeValue("64k")).bytes();
private FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, @Nullable IndexSettingsService indexSettingsService,
BigArrays bigArrays, Path location, @Nullable ThreadPool threadPool) throws IOException {
super(shardId, indexSettings);
ReadWriteLock rwl = new ReentrantReadWriteLock();
readLock = new ReleasableLock(rwl.readLock());
writeLock = new ReleasableLock(rwl.writeLock());
this.indexSettingsService = indexSettingsService;
this.bigArrays = bigArrays;
this.location = location;
Files.createDirectories(this.location);
this.threadPool = threadPool;
this.type = FsTranslogFile.Type.fromString(indexSettings.get(INDEX_TRANSLOG_FS_TYPE, FsTranslogFile.Type.BUFFERED.name()));
this.bufferSize = (int) indexSettings.getAsBytesSize(INDEX_TRANSLOG_BUFFER_SIZE, ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController...
syncInterval = indexSettings.getAsTime(INDEX_TRANSLOG_SYNC_INTERVAL, TimeValue.timeValueSeconds(5));
if (syncInterval.millis() > 0 && threadPool != null) {
syncOnEachOperation(false);
syncScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, new Sync());
} else if (syncInterval.millis() == 0) {
syncOnEachOperation(true);
}
if (indexSettingsService != null) {
indexSettingsService.addListener(applySettings);
}
recoverFromFiles();
// now that we know which files are there, create a new current one.
current = createTranslogFile(null);
}
/** recover all translog files found on disk */
private void recoverFromFiles() throws IOException {
boolean success = false;
ArrayList<FsChannelImmutableReader> foundTranslogs = new ArrayList<>();
try (ReleasableLock lock = writeLock.acquire()) {
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path file : stream) {
final long id = parseIdFromFileName(file);
if (id < 0) {
throw new TranslogException(shardId, "failed to parse id from file name matching pattern " + file);
}
idGenerator = Math.max(idGenerator, id + 1);
final ChannelReference raf = new InternalChannelReference(id, location.resolve(getFilename(id)), StandardOpenOption.READ);
foundTranslogs.add(new FsChannelImmutableReader(id, raf, raf.channel().size(), FsChannelReader.UNKNOWN_OP_COUNT));
logger.debug("found local translog with id [{}]", id);
}
}
CollectionUtil.timSort(foundTranslogs);
uncommittedTranslogs.addAll(foundTranslogs);
success = true;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(foundTranslogs);
}
}
}
/* extracts the translog id from a file name. returns -1 upon failure */
public static long parseIdFromFileName(Path translogFile) {
final String fileName = translogFile.getFileName().toString();
final Matcher matcher = PARSE_ID_PATTERN.matcher(fileName);
if (matcher.matches()) {
try {
return Long.parseLong(matcher.group(1));
} catch (NumberFormatException e) {
throw new ElasticsearchException("number formatting issue in a file that passed PARSE_ID_PATTERN: " + fileName + "]", e);
}
}
return -1;
}
@Override
public void updateBuffer(ByteSizeValue bufferSize) {
this.bufferSize = bufferSize.bytesAsInt();
rwl.writeLock().lock();
try {
FsTranslogFile current1 = this.current;
if (current1 != null) {
current1.updateBufferSize(this.bufferSize);
}
current1 = this.trans;
if (current1 != null) {
current1.updateBufferSize(this.bufferSize);
}
} finally {
rwl.writeLock().unlock();
try (ReleasableLock lock = writeLock.acquire()) {
current.updateBufferSize(this.bufferSize);
}
}
boolean isOpen() {
return closed.get() == false;
}
@Override
public void close() throws IOException {
if (indexSettingsService != null) {
indexSettingsService.removeListener(applySettings);
}
rwl.writeLock().lock();
try {
IOUtils.close(this.trans, this.current);
} finally {
rwl.writeLock().unlock();
if (closed.compareAndSet(false, true)) {
if (indexSettingsService != null) {
indexSettingsService.removeListener(applySettings);
}
try (ReleasableLock lock = writeLock.acquire()) {
try {
IOUtils.close(this.current);
} finally {
IOUtils.close(uncommittedTranslogs);
}
} finally {
FutureUtils.cancel(syncScheduler);
logger.debug("translog closed");
}
}
}
@ -157,137 +236,173 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
@Override
public long currentId() {
FsTranslogFile current1 = this.current;
if (current1 == null) {
return -1;
try (ReleasableLock lock = readLock.acquire()) {
return current.translogId();
}
return current1.id();
}
@Override
public int estimatedNumberOfOperations() {
FsTranslogFile current1 = this.current;
if (current1 == null) {
return 0;
public int totalOperations() {
int ops = 0;
try (ReleasableLock lock = readLock.acquire()) {
ops += current.totalOperations();
for (FsChannelReader translog : uncommittedTranslogs) {
int tops = translog.totalOperations();
if (tops == FsChannelReader.UNKNOWN_OP_COUNT) {
return FsChannelReader.UNKNOWN_OP_COUNT;
}
ops += tops;
}
}
return current1.estimatedNumberOfOperations();
return ops;
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public Collection<Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public long translogSizeInBytes() {
FsTranslogFile current1 = this.current;
if (current1 == null) {
return 0;
public long sizeInBytes() {
long size = 0;
try (ReleasableLock lock = readLock.acquire()) {
size += current.sizeInBytes();
for (FsChannelReader translog : uncommittedTranslogs) {
size += translog.sizeInBytes();
}
}
return current1.translogSizeInBytes();
return size;
}
@Override
public int clearUnreferenced() {
rwl.writeLock().lock();
int deleted = 0;
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path file : stream) {
if (isReferencedTranslogFile(file) == false) {
try {
logger.trace("delete unreferenced translog file: " + file);
Files.delete(file);
deleted++;
} catch (Exception ex) {
logger.debug("failed to delete " + file, ex);
/** asserts that all files were closed, if not throws an {@link AssertionError} with details regarding the open files */
public static void assertAllClosed() {
if (ChannelReference.openedFiles == null) {
return;
}
RuntimeException exampleAllocator = null;
ArrayList<String> files = new ArrayList<>();
for (Map.Entry<String, Map<FsChannelReader, RuntimeException>> file : ChannelReference.openedFiles.entrySet()) {
files.add(file.getKey());
for (RuntimeException allocator : file.getValue().values()) {
if (exampleAllocator == null) {
exampleAllocator = new RuntimeException(file.getKey() + " is still open", allocator);
} else {
exampleAllocator.addSuppressed(allocator);
}
}
}
if (exampleAllocator != null) {
throw new AssertionError("some translog files are still open [" + Strings.collectionToCommaDelimitedString(files) + "]", exampleAllocator);
}
}
/** force close an open reference captured in assertion code * */
public static void assertForceCloseAllReferences() {
if (ChannelReference.openedFiles == null) {
return;
}
for (Map.Entry<String, Map<FsChannelReader, RuntimeException>> file : ChannelReference.openedFiles.entrySet()) {
IOUtils.closeWhileHandlingException(file.getValue().keySet());
}
}
/** gets a list of unreferenced files (only works if assertions are enabled, returns an empty array otherwise) */
public String[] getUnreferenced() throws IOException {
if (ChannelReference.openedFiles == null) {
return Strings.EMPTY_ARRAY; // not supported
}
ArrayList<String> result = new ArrayList<>();
try (ReleasableLock lock = writeLock.acquire()) {
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path file : stream) {
final long id = parseIdFromFileName(file);
if (id < 0) {
logger.trace("failed to extract translog id from [{}]", file);
} else if (ChannelReference.openedFiles.containsKey(file.toString()) == false) {
result.add(file.toString());
}
}
}
} catch (IOException ex) {
logger.debug("failed to clear unreferenced files ", ex);
} finally {
rwl.writeLock().unlock();
}
return deleted;
return result.toArray(Strings.EMPTY_ARRAY);
}
@Override
public void newTranslog(long id) throws TranslogException, IOException {
rwl.writeLock().lock();
try {
FsTranslogFile newFile;
try {
newFile = type.create(shardId, id, new InternalChannelReference(location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), bufferSize);
} catch (IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
public void markCommitted(final long translogId) throws FileNotFoundException {
try (ReleasableLock lock = writeLock.acquire()) {
logger.trace("updating translogs on commit of [{}]", translogId);
if (translogId < lastCommittedTranslogId) {
throw new IllegalArgumentException("committed translog id can only go up (current ["
+ lastCommittedTranslogId + "], got [" + translogId + "]");
}
boolean found = false;
if (current.translogId() == translogId) {
found = true;
} else {
if (translogId > current.translogId()) {
throw new IllegalArgumentException("committed translog id must be lower or equal to current id (current ["
+ current.translogId() + "], got [" + translogId + "]");
}
}
if (found == false) {
// try to find it in uncommittedTranslogs
for (FsChannelImmutableReader translog : uncommittedTranslogs) {
if (translog.translogId() == translogId) {
found = true;
break;
}
}
}
if (found == false) {
ArrayList<Long> currentIds = new ArrayList<>();
for (FsChannelReader translog : Iterables.concat(uncommittedTranslogs, Collections.singletonList(current))) {
currentIds.add(translog.translogId());
}
throw new FileNotFoundException("committed translog id can not be found (current ["
+ Strings.collectionToCommaDelimitedString(currentIds) + "], got [" + translogId + "]");
}
lastCommittedTranslogId = translogId;
while (uncommittedTranslogs.isEmpty() == false && uncommittedTranslogs.get(0).translogId() < translogId) {
FsChannelReader old = uncommittedTranslogs.remove(0);
logger.trace("removed [{}] from uncommitted translog list", old.translogId());
try {
old.close();
} catch (IOException e) {
logger.error("failed to closed old translog [{}] (committed id [{}])", e, old, translogId);
}
}
FsTranslogFile old = current;
current = newFile;
IOUtils.close(old);
} finally {
rwl.writeLock().unlock();
}
}
@Override
public void newTransientTranslog(long id) throws TranslogException {
rwl.writeLock().lock();
public long newTranslog() throws TranslogException, IOException {
try (ReleasableLock lock = writeLock.acquire()) {
final FsTranslogFile old = current;
final FsTranslogFile newFile = createTranslogFile(old);
current = newFile;
FsChannelImmutableReader reader = old.immutableReader();
uncommittedTranslogs.add(reader);
// notify all outstanding views of the new translog (no views are created now as
// we hold a write lock).
for (FsView view : outstandingViews) {
view.onNewTranslog(old.immutableReader(), current.reader());
}
IOUtils.close(old);
logger.trace("current translog set to [{}]", current.translogId());
return current.translogId();
}
}
protected FsTranslogFile createTranslogFile(@Nullable FsTranslogFile reuse) throws IOException {
FsTranslogFile newFile;
long size = Long.MAX_VALUE;
try {
assert this.trans == null;
this.trans = type.create(shardId, id, new InternalChannelReference(location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), transientBufferSize);
long id = idGenerator++;
newFile = type.create(shardId, id, new InternalChannelReference(id, location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), bufferSize);
} catch (IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
} finally {
rwl.writeLock().unlock();
}
if (reuse != null) {
newFile.reuse(reuse);
}
return newFile;
}
@Override
public void makeTransientCurrent() throws IOException {
FsTranslogFile old;
rwl.writeLock().lock();
try {
assert this.trans != null;
old = current;
this.current = this.trans;
this.trans = null;
} finally {
rwl.writeLock().unlock();
}
old.close();
current.reuse(old);
}
@Override
public void revertTransient() throws IOException {
rwl.writeLock().lock();
try {
final FsTranslogFile toClose = this.trans;
this.trans = null;
IOUtils.close(toClose);
} finally {
rwl.writeLock().unlock();
}
}
/**
* Returns the translog that should be read for the specified location. If
* the transient or current translog does not match, returns null
*/
private FsTranslogFile translogForLocation(Location location) {
if (trans != null && trans.id() == location.translogId) {
return this.trans;
}
if (current.id() == location.translogId) {
return this.current;
}
return null;
}
/**
* Read the Operation object from the given location, returns null if the
@ -295,108 +410,112 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
*/
@Override
public Translog.Operation read(Location location) {
rwl.readLock().lock();
try {
FsTranslogFile translog = translogForLocation(location);
if (translog != null) {
byte[] data = translog.read(location);
try (BytesStreamInput in = new BytesStreamInput(data)) {
// Return the Operation using the current version of the
// stream based on which translog is being read
return translog.getStream().read(in);
try (ReleasableLock lock = readLock.acquire()) {
FsChannelReader reader = null;
if (current.translogId() == location.translogId) {
reader = current;
} else {
for (FsChannelReader translog : uncommittedTranslogs) {
if (translog.translogId() == location.translogId) {
reader = translog;
break;
}
}
}
return null;
return reader == null ? null : reader.read(location);
} catch (IOException e) {
throw new ElasticsearchException("failed to read source from translog location " + location, e);
} finally {
rwl.readLock().unlock();
}
}
@Override
public Location add(Operation operation) throws TranslogException {
rwl.readLock().lock();
boolean released = false;
ReleasableBytesStreamOutput out = null;
ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays);
try {
out = new ReleasableBytesStreamOutput(bigArrays);
TranslogStreams.writeTranslogOperation(out, operation);
ReleasablePagedBytesReference bytes = out.bytes();
Location location = current.add(bytes);
if (syncOnEachOperation) {
current.sync();
}
assert new BytesArray(current.read(location)).equals(bytes);
FsTranslogFile trans = this.trans;
if (trans != null) {
try {
location = trans.add(bytes);
} catch (ClosedChannelException e) {
// ignore
try (ReleasableLock lock = readLock.acquire()) {
Location location = current.add(bytes);
if (syncOnEachOperation) {
current.sync();
}
assert current.assertBytesAtLocation(location, bytes);
return location;
}
Releasables.close(bytes);
released = true;
return location;
} catch (Throwable e) {
throw new TranslogException(shardId, "Failed to write operation [" + operation + "]", e);
} finally {
rwl.readLock().unlock();
if (!released && out != null) {
Releasables.close(out.bytes());
Releasables.close(out.bytes());
}
}
@Override
public Snapshot newSnapshot() {
try (ReleasableLock lock = readLock.acquire()) {
// leave one place for current.
final FsChannelReader[] readers = uncommittedTranslogs.toArray(new FsChannelReader[uncommittedTranslogs.size() + 1]);
readers[readers.length - 1] = current;
return createdSnapshot(readers);
}
}
private Snapshot createdSnapshot(FsChannelReader... translogs) {
ArrayList<FsChannelSnapshot> channelSnapshots = new ArrayList<>();
boolean success = false;
try {
for (FsChannelReader translog : translogs) {
channelSnapshots.add(translog.newSnapshot());
}
Snapshot snapshot = new FsTranslogSnapshot(channelSnapshots, logger);
success = true;
return snapshot;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(channelSnapshots);
}
}
}
@Override
public FsChannelSnapshot snapshot() throws TranslogException {
while (true) {
FsTranslogFile current = this.current;
FsChannelSnapshot snapshot = current.snapshot();
if (snapshot != null) {
return snapshot;
public Translog.View newView() {
// we need to acquire the read lock to make sure new translog is created
// and will be missed by the view we're making
try (ReleasableLock lock = readLock.acquire()) {
ArrayList<FsChannelReader> translogs = new ArrayList<>();
try {
for (FsChannelImmutableReader translog : uncommittedTranslogs) {
translogs.add(translog.clone());
}
translogs.add(current.reader());
FsView view = new FsView(translogs);
// this is safe as we know that no new translog is being made at the moment
// (we hold a read lock) and the view will be notified of any future one
outstandingViews.add(view);
translogs.clear();
return view;
} finally {
// close if anything happend and we didn't reach the clear
IOUtils.closeWhileHandlingException(translogs);
}
if (current.closed() && this.current == current) {
// check if we are closed and if we are still current - then this translog is closed and we can exit
throw new TranslogException(shardId, "current translog is already closed");
}
Thread.yield();
}
}
@Override
public Snapshot snapshot(Snapshot snapshot) {
FsChannelSnapshot snap = snapshot();
if (snap.translogId() == snapshot.translogId()) {
snap.seekTo(snapshot.position());
}
return snap;
}
@Override
public void sync() throws IOException {
FsTranslogFile current1 = this.current;
if (current1 == null) {
return;
}
try {
current1.sync();
} catch (IOException e) {
// if we switches translots (!=), then this failure is not relevant
// we are working on a new translog
if (this.current == current1) {
throw e;
try (ReleasableLock lock = readLock.acquire()) {
if (closed.get()) {
return;
}
current.sync();
}
}
@Override
public boolean syncNeeded() {
FsTranslogFile current1 = this.current;
return current1 != null && current1.syncNeeded();
try (ReleasableLock lock = readLock.acquire()) {
return current.syncNeeded();
}
}
@Override
@ -409,138 +528,169 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
}
}
@Override
public String getFilename(long translogId) {
/** package private for testing */
String getFilename(long translogId) {
return TRANSLOG_FILE_PREFIX + translogId;
}
@Override
public TranslogStats stats() {
FsTranslogFile current = this.current;
if (current == null) {
return new TranslogStats(0, 0);
}
return new TranslogStats(current.estimatedNumberOfOperations(), current.translogSizeInBytes());
}
@Override
public long findLargestPresentTranslogId() throws IOException {
rwl.readLock().lock();
try {
long maxId = this.currentId();
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path translogFile : stream) {
try {
final String fileName = translogFile.getFileName().toString();
final Matcher matcher = PARSE_ID_PATTERN.matcher(fileName);
if (matcher.matches()) {
maxId = Math.max(maxId, Long.parseLong(matcher.group(1)));
}
} catch (NumberFormatException ex) {
logger.warn("Couldn't parse translog id from file " + translogFile + " skipping");
}
}
}
return maxId;
} finally {
rwl.readLock().unlock();
// acquire lock to make the two numbers roughly consistent (no file change half way)
try (ReleasableLock lock = readLock.acquire()) {
return new TranslogStats(totalOperations(), sizeInBytes());
}
}
@Override
public OperationIterator openIterator(long translogId) throws IOException {
final String translogName = getFilename(translogId);
Path recoveringTranslogFile = null;
logger.trace("try open translog file {} locations {}", translogName, location);
// we have to support .recovering since it's a leftover from previous version but might still be on the filesystem
// we used to rename the foo into foo.recovering since foo was reused / overwritten but we fixed that in 2.0
for (Path recoveryFiles : FileSystemUtils.files(location, translogName + "{.recovering,}")) {
logger.trace("translog file found in {}", recoveryFiles);
recoveringTranslogFile = recoveryFiles;
}
final boolean translogFileExists = recoveringTranslogFile != null && Files.exists(recoveringTranslogFile);
if (translogFileExists) {
if (logger.isTraceEnabled()) {
logger.trace("opening iterator for translog file: {} length: {}", recoveringTranslogFile, Files.size(recoveringTranslogFile));
}
final TranslogStream translogStream = TranslogStreams.translogStreamFor(recoveringTranslogFile);
return new OperationIteratorImpl(logger, translogStream, translogStream.openInput(recoveringTranslogFile));
}
logger.trace("translog file NOT found in {}", location);
throw new FileNotFoundException("no translog file found for id: " + translogId);
}
private boolean isReferencedTranslogFile(Path file) {
final FsTranslogFile theCurrent = this.current;
final FsTranslogFile theTrans = this.trans;
return (theCurrent != null && theCurrent.getPath().equals(file)) ||
(theTrans != null && theTrans.getPath().equals(file));
private boolean isReferencedTranslogId(long translogId) {
return translogId >= lastCommittedTranslogId;
}
private final class InternalChannelReference extends ChannelReference {
final long translogId;
public InternalChannelReference(Path file, OpenOption... openOptions) throws IOException {
public InternalChannelReference(long translogId, Path file, OpenOption... openOptions) throws IOException {
super(file, openOptions);
this.translogId = translogId;
}
@Override
protected void closeInternal() {
super.closeInternal();
rwl.writeLock().lock();
try {
if (isReferencedTranslogFile(file()) == false) {
try (ReleasableLock lock = writeLock.acquire()) {
if (isReferencedTranslogId(translogId) == false) {
// if the given path is not the current we can safely delete the file since all references are released
logger.trace("delete translog file - not referenced and not current anymore {}", file());
IOUtils.deleteFilesIgnoringExceptions(file());
}
} finally {
rwl.writeLock().unlock();
}
}
}
/**
* Iterator for translog operations.
* a view into the translog, capturing all translog file at the moment of creation
* and updated with any future translog.
*/
private static class OperationIteratorImpl implements org.elasticsearch.index.translog.Translog.OperationIterator {
class FsView implements View {
private final TranslogStream translogStream;
private final StreamInput input;
private final ESLogger logger;
boolean closed;
// last in this list is always FsTranslog.current
final List<FsChannelReader> orderedTranslogs;
OperationIteratorImpl(ESLogger logger, TranslogStream translogStream, StreamInput input) {
this.translogStream = translogStream;
this.input = input;
this.logger = logger;
FsView(List<FsChannelReader> orderedTranslogs) {
assert orderedTranslogs.isEmpty() == false;
// clone so we can safely mutate..
this.orderedTranslogs = new ArrayList<>(orderedTranslogs);
}
/**
* Returns the next operation in the translog or <code>null</code> if we reached the end of the stream.
* Called by the parent class when ever the current translog changes
*
* @param oldCurrent a new read only reader for the old current (should replace the previous reference)
* @param newCurrent a reader into the new current.
*/
public Translog.Operation next() throws IOException {
try {
if (translogStream instanceof LegacyTranslogStream) {
input.readInt(); // ignored opSize
synchronized void onNewTranslog(FsChannelReader oldCurrent, FsChannelReader newCurrent) throws IOException {
// even though the close method removes this view from outstandingViews, there is no synchronisation in place
// between that operation and an ongoing addition of a new translog, already having an iterator.
// As such, this method can be called despite of the fact that we are closed. We need to check and ignore.
if (closed) {
// we have to close the new references created for as as we will not hold them
IOUtils.close(oldCurrent, newCurrent);
return;
}
orderedTranslogs.remove(orderedTranslogs.size() - 1).close();
orderedTranslogs.add(oldCurrent);
orderedTranslogs.add(newCurrent);
}
@Override
public synchronized long minTranslogId() {
ensureOpen();
return orderedTranslogs.get(0).translogId();
}
@Override
public synchronized int totalOperations() {
int ops = 0;
for (FsChannelReader translog : orderedTranslogs) {
int tops = translog.totalOperations();
if (tops == FsChannelReader.UNKNOWN_OP_COUNT) {
return -1;
}
return translogStream.read(input);
} catch (TruncatedTranslogException | EOFException e) {
// ignore, not properly written the last op
logger.trace("ignoring translog EOF exception, the last operation was not properly written", e);
return null;
} catch (IOException e) {
// ignore, not properly written last op
logger.trace("ignoring translog IO exception, the last operation was not properly written", e);
return null;
ops += tops;
}
return ops;
}
@Override
public synchronized long sizeInBytes() {
long size = 0;
for (FsChannelReader translog : orderedTranslogs) {
size += translog.sizeInBytes();
}
return size;
}
public synchronized Snapshot snapshot() {
ensureOpen();
return createdSnapshot(orderedTranslogs.toArray(new FsChannelReader[orderedTranslogs.size()]));
}
void ensureOpen() {
if (closed) {
throw new ElasticsearchException("View is already closed");
}
}
@Override
public void close() {
List<FsChannelReader> toClose = new ArrayList<>();
try {
input.close();
} catch (IOException ex) {
throw new ElasticsearchException("failed to close stream input", ex);
synchronized (this) {
if (closed == false) {
logger.trace("closing view starting at translog [{}]", minTranslogId());
closed = true;
outstandingViews.remove(this);
toClose.addAll(orderedTranslogs);
orderedTranslogs.clear();
}
}
} finally {
try {
// Close out of lock to prevent deadlocks between channel close which checks for
// references in InternalChannelReference.closeInternal (waiting on a read lock)
// and other FsTranslog#newTranslog calling FsView.onNewTranslog (while having a write lock)
IOUtils.close(toClose);
} catch (Exception e) {
throw new ElasticsearchException("failed to close view", e);
}
}
}
}
class Sync implements Runnable {
@Override
public void run() {
// don't re-schedule if its closed..., we are done
if (closed.get()) {
return;
}
if (syncNeeded()) {
threadPool.executor(ThreadPool.Names.FLUSH).execute(new Runnable() {
@Override
public void run() {
try {
sync();
} catch (Exception e) {
logger.warn("failed to sync translog", e);
}
if (closed.get() == false) {
syncScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
}
}
});
} else {
syncScheduler = threadPool.schedule(syncInterval, ThreadPool.Names.SAME, Sync.this);
}
}
}

View File

@ -19,17 +19,32 @@
package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogException;
import org.elasticsearch.index.translog.TranslogStream;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.ByteBuffer;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
public abstract class FsTranslogFile extends FsChannelReader {
protected final ShardId shardId;
protected final ReleasableLock readLock;
protected final ReleasableLock writeLock;
public FsTranslogFile(ShardId shardId, long id, ChannelReference channelReference) {
super(id, channelReference);
this.shardId = shardId;
ReadWriteLock rwl = new ReentrantReadWriteLock();
readLock = new ReleasableLock(rwl.readLock());
writeLock = new ReleasableLock(rwl.writeLock());
}
public interface FsTranslogFile extends Closeable {
public static enum Type {
@ -58,29 +73,83 @@ public interface FsTranslogFile extends Closeable {
}
}
long id();
int estimatedNumberOfOperations();
/** add the given bytes to the translog and return the location they were written at */
public abstract Translog.Location add(BytesReference data) throws IOException;
long translogSizeInBytes();
/** reuse resources from another translog file, which is guaranteed not to be used anymore */
public abstract void reuse(FsTranslogFile other) throws TranslogException;
Translog.Location add(BytesReference data) throws IOException;
/** change the size of the internal buffer if relevant */
public abstract void updateBufferSize(int bufferSize) throws TranslogException;
byte[] read(Translog.Location location) throws IOException;
/** write all buffered ops to disk and fsync file */
public abstract void sync() throws IOException;
FsChannelSnapshot snapshot() throws TranslogException;
/** returns true if there are buffered ops */
public abstract boolean syncNeeded();
void reuse(FsTranslogFile other) throws TranslogException;
@Override
public FsChannelSnapshot newSnapshot() {
return new FsChannelSnapshot(immutableReader());
}
void updateBufferSize(int bufferSize) throws TranslogException;
/**
* returns a new reader that follows the current writes (most importantly allows making
* repeated snapshots that includes new content)
*/
public FsChannelReader reader() {
channelReference.incRef();
boolean success = false;
try {
FsChannelReader reader = new InnerReader(this.id, channelReference);
success = true;
return reader;
} finally {
if (!success) {
channelReference.decRef();
}
}
}
void sync() throws IOException;
boolean syncNeeded();
/** returns a new immutable reader which only exposes the current written operation * */
abstract public FsChannelImmutableReader immutableReader();
TranslogStream getStream();
boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException {
ByteBuffer buffer = ByteBuffer.allocate(location.size);
readBytes(buffer, location.translogLocation);
return new BytesArray(buffer.array()).equals(expectedBytes);
}
public Path getPath();
/**
* this class is used when one wants a reference to this file which exposes all recently written operation.
* as such it needs access to the internals of the current reader
*/
final class InnerReader extends FsChannelReader {
public boolean closed();
public InnerReader(long id, ChannelReference channelReference) {
super(id, channelReference);
}
@Override
public long sizeInBytes() {
return FsTranslogFile.this.sizeInBytes();
}
@Override
public int totalOperations() {
return FsTranslogFile.this.totalOperations();
}
@Override
protected void readBytes(ByteBuffer buffer, long position) throws IOException {
FsTranslogFile.this.readBytes(buffer, position);
}
@Override
public FsChannelSnapshot newSnapshot() {
return FsTranslogFile.this.newSnapshot();
}
}
}

View File

@ -0,0 +1,108 @@
/*
* 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.translog.fs;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TruncatedTranslogException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
public class FsTranslogSnapshot implements Translog.Snapshot {
private final List<FsChannelSnapshot> orderedTranslogs;
private final ESLogger logger;
private final ByteBuffer cacheBuffer;
private AtomicBoolean closed = new AtomicBoolean(false);
private final int estimatedTotalOperations;
private int currentTranslog;
/**
* Create a snapshot of translog file channel. The length parameter should be consistent with totalOperations and point
* at the end of the last operation in this snapshot.
*/
public FsTranslogSnapshot(List<FsChannelSnapshot> orderedTranslogs, ESLogger logger) {
this.orderedTranslogs = orderedTranslogs;
this.logger = logger;
int ops = 0;
for (FsChannelSnapshot translog : orderedTranslogs) {
final int tops = translog.estimatedTotalOperations();
if (tops < 0) {
ops = FsChannelReader.UNKNOWN_OP_COUNT;
break;
}
ops += tops;
}
estimatedTotalOperations = ops;
cacheBuffer = ByteBuffer.allocate(1024);
currentTranslog = 0;
}
@Override
public int estimatedTotalOperations() {
return estimatedTotalOperations;
}
@Override
public Translog.Operation next() throws IOException {
ensureOpen();
for (; currentTranslog < orderedTranslogs.size(); currentTranslog++) {
final FsChannelSnapshot current = orderedTranslogs.get(currentTranslog);
Translog.Operation op = null;
try {
op = current.next(cacheBuffer);
} catch (TruncatedTranslogException e) {
// file is empty or header has been half-written and should be ignored
logger.trace("ignoring truncation exception, the translog [{}] is either empty or half-written", e, current.translogId());
}
if (op != null) {
return op;
}
}
return null;
}
protected void ensureOpen() {
if (closed.get()) {
throw new AlreadyClosedException("snapshot already closed");
}
}
@Override
public void close() throws ElasticsearchException {
if (closed.compareAndSet(false, true)) {
try {
IOUtils.close(orderedTranslogs);
} catch (IOException e) {
throw new ElasticsearchException("failed to close channel snapshots", e);
} finally {
orderedTranslogs.clear();
}
}
}
}

View File

@ -19,128 +19,82 @@
package org.elasticsearch.index.translog.fs;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.TranslogStream;
import org.elasticsearch.index.translog.TranslogStreams;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogException;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.nio.ByteBuffer;
public class SimpleFsTranslogFile implements FsTranslogFile {
private final long id;
private final ShardId shardId;
private final ChannelReference channelReference;
private final AtomicBoolean closed = new AtomicBoolean();
private final ReadWriteLock rwl = new ReentrantReadWriteLock();
private final TranslogStream translogStream;
private final int headerSize;
public final class SimpleFsTranslogFile extends FsTranslogFile {
private volatile int operationCounter = 0;
private volatile long lastPosition = 0;
private volatile long lastWrittenPosition = 0;
private volatile long lastSyncPosition = 0;
public SimpleFsTranslogFile(ShardId shardId, long id, ChannelReference channelReference) throws IOException {
this.shardId = shardId;
this.id = id;
this.channelReference = channelReference;
this.translogStream = TranslogStreams.translogStreamFor(this.channelReference.file());
this.headerSize = this.translogStream.writeHeader(channelReference.channel());
super(shardId, id, channelReference);
int headerSize = this.channelReference.stream().writeHeader(channelReference.channel());
this.lastPosition += headerSize;
this.lastWrittenPosition += headerSize;
this.lastSyncPosition += headerSize;
}
@Override
public long id() {
return this.id;
}
@Override
public int estimatedNumberOfOperations() {
public int totalOperations() {
return operationCounter;
}
@Override
public long translogSizeInBytes() {
public long sizeInBytes() {
return lastWrittenPosition;
}
@Override
public Translog.Location add(BytesReference data) throws IOException {
rwl.writeLock().lock();
try {
try (ReleasableLock lock = writeLock.acquire()) {
long position = lastPosition;
data.writeTo(channelReference.channel());
lastPosition = lastPosition + data.length();
lastWrittenPosition = lastWrittenPosition + data.length();
operationCounter = operationCounter + 1;
return new Translog.Location(id, position, data.length());
} finally {
rwl.writeLock().unlock();
}
}
@Override
public byte[] read(Translog.Location location) throws IOException {
rwl.readLock().lock();
protected void readBytes(ByteBuffer buffer, long position) throws IOException {
try (ReleasableLock lock = readLock.acquire()) {
Channels.readFromFileChannelWithEofException(channelReference.channel(), position, buffer);
}
}
@Override
public void doClose() throws IOException {
try {
return Channels.readFromFileChannel(channelReference.channel(), location.translogLocation, location.size);
sync();
} finally {
rwl.readLock().unlock();
super.doClose();
}
}
@Override
public void close() throws IOException {
if (closed.compareAndSet(false, true)) {
try {
sync();
public FsChannelImmutableReader immutableReader() throws TranslogException {
if (channelReference.tryIncRef()) {
try (ReleasableLock lock = writeLock.acquire()) {
FsChannelImmutableReader reader = new FsChannelImmutableReader(this.id, channelReference, lastWrittenPosition, operationCounter);
channelReference.incRef(); // for the new object
return reader;
} finally {
channelReference.decRef();
}
} else {
throw new TranslogException(shardId, "can't increment channel [" + channelReference + "] channel ref count");
}
}
/**
* Returns a snapshot on this file, <tt>null</tt> if it failed to snapshot.
*/
@Override
public FsChannelSnapshot snapshot() throws TranslogException {
if (channelReference.tryIncRef()) {
boolean success = false;
try {
rwl.writeLock().lock();
try {
FsChannelSnapshot snapshot = new FsChannelSnapshot(this.id, channelReference, lastWrittenPosition, operationCounter);
snapshot.seekTo(this.headerSize);
success = true;
return snapshot;
} finally {
rwl.writeLock().unlock();
}
} catch (FileNotFoundException e) {
throw new TranslogException(shardId, "failed to create snapshot", e);
} finally {
if (!success) {
channelReference.decRef();
}
}
}
return null;
}
@Override
@ -148,28 +102,15 @@ public class SimpleFsTranslogFile implements FsTranslogFile {
return lastWrittenPosition != lastSyncPosition;
}
@Override
public TranslogStream getStream() {
return this.translogStream;
}
@Override
public Path getPath() {
return channelReference.file();
}
@Override
public void sync() throws IOException {
// check if we really need to sync here...
if (!syncNeeded()) {
return;
}
rwl.writeLock().lock();
try {
try (ReleasableLock lock = writeLock.acquire()) {
lastSyncPosition = lastWrittenPosition;
channelReference.channel().force(false);
} finally {
rwl.writeLock().unlock();
}
}
@ -182,10 +123,4 @@ public class SimpleFsTranslogFile implements FsTranslogFile {
public void updateBufferSize(int bufferSize) throws TranslogException {
// nothing to do here...
}
@Override
public boolean closed() {
return this.closed.get();
}
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.indices.memory;
import com.google.common.collect.Lists;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -191,9 +190,20 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
continue;
}
if (indexShard.canIndex() == false) {
// not relevant for memory related issues.
continue;
}
final Translog translog;
try {
translog = indexShard.translog();
} catch (EngineClosedException e) {
// not ready yet to be checked for in activity
continue;
}
final long time = threadPool.estimatedTimeInMillis();
Translog translog = indexShard.translog();
ShardIndexingStatus status = shardsIndicesStatus.get(indexShard.shardId());
if (status == null) {
status = new ShardIndexingStatus();
@ -201,7 +211,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
changes.add(ShardStatusChangeType.ADDED);
}
// check if it is deemed to be inactive (sam translogId and numberOfOperations over a long period of time)
if (status.translogId == translog.currentId() && translog.estimatedNumberOfOperations() == 0) {
if (status.translogId == translog.currentId() && translog.totalOperations() == 0) {
if (status.time == -1) { // first time
status.time = time;
}
@ -225,7 +235,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
status.time = -1;
}
status.translogId = translog.currentId();
status.translogNumberOfOperations = translog.estimatedNumberOfOperations();
status.translogNumberOfOperations = translog.totalOperations();
if (status.activeIndexing) {
activeShards++;

View File

@ -46,9 +46,6 @@ class RecoveryResponse extends TransportResponse {
int phase2Operations;
long phase2Time;
int phase3Operations;
long phase3Time;
RecoveryResponse() {
}
@ -84,8 +81,6 @@ class RecoveryResponse extends TransportResponse {
startTime = in.readVLong();
phase2Operations = in.readVInt();
phase2Time = in.readVLong();
phase3Operations = in.readVInt();
phase3Time = in.readVLong();
}
@Override
@ -116,7 +111,5 @@ class RecoveryResponse extends TransportResponse {
out.writeVLong(startTime);
out.writeVInt(phase2Operations);
out.writeVLong(phase2Time);
out.writeVInt(phase3Operations);
out.writeVLong(phase3Time);
}
}

View File

@ -122,11 +122,10 @@ public class RecoverySource extends AbstractComponent {
}
ongoingRecoveries.add(shard, handler);
try {
shard.recover(handler);
return handler.recoverToTarget();
} finally {
ongoingRecoveries.remove(shard, handler);
}
return handler.getResponse();
}
class StartRecoveryTransportRequestHandler implements TransportRequestHandler<StartRecoveryRequest> {

View File

@ -40,6 +40,7 @@ import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.compress.CompressorFactory;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
@ -49,7 +50,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.RecoveryEngineException;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShard;
@ -76,7 +77,7 @@ import java.util.concurrent.atomic.AtomicReference;
* everything relating to copying the segment files as well as sending translog
* operations across the wire once the segments have been copied.
*/
public class RecoverySourceHandler implements Engine.RecoveryHandler {
public class RecoverySourceHandler {
protected final ESLogger logger;
// Shard that is going to be recovered (the "source")
@ -126,27 +127,48 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
this.response = new RecoveryResponse();
}
/**
* @return the {@link RecoveryResponse} after the recovery has completed all three phases
*/
public RecoveryResponse getResponse() {
return this.response;
/** performs the recovery from the local engine to the target */
public RecoveryResponse recoverToTarget() {
try (Translog.View translogView = shard.translog().newView()) {
logger.trace("captured translog id [{}] for recovery", translogView.minTranslogId());
final SnapshotIndexCommit phase1Snapshot;
try {
phase1Snapshot = shard.snapshotIndex(false);
} catch (Throwable e) {
Releasables.closeWhileHandlingException(translogView);
throw new RecoveryEngineException(shard.shardId(), 1, "Snapshot failed", e);
}
try {
phase1(phase1Snapshot, translogView);
} catch (Throwable e) {
throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e);
} finally {
Releasables.closeWhileHandlingException(phase1Snapshot);
}
logger.trace("snapshot translog for recovery. current size is [{}]", translogView.totalOperations());
try (Translog.Snapshot phase2Snapshot = translogView.snapshot()) {
phase2(phase2Snapshot);
} catch (Throwable e) {
throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e);
}
finalizeRecovery();
}
return response;
}
/**
* Perform phase1 of the recovery operations. Once this {@link SnapshotIndexCommit}
* snapshot has been performed no commit operations (files being fsync'd)
* are effectively allowed on this index until all recovery phases are done
*
* <p/>
* Phase1 examines the segment files on the target node and copies over the
* segments that are missing. Only segments that have the same size and
* checksum can be reused
*
* {@code InternalEngine#recover} is responsible for snapshotting the index
* and releasing the snapshot once all 3 phases of recovery are complete
*/
@Override
public void phase1(final SnapshotIndexCommit snapshot) {
public void phase1(final SnapshotIndexCommit snapshot, final Translog.View translogView) {
cancellableThreads.checkForCancel();
// Total size of segment files that are recovered
long totalSize = 0;
@ -202,7 +224,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
public void run() throws InterruptedException {
RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(),
response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes,
shard.translog().estimatedNumberOfOperations());
translogView.totalOperations());
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest,
TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()),
EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
@ -298,7 +320,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
readCount += toRead;
final boolean lastChunk = readCount == len;
final RecoveryFileChunkRequest fileChunkRequest = new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position,
content, lastChunk, shard.translog().estimatedNumberOfOperations(), throttleTimeInNanos);
content, lastChunk, translogView.totalOperations(), throttleTimeInNanos);
cancellableThreads.execute(new Interruptable() {
@Override
public void run() throws InterruptedException {
@ -347,6 +369,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
});
if (corruptedEngine.get() != null) {
shard.engine().failEngine("recovery", corruptedEngine.get());
throw corruptedEngine.get();
} else {
ExceptionsHelper.rethrowAndSuppress(exceptions);
@ -365,7 +388,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
// are deleted
try {
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES,
new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, shard.translog().estimatedNumberOfOperations()),
new RecoveryCleanFilesRequest(request.recoveryId(), shard.shardId(), recoverySourceMetadata, translogView.totalOperations()),
TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()),
EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
} catch (RemoteTransportException remoteException) {
@ -406,7 +429,9 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
}
}
});
stopWatch.stop();
prepareTargetForTranslog(translogView);
logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime());
response.phase1Time = stopWatch.totalTime().millis();
} catch (Throwable e) {
@ -416,25 +441,10 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
}
}
/**
* Perform phase2 of the recovery process
*
* Phase2 takes a snapshot of the current translog *without* acquiring the
* write lock (however, the translog snapshot is a point-in-time view of
* the translog). It then sends each translog operation to the target node
* so it can be replayed into the new shard.
*
* {@code InternalEngine#recover} is responsible for taking the snapshot
* of the translog and releasing it once all 3 phases of recovery are complete
*/
@Override
public void phase2(Translog.Snapshot snapshot) {
if (shard.state() == IndexShardState.CLOSED) {
throw new IndexShardClosedException(request.shardId());
}
cancellableThreads.checkForCancel();
logger.trace("{} recovery [phase2] to {}: start", request.shardId(), request.targetNode());
protected void prepareTargetForTranslog(final Translog.View translogView) {
StopWatch stopWatch = new StopWatch().start();
logger.trace("{} recovery [phase1] to {}: prepare remote engine for translog", request.shardId(), request.targetNode());
final long startEngineStart = stopWatch.totalTime().millis();
cancellableThreads.execute(new Interruptable() {
@Override
public void run() throws InterruptedException {
@ -442,23 +452,38 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
// operations. This ensures the shard engine is started and disables
// garbage collection (not the JVM's GC!) of tombstone deletes
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.PREPARE_TRANSLOG,
new RecoveryPrepareForTranslogOperationsRequest(request.recoveryId(), request.shardId(), shard.translog().estimatedNumberOfOperations()),
new RecoveryPrepareForTranslogOperationsRequest(request.recoveryId(), request.shardId(), translogView.totalOperations()),
TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
}
});
stopWatch.stop();
response.startTime = stopWatch.totalTime().millis();
logger.trace("{} recovery [phase2] to {}: start took [{}]",
response.startTime = stopWatch.totalTime().millis() - startEngineStart;
logger.trace("{} recovery [phase1] to {}: remote engine start took [{}]",
request.shardId(), request.targetNode(), stopWatch.totalTime());
}
/**
* Perform phase2 of the recovery process
* <p/>
* Phase2 takes a snapshot of the current translog *without* acquiring the
* write lock (however, the translog snapshot is a point-in-time view of
* the translog). It then sends each translog operation to the target node
* so it can be replayed into the new shard.
*/
public void phase2(Translog.Snapshot snapshot) {
if (shard.state() == IndexShardState.CLOSED) {
throw new IndexShardClosedException(request.shardId());
}
cancellableThreads.checkForCancel();
StopWatch stopWatch = new StopWatch().start();
logger.trace("{} recovery [phase2] to {}: updating current mapping to master", request.shardId(), request.targetNode());
// Ensure that the mappings are synced with the master node
updateMappingOnMaster();
logger.trace("{} recovery [phase2] to {}: sending transaction log operations", request.shardId(), request.targetNode());
stopWatch = new StopWatch().start();
// Send all the snapshot's translog operations to the target
int totalOperations = sendSnapshot(snapshot);
stopWatch.stop();
@ -468,28 +493,16 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
}
/**
* Perform phase 3 of the recovery process
*
* Phase3 again takes a snapshot of the translog, however this time the
* snapshot is acquired under a write lock. The translog operations are
* sent to the target node where they are replayed.
*
* {@code InternalEngine#recover} is responsible for taking the snapshot
* of the translog, and after phase 3 completes the snapshots from all
* three phases are released.
* finalizes the recovery process
*/
@Override
public void phase3(Translog.Snapshot snapshot) {
public void finalizeRecovery() {
if (shard.state() == IndexShardState.CLOSED) {
throw new IndexShardClosedException(request.shardId());
}
cancellableThreads.checkForCancel();
StopWatch stopWatch = new StopWatch().start();
final int totalOperations;
logger.trace("[{}][{}] recovery [phase3] to {}: sending transaction log operations", indexName, shardId, request.targetNode());
logger.trace("[{}][{}] finalizing recovery to {}", indexName, shardId, request.targetNode());
// Send the translog operations to the target node
totalOperations = sendSnapshot(snapshot);
cancellableThreads.execute(new Interruptable() {
@Override
@ -518,10 +531,8 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
}
}
stopWatch.stop();
logger.trace("[{}][{}] recovery [phase3] to {}: took [{}]",
logger.trace("[{}][{}] finalizing recovery to {}: took [{}]",
indexName, shardId, request.targetNode(), stopWatch.totalTime());
response.phase3Time = stopWatch.totalTime().millis();
response.phase3Operations = totalOperations;
}
/**
@ -587,21 +598,21 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
/**
* Send the given snapshot's operations to this handler's target node.
*
* <p/>
* Operations are bulked into a single request depending on an operation
* count limit or size-in-bytes limit
*
* @return the total number of translog operations that were sent
*/
protected int sendSnapshot(Translog.Snapshot snapshot) {
protected int sendSnapshot(final Translog.Snapshot snapshot) {
int ops = 0;
long size = 0;
int totalOperations = 0;
final List<Translog.Operation> operations = Lists.newArrayList();
Translog.Operation operation;
try {
operation = snapshot.next(); // this ex should bubble up
} catch (IOException ex){
operation = snapshot.next(); // this ex should bubble up
} catch (IOException ex) {
throw new ElasticsearchException("failed to get next operation from translog", ex);
}
@ -641,7 +652,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
@Override
public void run() throws InterruptedException {
final RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest(
request.recoveryId(), request.shardId(), operations, shard.translog().estimatedNumberOfOperations());
request.recoveryId(), request.shardId(), operations, snapshot.estimatedTotalOperations());
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest,
recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
}
@ -649,7 +660,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
if (logger.isTraceEnabled()) {
logger.trace("[{}][{}] sent batch of [{}][{}] (total: [{}]) translog operations to {}",
indexName, shardId, ops, new ByteSizeValue(size),
shard.translog().estimatedNumberOfOperations(),
snapshot.estimatedTotalOperations(),
request.targetNode());
}
@ -659,16 +670,17 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
}
try {
operation = snapshot.next(); // this ex should bubble up
} catch (IOException ex){
} catch (IOException ex) {
throw new ElasticsearchException("failed to get next operation from translog", ex);
} }
}
}
// send the leftover
if (!operations.isEmpty()) {
cancellableThreads.execute(new Interruptable() {
@Override
public void run() throws InterruptedException {
RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest(
request.recoveryId(), request.shardId(), operations, shard.translog().estimatedNumberOfOperations());
request.recoveryId(), request.shardId(), operations, snapshot.estimatedTotalOperations());
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest,
recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
}
@ -678,7 +690,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
if (logger.isTraceEnabled()) {
logger.trace("[{}][{}] sent final batch of [{}][{}] (total: [{}]) translog operations to {}",
indexName, shardId, ops, new ByteSizeValue(size),
shard.translog().estimatedNumberOfOperations(),
snapshot.estimatedTotalOperations(),
request.targetNode());
}
return totalOperations;

View File

@ -42,7 +42,10 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.IndexShardMissingException;
import org.elasticsearch.index.engine.RecoveryEngineException;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardClosedException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.IndexMissingException;
@ -203,8 +206,6 @@ public class RecoveryTarget extends AbstractComponent {
sb.append(" : recovered [").append(recoveryResponse.phase2Operations).append("]").append(" transaction log operations")
.append(", took [").append(timeValueMillis(recoveryResponse.phase2Time)).append("]")
.append("\n");
sb.append(" phase3: recovered [").append(recoveryResponse.phase3Operations).append("]").append(" transaction log operations")
.append(", took [").append(timeValueMillis(recoveryResponse.phase3Time)).append("]");
logger.trace(sb.toString());
} else {
logger.debug("{} recovery done from [{}], took [{}]", request.shardId(), recoveryStatus.sourceNode(), recoveryTime);
@ -230,7 +231,7 @@ public class RecoveryTarget extends AbstractComponent {
// here, we would add checks against exception that need to be retried (and not removeAndClean in this case)
if (cause instanceof IndexShardNotStartedException || cause instanceof IndexMissingException || cause instanceof IndexShardMissingException) {
if (cause instanceof IllegalIndexShardStateException || cause instanceof IndexMissingException || cause instanceof IndexShardMissingException) {
// if the target is not ready yet, retry
retryRecovery(recoveryStatus, "remote shard not ready", recoverySettings.retryDelayStateSync(), request);
return;

View File

@ -45,7 +45,7 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler {
}
@Override
public void phase1(SnapshotIndexCommit snapshot) {
public void phase1(SnapshotIndexCommit snapshot, final Translog.View translogView) {
if (request.recoveryType() == RecoveryState.Type.RELOCATION && shard.routingEntry().primary()) {
// here we simply fail the primary shard since we can't move them (have 2 writers open at the same time)
// by failing the shard we play safe and just go through the entire reallocation procedure of the primary
@ -54,13 +54,14 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler {
shard.failShard("primary_relocation", exception);
throw exception;
}
logger.trace("{} recovery [phase2] to {}: skipping phase 1 for shared filesystem", request.shardId(), request.targetNode());
logger.trace("{} recovery [phase1] to {}: skipping phase 1 for shared filesystem", request.shardId(), request.targetNode());
prepareTargetForTranslog(translogView);
}
@Override
protected int sendSnapshot(Translog.Snapshot snapshot) {
logger.trace("{} recovery [phase3] to {}: skipping transaction log operations for file sync", shard.shardId(), request.targetNode());
logger.trace("{} recovery [phase2] to {}: skipping transaction log operations for file sync", shard.shardId(), request.targetNode());
return 0;
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.bwcompat;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
@ -60,16 +59,11 @@ import org.junit.Before;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.DirectoryStream;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.*;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
// needs at least 2 nodes since it bumps replicas to 1
@ -381,6 +375,7 @@ public class OldIndexBackwardsCompatibilityTests extends ElasticsearchIntegratio
// TODO: remove this once #10262 is fixed
return;
}
// these documents are supposed to be deleted by a delete by query operation in the translog
SearchRequestBuilder searchReq = client().prepareSearch(indexName).setQuery(QueryBuilders.queryStringQuery("long_sort:[10 TO 20]"));
assertEquals(0, searchReq.get().getHits().getTotalHits());
}

View File

@ -19,8 +19,6 @@
package org.elasticsearch.index.engine;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import com.carrotsearch.randomizedtesting.annotations.Seed;
import com.google.common.collect.ImmutableMap;
import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level;
@ -51,13 +49,13 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService;
@ -81,10 +79,10 @@ import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogSizeMatcher;
import org.elasticsearch.index.translog.fs.FsTranslog;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.ThreadPool;
import org.hamcrest.MatcherAssert;
import org.junit.After;
@ -92,6 +90,7 @@ import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CountDownLatch;
@ -113,15 +112,14 @@ public class InternalEngineTests extends ElasticsearchTestCase {
private Store store;
private Store storeReplica;
protected Translog translog;
protected Translog replicaTranslog;
protected InternalEngine engine;
protected InternalEngine replicaEngine;
private Settings defaultSettings;
private int indexConcurrency;
private String codecName;
private Path primaryTranslogDir;
private Path replicaTranslogDir;
@Override
@Before
@ -149,8 +147,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
storeReplica = createStore();
Lucene.cleanLuceneIndex(store.directory());
Lucene.cleanLuceneIndex(storeReplica.directory());
translog = createTranslog();
engine = createEngine(store, translog);
primaryTranslogDir = createTempDir("translog-primary");
engine = createEngine(store, createTranslog());
LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName());
@ -158,8 +156,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
if (randomBoolean()) {
engine.config().setEnableGcDeletes(false);
}
replicaTranslog = createTranslogReplica();
replicaEngine = createEngine(storeReplica, replicaTranslog);
replicaTranslogDir = createTempDir("translog-replica");
replicaEngine = createEngine(storeReplica, createTranslogReplica());
currentIndexWriterConfig = replicaEngine.getCurrentIndexWriterConfig();
assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName());
@ -173,13 +171,45 @@ public class InternalEngineTests extends ElasticsearchTestCase {
@After
public void tearDown() throws Exception {
super.tearDown();
IOUtils.close(
replicaEngine, storeReplica, replicaTranslog,
engine, store, translog);
terminate(threadPool);
try {
assertTranslogNotLeaking((FsTranslog) engine.translog());
assertTranslogNotLeaking((FsTranslog) replicaEngine.translog());
} finally {
IOUtils.close(
replicaEngine, storeReplica,
engine, store);
terminate(threadPool);
}
assertTranslogFilesClosed();
}
protected void assertTranslogNotLeaking(final FsTranslog translog) throws Exception {
assertBusy(new Runnable() {
@Override
public void run() {
try {
assertThat(translog.getUnreferenced(), emptyArray());
} catch (IOException e) {
throw new ElasticsearchException("error while checking for unreferenced files in translog", e);
}
}
});
}
protected void assertTranslogFilesClosed() throws Exception {
try {
assertBusy(new Runnable() {
@Override
public void run() {
FsTranslog.assertAllClosed();
}
});
} finally {
FsTranslog.assertForceCloseAllReferences();
}
}
private Document testDocumentWithTextField() {
Document document = testDocument();
document.add(new TextField("value", "test", Field.Store.YES));
@ -218,12 +248,16 @@ public class InternalEngineTests extends ElasticsearchTestCase {
return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId));
}
protected Translog createTranslog() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-primary"));
protected FsTranslog createTranslog() throws IOException {
return createTranslog(primaryTranslogDir);
}
protected Translog createTranslogReplica() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-replica"));
protected FsTranslog createTranslog(Path translogPath) throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE, translogPath);
}
protected FsTranslog createTranslogReplica() throws IOException {
return createTranslog(replicaTranslogDir);
}
protected IndexDeletionPolicy createIndexDeletionPolicy() {
@ -242,27 +276,25 @@ public class InternalEngineTests extends ElasticsearchTestCase {
return new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, indexSettingsService);
}
protected InternalEngine createEngine(Store store, Translog translog) {
protected InternalEngine createEngine(Store store, FsTranslog translog) {
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
return createEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService));
}
protected InternalEngine createEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new InternalEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider), false);
protected InternalEngine createEngine(IndexSettingsService indexSettingsService, Store store, FsTranslog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new InternalEngine(config(indexSettingsService, store, mergeSchedulerProvider), translog, false);
}
public EngineConfig config(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) {
public EngineConfig config(IndexSettingsService indexSettingsService, Store store, MergeSchedulerProvider mergeSchedulerProvider) {
IndexWriterConfig iwc = newIndexWriterConfig();
EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService
, null, store, createSnapshotDeletionPolicy(), translog, createMergePolicy(), mergeSchedulerProvider,
, null, store, createSnapshotDeletionPolicy(), createMergePolicy(), mergeSchedulerProvider,
iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(shardId.index()), new Engine.FailedEngineListener() {
@Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}
}, new TranslogHandler(shardId.index().getName()));
return config;
}
@ -313,7 +345,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
assertThat(segments.get(0).getDeletedDocs(), equalTo(0));
assertThat(segments.get(0).isCompound(), equalTo(defaultCompound));
((InternalEngine) engine).config().setCompoundOnFlush(false);
engine.config().setCompoundOnFlush(false);
ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, null);
engine.create(new Engine.Create(null, newUid("3"), doc3));
@ -361,7 +393,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
assertThat(segments.get(1).getDeletedDocs(), equalTo(0));
assertThat(segments.get(1).isCompound(), equalTo(false));
((InternalEngine) engine).config().setCompoundOnFlush(true);
engine.config().setCompoundOnFlush(true);
ParsedDocument doc4 = testParsedDocument("4", "4", "test", null, -1, -1, testDocumentWithTextField(), B_3, null);
engine.create(new Engine.Create(null, newUid("4"), doc4));
engine.refresh("test");
@ -422,8 +454,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
ConcurrentMergeSchedulerProvider mergeSchedulerProvider = new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, new IndexSettingsService(shardId.index(), EMPTY_SETTINGS));
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
try (Store store = createStore();
Translog translog = createTranslog();
Engine engine = createEngine(indexSettingsService, store, translog, mergeSchedulerProvider)) {
Engine engine = createEngine(indexSettingsService, store, createTranslog(createTempDir()), mergeSchedulerProvider)) {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(null, newUid("1"), doc);
@ -647,8 +678,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0));
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1));
searchResult.close();
engine.close();
}
@Test
@ -689,198 +718,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
searchResult.close();
}
@Test
public void testFailEngineOnCorruption() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
engine.create(new Engine.Create(null, newUid("1"), doc));
engine.flush();
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", "fake file description"));
}
}
@Override
public void phase2(Translog.Snapshot snapshot) throws EngineException {
if (failInPhase == 2) {
throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description"));
}
}
@Override
public void phase3(Translog.Snapshot snapshot) throws EngineException {
if (failInPhase == 3) {
throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description"));
}
}
});
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(), B_2, null);
engine.create(new Engine.Create(null, newUid("2"), doc2));
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();
fail("engine should have failed");
} catch (EngineClosedException ex) {
// expected
}
}
@Test
public void testSimpleRecover() throws Exception {
final ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
engine.create(new Engine.Create(null, newUid("1"), doc));
engine.flush();
engine.recover(new Engine.RecoveryHandler() {
@Override
public void phase1(SnapshotIndexCommit snapshot) throws EngineException {
try {
engine.flush();
assertThat("flush is not allowed in phase 1", false, equalTo(true));
} catch (FlushNotAllowedEngineException e) {
// all is well
}
}
@Override
public void phase2(Translog.Snapshot snapshot) throws EngineException {
MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
try {
engine.flush();
assertThat("flush is not allowed in phase 2", false, equalTo(true));
} catch (FlushNotAllowedEngineException e) {
// all is well
}
// but we can index
engine.index(new Engine.Index(null, newUid("1"), doc));
}
@Override
public void phase3(Translog.Snapshot snapshot) throws EngineException {
MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
try {
// we can do this here since we are on the same thread
engine.flush();
assertThat("flush is not allowed in phase 3", false, equalTo(true));
} catch (FlushNotAllowedEngineException e) {
// all is well
}
}
});
// post recovery should flush the translog
try (Translog.Snapshot snapshot = translog.snapshot()) {
MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
}
// and we should not leak files
assertThat("there are unreferenced translog files left", translog.clearUnreferenced(), equalTo(0));
engine.flush();
assertThat("there are unreferenced translog files left, post flush", translog.clearUnreferenced(), equalTo(0));
engine.close();
}
@Test
public void testRecoverWithOperationsBetweenPhase1AndPhase2() throws Exception {
ParsedDocument doc1 = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
engine.create(new Engine.Create(null, newUid("1"), doc1));
engine.flush();
ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null);
engine.create(new Engine.Create(null, newUid("2"), doc2));
engine.recover(new Engine.RecoveryHandler() {
@Override
public void phase1(SnapshotIndexCommit snapshot) throws EngineException {
}
@Override
public void phase2(Translog.Snapshot snapshot) throws EngineException {
try {
Translog.Create create = (Translog.Create) snapshot.next();
assertThat("translog snapshot should not read null", create != null, equalTo(true));
assertThat(create.source().toBytesArray(), equalTo(B_2));
assertThat(snapshot.next(), equalTo(null));
} catch (IOException ex) {
throw new ElasticsearchException("failed", ex);
}
}
@Override
public void phase3(Translog.Snapshot snapshot) throws EngineException {
MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
}
});
engine.flush();
engine.close();
}
@Test
public void testRecoverWithOperationsBetweenPhase1AndPhase2AndPhase3() throws Exception {
ParsedDocument doc1 = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
engine.create(new Engine.Create(null, newUid("1"), doc1));
engine.flush();
ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null);
engine.create(new Engine.Create(null, newUid("2"), doc2));
engine.recover(new Engine.RecoveryHandler() {
@Override
public void phase1(SnapshotIndexCommit snapshot) throws EngineException {
}
@Override
public void phase2(Translog.Snapshot snapshot) throws EngineException {
try {
Translog.Create create = (Translog.Create) snapshot.next();
assertThat(create != null, equalTo(true));
assertThat(snapshot.next(), equalTo(null));
assertThat(create.source().toBytesArray(), equalTo(B_2));
// add for phase3
ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, null);
engine.create(new Engine.Create(null, newUid("3"), doc3));
} catch (IOException ex) {
throw new ElasticsearchException("failed", ex);
}
}
@Override
public void phase3(Translog.Snapshot snapshot) throws EngineException {
try {
Translog.Create create = (Translog.Create) snapshot.next();
assertThat(create != null, equalTo(true));
assertThat(snapshot.next(), equalTo(null));
assertThat(create.source().toBytesArray(), equalTo(B_3));
} catch (IOException ex) {
throw new ElasticsearchException("failed", ex);
}
}
});
engine.flush();
engine.close();
}
@Test
public void testVersioningNewCreate() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null);
@ -1072,8 +909,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
int numIters = randomIntBetween(2, 10);
for (int j = 0; j < numIters; j++) {
try (Store store = createStore()) {
final Translog translog = createTranslog();
final InternalEngine engine = createEngine(store, translog);
final InternalEngine engine = createEngine(store, createTranslog(createTempDir()));
final CountDownLatch startGun = new CountDownLatch(1);
final CountDownLatch indexed = new CountDownLatch(1);
@ -1116,7 +952,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
engine.forceMerge(randomBoolean(), 1, false, randomBoolean(), randomBoolean());
}
indexed.await();
IOUtils.close(engine, translog);
IOUtils.close(engine);
}
}
@ -1452,7 +1288,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
assertNotNull(iwIFDLogger);
}
Level savedLevel = iwIFDLogger.getLevel();
iwIFDLogger.addAppender(mockAppender);
iwIFDLogger.setLevel(Level.DEBUG);
@ -1482,8 +1317,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
public void testEnableGcDeletes() throws Exception {
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
try (Store store = createStore();
Translog translog = createTranslog();
Engine engine = new InternalEngine(config(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)), false)) {
Engine engine = new InternalEngine(config(indexSettingsService, store, createMergeScheduler(indexSettingsService)),
createTranslog(createTempDir()), false)) {
engine.config().setEnableGcDeletes(false);
// Add document
@ -1549,7 +1384,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
try (Engine.Searcher test = this.engine.acquireSearcher("test")) {
ShardId shardId = ShardUtils.extractShardId(test.reader());
assertNotNull(shardId);
assertEquals(shardId, ((InternalEngine) engine).config().getShardId());
assertEquals(shardId, engine.config().getShardId());
}
}
@ -1567,10 +1402,9 @@ public class InternalEngineTests extends ElasticsearchTestCase {
wrapper.setAllowRandomFileNotFoundException(randomBoolean());
wrapper.setRandomIOExceptionRate(randomDouble());
wrapper.setRandomIOExceptionRateOnOpen(randomDouble());
try (Store store = createStore(wrapper)) {
try (Store store = createStore(wrapper); FsTranslog translog = createTranslog(createTempDir("testFailStart"))) {
int refCount = store.refCount();
assertTrue("refCount: " + store.refCount(), store.refCount() > 0);
Translog translog = createTranslog();
InternalEngine holder;
try {
holder = createEngine(store, translog);
@ -1592,7 +1426,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
break;
}
}
translog.close();
holder.close();
assertEquals(store.refCount(), refCount);
}
@ -1601,7 +1434,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
@Test
public void testSettings() {
InternalEngine engine = (InternalEngine) this.engine;
CodecService codecService = new CodecService(shardId.index());
LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
@ -1707,8 +1539,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
.put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb").build();
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), indexSettings);
try (Store store = createStore();
Translog translog = createTranslog();
final Engine engine = new InternalEngine(config(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)), false)) {
Engine engine = new InternalEngine(config(indexSettingsService, store, createMergeScheduler(indexSettingsService)),
createTranslog(createTempDir()), false)) {
for (int i = 0; i < 100; i++) {
String id = Integer.toString(i);
ParsedDocument doc = testParsedDocument(id, id, "test", null, -1, -1, testDocument(), B_1, null);
@ -1741,6 +1573,28 @@ public class InternalEngineTests extends ElasticsearchTestCase {
}
}
public void testMissingTranslog() throws IOException {
// test that we can force start the engine , even if the translog is missing.
engine.close();
// fake a new translog, causing the engine to point to a missing one.
FsTranslog translog = createTranslog();
translog.markCommitted(translog.currentId());
// we have to re-open the translog because o.w. it will complain about commit information going backwards, which is OK as we did a fake markComitted
translog.close();
translog = createTranslog();
try {
engine = createEngine(store, translog);
fail("engine shouldn't start without a valid translog id");
} catch (EngineCreationFailureException ex) {
// expected
}
// now it should be OK.
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings)
.put(EngineConfig.INDEX_IGNORE_UNKNOWN_TRANSLOG, true).build());
engine = createEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService));
}
@TestLogging("index.translog:TRACE")
public void testTranslogReplayWithFailure() throws IOException {
boolean canHaveDuplicates = true;
boolean autoGeneratedId = true;
@ -1770,21 +1624,13 @@ public class InternalEngineTests extends ElasticsearchTestCase {
directory.setRandomIOExceptionRate(randomDouble());
directory.setFailOnOpenInput(randomBoolean());
directory.setAllowRandomFileNotFoundException(randomBoolean());
final FsTranslog translog = createTranslog();
try {
engine = createEngine(store, translog);
started = true;
break;
} catch (EngineCreationFailureException ex) {
// sometimes we fail after we committed the recovered docs during the finaly refresh call
// that means hte index is consistent and recovered so we can't assert on the num recovered ops below.
try (IndexReader reader = DirectoryReader.open(directory.getDelegate())) {
if (reader.numDocs() == numDocs) {
recoveredButFailed = true;
break;
} else {
// skip - we just failed
}
}
translog.close();
}
}
@ -1793,20 +1639,16 @@ public class InternalEngineTests extends ElasticsearchTestCase {
directory.setFailOnOpenInput(false);
directory.setAllowRandomFileNotFoundException(false);
if (started == false) {
engine = createEngine(store, translog);
engine = createEngine(store, createTranslog());
}
} else {
// no mock directory, no fun.
engine = createEngine(store, translog);
engine = createEngine(store, createTranslog());
}
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
assertThat(topDocs.totalHits, equalTo(numDocs));
}
if (recoveredButFailed == false) {
TranslogHandler parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer();
assertEquals(numDocs, parser.recoveredOps.get());
}
}
@Test
@ -1831,11 +1673,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
// this so we have to disable the check explicitly
directory.setPreventDoubleWrite(false);
}
long currentTranslogId = translog.currentId();
engine.close();
engine = new InternalEngine(engine.config(), true);
assertTrue(currentTranslogId + "<" + translog.currentId(), currentTranslogId < translog.currentId());
assertEquals("translog ID must be incremented by 2 after initial recovery", currentTranslogId + 2, translog.currentId());
engine = new InternalEngine(engine.config(), createTranslog(), true);
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
@ -1875,11 +1714,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
TranslogHandler parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer();
parser.mappingUpdate = dynamicUpdate();
long currentTranslogId = translog.currentId();
engine.close();
engine = new InternalEngine(engine.config(), false); // we need to reuse the engine config otherwise the parser.mappingModified won't work
assertTrue(currentTranslogId + "<" + translog.currentId(), currentTranslogId < translog.currentId());
assertEquals("translog ID must be incremented by 2 after initial recovery", currentTranslogId + 2, translog.currentId());
engine = new InternalEngine(engine.config(), createTranslog(), false); // we need to reuse the engine config unless the parser.mappingModified won't work
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
@ -1895,7 +1731,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
}
engine.close();
engine = createEngine(store, translog);
engine = createEngine(store, createTranslog());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
assertThat(topDocs.totalHits, equalTo(numDocs));
@ -1925,7 +1761,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
}
engine.close();
engine = createEngine(store, translog);
engine = createEngine(store, createTranslog());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), numDocs + 1);
assertThat(topDocs.totalHits, equalTo(numDocs + 1));
@ -1937,7 +1773,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
engine.refresh("test");
} else {
engine.close();
engine = createEngine(store, translog);
engine = createEngine(store, createTranslog());
}
try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), numDocs);

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
@ -92,8 +93,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
private Store store;
private Store storeReplica;
protected Translog translog;
protected Translog replicaTranslog;
protected FsTranslog translog;
protected Engine primaryEngine;
protected Engine replicaEngine;
@ -139,8 +139,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
primaryEngine.config().setEnableGcDeletes(false);
}
replicaTranslog = createTranslogReplica();
replicaEngine = createShadowEngine(storeReplica, replicaTranslog);
replicaEngine = createShadowEngine(storeReplica);
assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName());
if (randomBoolean()) {
@ -154,11 +153,8 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
super.tearDown();
replicaEngine.close();
storeReplica.close();
translog.close();
replicaTranslog.close();
primaryEngine.close();
translog.close();
store.close();
terminate(threadPool);
}
@ -201,12 +197,8 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId));
}
protected Translog createTranslog() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-primary"));
}
protected Translog createTranslogReplica() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-replica"));
protected FsTranslog createTranslog() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE, createTempDir("translog-primary"));
}
protected IndexDeletionPolicy createIndexDeletionPolicy() {
@ -225,36 +217,34 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
return new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, indexSettingsService);
}
protected ShadowEngine createShadowEngine(Store store, Translog translog) {
protected ShadowEngine createShadowEngine(Store store) {
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
return createShadowEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService));
return createShadowEngine(indexSettingsService, store, createMergeScheduler(indexSettingsService));
}
protected InternalEngine createInternalEngine(Store store, Translog translog) {
protected InternalEngine createInternalEngine(Store store, FsTranslog translog) {
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
return createInternalEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService));
}
protected ShadowEngine createShadowEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new ShadowEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider));
protected ShadowEngine createShadowEngine(IndexSettingsService indexSettingsService, Store store, MergeSchedulerProvider mergeSchedulerProvider) {
return new ShadowEngine(config(indexSettingsService, store, mergeSchedulerProvider));
}
protected InternalEngine createInternalEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new InternalEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider), true);
protected InternalEngine createInternalEngine(IndexSettingsService indexSettingsService, Store store, FsTranslog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new InternalEngine(config(indexSettingsService, store, mergeSchedulerProvider), translog, true);
}
public EngineConfig config(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) {
public EngineConfig config(IndexSettingsService indexSettingsService, Store store, MergeSchedulerProvider mergeSchedulerProvider) {
IndexWriterConfig iwc = newIndexWriterConfig();
EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService
, null, store, createSnapshotDeletionPolicy(), translog, createMergePolicy(), mergeSchedulerProvider,
, null, store, createSnapshotDeletionPolicy(), createMergePolicy(), mergeSchedulerProvider,
iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(shardId.index()), new Engine.FailedEngineListener() {
@Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test
}
}, null);
return config;
}
@ -907,10 +897,9 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
try (Store store = createStore(wrapper)) {
int refCount = store.refCount();
assertTrue("refCount: "+ store.refCount(), store.refCount() > 0);
Translog translog = createTranslog();
ShadowEngine holder;
try {
holder = createShadowEngine(store, translog);
holder = createShadowEngine(store);
} catch (EngineCreationFailureException ex) {
assertEquals(store.refCount(), refCount);
continue;
@ -921,7 +910,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
try {
assertEquals(store.refCount(), refCount + 1);
holder.close();
holder = createShadowEngine(store, translog);
holder = createShadowEngine(store);
assertEquals(store.refCount(), refCount + 1);
} catch (EngineCreationFailureException ex) {
// all is fine
@ -948,7 +937,6 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
final Path srDir = createTempDir();
final Store srStore = createStore(srDir);
Lucene.cleanLuceneIndex(srStore.directory());
final Translog srTranslog = createTranslogReplica();
final AtomicBoolean succeeded = new AtomicBoolean(false);
final CountDownLatch latch = new CountDownLatch(1);
@ -963,7 +951,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
} catch (InterruptedException e) {
// ignore interruptions
}
try (ShadowEngine srEngine = createShadowEngine(srStore, srTranslog)) {
try (ShadowEngine srEngine = createShadowEngine(srStore)) {
succeeded.set(true);
} catch (Exception e) {
fail("should have been able to create the engine!");
@ -979,7 +967,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
// Create an InternalEngine, which creates the index so the shadow
// replica will handle it correctly
Store pStore = createStore(srDir);
Translog pTranslog = createTranslog();
FsTranslog pTranslog = createTranslog();
InternalEngine pEngine = createInternalEngine(pStore, pTranslog);
// create a document
@ -992,6 +980,6 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
t.join();
assertTrue("ShadowEngine should have been able to be created", succeeded.get());
// (shadow engine is already shut down in the try-with-resources)
IOUtils.close(srTranslog, srStore, pTranslog, pEngine, pStore);
IOUtils.close(srStore, pTranslog, pEngine, pStore);
}
}

View File

@ -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.index.translog;
import org.elasticsearch.ElasticsearchException;
import org.hamcrest.Description;
import org.hamcrest.Matcher;
import org.hamcrest.TypeSafeMatcher;
import java.io.IOException;
import java.util.ArrayList;
public final class SnapshotMatchers {
private SnapshotMatchers() {
}
/**
* Consumes a snapshot and make sure it's size is as expected
*/
public static Matcher<Translog.Snapshot> size(int size) {
return new SizeMatcher(size);
}
/**
* Consumes a snapshot and make sure it's content is as expected
*/
public static Matcher<Translog.Snapshot> equalsTo(Translog.Operation... ops) {
return new EqualMatcher(ops);
}
/**
* Consumes a snapshot and make sure it's content is as expected
*/
public static Matcher<Translog.Snapshot> equalsTo(ArrayList<Translog.Operation> ops) {
return new EqualMatcher(ops.toArray(new Translog.Operation[ops.size()]));
}
public static class SizeMatcher extends TypeSafeMatcher<Translog.Snapshot> {
private final int size;
public SizeMatcher(int size) {
this.size = size;
}
@Override
public boolean matchesSafely(Translog.Snapshot snapshot) {
int count = 0;
try {
while (snapshot.next() != null) {
count++;
}
} catch (IOException ex) {
throw new ElasticsearchException("failed to advance snapshot", ex);
}
return size == count;
}
@Override
public void describeTo(Description description) {
description.appendText("a snapshot with size ").appendValue(size);
}
}
public static class EqualMatcher extends TypeSafeMatcher<Translog.Snapshot> {
private final Translog.Operation[] expectedOps;
String failureMsg = null;
public EqualMatcher(Translog.Operation[] expectedOps) {
this.expectedOps = expectedOps;
}
@Override
protected boolean matchesSafely(Translog.Snapshot snapshot) {
try {
Translog.Operation op;
int i;
for (i = 0, op = snapshot.next(); op != null && i < expectedOps.length; i++, op = snapshot.next()) {
if (expectedOps[i].equals(op) == false) {
failureMsg = "position [" + i + "] expected [" + expectedOps[i] + "] but found [" + op + "]";
return false;
}
}
if (i < expectedOps.length) {
failureMsg = "expected [" + expectedOps.length + "] ops but only found [" + i + "]";
return false;
}
if (op != null) {
int count = 1; // to account for the op we already read
while (snapshot.next() != null) {
count++;
}
failureMsg = "expected [" + expectedOps.length + "] ops but got [" + (expectedOps.length + count) + "]";
return false;
}
return true;
} catch (IOException ex) {
throw new ElasticsearchException("failed to read snapshot content", ex);
}
}
@Override
public void describeTo(Description description) {
description.appendText(failureMsg);
}
}
}

View File

@ -1,66 +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.elasticsearch.index.translog;
import org.elasticsearch.ElasticsearchException;
import org.hamcrest.Description;
import org.hamcrest.Matcher;
import org.hamcrest.TypeSafeMatcher;
import java.io.IOException;
/**
*
*/
public class TranslogSizeMatcher extends TypeSafeMatcher<Translog.Snapshot> {
private final int size;
public TranslogSizeMatcher(int size) {
this.size = size;
}
@Override
public boolean matchesSafely(Translog.Snapshot snapshot) {
int count = 0;
long startingPosition = snapshot.position();
try {
while (snapshot.next() != null) {
count++;
}
return size == count;
} catch (IOException ex) {
throw new ElasticsearchException("failed to advance iterator", ex);
} finally {
// Since counting the translog size consumes the stream, reset it
// back to the origin position after reading
snapshot.seekTo(startingPosition);
}
}
@Override
public void describeTo(Description description) {
description.appendText("a translog with size ").appendValue(size);
}
public static Matcher<Translog.Snapshot> translogSize(int size) {
return new TranslogSizeMatcher(size);
}
}

View File

@ -44,7 +44,6 @@ public class TranslogVersionTests extends ElasticsearchTestCase {
StreamInput in = stream.openInput(translogFile);
in.readInt();
Translog.Operation operation = stream.read(in);
assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.SAVE, equalTo(true));
@ -60,7 +59,6 @@ public class TranslogVersionTests extends ElasticsearchTestCase {
assertThat(op.versionType(), equalTo(VersionType.INTERNAL));
try {
in.readInt();
stream.read(in);
fail("should have been the end of the file");
} catch (EOFException e) {
@ -143,7 +141,7 @@ public class TranslogVersionTests extends ElasticsearchTestCase {
fail("should have thrown an exception about the body being corrupted");
} catch (TranslogCorruptedException e) {
assertThat("translog corruption from body: " + e.getMessage(),
e.getMessage().contains("translog stream is corrupted"), equalTo(true));
e.getMessage().contains("translog corruption while reading from stream"), equalTo(true));
}
}

View File

@ -17,18 +17,22 @@
* under the License.
*/
package org.elasticsearch.index.translog;
package org.elasticsearch.index.translog.fs;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.stream.BytesStreamInput;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.*;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.hamcrest.Matchers;
import org.junit.After;
@ -42,12 +46,12 @@ import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.*;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import static com.google.common.collect.Lists.newArrayList;
import static org.hamcrest.Matchers.*;
@ -56,12 +60,29 @@ import static org.hamcrest.Matchers.*;
*
*/
@LuceneTestCase.SuppressFileSystems("ExtrasFS")
public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase {
public abstract class AbstractTranslogTests extends ElasticsearchTestCase {
protected final ShardId shardId = new ShardId(new Index("index"), 1);
protected FsTranslog translog;
protected Path translogDir;
protected Translog translog;
@Override
protected void afterIfSuccessful() throws Exception {
super.afterIfSuccessful();
if (translog.isOpen()) {
assertNotLeaking();
if (translog.currentId() > 1) {
translog.markCommitted(translog.currentId());
assertFileDeleted(translog, translog.currentId() - 1);
}
translog.close();
}
assertFileIsPresent(translog, translog.currentId());
IOUtils.rm(translog.location()); // delete all the locations
}
@Override
@Before
@ -69,8 +90,7 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
super.setUp();
// if a previous test failed we clean up things here
translogDir = createTempDir();
translog = create(translogDir);
translog.newTranslog(1);
translog = create();
}
@Override
@ -78,17 +98,56 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
public void tearDown() throws Exception {
try {
translog.close();
if (translog.currentId() > 1) {
// ensure all snapshots etc are closed if this fails something was not closed
assertFileDeleted(translog, translog.currentId() - 1);
}
assertFileIsPresent(translog, translog.currentId());
} finally {
super.tearDown();
}
}
protected abstract Translog create(Path translogDir) throws IOException;
protected abstract FsTranslog create() throws IOException;
protected void assertNotLeaking() throws Exception {
assertBusy(new Runnable() {
@Override
public void run() {
try {
assertThat(translog.getUnreferenced(), emptyArray());
} catch (IOException e) {
logger.warn("error while checking for unreferenced files in translog");
}
}
});
}
protected void assertTranslogFilesClosed() throws Exception {
assertBusy(new Runnable() {
@Override
public void run() {
FsTranslog.assertAllClosed();
}
});
}
protected void addToTranslogAndList(Translog translog, ArrayList<Translog.Operation> list, Translog.Operation op) {
list.add(op);
translog.add(op);
}
public void testIdParsingFromFile() {
long id = randomIntBetween(0, Integer.MAX_VALUE);
Path file = translogDir.resolve(FsTranslog.TRANSLOG_FILE_PREFIX + id);
assertThat(FsTranslog.parseIdFromFileName(file), equalTo(id));
file = translogDir.resolve(FsTranslog.TRANSLOG_FILE_PREFIX + id + ".recovering");
assertThat(FsTranslog.parseIdFromFileName(file), equalTo(id));
file = translogDir.resolve(FsTranslog.TRANSLOG_FILE_PREFIX + randomRealisticUnicodeOfCodepointLength(randomIntBetween(1, 10)) + id);
assertThat(FsTranslog.parseIdFromFileName(file), equalTo(-1l));
file = translogDir.resolve(randomRealisticUnicodeOfCodepointLength(randomIntBetween(1, FsTranslog.TRANSLOG_FILE_PREFIX.length() - 1)));
assertThat(FsTranslog.parseIdFromFileName(file), equalTo(-1l));
}
@Test
public void testRead() throws IOException {
@ -105,70 +164,32 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3})));
}
@Test
public void testTransientTranslog() throws IOException {
Translog.Snapshot snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close();
translog.newTransientTranslog(2);
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close();
translog.add(new Translog.Index("test", "2", new byte[]{2}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(2));
assertThat(snapshot.estimatedTotalOperations(), equalTo(2));
snapshot.close();
translog.makeTransientCurrent();
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); // now its one, since it only includes "2"
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close();
}
@Test
public void testSimpleOperations() throws IOException {
Translog.Snapshot snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
ArrayList<Translog.Operation> ops = new ArrayList<>();
Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
addToTranslogAndList(translog, ops, new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close();
translog.add(new Translog.Index("test", "2", new byte[]{2}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(2));
assertThat(snapshot.estimatedTotalOperations(), equalTo(2));
addToTranslogAndList(translog, ops, new Translog.Index("test", "2", new byte[]{2}));
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size()));
snapshot.close();
translog.add(new Translog.Delete(newUid("3")));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(3));
assertThat(snapshot.estimatedTotalOperations(), equalTo(3));
addToTranslogAndList(translog, ops, new Translog.Delete(newUid("3")));
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size()));
snapshot.close();
translog.add(new Translog.DeleteByQuery(new BytesArray(new byte[]{4}), null));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(4));
assertThat(snapshot.estimatedTotalOperations(), equalTo(4));
snapshot.close();
snapshot = translog.snapshot();
snapshot = translog.newSnapshot();
Translog.Create create = (Translog.Create) snapshot.next();
assertThat(create != null, equalTo(true));
@ -182,33 +203,26 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertThat(delete != null, equalTo(true));
assertThat(delete.uid(), equalTo(newUid("3")));
Translog.DeleteByQuery deleteByQuery = (Translog.DeleteByQuery) snapshot.next();
assertThat(deleteByQuery != null, equalTo(true));
assertThat(deleteByQuery.source().toBytes(), equalTo(new byte[]{4}));
assertThat(snapshot.next(), equalTo(null));
snapshot.close();
long firstId = translog.currentId();
translog.newTranslog(2);
translog.newTranslog();
assertThat(translog.currentId(), Matchers.not(equalTo(firstId)));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size()));
snapshot.close();
translog.markCommitted(translog.currentId());
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
assertThat(snapshot.estimatedTotalOperations(), equalTo(0));
snapshot.close();
}
@Test(expected = TranslogException.class)
public void testReuseFails() throws IOException {
if (randomBoolean()) {
translog.newTranslog(1);
} else {
translog.newTransientTranslog(1);
}
}
protected TranslogStats stats() throws IOException {
// force flushing and updating of stats
translog.sync();
@ -248,88 +262,69 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize));
lastSize = stats.translogSizeInBytes().bytes();
translog.add(new Translog.DeleteByQuery(new BytesArray(new byte[]{4}), null));
translog.add(new Translog.Delete(newUid("4")));
translog.newTranslog();
stats = stats();
assertThat(stats.estimatedNumberOfOperations(), equalTo(4l));
assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize));
translog.newTranslog(2);
translog.markCommitted(2);
stats = stats();
assertThat(stats.estimatedNumberOfOperations(), equalTo(0l));
assertThat(stats.translogSizeInBytes().bytes(), equalTo(17l));
}
@Test
public void testSnapshot() throws IOException {
Translog.Snapshot snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
public void testSnapshot() {
ArrayList<Translog.Operation> ops = new ArrayList<>();
Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
addToTranslogAndList(translog, ops, new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close();
snapshot = translog.snapshot();
Translog.Create create = (Translog.Create) snapshot.next();
assertThat(create != null, equalTo(true));
assertThat(create.source().toBytes(), equalTo(new byte[]{1}));
snapshot.close();
snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
Translog.Snapshot snapshot1 = translog.snapshot();
assertThat(snapshot1, TranslogSizeMatcher.translogSize(1));
// snapshot while another is open
Translog.Snapshot snapshot1 = translog.newSnapshot();
assertThat(snapshot1, SnapshotMatchers.size(1));
assertThat(snapshot1.estimatedTotalOperations(), equalTo(1));
// seek to the end of the translog snapshot
while (snapshot1.next() != null) {
// spin
}
translog.add(new Translog.Index("test", "2", new byte[]{2}));
snapshot = translog.snapshot(snapshot1);
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
assertThat(snapshot.estimatedTotalOperations(), equalTo(2));
snapshot.close();
snapshot = translog.snapshot(snapshot1);
Translog.Index index = (Translog.Index) snapshot.next();
assertThat(index != null, equalTo(true));
assertThat(index.source().toBytes(), equalTo(new byte[]{2}));
assertThat(snapshot.next(), equalTo(null));
assertThat(snapshot.estimatedTotalOperations(), equalTo(2));
snapshot.close();
snapshot1.close();
}
@Test
public void testSnapshotWithNewTranslog() throws IOException {
Translog.Snapshot snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
ArrayList<Translog.Operation> ops = new ArrayList<>();
Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot actualSnapshot = translog.snapshot();
addToTranslogAndList(translog, ops, new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot snapshot1 = translog.newSnapshot();
translog.add(new Translog.Index("test", "2", new byte[]{2}));
addToTranslogAndList(translog, ops, new Translog.Index("test", "2", new byte[]{2}));
translog.newTranslog(2);
translog.newTranslog();
translog.add(new Translog.Index("test", "3", new byte[]{3}));
addToTranslogAndList(translog, ops, new Translog.Index("test", "3", new byte[]{3}));
snapshot = translog.snapshot(actualSnapshot);
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
snapshot.close();
Translog.Snapshot snapshot2 = translog.newSnapshot();
assertThat(snapshot2, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot2.estimatedTotalOperations(), equalTo(ops.size()));
snapshot = translog.snapshot(actualSnapshot);
Translog.Index index = (Translog.Index) snapshot.next();
assertThat(index != null, equalTo(true));
assertThat(index.source().toBytes(), equalTo(new byte[]{3}));
assertThat(snapshot.next(), equalTo(null));
actualSnapshot.close();
snapshot.close();
assertThat(snapshot1, SnapshotMatchers.equalsTo(ops.get(0)));
snapshot1.close();
snapshot2.close();
}
public void testSnapshotOnClosedTranslog() throws IOException {
@ -337,106 +332,64 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
translog.add(new Translog.Create("test", "1", new byte[]{1}));
translog.close();
try {
Translog.Snapshot snapshot = translog.snapshot();
Translog.Snapshot snapshot = translog.newSnapshot();
fail("translog is closed");
} catch (TranslogException ex) {
assertEquals(ex.getMessage(), "current translog is already closed");
assertThat(ex.getMessage(), containsString("can't increment channel"));
}
}
@Test
public void deleteOnRollover() throws IOException {
translog.add(new Translog.Create("test", "1", new byte[]{1}));
public void deleteOnSnapshotRelease() throws Exception {
ArrayList<Translog.Operation> firstOps = new ArrayList<>();
addToTranslogAndList(translog, firstOps, new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot firstSnapshot = translog.snapshot();
assertThat(firstSnapshot, TranslogSizeMatcher.translogSize(1));
Translog.Snapshot firstSnapshot = translog.newSnapshot();
assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1));
translog.newTransientTranslog(2);
translog.newTranslog();
translog.markCommitted(translog.currentId());
assertFileIsPresent(translog, 1);
translog.add(new Translog.Index("test", "2", new byte[]{2}));
assertThat(firstSnapshot, TranslogSizeMatcher.translogSize(1));
ArrayList<Translog.Operation> secOps = new ArrayList<>();
addToTranslogAndList(translog, secOps, new Translog.Index("test", "2", new byte[]{2}));
assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1));
if (randomBoolean()) {
translog.clearUnreferenced();
}
translog.makeTransientCurrent();
Translog.Snapshot secondSnapshot = translog.snapshot();
assertNotLeaking();
Translog.Snapshot secondSnapshot = translog.newSnapshot();
translog.add(new Translog.Index("test", "3", new byte[]{3}));
assertThat(secondSnapshot, TranslogSizeMatcher.translogSize(1));
assertThat(secondSnapshot, SnapshotMatchers.equalsTo(secOps));
assertThat(secondSnapshot.estimatedTotalOperations(), equalTo(1));
assertFileIsPresent(translog, 1);
assertFileIsPresent(translog, 2);
if (randomBoolean()) {
translog.clearUnreferenced();
}
assertNotLeaking();
firstSnapshot.close();
assertFileDeleted(translog, 1);
assertFileIsPresent(translog, 2);
secondSnapshot.close();
assertFileIsPresent(translog, 2); // it's the current nothing should be deleted
if (randomBoolean()) {
translog.clearUnreferenced();
}
translog.newTransientTranslog(3);
translog.makeTransientCurrent();
if (randomBoolean()) {
translog.clearUnreferenced();
}
assertNotLeaking();
translog.newTranslog();
translog.markCommitted(translog.currentId());
assertNotLeaking();
assertFileIsPresent(translog, 3); // it's the current nothing should be deleted
assertFileDeleted(translog, 2);
assertEquals(3, translog.findLargestPresentTranslogId());
translog.newTransientTranslog(4);
translog.revertTransient();
assertFileIsPresent(translog, 3); // it's the current nothing should be deleted
assertFileDeleted(translog, 4);
}
public void assertFileIsPresent(Translog translog, long id) {
if(Files.exists(translog.location().resolve(translog.getFilename(id)))) {
public void assertFileIsPresent(FsTranslog translog, long id) {
if (Files.exists(translogDir.resolve(translog.getFilename(id)))) {
return;
}
fail(translog.getFilename(id) + " is not present in any location: " + translog.location());
}
public void assertFileDeleted(Translog translog, long id) {
assertFalse(Files.exists(translog.location().resolve(translog.getFilename(id))));
public void assertFileDeleted(FsTranslog translog, long id) {
assertFalse("translog [" + id + "] still exists", Files.exists(translog.location().resolve(translog.getFilename(id))));
}
@Test
public void testSnapshotWithSeekTo() throws IOException {
Translog.Snapshot snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0));
snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
// seek to the end of the translog snapshot
while (snapshot.next() != null) {
// spin
}
long lastPosition = snapshot.position();
snapshot.close();
translog.add(new Translog.Create("test", "2", new byte[]{1}));
snapshot = translog.snapshot();
snapshot.seekTo(lastPosition);
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
snapshot.close();
snapshot = translog.snapshot();
snapshot.seekTo(lastPosition);
Translog.Create create = (Translog.Create) snapshot.next();
assertThat(create != null, equalTo(true));
assertThat(create.id(), equalTo("2"));
snapshot.close();
}
static class LocationOperation {
final Translog.Operation operation;
final Translog.Location location;
@ -445,6 +398,7 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
this.operation = operation;
this.location = location;
}
}
@Test
@ -482,11 +436,8 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
randomFrom(VersionType.values()));
break;
case DELETE_BY_QUERY:
op = new Translog.DeleteByQuery(
new BytesArray(randomRealisticUnicodeOfLengthBetween(10, 400).getBytes("UTF-8")),
new String[]{randomRealisticUnicodeOfLengthBetween(10, 400)},
"test");
break;
// deprecated
continue;
default:
throw new ElasticsearchException("not supported op type");
}
@ -544,14 +495,6 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertEquals(expDelOp.version(), delOp.version());
assertEquals(expDelOp.versionType(), delOp.versionType());
break;
case DELETE_BY_QUERY:
Translog.DeleteByQuery delQueryOp = (Translog.DeleteByQuery) op;
Translog.DeleteByQuery expDelQueryOp = (Translog.DeleteByQuery) expectedOp;
assertThat(expDelQueryOp.source(), equalTo(delQueryOp.source()));
assertThat(expDelQueryOp.filteringAliases(), equalTo(delQueryOp.filteringAliases()));
assertThat(expDelQueryOp.types(), equalTo(delQueryOp.types()));
break;
default:
throw new ElasticsearchException("unsupported opType");
}
@ -655,10 +598,11 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
@Test
public void testVerifyTranslogIsNotDeleted() throws IOException {
assertTrue(Files.exists(translogDir.resolve("translog-1")));
assertFileIsPresent(translog, 1);
translog.add(new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1));
Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(1));
assertFileIsPresent(translog, 1);
assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
if (randomBoolean()) {
translog.close();
@ -668,6 +612,181 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
translog.close();
}
assertTrue(Files.exists(translogDir.resolve("translog-1")));
assertFileIsPresent(translog, 1);
}
/** Tests that concurrent readers and writes maintain view and snapshot semantics */
@Test
public void testConcurrentWriteViewsAndSnapshot() throws Throwable {
final Thread[] writers = new Thread[randomIntBetween(1, 10)];
final Thread[] readers = new Thread[randomIntBetween(1, 10)];
final int flushEveryOps = randomIntBetween(5, 100);
// used to notify main thread that so many operations have been written so it can simulate a flush
final AtomicReference<CountDownLatch> writtenOpsLatch = new AtomicReference<>(new CountDownLatch(0));
final AtomicLong idGenerator = new AtomicLong();
final CyclicBarrier barrier = new CyclicBarrier(writers.length + readers.length + 1);
// a map of all written ops and their returned location.
final Map<Translog.Operation, Translog.Location> writtenOps = ConcurrentCollections.newConcurrentMap();
// a signal for all threads to stop
final AtomicBoolean run = new AtomicBoolean(true);
// any errors on threads
final List<Throwable> errors = new CopyOnWriteArrayList<>();
logger.debug("using [{}] readers. [{}] writers. flushing every ~[{}] ops.", readers.length, writers.length, flushEveryOps);
for (int i = 0; i < writers.length; i++) {
final String threadId = "writer_" + i;
writers[i] = new Thread(new AbstractRunnable() {
@Override
public void doRun() throws BrokenBarrierException, InterruptedException {
barrier.await();
int counter = 0;
while (run.get()) {
long id = idGenerator.incrementAndGet();
final Translog.Operation op;
switch (Translog.Operation.Type.values()[((int) (id % Translog.Operation.Type.values().length))]) {
case CREATE:
op = new Translog.Create("type", "" + id, new byte[]{(byte) id});
break;
case SAVE:
op = new Translog.Index("type", "" + id, new byte[]{(byte) id});
break;
case DELETE:
op = new Translog.Delete(newUid("" + id));
break;
case DELETE_BY_QUERY:
// deprecated
continue;
default:
throw new ElasticsearchException("unknown type");
}
Translog.Location location = translog.add(op);
Translog.Location existing = writtenOps.put(op, location);
if (existing != null) {
fail("duplicate op [" + op + "], old entry at " + location);
}
writtenOpsLatch.get().countDown();
counter++;
}
logger.debug("--> [{}] done. wrote [{}] ops.", threadId, counter);
}
@Override
public void onFailure(Throwable t) {
logger.error("--> writer [{}] had an error", t, threadId);
errors.add(t);
}
}, threadId);
writers[i].start();
}
for (int i = 0; i < readers.length; i++) {
final String threadId = "reader_" + i;
readers[i] = new Thread(new AbstractRunnable() {
Translog.View view = null;
Set<Translog.Operation> writtenOpsAtView;
@Override
public void onFailure(Throwable t) {
logger.error("--> reader [{}] had an error", t, threadId);
errors.add(t);
closeView();
}
void closeView() {
if (view != null) {
view.close();
}
}
void newView() {
closeView();
view = translog.newView();
// captures the currently written ops so we know what to expect from the view
writtenOpsAtView = new HashSet<>(writtenOps.keySet());
logger.debug("--> [{}] opened view from [{}]", threadId, view.minTranslogId());
}
@Override
protected void doRun() throws Exception {
barrier.await();
int iter = 0;
while (run.get()) {
if (iter++ % 10 == 0) {
newView();
}
// captures al views that are written since the view was created (with a small caveat see bellow)
// these are what we expect the snapshot to return (and potentially some more).
Set<Translog.Operation> expectedOps = new HashSet<>(writtenOps.keySet());
expectedOps.removeAll(writtenOpsAtView);
try (Translog.Snapshot snapshot = view.snapshot()) {
Translog.Operation op;
while ((op = snapshot.next()) != null) {
expectedOps.remove(op);
}
}
if (expectedOps.isEmpty() == false) {
StringBuilder missed = new StringBuilder("missed ").append(expectedOps.size()).append(" operations");
boolean failed = false;
for (Translog.Operation op : expectedOps) {
final Translog.Location loc = writtenOps.get(op);
if (loc.translogId < view.minTranslogId()) {
// writtenOps is only updated after the op was written to the translog. This mean
// that ops written to the translog before the view was taken (and will be missing from the view)
// may yet be available in writtenOpsAtView, meaning we will erroneously expect them
continue;
}
failed = true;
missed.append("\n --> [").append(op).append("] written at ").append(loc);
}
if (failed) {
fail(missed.toString());
}
}
// slow down things a bit and spread out testing..
writtenOpsLatch.get().await(200, TimeUnit.MILLISECONDS);
}
closeView();
logger.debug("--> [{}] done. tested [{}] snapshots", threadId, iter);
}
}, threadId);
readers[i].start();
}
barrier.await();
try {
long previousId = translog.currentId();
for (int iterations = scaledRandomIntBetween(10, 200); iterations > 0 && errors.isEmpty(); iterations--) {
writtenOpsLatch.set(new CountDownLatch(flushEveryOps));
while (writtenOpsLatch.get().await(200, TimeUnit.MILLISECONDS) == false) {
if (errors.size() > 0) {
break;
}
}
long newId = translog.newTranslog();
translog.markCommitted(previousId);
previousId = newId;
}
} finally {
run.set(false);
logger.debug("--> waiting for threads to stop");
for (Thread thread : writers) {
thread.join();
}
for (Thread thread : readers) {
thread.join();
}
if (errors.size() > 0) {
Throwable e = errors.get(0);
for (Throwable suppress : errors.subList(1, errors.size())) {
e.addSuppressed(suppress);
}
throw e;
}
logger.info("--> test done. total ops written [{}]", writtenOps.size());
}
}
}

View File

@ -20,25 +20,25 @@
package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.translog.AbstractSimpleTranslogTests;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.test.junit.annotations.TestLogging;
import java.io.IOException;
import java.nio.file.Path;
/**
*
*/
public class FsBufferedTranslogTests extends AbstractSimpleTranslogTests {
@TestLogging("index.translog.fs:TRACE")
public class FsBufferedTranslogTests extends AbstractTranslogTests {
@Override
protected Translog create(Path translogDir) throws IOException {
protected FsTranslog create() throws IOException {
return new FsTranslog(shardId,
ImmutableSettings.settingsBuilder()
.put("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name())
.put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024))
.build(),
translogDir
BigArrays.NON_RECYCLING_INSTANCE, translogDir
);
}
}

View File

@ -20,22 +20,21 @@
package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.translog.AbstractSimpleTranslogTests;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.test.junit.annotations.TestLogging;
import java.io.IOException;
import java.nio.file.Path;
/**
*
*/
public class FsSimpleTranslogTests extends AbstractSimpleTranslogTests {
@TestLogging("index.translog.fs:TRACE")
public class FsSimpleTranslogTests extends AbstractTranslogTests {
@Override
protected Translog create(Path translogDir) throws IOException {
protected FsTranslog create() throws IOException {
return new FsTranslog(shardId,
ImmutableSettings.settingsBuilder().put("index.translog.fs.type", FsTranslogFile.Type.SIMPLE.name()).build(),
translogDir);
BigArrays.NON_RECYCLING_INSTANCE, translogDir);
}
}
}

View File

@ -164,7 +164,7 @@ public class BackgroundIndexer implements AutoCloseable {
continue;
}
id = idGenerator.incrementAndGet();
client.prepareIndex(index, type, Long.toString(id) + "-" + indexerId).setSource(generateSource(id, threadRandom)).get();
client.prepareIndex(index, type, Long.toString(id)).setSource(generateSource(id, threadRandom)).get();
indexCounter.incrementAndGet();
}
}

View File

@ -60,7 +60,6 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.junit.*;
import org.junit.rules.RuleChain;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.file.FileSystem;
@ -125,7 +124,7 @@ public abstract class ElasticsearchTestCase extends LuceneTestCase {
}
/** called after a test is finished, but only if succesfull */
protected void afterIfSuccessful() {
protected void afterIfSuccessful() throws Exception {
}
// TODO: Parent/child and other things does not work with the query cache

View File

@ -21,14 +21,15 @@ package org.elasticsearch.test.engine;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.translog.fs.FsTranslog;
/**
*
*/
public final class MockEngineFactory implements EngineFactory {
@Override
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery) {
return new MockInternalEngine(config, skipTranslogRecovery);
public Engine newReadWriteEngine(EngineConfig config, FsTranslog translog, boolean skipTranslogRecovery) {
return new MockInternalEngine(config, translog, skipTranslogRecovery);
}
@Override

View File

@ -18,21 +18,20 @@
*/
package org.elasticsearch.test.engine;
import org.apache.lucene.search.AssertingIndexSearcher;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherManager;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.InternalEngine;
import org.elasticsearch.index.translog.fs.FsTranslog;
import java.io.IOException;
final class MockInternalEngine extends InternalEngine {
private MockEngineSupport support;
MockInternalEngine(EngineConfig config, boolean skipInitialTranslogRecovery) throws EngineException {
super(config, skipInitialTranslogRecovery);
MockInternalEngine(EngineConfig config, FsTranslog translog, boolean skipInitialTranslogRecovery) throws EngineException {
super(config, translog, skipInitialTranslogRecovery);
}
private synchronized MockEngineSupport support() {
@ -45,7 +44,7 @@ final class MockInternalEngine extends InternalEngine {
@Override
public void close() throws IOException {
switch(support().flushOrClose(this, MockEngineSupport.CloseAction.CLOSE)) {
switch (support().flushOrClose(this, MockEngineSupport.CloseAction.CLOSE)) {
case FLUSH_AND_CLOSE:
super.flushAndClose();
break;
@ -53,13 +52,11 @@ final class MockInternalEngine extends InternalEngine {
super.close();
break;
}
logger.debug("Ongoing recoveries after engine close: " + onGoingRecoveries.get());
}
@Override
public void flushAndClose() throws IOException {
switch(support().flushOrClose(this, MockEngineSupport.CloseAction.FLUSH_AND_CLOSE)) {
switch (support().flushOrClose(this, MockEngineSupport.CloseAction.FLUSH_AND_CLOSE)) {
case FLUSH_AND_CLOSE:
super.flushAndClose();
break;
@ -67,7 +64,6 @@ final class MockInternalEngine extends InternalEngine {
super.close();
break;
}
logger.debug("Ongoing recoveries after engine close: " + onGoingRecoveries.get());
}
@Override

View File

@ -22,7 +22,6 @@ package org.elasticsearch.test.store;
import com.carrotsearch.randomizedtesting.SeedUtils;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.google.common.base.Charsets;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.store.*;
@ -39,14 +38,14 @@ import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreModule;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Assert;
import java.io.Closeable;
@ -65,7 +64,7 @@ public class MockFSDirectoryService extends FsDirectoryService {
public static final String CRASH_INDEX = "index.store.mock.random.crash_index";
private static final EnumSet<IndexShardState> validCheckIndexStates = EnumSet.of(
IndexShardState.STARTED, IndexShardState.RELOCATED , IndexShardState.POST_RECOVERY
IndexShardState.STARTED, IndexShardState.RELOCATED, IndexShardState.POST_RECOVERY
);
private final FsDirectoryService delegateService;
@ -115,7 +114,7 @@ public class MockFSDirectoryService extends FsDirectoryService {
// so that even in tests where don't flush we can check the integrity of the Lucene index
if (indexShard.engine().hasUncommittedChanges()) { // only if we have any changes
logger.info("{} flushing in order to run checkindex", indexShard.shardId());
Releasables.close(indexShard.engine().snapshotIndex()); // Keep translog for tests that rely on replaying it
Releasables.close(indexShard.engine().snapshotIndex(true)); // Keep translog for tests that rely on replaying it
}
logger.info("{} flush finished in beforeIndexShardClosed", indexShard.shardId());
canRun = true;
@ -138,12 +137,11 @@ public class MockFSDirectoryService extends FsDirectoryService {
}
@Override
public Directory newDirectory() throws IOException {
return wrap(delegateService.newDirectory());
}
@Override
protected synchronized Directory newFSDirectory(Path location, LockFactory lockFactory) throws IOException {
throw new UnsupportedOperationException();
@ -170,8 +168,8 @@ public class MockFSDirectoryService extends FsDirectoryService {
if (!status.clean) {
ElasticsearchTestCase.checkIndexFailed = true;
logger.warn("check index [failure] index files={}\n{}",
Arrays.toString(dir.listAll()),
new String(os.bytes().toBytes(), Charsets.UTF_8));
Arrays.toString(dir.listAll()),
new String(os.bytes().toBytes(), Charsets.UTF_8));
throw new IndexShardException(shardId, "index check failure");
} else {
if (logger.isDebugEnabled()) {