Merge pull request #11225 from s1monw/remove_flush_needed

[ENGINE] Remove flushNeeded in favor of IW#hasUncommittedChanges()
This commit is contained in:
Simon Willnauer 2015-05-19 14:53:09 +02:00
commit b3f1a5da4a
1 changed files with 1 additions and 11 deletions

View File

@ -92,9 +92,6 @@ public class InternalEngine extends Engine {
private final SearcherFactory searcherFactory;
private final SearcherManager searcherManager;
// we use flushNeeded here, since if there are no changes, then the commit won't write
// will not really happen, and then the commitUserData and the new translog will not be reflected
private volatile boolean flushNeeded = false;
private final Lock flushLock = new ReentrantLock();
private final ReentrantLock optimizeLock = new ReentrantLock();
@ -344,7 +341,6 @@ public class InternalEngine extends Engine {
innerCreate(create);
}
}
flushNeeded = true;
} catch (OutOfMemoryError | IllegalStateException | IOException t) {
maybeFailEngine("create", t);
throw new CreateFailedEngineException(shardId, create, t);
@ -451,7 +447,6 @@ public class InternalEngine extends Engine {
created = innerIndex(index);
}
}
flushNeeded = true;
} catch (OutOfMemoryError | IllegalStateException | IOException t) {
maybeFailEngine("index", t);
throw new IndexFailedEngineException(shardId, index, t);
@ -549,7 +544,6 @@ public class InternalEngine extends Engine {
ensureOpen();
// NOTE: we don't throttle this when merges fall behind because delete-by-id does not create new segments:
innerDelete(delete);
flushNeeded = true;
} catch (OutOfMemoryError | IllegalStateException | IOException t) {
maybeFailEngine("delete", t);
throw new DeleteFailedEngineException(shardId, delete, t);
@ -644,7 +638,6 @@ public class InternalEngine extends Engine {
indexWriter.deleteDocuments(query);
translog.add(new Translog.DeleteByQuery(delete));
flushNeeded = true;
} catch (Throwable t) {
maybeFailEngine("delete_by_query", t);
throw new DeleteByQueryFailedEngineException(shardId, delete, t);
@ -710,9 +703,7 @@ public class InternalEngine extends Engine {
logger.trace("acquired flush lock immediately");
}
try {
if (flushNeeded || force) {
flushNeeded = false;
final long translogId;
if (indexWriter.hasUncommittedChanges() || force) {
try {
translog.prepareCommit();
logger.trace("starting commit for flush; commitTranslog=true");
@ -721,7 +712,6 @@ public class InternalEngine extends Engine {
translog.commit();
// we need to refresh in order to clear older version values
refresh("version_table_flush");
} catch (Throwable e) {
throw new FlushFailedEngineException(shardId, e);
}