|  | @@ -28,16 +28,7 @@ import org.apache.lucene.codecs.Codec;
 | 
	
		
			
				|  |  |  import org.apache.lucene.document.Field;
 | 
	
		
			
				|  |  |  import org.apache.lucene.document.NumericDocValuesField;
 | 
	
		
			
				|  |  |  import org.apache.lucene.document.TextField;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.DirectoryReader;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.IndexWriterConfig;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.LiveIndexWriterConfig;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.LogByteSizeMergePolicy;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.MergePolicy;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.NoMergePolicy;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.SnapshotDeletionPolicy;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.Term;
 | 
	
		
			
				|  |  | -import org.apache.lucene.index.TieredMergePolicy;
 | 
	
		
			
				|  |  | +import org.apache.lucene.index.*;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.IndexSearcher;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.MatchAllDocsQuery;
 | 
	
		
			
				|  |  |  import org.apache.lucene.search.TermQuery;
 | 
	
	
		
			
				|  | @@ -62,6 +53,7 @@ import org.elasticsearch.common.lucene.uid.Versions;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.settings.Settings;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.unit.ByteSizeUnit;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.unit.ByteSizeValue;
 | 
	
		
			
				|  |  | +import org.elasticsearch.common.unit.TimeValue;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.util.BigArrays;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.Index;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.IndexSettings;
 | 
	
	
		
			
				|  | @@ -118,6 +110,7 @@ import java.util.Map;
 | 
	
		
			
				|  |  |  import java.util.concurrent.BrokenBarrierException;
 | 
	
		
			
				|  |  |  import java.util.concurrent.CountDownLatch;
 | 
	
		
			
				|  |  |  import java.util.concurrent.CyclicBarrier;
 | 
	
		
			
				|  |  | +import java.util.concurrent.TimeUnit;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicBoolean;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicInteger;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicReference;
 | 
	
	
		
			
				|  | @@ -276,7 +269,7 @@ public class InternalEngineTests extends ESTestCase {
 | 
	
		
			
				|  |  |              public void onFailedEngine(String reason, @Nullable Throwable t) {
 | 
	
		
			
				|  |  |                  // we don't need to notify anybody in this test
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | -        }, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
 | 
	
		
			
				|  |  | +        }, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5));
 | 
	
		
			
				|  |  |          try {
 | 
	
		
			
				|  |  |              config.setCreate(Lucene.indexExists(store.directory()) == false);
 | 
	
		
			
				|  |  |          } catch (IOException e) {
 | 
	
	
		
			
				|  | @@ -796,6 +789,64 @@ public class InternalEngineTests extends ESTestCase {
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    public void testRenewSyncFlush() throws Exception {
 | 
	
		
			
				|  |  | +        final int iters = randomIntBetween(2, 5); // run this a couple of times to get some coverage
 | 
	
		
			
				|  |  | +        for (int i = 0; i < iters; i++) {
 | 
	
		
			
				|  |  | +            try (Store store = createStore();
 | 
	
		
			
				|  |  | +                 InternalEngine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings),
 | 
	
		
			
				|  |  | +                         new LogDocMergePolicy()), false)) {
 | 
	
		
			
				|  |  | +                final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20);
 | 
	
		
			
				|  |  | +                ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
 | 
	
		
			
				|  |  | +                Engine.Index doc1 = new Engine.Index(newUid("1"), doc);
 | 
	
		
			
				|  |  | +                engine.index(doc1);
 | 
	
		
			
				|  |  | +                assertEquals(engine.getLastWriteNanos(), doc1.startTime());
 | 
	
		
			
				|  |  | +                engine.flush();
 | 
	
		
			
				|  |  | +                Engine.Index doc2 = new Engine.Index(newUid("2"), doc);
 | 
	
		
			
				|  |  | +                engine.index(doc2);
 | 
	
		
			
				|  |  | +                assertEquals(engine.getLastWriteNanos(), doc2.startTime());
 | 
	
		
			
				|  |  | +                engine.flush();
 | 
	
		
			
				|  |  | +                final boolean forceMergeFlushes = randomBoolean();
 | 
	
		
			
				|  |  | +                if (forceMergeFlushes) {
 | 
	
		
			
				|  |  | +                    engine.index(new Engine.Index(newUid("3"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos()));
 | 
	
		
			
				|  |  | +                } else {
 | 
	
		
			
				|  |  | +                    engine.index(new Engine.Index(newUid("3"), doc));
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +                Engine.CommitId commitID = engine.flush();
 | 
	
		
			
				|  |  | +                assertEquals("should succeed to flush commit with right id and no pending doc", engine.syncFlush(syncId, commitID),
 | 
	
		
			
				|  |  | +                        Engine.SyncedFlushResult.SUCCESS);
 | 
	
		
			
				|  |  | +                assertEquals(3, engine.segments(false).size());
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +                engine.forceMerge(false, 1, false, false, false);
 | 
	
		
			
				|  |  | +                if (forceMergeFlushes == false) {
 | 
	
		
			
				|  |  | +                    engine.refresh("make all segments visible");
 | 
	
		
			
				|  |  | +                    assertEquals(4, engine.segments(false).size());
 | 
	
		
			
				|  |  | +                    assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId);
 | 
	
		
			
				|  |  | +                    assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId);
 | 
	
		
			
				|  |  | +                    assertTrue(engine.tryRenewSyncCommit());
 | 
	
		
			
				|  |  | +                    assertEquals(1, engine.segments(false).size());
 | 
	
		
			
				|  |  | +                } else {
 | 
	
		
			
				|  |  | +                    assertBusy(() -> assertEquals(1, engine.segments(false).size()));
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +                assertEquals(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID), syncId);
 | 
	
		
			
				|  |  | +                assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId);
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +                if (randomBoolean()) {
 | 
	
		
			
				|  |  | +                    Engine.Index doc4 = new Engine.Index(newUid("4"), doc);
 | 
	
		
			
				|  |  | +                    engine.index(doc4);
 | 
	
		
			
				|  |  | +                    assertEquals(engine.getLastWriteNanos(), doc4.startTime());
 | 
	
		
			
				|  |  | +                } else {
 | 
	
		
			
				|  |  | +                    Engine.Delete delete = new Engine.Delete(doc1.type(), doc1.id(), doc1.uid());
 | 
	
		
			
				|  |  | +                    engine.delete(delete);
 | 
	
		
			
				|  |  | +                    assertEquals(engine.getLastWriteNanos(), delete.startTime());
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +                assertFalse(engine.tryRenewSyncCommit());
 | 
	
		
			
				|  |  | +                engine.flush();
 | 
	
		
			
				|  |  | +                assertNull(store.readLastCommittedSegmentsInfo().getUserData().get(Engine.SYNC_COMMIT_ID));
 | 
	
		
			
				|  |  | +                assertNull(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID));
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      public void testSycnedFlushSurvivesEngineRestart() throws IOException {
 | 
	
		
			
				|  |  |          final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20);
 | 
	
		
			
				|  |  |          ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
 | 
	
	
		
			
				|  | @@ -1923,7 +1974,7 @@ public class InternalEngineTests extends ESTestCase {
 | 
	
		
			
				|  |  |          EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettings()
 | 
	
		
			
				|  |  |                  , null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(),
 | 
	
		
			
				|  |  |                  config.getAnalyzer(), config.getSimilarity(), new CodecService(INDEX_SETTINGS, null), config.getEventListener()
 | 
	
		
			
				|  |  | -        , config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig);
 | 
	
		
			
				|  |  | +        , config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5));
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          try {
 | 
	
		
			
				|  |  |              new InternalEngine(brokenConfig, false);
 |