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 How often to check if a flush is needed, randomized
between the interval value and 2x the interval value. Defaults to `5s`. 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`. 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); 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. * 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(); return this.refCount.get();
} }
/** gets the name of this instance */
public String getName() {
return name;
}
protected abstract void closeInternal(); protected abstract void closeInternal();
} }

View File

@ -30,17 +30,48 @@ import java.util.concurrent.locks.Lock;
public class ReleasableLock implements Releasable { public class ReleasableLock implements Releasable {
private final Lock lock; 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) { public ReleasableLock(Lock lock) {
this.lock = lock; this.lock = lock;
boolean useHoldingThreads = false;
assert (useHoldingThreads = true);
if (useHoldingThreads) {
holdingThreads = new ThreadLocal<>();
} else {
holdingThreads = null;
}
} }
@Override @Override
public void close() { public void close() {
lock.unlock(); lock.unlock();
assert removeCurrentThread();
} }
public ReleasableLock acquire() throws EngineException { public ReleasableLock acquire() throws EngineException {
lock.lock(); lock.lock();
assert addCurrentThread();
return this; 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.store.StoreModule;
import org.elasticsearch.index.suggest.SuggestShardModule; import org.elasticsearch.index.suggest.SuggestShardModule;
import org.elasticsearch.index.termvectors.ShardTermVectorsModule; 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.index.translog.TranslogService;
import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
@ -187,6 +185,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
} }
return null; return null;
} }
/** /**
* Return the shard with the provided id, or throw an exception if it doesn't exist. * 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 ShardQueryCacheModule());
modules.add(new ShardBitsetFilterCacheModule()); modules.add(new ShardBitsetFilterCacheModule());
modules.add(new ShardFieldDataModule()); modules.add(new ShardFieldDataModule());
modules.add(new TranslogModule(indexSettings));
modules.add(new IndexShardGatewayModule()); modules.add(new IndexShardGatewayModule());
modules.add(new PercolatorShardModule()); modules.add(new PercolatorShardModule());
modules.add(new ShardTermVectorsModule()); 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 // 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 // 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 // and close the shard so no operations are allowed to it
if (indexShard != null) { if (indexShard != null) {
@ -402,7 +401,6 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
MergeSchedulerProvider.class, MergeSchedulerProvider.class,
MergePolicyProvider.class, MergePolicyProvider.class,
IndexShardGatewayService.class, IndexShardGatewayService.class,
Translog.class,
PercolatorQueriesRegistry.class); PercolatorQueriesRegistry.class);
// call this before we close the store, so we can release resources for it // 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) { private void closeInjectorResource(ShardId shardId, Injector shardInjector, Class<? extends Closeable>... toClose) {
for (Class<? extends Closeable> closeable : toClose) { for (Class<? extends Closeable> closeable : toClose) {
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 { try {
final Closeable instance = shardInjector.getInstance(closeable); final Closeable instance = shardInjector.getInstance(toClose);
if (instance == null) { if (instance == null) {
throw new NullPointerException("No instance available for " + closeable.getName()); return false;
} }
IOUtils.close(instance); IOUtils.close(instance);
} catch (Throwable t) { } catch (Throwable t) {
logger.debug("{} failed to close {}", t, shardId, Strings.toUnderscoreCase(closeable.getSimpleName())); logger.debug("{} failed to close {}", t, shardId, Strings.toUnderscoreCase(toClose.getSimpleName()));
}
} }
return true;
} }
private void onShardClose(ShardLock lock, boolean ownsShard) { private void onShardClose(ShardLock lock, boolean ownsShard) {
if (deleted.get()) { // we remove that shards content if this index has been deleted if (deleted.get()) { // we remove that shards content if this index has been deleted
try { try {
@ -464,7 +474,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Override @Override
public void handle(ShardLock lock) { 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); onShardClose(lock, ownsShard);
} }
} }

View File

@ -82,7 +82,6 @@ public abstract class Engine implements Closeable {
protected Engine(EngineConfig engineConfig) { protected Engine(EngineConfig engineConfig) {
Preconditions.checkNotNull(engineConfig.getStore(), "Store must be provided to the engine"); 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.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.engineConfig = engineConfig;
this.shardId = engineConfig.getShardId(); 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() { protected void ensureOpen() {
if (isClosed.get()) { if (isClosed.get()) {
throw new EngineClosedException(shardId, failedEngine); 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; 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. * 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 SnapshotIndexCommit snapshotIndex(boolean flushFirst) throws EngineException;
public abstract void recover(RecoveryHandler recoveryHandler) throws EngineException;
/** fail engine due to some error. the engine will also be closed. */ /** fail engine due to some error. the engine will also be closed. */
public void failEngine(String reason, Throwable failure) { public void failEngine(String reason, Throwable failure) {
@ -1048,7 +1050,7 @@ public abstract class Engine implements Closeable {
public void flushAndClose() throws IOException { public void flushAndClose() throws IOException {
if (isClosed.get() == false) { if (isClosed.get() == false) {
logger.trace("flushAndClose now acquire writeLock"); logger.trace("flushAndClose now acquire writeLock");
try (ReleasableLock _ = writeLock.acquire()) { try (ReleasableLock lock = writeLock.acquire()) {
logger.trace("flushAndClose now acquired writeLock"); logger.trace("flushAndClose now acquired writeLock");
try { try {
logger.debug("flushing shard on close - this might take some time to sync files to disk"); 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 { public void close() throws IOException {
if (isClosed.get() == false) { // don't acquire the write lock if we are already closed if (isClosed.get() == false) { // don't acquire the write lock if we are already closed
logger.debug("close now acquiring writeLock"); logger.debug("close now acquiring writeLock");
try (ReleasableLock _ = writeLock.acquire()) { try (ReleasableLock lock = writeLock.acquire()) {
logger.debug("close acquired writeLock"); logger.debug("close acquired writeLock");
closeNoLock("api"); 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.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -67,13 +66,13 @@ public final class EngineConfig {
private final IndicesWarmer warmer; private final IndicesWarmer warmer;
private final Store store; private final Store store;
private final SnapshotDeletionPolicy deletionPolicy; private final SnapshotDeletionPolicy deletionPolicy;
private final Translog translog;
private final MergePolicyProvider mergePolicyProvider; private final MergePolicyProvider mergePolicyProvider;
private final MergeSchedulerProvider mergeScheduler; private final MergeSchedulerProvider mergeScheduler;
private final Analyzer analyzer; private final Analyzer analyzer;
private final Similarity similarity; private final Similarity similarity;
private final CodecService codecService; private final CodecService codecService;
private final Engine.FailedEngineListener failedEngineListener; private final Engine.FailedEngineListener failedEngineListener;
private final boolean ignoreUnknownTranslog;
/** /**
* Index setting for index concurrency / number of threadstates in the indexwriter. * 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"; 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_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60); public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB); 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} * 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.shardId = shardId;
this.threadPool = threadPool; this.threadPool = threadPool;
this.indexingService = indexingService; this.indexingService = indexingService;
@ -138,7 +142,6 @@ public final class EngineConfig {
this.warmer = warmer; this.warmer = warmer;
this.store = store; this.store = store;
this.deletionPolicy = deletionPolicy; this.deletionPolicy = deletionPolicy;
this.translog = translog;
this.mergePolicyProvider = mergePolicyProvider; this.mergePolicyProvider = mergePolicyProvider;
this.mergeScheduler = mergeScheduler; this.mergeScheduler = mergeScheduler;
this.analyzer = analyzer; this.analyzer = analyzer;
@ -155,6 +158,7 @@ public final class EngineConfig {
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE); versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize(); updateVersionMapSize();
this.translogRecoveryPerformer = translogRecoveryPerformer; this.translogRecoveryPerformer = translogRecoveryPerformer;
this.ignoreUnknownTranslog = indexSettings.getAsBoolean(INDEX_IGNORE_UNKNOWN_TRANSLOG, false);
} }
/** updates {@link #versionMapSize} based on current setting and {@link #indexingBufferSize} */ /** updates {@link #versionMapSize} based on current setting and {@link #indexingBufferSize} */
@ -182,6 +186,10 @@ public final class EngineConfig {
return versionMapSizeSetting; 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 * returns the size of the version map that should trigger a refresh
@ -318,13 +326,6 @@ public final class EngineConfig {
return deletionPolicy; return deletionPolicy;
} }
/**
* Returns a {@link Translog instance}
*/
public Translog getTranslog() {
return translog;
}
/** /**
* Returns the {@link org.elasticsearch.index.merge.policy.MergePolicyProvider} used to obtain * 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} * 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; package org.elasticsearch.index.engine;
import org.elasticsearch.index.translog.fs.FsTranslog;
/** /**
* Simple Engine Factory * Simple Engine Factory
*/ */
public interface EngineFactory { public interface EngineFactory {
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery); public Engine newReadWriteEngine(EngineConfig config, FsTranslog translog, boolean skipTranslogRecovery);
public Engine newReadOnlyEngine(EngineConfig config); public Engine newReadOnlyEngine(EngineConfig config);
} }

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.engine; package org.elasticsearch.index.engine;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.lucene.index.*; import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer; import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
@ -31,9 +32,7 @@ import org.apache.lucene.util.InfoStream;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.routing.DjbHashFunction; import org.elasticsearch.cluster.routing.DjbHashFunction;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.LoggerInfoStream;
import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; 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.search.nested.IncludeNestedDocsQuery;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.translog.Translog; 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.indices.IndicesWarmer;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -61,7 +60,6 @@ import java.io.IOException;
import java.util.*; import java.util.*;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
@ -79,7 +77,7 @@ public class InternalEngine extends Engine {
private final ShardIndexingService indexingService; private final ShardIndexingService indexingService;
@Nullable @Nullable
private final IndicesWarmer warmer; private final IndicesWarmer warmer;
private final Translog translog; private final FsTranslog translog;
private final MergePolicyProvider mergePolicyProvider; private final MergePolicyProvider mergePolicyProvider;
private final MergeSchedulerProvider mergeScheduler; private final MergeSchedulerProvider mergeScheduler;
@ -94,33 +92,31 @@ public class InternalEngine extends Engine {
private final Lock flushLock = new ReentrantLock(); private final Lock flushLock = new ReentrantLock();
private final ReentrantLock optimizeLock = new ReentrantLock(); private final ReentrantLock optimizeLock = new ReentrantLock();
protected final FlushingRecoveryCounter onGoingRecoveries;
// A uid (in the form of BytesRef) to the version map // 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 // we use the hashed variant since we iterate over it and check removal and additions on existing keys
private final LiveVersionMap versionMap; private final LiveVersionMap versionMap;
private final Object[] dirtyLocks; private final Object[] dirtyLocks;
private final AtomicLong translogIdGenerator = new AtomicLong();
private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean(); private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean();
private volatile SegmentInfos lastCommittedSegmentInfos; private volatile SegmentInfos lastCommittedSegmentInfos;
private final IndexThrottle throttle; private final IndexThrottle throttle;
public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogRecovery) throws EngineException { public InternalEngine(EngineConfig engineConfig, FsTranslog translog, boolean skipInitialTranslogRecovery) throws EngineException {
super(engineConfig); super(engineConfig);
Preconditions.checkNotNull(translog, "Translog must be provided to the engine");
this.versionMap = new LiveVersionMap(); this.versionMap = new LiveVersionMap();
store.incRef(); store.incRef();
IndexWriter writer = null; IndexWriter writer = null;
SearcherManager manager = null; SearcherManager manager = null;
boolean success = false; boolean success = false;
try { try {
this.onGoingRecoveries = new FlushingRecoveryCounter(this, store, logger);
this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis(); this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis();
this.indexingService = engineConfig.getIndexingService(); this.indexingService = engineConfig.getIndexingService();
this.warmer = engineConfig.getWarmer(); this.warmer = engineConfig.getWarmer();
this.translog = engineConfig.getTranslog(); this.translog = translog;
this.mergePolicyProvider = engineConfig.getMergePolicyProvider(); this.mergePolicyProvider = engineConfig.getMergePolicyProvider();
this.mergeScheduler = engineConfig.getMergeScheduler(); this.mergeScheduler = engineConfig.getMergeScheduler();
this.dirtyLocks = new Object[engineConfig.getIndexConcurrency() * 50]; // we multiply it to have enough... 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(); throttle = new IndexThrottle();
this.searcherFactory = new SearchFactory(engineConfig); this.searcherFactory = new SearchFactory(engineConfig);
final Tuple<Long, Long> translogId; // nextTranslogId, currentTranslogId final Long committedTranslogId;
try { try {
writer = createWriter(); writer = createWriter();
indexWriter = writer; indexWriter = writer;
translogId = loadTranslogIds(writer, translog); committedTranslogId = loadCommittedTranslogId(writer, translog);
} catch (IOException e) { } catch (IOException e) {
throw new EngineCreationFailureException(shardId, "failed to create engine", e); throw new EngineCreationFailureException(shardId, "failed to create engine", e);
} }
@ -145,15 +141,12 @@ public class InternalEngine extends Engine {
this.mergeSchedulerListener = new MergeSchedulerListener(); this.mergeSchedulerListener = new MergeSchedulerListener();
this.mergeScheduler.addListener(mergeSchedulerListener); this.mergeScheduler.addListener(mergeSchedulerListener);
this.mergeScheduler.addFailureListener(mergeSchedulerFailureListener); this.mergeScheduler.addFailureListener(mergeSchedulerFailureListener);
final TranslogRecoveryPerformer transformer = engineConfig.getTranslogRecoveryPerformer();
try { try {
long nextTranslogID = translogId.v2(); if (skipInitialTranslogRecovery) {
translog.newTranslog(nextTranslogID); // make sure we point at the latest translog from now on..
translogIdGenerator.set(nextTranslogID); commitIndexWriter(writer, translog.currentId());
if (translogId.v1() != null && skipInitialTranslogRecovery == false) {
recoverFromTranslog(translogId.v1(), transformer);
} else { } else {
flush(true, true); recoverFromTranslog(engineConfig, committedTranslogId);
} }
} catch (IOException | EngineException ex) { } catch (IOException | EngineException ex) {
throw new EngineCreationFailureException(shardId, "failed to recover from translog", 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"); 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) * Reads the current stored translog ID from the IW commit data. If the id is not found, recommits the current
* from the largest present translog ID. If there is no stored translog ID v1 is <code>null</code> * 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 // 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 // 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(); final Map<String, String> commitUserData = writer.getCommitData();
if (commitUserData.containsKey(Translog.TRANSLOG_ID_KEY)) { if (commitUserData.containsKey(Translog.TRANSLOG_ID_KEY)) {
final long currentTranslogId = Long.parseLong(commitUserData.get(Translog.TRANSLOG_ID_KEY)); return Long.parseLong(commitUserData.get(Translog.TRANSLOG_ID_KEY));
return new Tuple<>(currentTranslogId, nextTranslogId);
} }
// 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"); 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 { 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 { private void flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) throws EngineException {
ensureOpen(); 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 * 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: * 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 { try {
if (commitTranslog) { if (commitTranslog) {
if (onGoingRecoveries.get() > 0) {
throw new FlushNotAllowedEngineException(shardId, "Recovery is in progress, flush is not allowed");
}
if (flushNeeded || force) { if (flushNeeded || force) {
flushNeeded = false; flushNeeded = false;
final long translogId;
try { try {
long translogId = translogIdGenerator.incrementAndGet(); translogId = translog.newTranslog();
translog.newTransientTranslog(translogId);
indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)));
logger.trace("starting commit for flush; commitTranslog=true"); logger.trace("starting commit for flush; commitTranslog=true");
commitIndexWriter(indexWriter); commitIndexWriter(indexWriter, translogId);
logger.trace("finished commit for flush"); logger.trace("finished commit for flush");
// we need to refresh in order to clear older version values // we need to refresh in order to clear older version values
refresh("version_table_flush"); refresh("version_table_flush");
// we need to move transient to current only after we refresh translog.markCommitted(translogId);
// so items added to current will still be around for realtime get
// when tans overrides it
translog.makeTransientCurrent();
} catch (Throwable e) { } catch (Throwable e) {
try {
translog.revertTransient();
} catch (IOException ex) {
e.addSuppressed(ex);
}
throw new FlushFailedEngineException(shardId, e); 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 // its ok to use this, only a flush will cause a new translogId, and we are locked here from
// other flushes use flushLock // other flushes use flushLock
try { try {
long translogId = translog.currentId();
indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)));
logger.trace("starting commit for flush; commitTranslog=false"); logger.trace("starting commit for flush; commitTranslog=false");
commitIndexWriter(indexWriter); commitIndexWriter(indexWriter, translog.currentId());
logger.trace("finished commit for flush"); logger.trace("finished commit for flush");
} catch (Throwable e) { } catch (Throwable e) {
throw new FlushFailedEngineException(shardId, e); throw new FlushFailedEngineException(shardId, e);
@ -807,12 +826,14 @@ public class InternalEngine extends Engine {
} }
@Override @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 // 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 // the to a write lock when we fail the engine in this operation
if (flushFirst) {
logger.trace("start flush for snapshot"); logger.trace("start flush for snapshot");
flush(false, false, true); flush(false, false, true);
logger.trace("finish flush for snapshot"); logger.trace("finish flush for snapshot");
}
try (ReleasableLock lock = readLock.acquire()) { try (ReleasableLock lock = readLock.acquire()) {
ensureOpen(); ensureOpen();
logger.trace("pulling snapshot"); 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 @Override
protected boolean maybeFailEngine(String source, Throwable t) { protected boolean maybeFailEngine(String source, Throwable t) {
boolean shouldFail = super.maybeFailEngine(source, t); boolean shouldFail = super.maybeFailEngine(source, t);
@ -950,11 +912,6 @@ public class InternalEngine extends Engine {
if (isClosed.compareAndSet(false, true)) { 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"; assert rwl.isWriteLockedByCurrentThread() || failEngineLock.isHeldByCurrentThread() : "Either the write lock must be held or the engine must be currently be failing itself";
try { try {
try {
IOUtils.close(this.translog);
} catch (IOException ex) {
logger.warn("failed to close translog", ex);
}
this.versionMap.clear(); this.versionMap.clear();
logger.trace("close searcherManager"); logger.trace("close searcherManager");
try { try {
@ -962,6 +919,11 @@ public class InternalEngine extends Engine {
} catch (Throwable t) { } catch (Throwable t) {
logger.warn("Failed to close SearcherManager", 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 // no need to commit in this case!, we snapshot before we close the shard, so translog and all sync'ed
logger.trace("rollback indexWriter"); logger.trace("rollback indexWriter");
try { 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 { try {
logger.trace("committing writer with translog id [{}] ", translogId);
indexWriter.setCommitData(Collections.singletonMap(Translog.TRANSLOG_ID_KEY, Long.toString(translogId)));
writer.commit(); writer.commit();
} catch (Throwable ex) { } catch (Throwable ex) {
failEngine("lucene commit failed", ex); failEngine("lucene commit failed", ex);
throw 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; package org.elasticsearch.index.engine;
import org.elasticsearch.index.translog.fs.FsTranslog;
public class InternalEngineFactory implements EngineFactory { public class InternalEngineFactory implements EngineFactory {
@Override @Override
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery) { public Engine newReadWriteEngine(EngineConfig config, FsTranslog translog, boolean skipTranslogRecovery) {
return new InternalEngine(config, skipTranslogRecovery); return new InternalEngine(config, translog, skipTranslogRecovery);
} }
@Override @Override

View File

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

View File

@ -30,14 +30,12 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.CancellableThreads;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -47,7 +45,6 @@ import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -61,9 +58,7 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl
private final IndexService indexService; private final IndexService indexService;
private final IndexShard indexShard; private final IndexShard indexShard;
private final TimeValue waitForMappingUpdatePostRecovery; private final TimeValue waitForMappingUpdatePostRecovery;
private final TimeValue syncInterval;
private volatile ScheduledFuture<?> flushScheduler;
private final CancellableThreads cancellableThreads = new CancellableThreads(); private final CancellableThreads cancellableThreads = new CancellableThreads();
@ -76,17 +71,7 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl
this.indexService = indexService; this.indexService = indexService;
this.indexShard = indexShard; this.indexShard = indexShard;
this.waitForMappingUpdatePostRecovery = indexSettings.getAsTime("index.gateway.wait_for_mapping_update_post_recovery", TimeValue.timeValueMinutes(15)); this.waitForMappingUpdatePostRecovery = indexSettings.getAsTime("index.gateway.wait_for_mapping_update_post_recovery", TimeValue.timeValueSeconds(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;
}
} }
/** /**
@ -198,39 +183,9 @@ public class IndexShardGateway extends AbstractIndexShardComponent implements Cl
@Override @Override
public void close() { public void close() {
FutureUtils.cancel(flushScheduler);
cancellableThreads.cancel("closed"); 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 @Override
public String toString() { public String toString() {
return "shard_gateway"; return "shard_gateway";

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.shard; package org.elasticsearch.index.shard;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex; 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.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.MetaDataStateFormat; 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.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.*;
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.fielddata.FieldDataStats; import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.fielddata.ShardFieldData; 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.get.ShardGetService;
import org.elasticsearch.index.indexing.IndexingStats; import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.*;
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.internal.ParentFieldMapper; import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.merge.policy.MergePolicyProvider; 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.termvectors.ShardTermVectorsService;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.index.translog.fs.FsTranslog;
import org.elasticsearch.index.warmer.ShardIndexWarmerService; import org.elasticsearch.index.warmer.ShardIndexWarmerService;
import org.elasticsearch.index.warmer.WarmerStats; import org.elasticsearch.index.warmer.WarmerStats;
import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesLifecycle;
@ -145,7 +134,6 @@ public class IndexShard extends AbstractIndexShardComponent {
private final InternalIndicesLifecycle indicesLifecycle; private final InternalIndicesLifecycle indicesLifecycle;
private final Store store; private final Store store;
private final MergeSchedulerProvider mergeScheduler; private final MergeSchedulerProvider mergeScheduler;
private final Translog translog;
private final IndexAliasesService indexAliasesService; private final IndexAliasesService indexAliasesService;
private final ShardIndexingService indexingService; private final ShardIndexingService indexingService;
private final ShardSearchService searchService; private final ShardSearchService searchService;
@ -171,6 +159,7 @@ public class IndexShard extends AbstractIndexShardComponent {
private final SnapshotDeletionPolicy deletionPolicy; private final SnapshotDeletionPolicy deletionPolicy;
private final SimilarityService similarityService; private final SimilarityService similarityService;
private final MergePolicyProvider mergePolicyProvider; private final MergePolicyProvider mergePolicyProvider;
private final BigArrays bigArrays;
private final EngineConfig engineConfig; private final EngineConfig engineConfig;
private TimeValue refreshInterval; private TimeValue refreshInterval;
@ -205,27 +194,26 @@ public class IndexShard extends AbstractIndexShardComponent {
private final ShardPath path; private final ShardPath path;
@Inject @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, 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, ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache,
@Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory, @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()); super(shardId, indexSettingsService.getSettings());
this.codecService = codecService; this.codecService = codecService;
this.warmer = warmer; this.warmer = warmer;
this.deletionPolicy = deletionPolicy; this.deletionPolicy = deletionPolicy;
this.similarityService = similarityService; this.similarityService = similarityService;
this.mergePolicyProvider = mergePolicyProvider; this.mergePolicyProvider = mergePolicyProvider;
this.bigArrays = bigArrays;
Preconditions.checkNotNull(store, "Store must be provided to the index shard"); 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(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.engineFactory = factory;
this.indicesLifecycle = (InternalIndicesLifecycle) indicesLifecycle; this.indicesLifecycle = (InternalIndicesLifecycle) indicesLifecycle;
this.indexSettingsService = indexSettingsService; this.indexSettingsService = indexSettingsService;
this.store = store; this.store = store;
this.mergeScheduler = mergeScheduler; this.mergeScheduler = mergeScheduler;
this.translog = translog;
this.threadPool = threadPool; this.threadPool = threadPool;
this.mapperService = mapperService; this.mapperService = mapperService;
this.queryParserService = queryParserService; this.queryParserService = queryParserService;
@ -266,8 +254,13 @@ public class IndexShard extends AbstractIndexShardComponent {
return this.store; return this.store;
} }
/** returns true if this shard supports indexing (i.e., write) operations. */
public boolean canIndex() {
return true;
}
public Translog translog() { public Translog translog() {
return translog; return engine().translog();
} }
public ShardIndexingService indexingService() { public ShardIndexingService indexingService() {
@ -662,7 +655,7 @@ public class IndexShard extends AbstractIndexShardComponent {
} }
public TranslogStats translogStats() { public TranslogStats translogStats() {
return translog.stats(); return engine().translog().stats();
} }
public SuggestStats suggestStats() { public SuggestStats suggestStats() {
@ -706,21 +699,16 @@ public class IndexShard extends AbstractIndexShardComponent {
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 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 // 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) { if (state == IndexShardState.STARTED || state == IndexShardState.RELOCATED || state == IndexShardState.CLOSED) {
return engine().snapshotIndex(); return engine().snapshotIndex(flushFirst);
} else { } else {
throw new IllegalIndexShardStateException(shardId, state, "snapshot is not allowed"); 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) { 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. // fail the engine. This will cause this shard to also be removed from the node's index service.
engine().failEngine(reason, e); engine().failEngine(reason, e);
@ -817,7 +805,6 @@ public class IndexShard extends AbstractIndexShardComponent {
if (Booleans.parseBoolean(checkIndexOnStartup, false)) { if (Booleans.parseBoolean(checkIndexOnStartup, false)) {
checkIndex(); checkIndex();
} }
recoveryState.setStage(RecoveryState.Stage.TRANSLOG); recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
// we disable deletes since we allow for operations to be executed against the shard while recovering // 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 // but we need to make sure we don't loose deletes until we are done recovering
@ -869,8 +856,6 @@ public class IndexShard extends AbstractIndexShardComponent {
*/ */
public void finalizeRecovery() { public void finalizeRecovery() {
recoveryState().setStage(RecoveryState.Stage.FINALIZE); recoveryState().setStage(RecoveryState.Stage.FINALIZE);
// clear unreferenced files
translog.clearUnreferenced();
engine().refresh("recovery_finalization"); engine().refresh("recovery_finalization");
startScheduledTasksIfNeeded(); startScheduledTasksIfNeeded();
engineConfig.setEnableGcDeletes(true); engineConfig.setEnableGcDeletes(true);
@ -977,7 +962,10 @@ public class IndexShard extends AbstractIndexShardComponent {
logger.debug("updating index_buffer_size from [{}] to [{}]", preValue, shardIndexingBufferSize); 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() { public void markAsInactive() {
@ -999,6 +987,7 @@ public class IndexShard extends AbstractIndexShardComponent {
} }
MetaDataStateFormat.deleteMetaState(shardPath().getDataPath()); MetaDataStateFormat.deleteMetaState(shardPath().getDataPath());
} }
public ShardPath shardPath() { public ShardPath shardPath() {
return path; return path;
} }
@ -1229,9 +1218,22 @@ public class IndexShard extends AbstractIndexShardComponent {
} }
protected Engine newEngine(boolean skipTranslogRecovery, EngineConfig config) { 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> * 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, 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); mapperAnalyzer, similarityService.similarity(), codecService, failedEngineListener, translogRecoveryPerformer);
} }
} }

View File

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

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.translog.TranslogService;
import org.elasticsearch.index.warmer.ShardIndexWarmerService; import org.elasticsearch.index.warmer.ShardIndexWarmerService;
/** /**
@ -47,6 +48,9 @@ public class IndexShardModule extends AbstractModule {
this.settings = settings; this.settings = settings;
this.shardId = shardId; this.shardId = shardId;
this.primary = primary; 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 */ /** 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(); bind(IndexShard.class).to(ShadowIndexShard.class).asEagerSingleton();
} else { } else {
bind(IndexShard.class).asEagerSingleton(); bind(IndexShard.class).asEagerSingleton();
bind(TranslogService.class).asEagerSingleton();
} }
bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, ENGINE_PREFIX, ENGINE_SUFFIX)); 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.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.aliases.IndexAliasesService; import org.elasticsearch.index.aliases.IndexAliasesService;
@ -56,6 +57,8 @@ import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
/** /**
* ShadowIndexShard extends {@link IndexShard} to add file synchronization * ShadowIndexShard extends {@link IndexShard} to add file synchronization
* from the primary when a flush happens. It also ensures that a replica being * 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 @Inject
public ShadowIndexShard(ShardId shardId, IndexSettingsService indexSettingsService, public ShadowIndexShard(ShardId shardId, IndexSettingsService indexSettingsService,
IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler,
Translog translog, ThreadPool threadPool, MapperService mapperService, ThreadPool threadPool, MapperService mapperService,
IndexQueryParserService queryParserService, IndexCache indexCache, IndexQueryParserService queryParserService, IndexCache indexCache,
IndexAliasesService indexAliasesService, ShardIndexingService indexingService, IndexAliasesService indexAliasesService, ShardIndexingService indexingService,
ShardGetService getService, ShardSearchService searchService, ShardGetService getService, ShardSearchService searchService,
@ -78,14 +81,15 @@ public final class ShadowIndexShard extends IndexShard {
IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache,
ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer, ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer,
SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, 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, super(shardId, indexSettingsService, indicesLifecycle, store, mergeScheduler,
translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService, threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indexingService, getService, searchService, shardWarmerService, shardFilterCache, indexingService, getService, searchService, shardWarmerService, shardFilterCache,
shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService, shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService,
termVectorsService, indexFieldDataService, indexService, shardSuggestService, termVectorsService, indexFieldDataService, indexService, shardSuggestService,
shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService, 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); super.updateRoutingEntry(newRouting, persistState);
} }
@Override
public boolean canIndex() {
return false;
}
@Override @Override
protected Engine newEngine(boolean skipInitialTranslogRecovery, EngineConfig config) { protected Engine newEngine(boolean skipInitialTranslogRecovery, EngineConfig config) {
assert this.shardRouting.primary() == false; assert this.shardRouting.primary() == false;
@ -112,4 +121,10 @@ public final class ShadowIndexShard extends IndexShard {
public boolean allowsPrimaryPromotion() { public boolean allowsPrimaryPromotion() {
return false; 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 { try {
SnapshotIndexCommit snapshotIndexCommit = indexShard.snapshotIndex(); // we flush first to make sure we get the latest writes snapshotted
SnapshotIndexCommit snapshotIndexCommit = indexShard.snapshotIndex(true);
try { try {
indexShardRepository.snapshot(snapshotId, shardId, snapshotIndexCommit, snapshotStatus); indexShardRepository.snapshot(snapshotId, shardId, snapshotIndexCommit, snapshotStatus);
if (logger.isDebugEnabled()) { 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.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput; import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.IOUtils; 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.Channels;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.nio.file.Files; import java.nio.file.Files;
@ -65,12 +70,12 @@ public class ChecksummedTranslogStream implements TranslogStream {
Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte()); Translog.Operation.Type type = Translog.Operation.Type.fromId(in.readByte());
operation = TranslogStreams.newOperationFromType(type); operation = TranslogStreams.newOperationFromType(type);
operation.readFrom(in); operation.readFrom(in);
verifyChecksum(in);
} catch (EOFException e) { } catch (EOFException e) {
throw new TruncatedTranslogException("reached premature end of file, translog is truncated", e); throw new TruncatedTranslogException("reached premature end of file, translog is truncated", e);
} catch (AssertionError|Exception e) { } catch (AssertionError|Exception e) {
throw new TranslogCorruptedException("translog corruption while reading from stream", e); throw new TranslogCorruptedException("translog corruption while reading from stream", e);
} }
verifyChecksum(in);
return operation; return operation;
} }
@ -103,6 +108,11 @@ public class ChecksummedTranslogStream implements TranslogStream {
// closing it will close the FileChannel // closing it will close the FileChannel
OutputStreamDataOutput out = new OutputStreamDataOutput(Channels.newOutputStream(channel)); OutputStreamDataOutput out = new OutputStreamDataOutput(Channels.newOutputStream(channel));
CodecUtil.writeHeader(out, TranslogStreams.TRANSLOG_CODEC, VERSION); CodecUtil.writeHeader(out, TranslogStreams.TRANSLOG_CODEC, VERSION);
return headerLength();
}
@Override
public int headerLength() {
return CodecUtil.headerLength(TranslogStreams.TRANSLOG_CODEC); 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.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; 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.io.IOException;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.nio.file.Files; import java.nio.file.Files;
@ -41,6 +38,10 @@ public class LegacyTranslogStream implements TranslogStream {
@Override @Override
public Translog.Operation read(StreamInput in) throws IOException { 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.Type type = Translog.Operation.Type.fromId(in.readByte());
Translog.Operation operation = TranslogStreams.newOperationFromType(type); Translog.Operation operation = TranslogStreams.newOperationFromType(type);
operation.readFrom(in); operation.readFrom(in);
@ -49,8 +50,7 @@ public class LegacyTranslogStream implements TranslogStream {
@Override @Override
public void write(StreamOutput out, Translog.Operation op) throws IOException { public void write(StreamOutput out, Translog.Operation op) throws IOException {
out.writeByte(op.opType().id()); throw new UnsupportedOperationException("LegacyTranslogStream is depracated. Use TranslogStreams.LATEST");
op.writeTo(out);
} }
@Override @Override
@ -59,6 +59,11 @@ public class LegacyTranslogStream implements TranslogStream {
return 0; return 0;
} }
@Override
public int headerLength() {
return 0;
}
@Override @Override
public StreamInput openInput(Path translogFile) throws IOException { public StreamInput openInput(Path translogFile) throws IOException {
// nothing to do, legacy translogs have no header // 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.engine.Engine;
import org.elasticsearch.index.shard.IndexShardComponent; import org.elasticsearch.index.shard.IndexShardComponent;
import java.io.Closeable; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; 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"); static ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb");
@ -61,42 +61,21 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
long currentId(); 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. * Creates a new transaction log file internally. That new file will be visible to all outstanding views.
* <p/> * The id of the new translog file is returned.
* <p>Can only be called by one thread.
* @param id the translog id for the new translog
*/ */
void newTranslog(long id) throws TranslogException, IOException; long newTranslog() 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;
/** /**
* Adds a create operation to the transaction log. * 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 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 * Returns a view into the current translog that is guaranteed to retain all current operations
* by the provided snapshot. If a new translog has been created after the provided snapshot * while receiving future ones as well
* has been take, will return a snapshot on the current trasnlog.
*/ */
Snapshot snapshot(Snapshot snapshot); View newView();
/**
* Clears unreferenced transaction logs.
*
* @return the number of clean up files
*/
int clearUnreferenced();
/** /**
* Sync's the translog. * Sync's the translog.
@ -140,36 +112,19 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
*/ */
public Path location(); public Path location();
/**
* Returns the translog filename for the given id.
*/
String getFilename(long translogId);
/** /**
* return stats * return stats
*/ */
TranslogStats 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 { static class Location implements Accountable {
@ -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. * A snapshot of the transaction log, allows to iterate over all the transaction log operations.
*/ */
static interface Snapshot extends OperationIterator { static interface Snapshot extends Releasable {
/**
* 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();
/** /**
* The total number of operations in the translog. * The total number of operations in the translog.
@ -225,14 +165,31 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
int estimatedTotalOperations(); 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(); long estimateSize();
Source getSource(); Source getSource();
} }
static class Source { static class Source {
@ -435,6 +393,57 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeLong(ttl); out.writeLong(ttl);
out.writeByte(versionType.getValue()); 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 { static class Index implements Operation {
@ -581,6 +590,55 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeLong(ttl); out.writeLong(ttl);
out.writeByte(versionType.getValue()); 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 { static class Delete implements Operation {
@ -658,6 +716,37 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
out.writeLong(version); out.writeLong(version);
out.writeByte(versionType.getValue()); 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. */ /** @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); 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.engine.FlushNotAllowedEngineException;
import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.*;
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.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable; import java.io.Closeable;
@ -57,7 +53,7 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
private final ThreadPool threadPool; private final ThreadPool threadPool;
private final IndexSettingsService indexSettingsService; private final IndexSettingsService indexSettingsService;
private final IndexShard indexShard; private final IndexShard indexShard;
private final Translog translog; private volatile Translog translog;
private volatile TimeValue interval; private volatile TimeValue interval;
private volatile int flushThresholdOperations; private volatile int flushThresholdOperations;
@ -69,12 +65,11 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
private final ApplySettings applySettings = new ApplySettings(); private final ApplySettings applySettings = new ApplySettings();
@Inject @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); super(shardId, indexSettings);
this.threadPool = threadPool; this.threadPool = threadPool;
this.indexSettingsService = indexSettingsService; this.indexSettingsService = indexSettingsService;
this.indexShard = indexShard; 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.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.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)); 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); indexSettingsService.addListener(applySettings);
} }
@Override @Override
public void close() { public void close() {
indexSettingsService.removeListener(applySettings); indexSettingsService.removeListener(applySettings);
@ -148,12 +142,12 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
return; return;
} }
if (indexShard.state() == IndexShardState.CREATED) { if (indexShard.translog() == null) {
reschedule(); reschedule();
return; return;
} }
int currentNumberOfOperations = translog.estimatedNumberOfOperations(); int currentNumberOfOperations = translog.totalOperations();
if (currentNumberOfOperations == 0) { if (currentNumberOfOperations == 0) {
reschedule(); reschedule();
return; return;
@ -168,7 +162,7 @@ public class TranslogService extends AbstractIndexShardComponent implements Clos
} }
if (flushThresholdSize.bytes() > 0) { if (flushThresholdSize.bytes() > 0) {
long sizeInBytes = translog.translogSizeInBytes(); long sizeInBytes = translog.sizeInBytes();
if (sizeInBytes > flushThresholdSize.bytes()) { if (sizeInBytes > flushThresholdSize.bytes()) {
logger.trace("flushing translog, size [{}], breached [{}]", new ByteSizeValue(sizeInBytes), flushThresholdSize); logger.trace("flushing translog, size [{}], breached [{}]", new ByteSizeValue(sizeInBytes), flushThresholdSize);
asyncFlushAndReschedule(); asyncFlushAndReschedule();

View File

@ -34,13 +34,12 @@ import java.io.IOException;
public class TranslogStats implements ToXContent, Streamable { public class TranslogStats implements ToXContent, Streamable {
private long translogSizeInBytes = 0; private long translogSizeInBytes = 0;
private int estimatedNumberOfOperations = 0; private int estimatedNumberOfOperations = -1;
public TranslogStats() { public TranslogStats() {
} }
public TranslogStats(int estimatedNumberOfOperations, long translogSizeInBytes) { 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.estimatedNumberOfOperations = estimatedNumberOfOperations;
this.translogSizeInBytes = translogSizeInBytes; 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.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.nio.file.Path; import java.nio.file.Path;
@ -49,6 +48,11 @@ public interface TranslogStream {
*/ */
public int writeHeader(FileChannel channel) throws IOException; 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 * 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.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
@ -91,7 +89,6 @@ public class TranslogStreams {
* has zero length, returns the latest version. If the header does not * has zero length, returns the latest version. If the header does not
* exist, assumes Version 0 of the translog file format. * exist, assumes Version 0 of the translog file format.
* <p/> * <p/>
* The caller is responsible for closing the TranslogStream.
* *
* @throws IOException * @throws IOException
*/ */

View File

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

View File

@ -19,33 +19,40 @@
package org.elasticsearch.index.translog.fs; package org.elasticsearch.index.translog.fs;
import com.google.common.collect.Iterables;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.util.concurrent.AbstractRefCounted; 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.IOException;
import java.io.RandomAccessFile;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.OpenOption; import java.nio.file.OpenOption;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.StandardOpenOption; import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
/** class ChannelReference extends AbstractRefCounted {
*
*/
abstract class ChannelReference extends AbstractRefCounted {
private final Path file; private final Path file;
private final FileChannel channel; private final FileChannel channel;
private final TranslogStream stream;
public ChannelReference(Path file, OpenOption... openOptions) throws IOException { public ChannelReference(Path file, OpenOption... openOptions) throws IOException {
super(file.toString()); super(file.toString());
this.file = file; this.file = file;
this.channel = FileChannel.open(file, openOptions); 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() { public Path file() {
@ -56,8 +63,54 @@ abstract class ChannelReference extends AbstractRefCounted {
return this.channel; 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 @Override
protected void closeInternal() { protected void closeInternal() {
IOUtils.closeWhileHandlingException(channel); 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 * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.index.translog.fs; 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.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 org.elasticsearch.index.translog.Translog;
import java.io.EOFException; import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.concurrent.atomic.AtomicBoolean; 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; public FsChannelSnapshot(FsChannelReader reader) {
this.reader = reader;
private final FileChannel channel; this.position = reader.firstPosition();
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;
} }
@Override
public long translogId() { 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() { public int estimatedTotalOperations() {
return this.totalOperations; return reader.totalOperations();
} }
@Override public Translog.Operation next(ByteBuffer reusableBuffer) throws IOException {
public long lengthInBytes() { if (position >= reader.sizeInBytes()) {
return length - position;
}
@Override
public Translog.Operation next() {
try {
if (position >= length) {
return null; return null;
} }
if (cacheBuffer == null) { final int opSize = reader.readSize(reusableBuffer, position);
cacheBuffer = ByteBuffer.allocate(1024); Translog.Operation op = reader.read(reusableBuffer, position, opSize);
}
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; position += opSize;
BytesArray bytesArray = new BytesArray(cacheBuffer.array(), 0, opSize); return op;
return TranslogStreams.readTranslogOperation(new BytesStreamInput(bytesArray.copyBytesArray()));
} catch (IOException e) {
throw new ElasticsearchException("unexpected exception reading from translog snapshot of " + this.channelReference.file(), e);
}
}
@Override
public void seekTo(long position) {
this.position = position;
} }
@Override @Override
public void close() { public void close() {
if (closed.compareAndSet(false, true)) { 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; 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.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException; 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.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.ReleasableBytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays; 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.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.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.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.nio.channels.ClosedChannelException;
import java.nio.file.*; import java.nio.file.*;
import java.util.Arrays; import java.util.*;
import java.util.Collection; import java.util.concurrent.ScheduledFuture;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock; 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"; 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 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 { class ApplySettings implements IndexSettingsService.Listener {
@Override @Override
@ -78,75 +90,142 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
private final IndexSettingsService indexSettingsService; private final IndexSettingsService indexSettingsService;
private final BigArrays bigArrays; 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 final Path location;
private volatile FsTranslogFile current; // protected by the write lock
private volatile FsTranslogFile trans; 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 FsTranslogFile.Type type;
private boolean syncOnEachOperation = false; private boolean syncOnEachOperation = false;
private volatile int bufferSize; private volatile int bufferSize;
private volatile int transientBufferSize;
private final ApplySettings applySettings = new ApplySettings(); private final ApplySettings applySettings = new ApplySettings();
@Inject private final AtomicBoolean closed = new AtomicBoolean();
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService,
BigArrays bigArrays, ShardPath shardPath) throws IOException { public FsTranslog(ShardId shardId, IndexSettingsService indexSettingsService,
super(shardId, indexSettings); BigArrays bigArrays, ShardPath shardPath, ThreadPool threadPool) throws IOException {
this.indexSettingsService = indexSettingsService; this(shardId, indexSettingsService.getSettings(), indexSettingsService, bigArrays, shardPath.resolveTranslog(), threadPool);
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);
} }
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, Path location) throws IOException { public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings,
super(shardId, indexSettings); BigArrays bigArrays, Path location) throws IOException {
this.indexSettingsService = null; this(shardId, indexSettings, null, bigArrays, location, null);
this.location = location; }
Files.createDirectories(location);
this.bigArrays = BigArrays.NON_RECYCLING_INSTANCE;
this.type = FsTranslogFile.Type.fromString(indexSettings.get("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name())); private FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, @Nullable IndexSettingsService indexSettingsService,
this.bufferSize = (int) indexSettings.getAsBytesSize("index.translog.fs.buffer_size", ByteSizeValue.parseBytesSizeValue("64k")).bytes(); 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 @Override
public void updateBuffer(ByteSizeValue bufferSize) { public void updateBuffer(ByteSizeValue bufferSize) {
this.bufferSize = bufferSize.bytesAsInt(); this.bufferSize = bufferSize.bytesAsInt();
rwl.writeLock().lock(); try (ReleasableLock lock = writeLock.acquire()) {
try { current.updateBufferSize(this.bufferSize);
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();
} }
boolean isOpen() {
return closed.get() == false;
} }
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (closed.compareAndSet(false, true)) {
if (indexSettingsService != null) { if (indexSettingsService != null) {
indexSettingsService.removeListener(applySettings); indexSettingsService.removeListener(applySettings);
} }
rwl.writeLock().lock();
try (ReleasableLock lock = writeLock.acquire()) {
try { try {
IOUtils.close(this.trans, this.current); IOUtils.close(this.current);
} finally { } finally {
rwl.writeLock().unlock(); IOUtils.close(uncommittedTranslogs);
}
} finally {
FutureUtils.cancel(syncScheduler);
logger.debug("translog closed");
}
} }
} }
@ -157,246 +236,286 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
@Override @Override
public long currentId() { public long currentId() {
FsTranslogFile current1 = this.current; try (ReleasableLock lock = readLock.acquire()) {
if (current1 == null) { return current.translogId();
return -1;
} }
return current1.id();
} }
@Override @Override
public int estimatedNumberOfOperations() { public int totalOperations() {
FsTranslogFile current1 = this.current; int ops = 0;
if (current1 == null) { try (ReleasableLock lock = readLock.acquire()) {
return 0; ops += current.totalOperations();
for (FsChannelReader translog : uncommittedTranslogs) {
int tops = translog.totalOperations();
if (tops == FsChannelReader.UNKNOWN_OP_COUNT) {
return FsChannelReader.UNKNOWN_OP_COUNT;
} }
return current1.estimatedNumberOfOperations(); ops += tops;
}
}
return ops;
} }
@Override @Override
public long ramBytesUsed() { public long sizeInBytes() {
return 0; long size = 0;
try (ReleasableLock lock = readLock.acquire()) {
size += current.sizeInBytes();
for (FsChannelReader translog : uncommittedTranslogs) {
size += translog.sizeInBytes();
}
}
return size;
} }
@Override /** asserts that all files were closed, if not throws an {@link AssertionError} with details regarding the open files */
public Collection<Accountable> getChildResources() { public static void assertAllClosed() {
return Collections.emptyList(); 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);
}
} }
@Override /** force close an open reference captured in assertion code * */
public long translogSizeInBytes() { public static void assertForceCloseAllReferences() {
FsTranslogFile current1 = this.current; if (ChannelReference.openedFiles == null) {
if (current1 == null) { return;
return 0; }
for (Map.Entry<String, Map<FsChannelReader, RuntimeException>> file : ChannelReference.openedFiles.entrySet()) {
IOUtils.closeWhileHandlingException(file.getValue().keySet());
} }
return current1.translogSizeInBytes();
} }
@Override /** gets a list of unreferenced files (only works if assertions are enabled, returns an empty array otherwise) */
public int clearUnreferenced() { public String[] getUnreferenced() throws IOException {
rwl.writeLock().lock(); if (ChannelReference.openedFiles == null) {
int deleted = 0; 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]*")) { try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path file : stream) { for (Path file : stream) {
if (isReferencedTranslogFile(file) == false) { final long id = parseIdFromFileName(file);
try { if (id < 0) {
logger.trace("delete unreferenced translog file: " + file); logger.trace("failed to extract translog id from [{}]", file);
Files.delete(file); } else if (ChannelReference.openedFiles.containsKey(file.toString()) == false) {
deleted++; result.add(file.toString());
} catch (Exception ex) {
logger.debug("failed to delete " + file, ex);
} }
} }
} }
} catch (IOException ex) {
logger.debug("failed to clear unreferenced files ", ex);
} finally {
rwl.writeLock().unlock();
} }
return deleted; return result.toArray(Strings.EMPTY_ARRAY);
} }
@Override @Override
public void newTranslog(long id) throws TranslogException, IOException { public void markCommitted(final long translogId) throws FileNotFoundException {
rwl.writeLock().lock(); 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 { 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);
}
FsTranslogFile old = current;
current = newFile;
IOUtils.close(old);
} finally {
rwl.writeLock().unlock();
}
}
@Override
public void newTransientTranslog(long id) throws TranslogException {
rwl.writeLock().lock();
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);
} catch (IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
} finally {
rwl.writeLock().unlock();
}
}
@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(); old.close();
current.reuse(old); } catch (IOException e) {
logger.error("failed to closed old translog [{}] (committed id [{}])", e, old, translogId);
}
}
}
} }
@Override @Override
public void revertTransient() throws IOException { public long newTranslog() throws TranslogException, IOException {
rwl.writeLock().lock(); try (ReleasableLock lock = writeLock.acquire()) {
try { final FsTranslogFile old = current;
final FsTranslogFile toClose = this.trans; final FsTranslogFile newFile = createTranslogFile(old);
this.trans = null; current = newFile;
IOUtils.close(toClose); FsChannelImmutableReader reader = old.immutableReader();
} finally { uncommittedTranslogs.add(reader);
rwl.writeLock().unlock(); // 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 {
* Returns the translog that should be read for the specified location. If FsTranslogFile newFile;
* the transient or current translog does not match, returns null long size = Long.MAX_VALUE;
*/ try {
private FsTranslogFile translogForLocation(Location location) { long id = idGenerator++;
if (trans != null && trans.id() == location.translogId) { newFile = type.create(shardId, id, new InternalChannelReference(id, location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), bufferSize);
return this.trans; } catch (IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
} }
if (current.id() == location.translogId) { if (reuse != null) {
return this.current; newFile.reuse(reuse);
} }
return null; return newFile;
} }
/** /**
* Read the Operation object from the given location, returns null if the * Read the Operation object from the given location, returns null if the
* Operation could not be read. * Operation could not be read.
*/ */
@Override @Override
public Translog.Operation read(Location location) { public Translog.Operation read(Location location) {
rwl.readLock().lock(); try (ReleasableLock lock = readLock.acquire()) {
try { FsChannelReader reader = null;
FsTranslogFile translog = translogForLocation(location); if (current.translogId() == location.translogId) {
if (translog != null) { reader = current;
byte[] data = translog.read(location); } else {
try (BytesStreamInput in = new BytesStreamInput(data)) { for (FsChannelReader translog : uncommittedTranslogs) {
// Return the Operation using the current version of the if (translog.translogId() == location.translogId) {
// stream based on which translog is being read reader = translog;
return translog.getStream().read(in); break;
} }
} }
return null; }
return reader == null ? null : reader.read(location);
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("failed to read source from translog location " + location, e); throw new ElasticsearchException("failed to read source from translog location " + location, e);
} finally {
rwl.readLock().unlock();
} }
} }
@Override @Override
public Location add(Operation operation) throws TranslogException { public Location add(Operation operation) throws TranslogException {
rwl.readLock().lock(); ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(bigArrays);
boolean released = false;
ReleasableBytesStreamOutput out = null;
try { try {
out = new ReleasableBytesStreamOutput(bigArrays);
TranslogStreams.writeTranslogOperation(out, operation); TranslogStreams.writeTranslogOperation(out, operation);
ReleasablePagedBytesReference bytes = out.bytes(); ReleasablePagedBytesReference bytes = out.bytes();
try (ReleasableLock lock = readLock.acquire()) {
Location location = current.add(bytes); Location location = current.add(bytes);
if (syncOnEachOperation) { if (syncOnEachOperation) {
current.sync(); current.sync();
} }
assert new BytesArray(current.read(location)).equals(bytes); assert current.assertBytesAtLocation(location, bytes);
FsTranslogFile trans = this.trans;
if (trans != null) {
try {
location = trans.add(bytes);
} catch (ClosedChannelException e) {
// ignore
}
}
Releasables.close(bytes);
released = true;
return location; return location;
}
} catch (Throwable e) { } catch (Throwable e) {
throw new TranslogException(shardId, "Failed to write operation [" + operation + "]", e); throw new TranslogException(shardId, "Failed to write operation [" + operation + "]", e);
} finally { } finally {
rwl.readLock().unlock();
if (!released && out != null) {
Releasables.close(out.bytes()); Releasables.close(out.bytes());
} }
} }
}
@Override @Override
public FsChannelSnapshot snapshot() throws TranslogException { public Snapshot newSnapshot() {
while (true) { try (ReleasableLock lock = readLock.acquire()) {
FsTranslogFile current = this.current; // leave one place for current.
FsChannelSnapshot snapshot = current.snapshot(); final FsChannelReader[] readers = uncommittedTranslogs.toArray(new FsChannelReader[uncommittedTranslogs.size() + 1]);
if (snapshot != null) { 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; return snapshot;
} finally {
if (success == false) {
IOUtils.closeWhileHandlingException(channelSnapshots);
} }
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 @Override
public Snapshot snapshot(Snapshot snapshot) { public Translog.View newView() {
FsChannelSnapshot snap = snapshot(); // we need to acquire the read lock to make sure new translog is created
if (snap.translogId() == snapshot.translogId()) { // and will be missed by the view we're making
snap.seekTo(snapshot.position()); 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);
}
} }
return snap;
} }
@Override @Override
public void sync() throws IOException { public void sync() throws IOException {
FsTranslogFile current1 = this.current; try (ReleasableLock lock = readLock.acquire()) {
if (current1 == null) { if (closed.get()) {
return; return;
} }
try { current.sync();
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;
}
} }
} }
@Override @Override
public boolean syncNeeded() { public boolean syncNeeded() {
FsTranslogFile current1 = this.current; try (ReleasableLock lock = readLock.acquire()) {
return current1 != null && current1.syncNeeded(); return current.syncNeeded();
}
} }
@Override @Override
@ -409,138 +528,169 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
} }
} }
@Override /** package private for testing */
public String getFilename(long translogId) { String getFilename(long translogId) {
return TRANSLOG_FILE_PREFIX + translogId; return TRANSLOG_FILE_PREFIX + translogId;
} }
@Override @Override
public TranslogStats stats() { public TranslogStats stats() {
FsTranslogFile current = this.current; // acquire lock to make the two numbers roughly consistent (no file change half way)
if (current == null) { try (ReleasableLock lock = readLock.acquire()) {
return new TranslogStats(0, 0); return new TranslogStats(totalOperations(), sizeInBytes());
}
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();
} }
} }
@Override private boolean isReferencedTranslogId(long translogId) {
public OperationIterator openIterator(long translogId) throws IOException { return translogId >= lastCommittedTranslogId;
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 final class InternalChannelReference extends ChannelReference { 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); super(file, openOptions);
this.translogId = translogId;
} }
@Override @Override
protected void closeInternal() { protected void closeInternal() {
super.closeInternal(); super.closeInternal();
rwl.writeLock().lock(); try (ReleasableLock lock = writeLock.acquire()) {
try { if (isReferencedTranslogId(translogId) == false) {
if (isReferencedTranslogFile(file()) == false) {
// if the given path is not the current we can safely delete the file since all references are released // 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()); logger.trace("delete translog file - not referenced and not current anymore {}", file());
IOUtils.deleteFilesIgnoringExceptions(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; boolean closed;
private final StreamInput input; // last in this list is always FsTranslog.current
private final ESLogger logger; final List<FsChannelReader> orderedTranslogs;
OperationIteratorImpl(ESLogger logger, TranslogStream translogStream, StreamInput input) { FsView(List<FsChannelReader> orderedTranslogs) {
this.translogStream = translogStream; assert orderedTranslogs.isEmpty() == false;
this.input = input; // clone so we can safely mutate..
this.logger = logger; 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 { synchronized void onNewTranslog(FsChannelReader oldCurrent, FsChannelReader newCurrent) throws IOException {
try { // even though the close method removes this view from outstandingViews, there is no synchronisation in place
if (translogStream instanceof LegacyTranslogStream) { // between that operation and an ongoing addition of a new translog, already having an iterator.
input.readInt(); // ignored opSize // 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;
} }
return translogStream.read(input); orderedTranslogs.remove(orderedTranslogs.size() - 1).close();
} catch (TruncatedTranslogException | EOFException e) { orderedTranslogs.add(oldCurrent);
// ignore, not properly written the last op orderedTranslogs.add(newCurrent);
logger.trace("ignoring translog EOF exception, the last operation was not properly written", e); }
return null;
} catch (IOException e) { @Override
// ignore, not properly written last op public synchronized long minTranslogId() {
logger.trace("ignoring translog IO exception, the last operation was not properly written", e); ensureOpen();
return null; 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;
}
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 @Override
public void close() { public void close() {
List<FsChannelReader> toClose = new ArrayList<>();
try { try {
input.close(); synchronized (this) {
} catch (IOException ex) { if (closed == false) {
throw new ElasticsearchException("failed to close stream input", ex); 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; package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogException; import org.elasticsearch.index.translog.TranslogException;
import org.elasticsearch.index.translog.TranslogStream;
import java.io.Closeable;
import java.io.IOException; 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 { 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() {
void updateBufferSize(int bufferSize) throws TranslogException; return new FsChannelSnapshot(immutableReader());
}
void sync() throws IOException;
/**
boolean syncNeeded(); * returns a new reader that follows the current writes (most importantly allows making
* repeated snapshots that includes new content)
TranslogStream getStream(); */
public FsChannelReader reader() {
public Path getPath(); channelReference.incRef();
boolean success = false;
public boolean closed(); try {
FsChannelReader reader = new InnerReader(this.id, channelReference);
success = true;
return reader;
} finally {
if (!success) {
channelReference.decRef();
}
}
}
/** returns a new immutable reader which only exposes the current written operation * */
abstract public FsChannelImmutableReader immutableReader();
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);
}
/**
* 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 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; package org.elasticsearch.index.translog.fs;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.Channels; import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.ShardId; 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.Translog;
import org.elasticsearch.index.translog.TranslogException; import org.elasticsearch.index.translog.TranslogException;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.ByteBuffer;
import java.nio.file.Path;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
public class SimpleFsTranslogFile implements FsTranslogFile { public final class SimpleFsTranslogFile extends 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;
private volatile int operationCounter = 0; private volatile int operationCounter = 0;
private volatile long lastPosition = 0; private volatile long lastPosition = 0;
private volatile long lastWrittenPosition = 0; private volatile long lastWrittenPosition = 0;
private volatile long lastSyncPosition = 0; private volatile long lastSyncPosition = 0;
public SimpleFsTranslogFile(ShardId shardId, long id, ChannelReference channelReference) throws IOException { public SimpleFsTranslogFile(ShardId shardId, long id, ChannelReference channelReference) throws IOException {
this.shardId = shardId; super(shardId, id, channelReference);
this.id = id; int headerSize = this.channelReference.stream().writeHeader(channelReference.channel());
this.channelReference = channelReference;
this.translogStream = TranslogStreams.translogStreamFor(this.channelReference.file());
this.headerSize = this.translogStream.writeHeader(channelReference.channel());
this.lastPosition += headerSize; this.lastPosition += headerSize;
this.lastWrittenPosition += headerSize; this.lastWrittenPosition += headerSize;
this.lastSyncPosition += headerSize; this.lastSyncPosition += headerSize;
} }
@Override @Override
public long id() { public int totalOperations() {
return this.id;
}
@Override
public int estimatedNumberOfOperations() {
return operationCounter; return operationCounter;
} }
@Override @Override
public long translogSizeInBytes() { public long sizeInBytes() {
return lastWrittenPosition; return lastWrittenPosition;
} }
@Override @Override
public Translog.Location add(BytesReference data) throws IOException { public Translog.Location add(BytesReference data) throws IOException {
rwl.writeLock().lock(); try (ReleasableLock lock = writeLock.acquire()) {
try {
long position = lastPosition; long position = lastPosition;
data.writeTo(channelReference.channel()); data.writeTo(channelReference.channel());
lastPosition = lastPosition + data.length(); lastPosition = lastPosition + data.length();
lastWrittenPosition = lastWrittenPosition + data.length(); lastWrittenPosition = lastWrittenPosition + data.length();
operationCounter = operationCounter + 1; operationCounter = operationCounter + 1;
return new Translog.Location(id, position, data.length()); return new Translog.Location(id, position, data.length());
} finally {
rwl.writeLock().unlock();
} }
} }
@Override @Override
public byte[] read(Translog.Location location) throws IOException { protected void readBytes(ByteBuffer buffer, long position) throws IOException {
rwl.readLock().lock(); try (ReleasableLock lock = readLock.acquire()) {
try { Channels.readFromFileChannelWithEofException(channelReference.channel(), position, buffer);
return Channels.readFromFileChannel(channelReference.channel(), location.translogLocation, location.size);
} finally {
rwl.readLock().unlock();
} }
} }
@Override @Override
public void close() throws IOException { public void doClose() throws IOException {
if (closed.compareAndSet(false, true)) {
try { try {
sync(); sync();
} finally { } finally {
channelReference.decRef(); super.doClose();
}
} }
} }
/** public FsChannelImmutableReader immutableReader() throws TranslogException {
* Returns a snapshot on this file, <tt>null</tt> if it failed to snapshot.
*/
@Override
public FsChannelSnapshot snapshot() throws TranslogException {
if (channelReference.tryIncRef()) { if (channelReference.tryIncRef()) {
boolean success = false; try (ReleasableLock lock = writeLock.acquire()) {
try { FsChannelImmutableReader reader = new FsChannelImmutableReader(this.id, channelReference, lastWrittenPosition, operationCounter);
rwl.writeLock().lock(); channelReference.incRef(); // for the new object
try { return reader;
FsChannelSnapshot snapshot = new FsChannelSnapshot(this.id, channelReference, lastWrittenPosition, operationCounter);
snapshot.seekTo(this.headerSize);
success = true;
return snapshot;
} finally { } finally {
rwl.writeLock().unlock();
}
} catch (FileNotFoundException e) {
throw new TranslogException(shardId, "failed to create snapshot", e);
} finally {
if (!success) {
channelReference.decRef(); channelReference.decRef();
} }
} else {
throw new TranslogException(shardId, "can't increment channel [" + channelReference + "] channel ref count");
} }
}
return null;
} }
@Override @Override
@ -148,28 +102,15 @@ public class SimpleFsTranslogFile implements FsTranslogFile {
return lastWrittenPosition != lastSyncPosition; return lastWrittenPosition != lastSyncPosition;
} }
@Override
public TranslogStream getStream() {
return this.translogStream;
}
@Override
public Path getPath() {
return channelReference.file();
}
@Override @Override
public void sync() throws IOException { public void sync() throws IOException {
// check if we really need to sync here... // check if we really need to sync here...
if (!syncNeeded()) { if (!syncNeeded()) {
return; return;
} }
rwl.writeLock().lock(); try (ReleasableLock lock = writeLock.acquire()) {
try {
lastSyncPosition = lastWrittenPosition; lastSyncPosition = lastWrittenPosition;
channelReference.channel().force(false); channelReference.channel().force(false);
} finally {
rwl.writeLock().unlock();
} }
} }
@ -182,10 +123,4 @@ public class SimpleFsTranslogFile implements FsTranslogFile {
public void updateBufferSize(int bufferSize) throws TranslogException { public void updateBufferSize(int bufferSize) throws TranslogException {
// nothing to do here... // nothing to do here...
} }
@Override
public boolean closed() {
return this.closed.get();
}
} }

View File

@ -20,7 +20,6 @@
package org.elasticsearch.indices.memory; package org.elasticsearch.indices.memory;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -191,9 +190,20 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
continue; 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(); final long time = threadPool.estimatedTimeInMillis();
Translog translog = indexShard.translog();
ShardIndexingStatus status = shardsIndicesStatus.get(indexShard.shardId()); ShardIndexingStatus status = shardsIndicesStatus.get(indexShard.shardId());
if (status == null) { if (status == null) {
status = new ShardIndexingStatus(); status = new ShardIndexingStatus();
@ -201,7 +211,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
changes.add(ShardStatusChangeType.ADDED); changes.add(ShardStatusChangeType.ADDED);
} }
// check if it is deemed to be inactive (sam translogId and numberOfOperations over a long period of time) // 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 if (status.time == -1) { // first time
status.time = time; status.time = time;
} }
@ -225,7 +235,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
status.time = -1; status.time = -1;
} }
status.translogId = translog.currentId(); status.translogId = translog.currentId();
status.translogNumberOfOperations = translog.estimatedNumberOfOperations(); status.translogNumberOfOperations = translog.totalOperations();
if (status.activeIndexing) { if (status.activeIndexing) {
activeShards++; activeShards++;

View File

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

View File

@ -122,11 +122,10 @@ public class RecoverySource extends AbstractComponent {
} }
ongoingRecoveries.add(shard, handler); ongoingRecoveries.add(shard, handler);
try { try {
shard.recover(handler); return handler.recoverToTarget();
} finally { } finally {
ongoingRecoveries.remove(shard, handler); ongoingRecoveries.remove(shard, handler);
} }
return handler.getResponse();
} }
class StartRecoveryTransportRequestHandler implements TransportRequestHandler<StartRecoveryRequest> { 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.bytes.BytesArray;
import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.compress.CompressorFactory; import org.elasticsearch.common.compress.CompressorFactory;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; 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.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; 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.mapper.DocumentMapper;
import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShard; 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 * everything relating to copying the segment files as well as sending translog
* operations across the wire once the segments have been copied. * operations across the wire once the segments have been copied.
*/ */
public class RecoverySourceHandler implements Engine.RecoveryHandler { public class RecoverySourceHandler {
protected final ESLogger logger; protected final ESLogger logger;
// Shard that is going to be recovered (the "source") // Shard that is going to be recovered (the "source")
@ -126,27 +127,48 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
this.response = new RecoveryResponse(); this.response = new RecoveryResponse();
} }
/** /** performs the recovery from the local engine to the target */
* @return the {@link RecoveryResponse} after the recovery has completed all three phases public RecoveryResponse recoverToTarget() {
*/ try (Translog.View translogView = shard.translog().newView()) {
public RecoveryResponse getResponse() { logger.trace("captured translog id [{}] for recovery", translogView.minTranslogId());
return this.response; 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} * Perform phase1 of the recovery operations. Once this {@link SnapshotIndexCommit}
* snapshot has been performed no commit operations (files being fsync'd) * snapshot has been performed no commit operations (files being fsync'd)
* are effectively allowed on this index until all recovery phases are done * 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 * 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 * segments that are missing. Only segments that have the same size and
* checksum can be reused * 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, final Translog.View translogView) {
public void phase1(final SnapshotIndexCommit snapshot) {
cancellableThreads.checkForCancel(); cancellableThreads.checkForCancel();
// Total size of segment files that are recovered // Total size of segment files that are recovered
long totalSize = 0; long totalSize = 0;
@ -202,7 +224,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
public void run() throws InterruptedException { public void run() throws InterruptedException {
RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(), RecoveryFilesInfoRequest recoveryInfoFilesRequest = new RecoveryFilesInfoRequest(request.recoveryId(), request.shardId(),
response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes, response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, response.phase1ExistingFileSizes,
shard.translog().estimatedNumberOfOperations()); translogView.totalOperations());
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest, transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.FILES_INFO, recoveryInfoFilesRequest,
TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()),
EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
@ -298,7 +320,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
readCount += toRead; readCount += toRead;
final boolean lastChunk = readCount == len; final boolean lastChunk = readCount == len;
final RecoveryFileChunkRequest fileChunkRequest = new RecoveryFileChunkRequest(request.recoveryId(), request.shardId(), md, position, 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() { cancellableThreads.execute(new Interruptable() {
@Override @Override
public void run() throws InterruptedException { public void run() throws InterruptedException {
@ -347,6 +369,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
}); });
if (corruptedEngine.get() != null) { if (corruptedEngine.get() != null) {
shard.engine().failEngine("recovery", corruptedEngine.get());
throw corruptedEngine.get(); throw corruptedEngine.get();
} else { } else {
ExceptionsHelper.rethrowAndSuppress(exceptions); ExceptionsHelper.rethrowAndSuppress(exceptions);
@ -365,7 +388,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
// are deleted // are deleted
try { try {
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.CLEAN_FILES, 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()), TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()),
EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
} catch (RemoteTransportException remoteException) { } 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()); logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime());
response.phase1Time = stopWatch.totalTime().millis(); response.phase1Time = stopWatch.totalTime().millis();
} catch (Throwable e) { } catch (Throwable e) {
@ -416,25 +441,10 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
} }
} }
/** protected void prepareTargetForTranslog(final Translog.View translogView) {
* 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());
StopWatch stopWatch = new StopWatch().start(); 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() { cancellableThreads.execute(new Interruptable() {
@Override @Override
public void run() throws InterruptedException { 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 // operations. This ensures the shard engine is started and disables
// garbage collection (not the JVM's GC!) of tombstone deletes // garbage collection (not the JVM's GC!) of tombstone deletes
transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.PREPARE_TRANSLOG, 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(); TransportRequestOptions.options().withTimeout(recoverySettings.internalActionTimeout()), EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
} }
}); });
stopWatch.stop(); 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()); 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()); logger.trace("{} recovery [phase2] to {}: updating current mapping to master", request.shardId(), request.targetNode());
// Ensure that the mappings are synced with the master node // Ensure that the mappings are synced with the master node
updateMappingOnMaster(); updateMappingOnMaster();
logger.trace("{} recovery [phase2] to {}: sending transaction log operations", request.shardId(), request.targetNode()); 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 // Send all the snapshot's translog operations to the target
int totalOperations = sendSnapshot(snapshot); int totalOperations = sendSnapshot(snapshot);
stopWatch.stop(); stopWatch.stop();
@ -468,28 +493,16 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
} }
/** /**
* Perform phase 3 of the recovery process * finalizes 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.
*/ */
@Override public void finalizeRecovery() {
public void phase3(Translog.Snapshot snapshot) {
if (shard.state() == IndexShardState.CLOSED) { if (shard.state() == IndexShardState.CLOSED) {
throw new IndexShardClosedException(request.shardId()); throw new IndexShardClosedException(request.shardId());
} }
cancellableThreads.checkForCancel(); cancellableThreads.checkForCancel();
StopWatch stopWatch = new StopWatch().start(); StopWatch stopWatch = new StopWatch().start();
final int totalOperations; logger.trace("[{}][{}] finalizing recovery to {}", indexName, shardId, request.targetNode());
logger.trace("[{}][{}] recovery [phase3] to {}: sending transaction log operations", indexName, shardId, request.targetNode());
// Send the translog operations to the target node
totalOperations = sendSnapshot(snapshot);
cancellableThreads.execute(new Interruptable() { cancellableThreads.execute(new Interruptable() {
@Override @Override
@ -518,10 +531,8 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
} }
} }
stopWatch.stop(); stopWatch.stop();
logger.trace("[{}][{}] recovery [phase3] to {}: took [{}]", logger.trace("[{}][{}] finalizing recovery to {}: took [{}]",
indexName, shardId, request.targetNode(), stopWatch.totalTime()); indexName, shardId, request.targetNode(), stopWatch.totalTime());
response.phase3Time = stopWatch.totalTime().millis();
response.phase3Operations = totalOperations;
} }
/** /**
@ -587,13 +598,13 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
/** /**
* Send the given snapshot's operations to this handler's target node. * Send the given snapshot's operations to this handler's target node.
* * <p/>
* Operations are bulked into a single request depending on an operation * Operations are bulked into a single request depending on an operation
* count limit or size-in-bytes limit * count limit or size-in-bytes limit
* *
* @return the total number of translog operations that were sent * @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; int ops = 0;
long size = 0; long size = 0;
int totalOperations = 0; int totalOperations = 0;
@ -641,7 +652,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
@Override @Override
public void run() throws InterruptedException { public void run() throws InterruptedException {
final RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest( 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, transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest,
recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
} }
@ -649,7 +660,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("[{}][{}] sent batch of [{}][{}] (total: [{}]) translog operations to {}", logger.trace("[{}][{}] sent batch of [{}][{}] (total: [{}]) translog operations to {}",
indexName, shardId, ops, new ByteSizeValue(size), indexName, shardId, ops, new ByteSizeValue(size),
shard.translog().estimatedNumberOfOperations(), snapshot.estimatedTotalOperations(),
request.targetNode()); request.targetNode());
} }
@ -661,14 +672,15 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
operation = snapshot.next(); // this ex should bubble up 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); throw new ElasticsearchException("failed to get next operation from translog", ex);
} } }
}
// send the leftover // send the leftover
if (!operations.isEmpty()) { if (!operations.isEmpty()) {
cancellableThreads.execute(new Interruptable() { cancellableThreads.execute(new Interruptable() {
@Override @Override
public void run() throws InterruptedException { public void run() throws InterruptedException {
RecoveryTranslogOperationsRequest translogOperationsRequest = new RecoveryTranslogOperationsRequest( 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, transportService.submitRequest(request.targetNode(), RecoveryTarget.Actions.TRANSLOG_OPS, translogOperationsRequest,
recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); recoveryOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
} }
@ -678,7 +690,7 @@ public class RecoverySourceHandler implements Engine.RecoveryHandler {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("[{}][{}] sent final batch of [{}][{}] (total: [{}]) translog operations to {}", logger.trace("[{}][{}] sent final batch of [{}][{}] (total: [{}]) translog operations to {}",
indexName, shardId, ops, new ByteSizeValue(size), indexName, shardId, ops, new ByteSizeValue(size),
shard.translog().estimatedNumberOfOperations(), snapshot.estimatedTotalOperations(),
request.targetNode()); request.targetNode());
} }
return totalOperations; return totalOperations;

View File

@ -42,7 +42,10 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.IndexShardMissingException; import org.elasticsearch.index.IndexShardMissingException;
import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.engine.RecoveryEngineException;
import org.elasticsearch.index.settings.IndexSettings; 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.Store;
import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.IndexMissingException; 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") sb.append(" : recovered [").append(recoveryResponse.phase2Operations).append("]").append(" transaction log operations")
.append(", took [").append(timeValueMillis(recoveryResponse.phase2Time)).append("]") .append(", took [").append(timeValueMillis(recoveryResponse.phase2Time)).append("]")
.append("\n"); .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()); logger.trace(sb.toString());
} else { } else {
logger.debug("{} recovery done from [{}], took [{}]", request.shardId(), recoveryStatus.sourceNode(), recoveryTime); 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) // 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 // if the target is not ready yet, retry
retryRecovery(recoveryStatus, "remote shard not ready", recoverySettings.retryDelayStateSync(), request); retryRecovery(recoveryStatus, "remote shard not ready", recoverySettings.retryDelayStateSync(), request);
return; return;

View File

@ -45,7 +45,7 @@ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler {
} }
@Override @Override
public void phase1(SnapshotIndexCommit snapshot) { public void phase1(SnapshotIndexCommit snapshot, final Translog.View translogView) {
if (request.recoveryType() == RecoveryState.Type.RELOCATION && shard.routingEntry().primary()) { 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) // 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 // 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); shard.failShard("primary_relocation", exception);
throw 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 @Override
protected int sendSnapshot(Translog.Snapshot snapshot) { 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; return 0;
} }

View File

@ -20,7 +20,6 @@
package org.elasticsearch.bwcompat; package org.elasticsearch.bwcompat;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TestUtil;
@ -60,16 +59,11 @@ import org.junit.Before;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.nio.file.DirectoryStream; import java.nio.file.*;
import java.nio.file.FileVisitResult;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes; import java.nio.file.attribute.BasicFileAttributes;
import java.util.*; import java.util.*;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo;
// needs at least 2 nodes since it bumps replicas to 1 // 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 // TODO: remove this once #10262 is fixed
return; 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]")); SearchRequestBuilder searchReq = client().prepareSearch(indexName).setQuery(QueryBuilders.queryStringQuery("long_sort:[10 TO 20]"));
assertEquals(0, searchReq.get().getHits().getTotalHits()); assertEquals(0, searchReq.get().getHits().getTotalHits());
} }

View File

@ -19,8 +19,6 @@
package org.elasticsearch.index.engine; package org.elasticsearch.index.engine;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import com.carrotsearch.randomizedtesting.annotations.Seed;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.log4j.AppenderSkeleton; import org.apache.log4j.AppenderSkeleton;
import org.apache.log4j.Level; 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.lucene.uid.Versions;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy; import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
import org.elasticsearch.index.engine.Engine.Searcher; import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService; 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.DirectoryUtils;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogSizeMatcher;
import org.elasticsearch.index.translog.fs.FsTranslog; import org.elasticsearch.index.translog.fs.FsTranslog;
import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.hamcrest.MatcherAssert; import org.hamcrest.MatcherAssert;
import org.junit.After; import org.junit.After;
@ -92,6 +90,7 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
@ -113,15 +112,14 @@ public class InternalEngineTests extends ElasticsearchTestCase {
private Store store; private Store store;
private Store storeReplica; private Store storeReplica;
protected Translog translog;
protected Translog replicaTranslog;
protected InternalEngine engine; protected InternalEngine engine;
protected InternalEngine replicaEngine; protected InternalEngine replicaEngine;
private Settings defaultSettings; private Settings defaultSettings;
private int indexConcurrency; private int indexConcurrency;
private String codecName; private String codecName;
private Path primaryTranslogDir;
private Path replicaTranslogDir;
@Override @Override
@Before @Before
@ -149,8 +147,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
storeReplica = createStore(); storeReplica = createStore();
Lucene.cleanLuceneIndex(store.directory()); Lucene.cleanLuceneIndex(store.directory());
Lucene.cleanLuceneIndex(storeReplica.directory()); Lucene.cleanLuceneIndex(storeReplica.directory());
translog = createTranslog(); primaryTranslogDir = createTempDir("translog-primary");
engine = createEngine(store, translog); engine = createEngine(store, createTranslog());
LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig(); LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName()); assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName());
@ -158,8 +156,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
if (randomBoolean()) { if (randomBoolean()) {
engine.config().setEnableGcDeletes(false); engine.config().setEnableGcDeletes(false);
} }
replicaTranslog = createTranslogReplica(); replicaTranslogDir = createTempDir("translog-replica");
replicaEngine = createEngine(storeReplica, replicaTranslog); replicaEngine = createEngine(storeReplica, createTranslogReplica());
currentIndexWriterConfig = replicaEngine.getCurrentIndexWriterConfig(); currentIndexWriterConfig = replicaEngine.getCurrentIndexWriterConfig();
assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName());
@ -173,12 +171,44 @@ public class InternalEngineTests extends ElasticsearchTestCase {
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
super.tearDown(); super.tearDown();
try {
assertTranslogNotLeaking((FsTranslog) engine.translog());
assertTranslogNotLeaking((FsTranslog) replicaEngine.translog());
} finally {
IOUtils.close( IOUtils.close(
replicaEngine, storeReplica, replicaTranslog, replicaEngine, storeReplica,
engine, store, translog); engine, store);
terminate(threadPool); 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() { private Document testDocumentWithTextField() {
Document document = testDocument(); Document document = testDocument();
@ -218,12 +248,16 @@ public class InternalEngineTests extends ElasticsearchTestCase {
return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId)); return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId));
} }
protected Translog createTranslog() throws IOException { protected FsTranslog createTranslog() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-primary")); return createTranslog(primaryTranslogDir);
} }
protected Translog createTranslogReplica() throws IOException { protected FsTranslog createTranslog(Path translogPath) throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-replica")); return new FsTranslog(shardId, EMPTY_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE, translogPath);
}
protected FsTranslog createTranslogReplica() throws IOException {
return createTranslog(replicaTranslogDir);
} }
protected IndexDeletionPolicy createIndexDeletionPolicy() { protected IndexDeletionPolicy createIndexDeletionPolicy() {
@ -242,27 +276,25 @@ public class InternalEngineTests extends ElasticsearchTestCase {
return new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, indexSettingsService); 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()); 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)); return createEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService));
} }
protected InternalEngine createEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) { protected InternalEngine createEngine(IndexSettingsService indexSettingsService, Store store, FsTranslog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new InternalEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider), false); 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(); IndexWriterConfig iwc = newIndexWriterConfig();
EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService 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() { iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(shardId.index()), new Engine.FailedEngineListener() {
@Override @Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) { public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test // we don't need to notify anybody in this test
} }
}, new TranslogHandler(shardId.index().getName())); }, new TranslogHandler(shardId.index().getName()));
return config; return config;
} }
@ -313,7 +345,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); assertThat(segments.get(0).getDeletedDocs(), equalTo(0));
assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); 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); ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, null);
engine.create(new Engine.Create(null, newUid("3"), doc3)); 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).getDeletedDocs(), equalTo(0));
assertThat(segments.get(1).isCompound(), equalTo(false)); 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); ParsedDocument doc4 = testParsedDocument("4", "4", "test", null, -1, -1, testDocumentWithTextField(), B_3, null);
engine.create(new Engine.Create(null, newUid("4"), doc4)); engine.create(new Engine.Create(null, newUid("4"), doc4));
engine.refresh("test"); 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)); 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()); IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
try (Store store = createStore(); try (Store store = createStore();
Translog translog = createTranslog(); Engine engine = createEngine(indexSettingsService, store, createTranslog(createTempDir()), mergeSchedulerProvider)) {
Engine engine = createEngine(indexSettingsService, store, translog, mergeSchedulerProvider)) {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null);
Engine.Index index = new Engine.Index(null, newUid("1"), doc); 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", "test")), 0));
MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1)); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1));
searchResult.close(); searchResult.close();
engine.close();
} }
@Test @Test
@ -689,198 +718,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
searchResult.close(); 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 @Test
public void testVersioningNewCreate() { public void testVersioningNewCreate() {
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); 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); int numIters = randomIntBetween(2, 10);
for (int j = 0; j < numIters; j++) { for (int j = 0; j < numIters; j++) {
try (Store store = createStore()) { try (Store store = createStore()) {
final Translog translog = createTranslog(); final InternalEngine engine = createEngine(store, createTranslog(createTempDir()));
final InternalEngine engine = createEngine(store, translog);
final CountDownLatch startGun = new CountDownLatch(1); final CountDownLatch startGun = new CountDownLatch(1);
final CountDownLatch indexed = 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()); engine.forceMerge(randomBoolean(), 1, false, randomBoolean(), randomBoolean());
} }
indexed.await(); indexed.await();
IOUtils.close(engine, translog); IOUtils.close(engine);
} }
} }
@ -1452,7 +1288,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
assertNotNull(iwIFDLogger); assertNotNull(iwIFDLogger);
} }
Level savedLevel = iwIFDLogger.getLevel();
iwIFDLogger.addAppender(mockAppender); iwIFDLogger.addAppender(mockAppender);
iwIFDLogger.setLevel(Level.DEBUG); iwIFDLogger.setLevel(Level.DEBUG);
@ -1482,8 +1317,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
public void testEnableGcDeletes() throws Exception { public void testEnableGcDeletes() throws Exception {
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
try (Store store = createStore(); try (Store store = createStore();
Translog translog = createTranslog(); Engine engine = new InternalEngine(config(indexSettingsService, store, createMergeScheduler(indexSettingsService)),
Engine engine = new InternalEngine(config(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)), false)) { createTranslog(createTempDir()), false)) {
engine.config().setEnableGcDeletes(false); engine.config().setEnableGcDeletes(false);
// Add document // Add document
@ -1549,7 +1384,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
try (Engine.Searcher test = this.engine.acquireSearcher("test")) { try (Engine.Searcher test = this.engine.acquireSearcher("test")) {
ShardId shardId = ShardUtils.extractShardId(test.reader()); ShardId shardId = ShardUtils.extractShardId(test.reader());
assertNotNull(shardId); 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.setAllowRandomFileNotFoundException(randomBoolean());
wrapper.setRandomIOExceptionRate(randomDouble()); wrapper.setRandomIOExceptionRate(randomDouble());
wrapper.setRandomIOExceptionRateOnOpen(randomDouble()); wrapper.setRandomIOExceptionRateOnOpen(randomDouble());
try (Store store = createStore(wrapper)) { try (Store store = createStore(wrapper); FsTranslog translog = createTranslog(createTempDir("testFailStart"))) {
int refCount = store.refCount(); int refCount = store.refCount();
assertTrue("refCount: " + store.refCount(), store.refCount() > 0); assertTrue("refCount: " + store.refCount(), store.refCount() > 0);
Translog translog = createTranslog();
InternalEngine holder; InternalEngine holder;
try { try {
holder = createEngine(store, translog); holder = createEngine(store, translog);
@ -1592,7 +1426,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
break; break;
} }
} }
translog.close();
holder.close(); holder.close();
assertEquals(store.refCount(), refCount); assertEquals(store.refCount(), refCount);
} }
@ -1601,7 +1434,6 @@ public class InternalEngineTests extends ElasticsearchTestCase {
@Test @Test
public void testSettings() { public void testSettings() {
InternalEngine engine = (InternalEngine) this.engine;
CodecService codecService = new CodecService(shardId.index()); CodecService codecService = new CodecService(shardId.index());
LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig(); LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig();
@ -1707,8 +1539,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
.put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb").build(); .put(EngineConfig.INDEX_BUFFER_SIZE_SETTING, "1kb").build();
IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), indexSettings); IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), indexSettings);
try (Store store = createStore(); try (Store store = createStore();
Translog translog = createTranslog(); Engine engine = new InternalEngine(config(indexSettingsService, store, createMergeScheduler(indexSettingsService)),
final Engine engine = new InternalEngine(config(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)), false)) { createTranslog(createTempDir()), false)) {
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
String id = Integer.toString(i); String id = Integer.toString(i);
ParsedDocument doc = testParsedDocument(id, id, "test", null, -1, -1, testDocument(), B_1, null); 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 { public void testTranslogReplayWithFailure() throws IOException {
boolean canHaveDuplicates = true; boolean canHaveDuplicates = true;
boolean autoGeneratedId = true; boolean autoGeneratedId = true;
@ -1770,21 +1624,13 @@ public class InternalEngineTests extends ElasticsearchTestCase {
directory.setRandomIOExceptionRate(randomDouble()); directory.setRandomIOExceptionRate(randomDouble());
directory.setFailOnOpenInput(randomBoolean()); directory.setFailOnOpenInput(randomBoolean());
directory.setAllowRandomFileNotFoundException(randomBoolean()); directory.setAllowRandomFileNotFoundException(randomBoolean());
final FsTranslog translog = createTranslog();
try { try {
engine = createEngine(store, translog); engine = createEngine(store, translog);
started = true; started = true;
break; break;
} catch (EngineCreationFailureException ex) { } catch (EngineCreationFailureException ex) {
// sometimes we fail after we committed the recovered docs during the finaly refresh call translog.close();
// 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
}
}
} }
} }
@ -1793,20 +1639,16 @@ public class InternalEngineTests extends ElasticsearchTestCase {
directory.setFailOnOpenInput(false); directory.setFailOnOpenInput(false);
directory.setAllowRandomFileNotFoundException(false); directory.setAllowRandomFileNotFoundException(false);
if (started == false) { if (started == false) {
engine = createEngine(store, translog); engine = createEngine(store, createTranslog());
} }
} else { } else {
// no mock directory, no fun. // no mock directory, no fun.
engine = createEngine(store, translog); engine = createEngine(store, createTranslog());
} }
try (Engine.Searcher searcher = engine.acquireSearcher("test")) { try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
assertThat(topDocs.totalHits, equalTo(numDocs)); assertThat(topDocs.totalHits, equalTo(numDocs));
} }
if (recoveredButFailed == false) {
TranslogHandler parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer();
assertEquals(numDocs, parser.recoveredOps.get());
}
} }
@Test @Test
@ -1831,11 +1673,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
// this so we have to disable the check explicitly // this so we have to disable the check explicitly
directory.setPreventDoubleWrite(false); directory.setPreventDoubleWrite(false);
} }
long currentTranslogId = translog.currentId();
engine.close(); engine.close();
engine = new InternalEngine(engine.config(), true); engine = new InternalEngine(engine.config(), createTranslog(), true);
assertTrue(currentTranslogId + "<" + translog.currentId(), currentTranslogId < translog.currentId());
assertEquals("translog ID must be incremented by 2 after initial recovery", currentTranslogId + 2, translog.currentId());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) { try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); 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(); TranslogHandler parser = (TranslogHandler) engine.config().getTranslogRecoveryPerformer();
parser.mappingUpdate = dynamicUpdate(); parser.mappingUpdate = dynamicUpdate();
long currentTranslogId = translog.currentId();
engine.close(); engine.close();
engine = new InternalEngine(engine.config(), false); // we need to reuse the engine config otherwise the parser.mappingModified won't work engine = new InternalEngine(engine.config(), createTranslog(), false); // we need to reuse the engine config unless 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());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) { try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
@ -1895,7 +1731,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
} }
engine.close(); engine.close();
engine = createEngine(store, translog); engine = createEngine(store, createTranslog());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) { try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10));
assertThat(topDocs.totalHits, equalTo(numDocs)); assertThat(topDocs.totalHits, equalTo(numDocs));
@ -1925,7 +1761,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
} }
engine.close(); engine.close();
engine = createEngine(store, translog); engine = createEngine(store, createTranslog());
try (Engine.Searcher searcher = engine.acquireSearcher("test")) { try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), numDocs + 1); TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), numDocs + 1);
assertThat(topDocs.totalHits, equalTo(numDocs + 1)); assertThat(topDocs.totalHits, equalTo(numDocs + 1));
@ -1937,7 +1773,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
engine.refresh("test"); engine.refresh("test");
} else { } else {
engine.close(); engine.close();
engine = createEngine(store, translog); engine = createEngine(store, createTranslog());
} }
try (Engine.Searcher searcher = engine.acquireSearcher("test")) { try (Engine.Searcher searcher = engine.acquireSearcher("test")) {
TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), numDocs); 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.lucene.Lucene;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy; import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
@ -92,8 +93,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
private Store store; private Store store;
private Store storeReplica; private Store storeReplica;
protected Translog translog; protected FsTranslog translog;
protected Translog replicaTranslog;
protected Engine primaryEngine; protected Engine primaryEngine;
protected Engine replicaEngine; protected Engine replicaEngine;
@ -139,8 +139,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
primaryEngine.config().setEnableGcDeletes(false); primaryEngine.config().setEnableGcDeletes(false);
} }
replicaTranslog = createTranslogReplica(); replicaEngine = createShadowEngine(storeReplica);
replicaEngine = createShadowEngine(storeReplica, replicaTranslog);
assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName());
if (randomBoolean()) { if (randomBoolean()) {
@ -154,11 +153,8 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
super.tearDown(); super.tearDown();
replicaEngine.close(); replicaEngine.close();
storeReplica.close(); storeReplica.close();
translog.close();
replicaTranslog.close();
primaryEngine.close(); primaryEngine.close();
translog.close();
store.close(); store.close();
terminate(threadPool); terminate(threadPool);
} }
@ -201,12 +197,8 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId)); return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId));
} }
protected Translog createTranslog() throws IOException { protected FsTranslog createTranslog() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-primary")); return new FsTranslog(shardId, EMPTY_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE, createTempDir("translog-primary"));
}
protected Translog createTranslogReplica() throws IOException {
return new FsTranslog(shardId, EMPTY_SETTINGS, createTempDir("translog-replica"));
} }
protected IndexDeletionPolicy createIndexDeletionPolicy() { protected IndexDeletionPolicy createIndexDeletionPolicy() {
@ -225,36 +217,34 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
return new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, indexSettingsService); 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()); 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()); 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)); return createInternalEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService));
} }
protected ShadowEngine createShadowEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) { protected ShadowEngine createShadowEngine(IndexSettingsService indexSettingsService, Store store, MergeSchedulerProvider mergeSchedulerProvider) {
return new ShadowEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider)); return new ShadowEngine(config(indexSettingsService, store, mergeSchedulerProvider));
} }
protected InternalEngine createInternalEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) { protected InternalEngine createInternalEngine(IndexSettingsService indexSettingsService, Store store, FsTranslog translog, MergeSchedulerProvider mergeSchedulerProvider) {
return new InternalEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider), true); 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(); IndexWriterConfig iwc = newIndexWriterConfig();
EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService 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() { iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(shardId.index()), new Engine.FailedEngineListener() {
@Override @Override
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) { public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) {
// we don't need to notify anybody in this test // we don't need to notify anybody in this test
} }
}, null); }, null);
return config; return config;
} }
@ -907,10 +897,9 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
try (Store store = createStore(wrapper)) { try (Store store = createStore(wrapper)) {
int refCount = store.refCount(); int refCount = store.refCount();
assertTrue("refCount: "+ store.refCount(), store.refCount() > 0); assertTrue("refCount: "+ store.refCount(), store.refCount() > 0);
Translog translog = createTranslog();
ShadowEngine holder; ShadowEngine holder;
try { try {
holder = createShadowEngine(store, translog); holder = createShadowEngine(store);
} catch (EngineCreationFailureException ex) { } catch (EngineCreationFailureException ex) {
assertEquals(store.refCount(), refCount); assertEquals(store.refCount(), refCount);
continue; continue;
@ -921,7 +910,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
try { try {
assertEquals(store.refCount(), refCount + 1); assertEquals(store.refCount(), refCount + 1);
holder.close(); holder.close();
holder = createShadowEngine(store, translog); holder = createShadowEngine(store);
assertEquals(store.refCount(), refCount + 1); assertEquals(store.refCount(), refCount + 1);
} catch (EngineCreationFailureException ex) { } catch (EngineCreationFailureException ex) {
// all is fine // all is fine
@ -948,7 +937,6 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
final Path srDir = createTempDir(); final Path srDir = createTempDir();
final Store srStore = createStore(srDir); final Store srStore = createStore(srDir);
Lucene.cleanLuceneIndex(srStore.directory()); Lucene.cleanLuceneIndex(srStore.directory());
final Translog srTranslog = createTranslogReplica();
final AtomicBoolean succeeded = new AtomicBoolean(false); final AtomicBoolean succeeded = new AtomicBoolean(false);
final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1);
@ -963,7 +951,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
} catch (InterruptedException e) { } catch (InterruptedException e) {
// ignore interruptions // ignore interruptions
} }
try (ShadowEngine srEngine = createShadowEngine(srStore, srTranslog)) { try (ShadowEngine srEngine = createShadowEngine(srStore)) {
succeeded.set(true); succeeded.set(true);
} catch (Exception e) { } catch (Exception e) {
fail("should have been able to create the engine!"); 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 // Create an InternalEngine, which creates the index so the shadow
// replica will handle it correctly // replica will handle it correctly
Store pStore = createStore(srDir); Store pStore = createStore(srDir);
Translog pTranslog = createTranslog(); FsTranslog pTranslog = createTranslog();
InternalEngine pEngine = createInternalEngine(pStore, pTranslog); InternalEngine pEngine = createInternalEngine(pStore, pTranslog);
// create a document // create a document
@ -992,6 +980,6 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
t.join(); t.join();
assertTrue("ShadowEngine should have been able to be created", succeeded.get()); assertTrue("ShadowEngine should have been able to be created", succeeded.get());
// (shadow engine is already shut down in the try-with-resources) // (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); StreamInput in = stream.openInput(translogFile);
in.readInt();
Translog.Operation operation = stream.read(in); Translog.Operation operation = stream.read(in);
assertThat("operation is the correct type correctly", operation.opType() == Translog.Operation.Type.SAVE, equalTo(true)); 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)); assertThat(op.versionType(), equalTo(VersionType.INTERNAL));
try { try {
in.readInt();
stream.read(in); stream.read(in);
fail("should have been the end of the file"); fail("should have been the end of the file");
} catch (EOFException e) { } catch (EOFException e) {
@ -143,7 +141,7 @@ public class TranslogVersionTests extends ElasticsearchTestCase {
fail("should have thrown an exception about the body being corrupted"); fail("should have thrown an exception about the body being corrupted");
} catch (TranslogCorruptedException e) { } catch (TranslogCorruptedException e) {
assertThat("translog corruption from body: " + e.getMessage(), 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. * under the License.
*/ */
package org.elasticsearch.index.translog; package org.elasticsearch.index.translog.fs;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.stream.BytesStreamInput; import org.elasticsearch.common.io.stream.BytesStreamInput;
import org.elasticsearch.common.io.stream.BytesStreamOutput; 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.Index;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.*;
import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.ElasticsearchTestCase;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.junit.After; import org.junit.After;
@ -42,12 +46,12 @@ import java.nio.channels.FileChannel;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.StandardOpenOption; import java.nio.file.StandardOpenOption;
import java.util.Arrays; import java.util.*;
import java.util.List; import java.util.concurrent.*;
import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger; 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 com.google.common.collect.Lists.newArrayList;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.*;
@ -56,12 +60,29 @@ import static org.hamcrest.Matchers.*;
* *
*/ */
@LuceneTestCase.SuppressFileSystems("ExtrasFS") @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 final ShardId shardId = new ShardId(new Index("index"), 1);
protected FsTranslog translog;
protected Path translogDir; 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 @Override
@Before @Before
@ -69,8 +90,7 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
super.setUp(); super.setUp();
// if a previous test failed we clean up things here // if a previous test failed we clean up things here
translogDir = createTempDir(); translogDir = createTempDir();
translog = create(translogDir); translog = create();
translog.newTranslog(1);
} }
@Override @Override
@ -78,17 +98,56 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
public void tearDown() throws Exception { public void tearDown() throws Exception {
try { try {
translog.close(); 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 { } finally {
super.tearDown(); 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 @Test
public void testRead() throws IOException { 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}))); 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 @Test
public void testSimpleOperations() throws IOException { public void testSimpleOperations() throws IOException {
Translog.Snapshot snapshot = translog.snapshot(); ArrayList<Translog.Operation> ops = new ArrayList<>();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
snapshot.close(); snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1})); addToTranslogAndList(translog, ops, new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.snapshot(); snapshot = translog.newSnapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1)); assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close(); snapshot.close();
translog.add(new Translog.Index("test", "2", new byte[]{2})); addToTranslogAndList(translog, ops, new Translog.Index("test", "2", new byte[]{2}));
snapshot = translog.snapshot(); snapshot = translog.newSnapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(2)); assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(2)); assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size()));
snapshot.close(); snapshot.close();
translog.add(new Translog.Delete(newUid("3"))); addToTranslogAndList(translog, ops, new Translog.Delete(newUid("3")));
snapshot = translog.snapshot(); snapshot = translog.newSnapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(3)); assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(3)); assertThat(snapshot.estimatedTotalOperations(), equalTo(ops.size()));
snapshot.close(); snapshot.close();
translog.add(new Translog.DeleteByQuery(new BytesArray(new byte[]{4}), null)); snapshot = translog.newSnapshot();
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(4));
assertThat(snapshot.estimatedTotalOperations(), equalTo(4));
snapshot.close();
snapshot = translog.snapshot();
Translog.Create create = (Translog.Create) snapshot.next(); Translog.Create create = (Translog.Create) snapshot.next();
assertThat(create != null, equalTo(true)); assertThat(create != null, equalTo(true));
@ -182,33 +203,26 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertThat(delete != null, equalTo(true)); assertThat(delete != null, equalTo(true));
assertThat(delete.uid(), equalTo(newUid("3"))); 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)); assertThat(snapshot.next(), equalTo(null));
snapshot.close(); snapshot.close();
long firstId = translog.currentId(); long firstId = translog.currentId();
translog.newTranslog(2); translog.newTranslog();
assertThat(translog.currentId(), Matchers.not(equalTo(firstId))); assertThat(translog.currentId(), Matchers.not(equalTo(firstId)));
snapshot = translog.snapshot(); snapshot = translog.newSnapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); 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)); assertThat(snapshot.estimatedTotalOperations(), equalTo(0));
snapshot.close(); 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 { protected TranslogStats stats() throws IOException {
// force flushing and updating of stats // force flushing and updating of stats
translog.sync(); translog.sync();
@ -248,88 +262,69 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize)); assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize));
lastSize = stats.translogSizeInBytes().bytes(); lastSize = stats.translogSizeInBytes().bytes();
translog.add(new Translog.Delete(newUid("4")));
translog.add(new Translog.DeleteByQuery(new BytesArray(new byte[]{4}), null)); translog.newTranslog();
stats = stats(); stats = stats();
assertThat(stats.estimatedNumberOfOperations(), equalTo(4l)); assertThat(stats.estimatedNumberOfOperations(), equalTo(4l));
assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize)); assertThat(stats.translogSizeInBytes().bytes(), greaterThan(lastSize));
translog.newTranslog(2); translog.markCommitted(2);
stats = stats(); stats = stats();
assertThat(stats.estimatedNumberOfOperations(), equalTo(0l)); assertThat(stats.estimatedNumberOfOperations(), equalTo(0l));
assertThat(stats.translogSizeInBytes().bytes(), equalTo(17l)); assertThat(stats.translogSizeInBytes().bytes(), equalTo(17l));
} }
@Test @Test
public void testSnapshot() throws IOException { public void testSnapshot() {
Translog.Snapshot snapshot = translog.snapshot(); ArrayList<Translog.Operation> ops = new ArrayList<>();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
snapshot.close(); snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1})); addToTranslogAndList(translog, ops, new Translog.Create("test", "1", new byte[]{1}));
snapshot = translog.snapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(snapshot.estimatedTotalOperations(), equalTo(1)); assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
snapshot.close(); snapshot.close();
snapshot = translog.snapshot(); snapshot = translog.newSnapshot();
Translog.Create create = (Translog.Create) snapshot.next(); assertThat(snapshot, SnapshotMatchers.equalsTo(ops));
assertThat(create != null, equalTo(true)); assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
assertThat(create.source().toBytes(), equalTo(new byte[]{1}));
snapshot.close();
Translog.Snapshot snapshot1 = translog.snapshot(); // snapshot while another is open
assertThat(snapshot1, TranslogSizeMatcher.translogSize(1)); Translog.Snapshot snapshot1 = translog.newSnapshot();
assertThat(snapshot1, SnapshotMatchers.size(1));
assertThat(snapshot1.estimatedTotalOperations(), equalTo(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(); snapshot.close();
snapshot1.close(); snapshot1.close();
} }
@Test @Test
public void testSnapshotWithNewTranslog() throws IOException { public void testSnapshotWithNewTranslog() throws IOException {
Translog.Snapshot snapshot = translog.snapshot(); ArrayList<Translog.Operation> ops = new ArrayList<>();
assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, SnapshotMatchers.size(0));
snapshot.close(); snapshot.close();
translog.add(new Translog.Create("test", "1", new byte[]{1})); addToTranslogAndList(translog, ops, new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot actualSnapshot = translog.snapshot(); 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); Translog.Snapshot snapshot2 = translog.newSnapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); assertThat(snapshot2, SnapshotMatchers.equalsTo(ops));
snapshot.close(); 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(); assertThat(snapshot1, SnapshotMatchers.equalsTo(ops.get(0)));
snapshot.close(); snapshot1.close();
snapshot2.close();
} }
public void testSnapshotOnClosedTranslog() throws IOException { 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.add(new Translog.Create("test", "1", new byte[]{1}));
translog.close(); translog.close();
try { try {
Translog.Snapshot snapshot = translog.snapshot(); Translog.Snapshot snapshot = translog.newSnapshot();
fail("translog is closed"); fail("translog is closed");
} catch (TranslogException ex) { } catch (TranslogException ex) {
assertEquals(ex.getMessage(), "current translog is already closed"); assertThat(ex.getMessage(), containsString("can't increment channel"));
} }
} }
@Test @Test
public void deleteOnRollover() throws IOException { public void deleteOnSnapshotRelease() throws Exception {
translog.add(new Translog.Create("test", "1", new byte[]{1})); ArrayList<Translog.Operation> firstOps = new ArrayList<>();
addToTranslogAndList(translog, firstOps, new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot firstSnapshot = translog.snapshot(); Translog.Snapshot firstSnapshot = translog.newSnapshot();
assertThat(firstSnapshot, TranslogSizeMatcher.translogSize(1));
assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1)); assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1));
translog.newTransientTranslog(2); translog.newTranslog();
translog.markCommitted(translog.currentId());
assertFileIsPresent(translog, 1); assertFileIsPresent(translog, 1);
translog.add(new Translog.Index("test", "2", new byte[]{2})); ArrayList<Translog.Operation> secOps = new ArrayList<>();
assertThat(firstSnapshot, TranslogSizeMatcher.translogSize(1)); addToTranslogAndList(translog, secOps, new Translog.Index("test", "2", new byte[]{2}));
assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1)); assertThat(firstSnapshot.estimatedTotalOperations(), equalTo(1));
if (randomBoolean()) { assertNotLeaking();
translog.clearUnreferenced();
} Translog.Snapshot secondSnapshot = translog.newSnapshot();
translog.makeTransientCurrent();
Translog.Snapshot secondSnapshot = translog.snapshot();
translog.add(new Translog.Index("test", "3", new byte[]{3})); 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)); assertThat(secondSnapshot.estimatedTotalOperations(), equalTo(1));
assertFileIsPresent(translog, 1); assertFileIsPresent(translog, 1);
assertFileIsPresent(translog, 2); assertFileIsPresent(translog, 2);
if (randomBoolean()) { assertNotLeaking();
translog.clearUnreferenced();
}
firstSnapshot.close(); firstSnapshot.close();
assertFileDeleted(translog, 1); assertFileDeleted(translog, 1);
assertFileIsPresent(translog, 2); assertFileIsPresent(translog, 2);
secondSnapshot.close(); secondSnapshot.close();
assertFileIsPresent(translog, 2); // it's the current nothing should be deleted assertFileIsPresent(translog, 2); // it's the current nothing should be deleted
if (randomBoolean()) { assertNotLeaking();
translog.clearUnreferenced(); translog.newTranslog();
} translog.markCommitted(translog.currentId());
translog.newTransientTranslog(3); assertNotLeaking();
translog.makeTransientCurrent();
if (randomBoolean()) {
translog.clearUnreferenced();
}
assertFileIsPresent(translog, 3); // it's the current nothing should be deleted assertFileIsPresent(translog, 3); // it's the current nothing should be deleted
assertFileDeleted(translog, 2); 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; return;
} }
fail(translog.getFilename(id) + " is not present in any location: " + translog.location()); fail(translog.getFilename(id) + " is not present in any location: " + translog.location());
} }
public void assertFileDeleted(Translog translog, long id) { public void assertFileDeleted(FsTranslog translog, long id) {
assertFalse(Files.exists(translog.location().resolve(translog.getFilename(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 { static class LocationOperation {
final Translog.Operation operation; final Translog.Operation operation;
final Translog.Location location; final Translog.Location location;
@ -445,6 +398,7 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
this.operation = operation; this.operation = operation;
this.location = location; this.location = location;
} }
} }
@Test @Test
@ -482,11 +436,8 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
randomFrom(VersionType.values())); randomFrom(VersionType.values()));
break; break;
case DELETE_BY_QUERY: case DELETE_BY_QUERY:
op = new Translog.DeleteByQuery( // deprecated
new BytesArray(randomRealisticUnicodeOfLengthBetween(10, 400).getBytes("UTF-8")), continue;
new String[]{randomRealisticUnicodeOfLengthBetween(10, 400)},
"test");
break;
default: default:
throw new ElasticsearchException("not supported op type"); throw new ElasticsearchException("not supported op type");
} }
@ -544,14 +495,6 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
assertEquals(expDelOp.version(), delOp.version()); assertEquals(expDelOp.version(), delOp.version());
assertEquals(expDelOp.versionType(), delOp.versionType()); assertEquals(expDelOp.versionType(), delOp.versionType());
break; 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: default:
throw new ElasticsearchException("unsupported opType"); throw new ElasticsearchException("unsupported opType");
} }
@ -655,10 +598,11 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
@Test @Test
public void testVerifyTranslogIsNotDeleted() throws IOException { 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.add(new Translog.Create("test", "1", new byte[]{1}));
Translog.Snapshot snapshot = translog.snapshot(); Translog.Snapshot snapshot = translog.newSnapshot();
assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); assertThat(snapshot, SnapshotMatchers.size(1));
assertFileIsPresent(translog, 1);
assertThat(snapshot.estimatedTotalOperations(), equalTo(1)); assertThat(snapshot.estimatedTotalOperations(), equalTo(1));
if (randomBoolean()) { if (randomBoolean()) {
translog.close(); translog.close();
@ -668,6 +612,181 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
translog.close(); 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; package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.translog.AbstractSimpleTranslogTests; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.junit.annotations.TestLogging;
import java.io.IOException; 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 @Override
protected Translog create(Path translogDir) throws IOException { protected FsTranslog create() throws IOException {
return new FsTranslog(shardId, return new FsTranslog(shardId,
ImmutableSettings.settingsBuilder() ImmutableSettings.settingsBuilder()
.put("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name()) .put("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name())
.put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024)) .put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024))
.build(), .build(),
translogDir BigArrays.NON_RECYCLING_INSTANCE, translogDir
); );
} }
} }

View File

@ -20,22 +20,21 @@
package org.elasticsearch.index.translog.fs; package org.elasticsearch.index.translog.fs;
import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.translog.AbstractSimpleTranslogTests; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.test.junit.annotations.TestLogging;
import java.io.IOException; 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 @Override
protected Translog create(Path translogDir) throws IOException { protected FsTranslog create() throws IOException {
return new FsTranslog(shardId, return new FsTranslog(shardId,
ImmutableSettings.settingsBuilder().put("index.translog.fs.type", FsTranslogFile.Type.SIMPLE.name()).build(), 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; continue;
} }
id = idGenerator.incrementAndGet(); 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(); indexCounter.incrementAndGet();
} }
} }

View File

@ -60,7 +60,6 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.junit.*; import org.junit.*;
import org.junit.rules.RuleChain; import org.junit.rules.RuleChain;
import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.nio.file.FileSystem; 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 */ /** 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 // 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.Engine;
import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.translog.fs.FsTranslog;
/** /**
* *
*/ */
public final class MockEngineFactory implements EngineFactory { public final class MockEngineFactory implements EngineFactory {
@Override @Override
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery) { public Engine newReadWriteEngine(EngineConfig config, FsTranslog translog, boolean skipTranslogRecovery) {
return new MockInternalEngine(config, skipTranslogRecovery); return new MockInternalEngine(config, translog, skipTranslogRecovery);
} }
@Override @Override

View File

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

View File

@ -22,7 +22,6 @@ package org.elasticsearch.test.store;
import com.carrotsearch.randomizedtesting.SeedUtils; import com.carrotsearch.randomizedtesting.SeedUtils;
import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.google.common.base.Charsets; import com.google.common.base.Charsets;
import org.apache.lucene.index.CheckIndex; import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.store.*; import org.apache.lucene.store.*;
@ -39,14 +38,14 @@ import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.*; import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreModule; import org.elasticsearch.index.store.IndexStoreModule;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Assert; import org.junit.Assert;
import java.io.Closeable; import java.io.Closeable;
@ -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 // 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 if (indexShard.engine().hasUncommittedChanges()) { // only if we have any changes
logger.info("{} flushing in order to run checkindex", indexShard.shardId()); 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()); logger.info("{} flush finished in beforeIndexShardClosed", indexShard.shardId());
canRun = true; canRun = true;
@ -138,7 +137,6 @@ public class MockFSDirectoryService extends FsDirectoryService {
} }
@Override @Override
public Directory newDirectory() throws IOException { public Directory newDirectory() throws IOException {
return wrap(delegateService.newDirectory()); return wrap(delegateService.newDirectory());