|  | @@ -22,11 +22,7 @@ package org.elasticsearch.index.engine.internal;
 | 
	
		
			
				|  |  |  import com.google.common.collect.Lists;
 | 
	
		
			
				|  |  |  import org.apache.lucene.index.*;
 | 
	
		
			
				|  |  |  import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
 | 
	
		
			
				|  |  | -import org.apache.lucene.search.FilteredQuery;
 | 
	
		
			
				|  |  | -import org.apache.lucene.search.IndexSearcher;
 | 
	
		
			
				|  |  | -import org.apache.lucene.search.Query;
 | 
	
		
			
				|  |  | -import org.apache.lucene.search.SearcherFactory;
 | 
	
		
			
				|  |  | -import org.apache.lucene.search.SearcherManager;
 | 
	
		
			
				|  |  | +import org.apache.lucene.search.*;
 | 
	
		
			
				|  |  |  import org.apache.lucene.store.AlreadyClosedException;
 | 
	
		
			
				|  |  |  import org.apache.lucene.store.LockObtainFailedException;
 | 
	
		
			
				|  |  |  import org.apache.lucene.util.BytesRef;
 | 
	
	
		
			
				|  | @@ -37,7 +33,6 @@ import org.elasticsearch.ExceptionsHelper;
 | 
	
		
			
				|  |  |  import org.elasticsearch.cluster.routing.operation.hash.djb.DjbHashFunction;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.Nullable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.Preconditions;
 | 
	
		
			
				|  |  | -import org.elasticsearch.common.inject.Inject;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lease.Releasable;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lease.Releasables;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.logging.ESLogger;
 | 
	
	
		
			
				|  | @@ -46,36 +41,15 @@ import org.elasticsearch.common.lucene.Lucene;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.lucene.uid.Versions;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.math.MathUtils;
 | 
	
		
			
				|  |  | -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.concurrent.AbstractRunnable;
 | 
	
		
			
				|  |  | -import org.elasticsearch.common.util.concurrent.EsExecutors;
 | 
	
		
			
				|  |  |  import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.analysis.AnalysisService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.codec.CodecService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.CreateFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.DeleteByQueryFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.DeleteFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.DocumentAlreadyExistsException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.Engine;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.EngineAlreadyStartedException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.EngineClosedException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.EngineCreationFailureException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.EngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.FlushFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.IndexFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.OptimizeFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.RecoveryEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.RefreshFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.Segment;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.SegmentsStats;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.SnapshotFailedEngineException;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.engine.VersionConflictEngineException;
 | 
	
		
			
				|  |  | +import org.elasticsearch.index.engine.*;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.indexing.ShardIndexingService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.mapper.Uid;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.merge.OnGoingMerge;
 | 
	
	
		
			
				|  | @@ -83,9 +57,6 @@ import org.elasticsearch.index.merge.policy.ElasticsearchMergePolicy;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.merge.policy.MergePolicyProvider;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.search.nested.IncludeNestedDocsQuery;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.settings.IndexSettings;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.settings.IndexSettingsService;
 | 
	
		
			
				|  |  | -import org.elasticsearch.index.shard.AbstractIndexShardComponent;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.shard.ShardId;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.similarity.SimilarityService;
 | 
	
		
			
				|  |  |  import org.elasticsearch.index.store.Store;
 | 
	
	
		
			
				|  | @@ -95,14 +66,7 @@ import org.elasticsearch.indices.warmer.InternalIndicesWarmer;
 | 
	
		
			
				|  |  |  import org.elasticsearch.threadpool.ThreadPool;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  import java.io.IOException;
 | 
	
		
			
				|  |  | -import java.util.Arrays;
 | 
	
		
			
				|  |  | -import java.util.Collections;
 | 
	
		
			
				|  |  | -import java.util.Comparator;
 | 
	
		
			
				|  |  | -import java.util.HashMap;
 | 
	
		
			
				|  |  | -import java.util.List;
 | 
	
		
			
				|  |  | -import java.util.Map;
 | 
	
		
			
				|  |  | -import java.util.Set;
 | 
	
		
			
				|  |  | -import java.util.concurrent.CopyOnWriteArrayList;
 | 
	
		
			
				|  |  | +import java.util.*;
 | 
	
		
			
				|  |  |  import java.util.concurrent.TimeUnit;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicBoolean;
 | 
	
		
			
				|  |  |  import java.util.concurrent.atomic.AtomicInteger;
 | 
	
	
		
			
				|  | @@ -115,14 +79,16 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 | 
	
		
			
				|  |  |  /**
 | 
	
		
			
				|  |  |   *
 | 
	
		
			
				|  |  |   */
 | 
	
		
			
				|  |  | -public class InternalEngine extends AbstractIndexShardComponent implements Engine {
 | 
	
		
			
				|  |  | +public class InternalEngine implements Engine {
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    protected final ESLogger logger;
 | 
	
		
			
				|  |  | +    protected final ShardId shardId;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private volatile boolean failEngineOnCorruption;
 | 
	
		
			
				|  |  |      private volatile ByteSizeValue indexingBufferSize;
 | 
	
		
			
				|  |  |      private volatile int indexConcurrency;
 | 
	
		
			
				|  |  | -    private volatile boolean compoundOnFlush = true;
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    private long gcDeletesInMillis;
 | 
	
		
			
				|  |  | +    private volatile boolean compoundOnFlush;
 | 
	
		
			
				|  |  | +    private volatile long gcDeletesInMillis;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      /** When we last pruned expired tombstones from versionMap.deletes: */
 | 
	
		
			
				|  |  |      private volatile long lastDeleteVersionPruneTimeMSec;
 | 
	
	
		
			
				|  | @@ -134,7 +100,6 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |      private final ThreadPool threadPool;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private final ShardIndexingService indexingService;
 | 
	
		
			
				|  |  | -    private final IndexSettingsService indexSettingsService;
 | 
	
		
			
				|  |  |      @Nullable
 | 
	
		
			
				|  |  |      private final InternalIndicesWarmer warmer;
 | 
	
		
			
				|  |  |      private final Store store;
 | 
	
	
		
			
				|  | @@ -181,38 +146,38 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private final Object refreshMutex = new Object();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private final ApplySettings applySettings = new ApplySettings();
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    private volatile boolean failOnMergeFailure;
 | 
	
		
			
				|  |  |      private Throwable failedEngine = null;
 | 
	
		
			
				|  |  |      private final Lock failEngineLock = new ReentrantLock();
 | 
	
		
			
				|  |  | -    private final CopyOnWriteArrayList<FailedEngineListener> failedEngineListeners = new CopyOnWriteArrayList<>();
 | 
	
		
			
				|  |  | +    private final FailedEngineListener failedEngineListener;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private final AtomicLong translogIdGenerator = new AtomicLong();
 | 
	
		
			
				|  |  |      private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean();
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private SegmentInfos lastCommittedSegmentInfos;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    private IndexThrottle throttle;
 | 
	
		
			
				|  |  | +    private final IndexThrottle throttle;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    @Inject
 | 
	
		
			
				|  |  | -    public InternalEngine(ShardId shardId, @IndexSettings Settings indexSettings, ThreadPool threadPool,
 | 
	
		
			
				|  |  | -                          IndexSettingsService indexSettingsService, ShardIndexingService indexingService, @Nullable IndicesWarmer warmer,
 | 
	
		
			
				|  |  | +    public InternalEngine(ShardId shardId, ESLogger logger, CodecService codecService, ThreadPool threadPool,
 | 
	
		
			
				|  |  | +                          ShardIndexingService indexingService, @Nullable IndicesWarmer warmer,
 | 
	
		
			
				|  |  |                            Store store, SnapshotDeletionPolicy deletionPolicy, Translog translog,
 | 
	
		
			
				|  |  |                            MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler,
 | 
	
		
			
				|  |  | -                          AnalysisService analysisService, SimilarityService similarityService, CodecService codecService) throws EngineException {
 | 
	
		
			
				|  |  | -        super(shardId, indexSettings);
 | 
	
		
			
				|  |  | +                          AnalysisService analysisService, SimilarityService similarityService,
 | 
	
		
			
				|  |  | +                          boolean enableGcDeletes, long gcDeletesInMillis, ByteSizeValue indexingBufferSize, String codecName,
 | 
	
		
			
				|  |  | +                          boolean compoundOnFlush, int indexConcurrency, boolean optimizeAutoGenerateId, boolean failEngineOnCorruption,
 | 
	
		
			
				|  |  | +                          FailedEngineListener failedEngineListener) throws EngineException {
 | 
	
		
			
				|  |  |          Preconditions.checkNotNull(store, "Store must be provided to the engine");
 | 
	
		
			
				|  |  |          Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the engine");
 | 
	
		
			
				|  |  |          Preconditions.checkNotNull(translog, "Translog must be provided to the engine");
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        this.gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES, TimeValue.timeValueSeconds(60)).millis();
 | 
	
		
			
				|  |  | -        this.indexingBufferSize = componentSettings.getAsBytesSize("index_buffer_size", new ByteSizeValue(64, ByteSizeUnit.MB)); // not really important, as it is set by the IndexingMemory manager
 | 
	
		
			
				|  |  | -        this.codecName = indexSettings.get(INDEX_CODEC, "default");
 | 
	
		
			
				|  |  | +        this.shardId = shardId;
 | 
	
		
			
				|  |  | +        this.logger = logger;
 | 
	
		
			
				|  |  | +        this.gcDeletesInMillis = gcDeletesInMillis;
 | 
	
		
			
				|  |  | +        this.enableGcDeletes = enableGcDeletes;
 | 
	
		
			
				|  |  | +        this.indexingBufferSize = indexingBufferSize;
 | 
	
		
			
				|  |  | +        this.codecName = codecName;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          this.threadPool = threadPool;
 | 
	
		
			
				|  |  |          this.lastDeleteVersionPruneTimeMSec = threadPool.estimatedTimeInMillis();
 | 
	
		
			
				|  |  | -        this.indexSettingsService = indexSettingsService;
 | 
	
		
			
				|  |  |          this.indexingService = indexingService;
 | 
	
		
			
				|  |  |          this.warmer = (InternalIndicesWarmer) warmer;
 | 
	
		
			
				|  |  |          this.store = store;
 | 
	
	
		
			
				|  | @@ -223,22 +188,20 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |          this.analysisService = analysisService;
 | 
	
		
			
				|  |  |          this.similarityService = similarityService;
 | 
	
		
			
				|  |  |          this.codecService = codecService;
 | 
	
		
			
				|  |  | -        this.compoundOnFlush = indexSettings.getAsBoolean(INDEX_COMPOUND_ON_FLUSH, this.compoundOnFlush);
 | 
	
		
			
				|  |  | -        this.indexConcurrency = indexSettings.getAsInt(INDEX_INDEX_CONCURRENCY, Math.max(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, (int) (EsExecutors.boundedNumberOfProcessors(indexSettings) * 0.65)));
 | 
	
		
			
				|  |  | +        this.compoundOnFlush = compoundOnFlush;
 | 
	
		
			
				|  |  | +        this.indexConcurrency = indexConcurrency;
 | 
	
		
			
				|  |  |          this.versionMap = new LiveVersionMap();
 | 
	
		
			
				|  |  |          this.dirtyLocks = new Object[indexConcurrency * 50]; // we multiply it to have enough...
 | 
	
		
			
				|  |  |          for (int i = 0; i < dirtyLocks.length; i++) {
 | 
	
		
			
				|  |  |              dirtyLocks[i] = new Object();
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | -        this.optimizeAutoGenerateId = indexSettings.getAsBoolean("index.optimize_auto_generated_id", true);
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -        this.indexSettingsService.addListener(applySettings);
 | 
	
		
			
				|  |  | -        this.failEngineOnCorruption = indexSettings.getAsBoolean(INDEX_FAIL_ON_CORRUPTION, true);
 | 
	
		
			
				|  |  | -        this.failOnMergeFailure = indexSettings.getAsBoolean(INDEX_FAIL_ON_MERGE_FAILURE, true);
 | 
	
		
			
				|  |  | -        if (failOnMergeFailure) {
 | 
	
		
			
				|  |  | -            this.mergeScheduler.addFailureListener(new FailEngineOnMergeFailure());
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | +        this.optimizeAutoGenerateId = optimizeAutoGenerateId;
 | 
	
		
			
				|  |  | +        this.failEngineOnCorruption = failEngineOnCorruption;
 | 
	
		
			
				|  |  | +        this.failedEngineListener = failedEngineListener;
 | 
	
		
			
				|  |  | +        // will be decremented in close()
 | 
	
		
			
				|  |  |          store.incRef();
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +        throttle = new IndexThrottle();
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      @Override
 | 
	
	
		
			
				|  | @@ -273,7 +236,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      @Override
 | 
	
		
			
				|  |  |      public void addFailedEngineListener(FailedEngineListener listener) {
 | 
	
		
			
				|  |  | -        failedEngineListeners.add(listener);
 | 
	
		
			
				|  |  | +        throw new UnsupportedOperationException("addFailedEngineListener is not supported by InternalEngineImpl. Use InternalEngine.");
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      @Override
 | 
	
	
		
			
				|  | @@ -287,14 +250,12 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |              if (closed) {
 | 
	
		
			
				|  |  |                  throw new EngineClosedException(shardId);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |              if (logger.isDebugEnabled()) {
 | 
	
		
			
				|  |  |                  logger.debug("starting engine");
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |              try {
 | 
	
		
			
				|  |  |                  this.indexWriter = createWriter();
 | 
	
		
			
				|  |  | -                mergeScheduler.removeListener(this.throttle);
 | 
	
		
			
				|  |  | -                this.throttle = new IndexThrottle(mergeScheduler, logger, indexingService);
 | 
	
		
			
				|  |  | -                mergeScheduler.addListener(throttle);
 | 
	
		
			
				|  |  |              } catch (IOException e) {
 | 
	
		
			
				|  |  |                  maybeFailEngine(e, "start");
 | 
	
		
			
				|  |  |                  if (this.indexWriter != null) {
 | 
	
	
		
			
				|  | @@ -303,7 +264,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                          indexWriter = null;
 | 
	
		
			
				|  |  |                          pending.rollback();
 | 
	
		
			
				|  |  |                      } catch (IOException e1) {
 | 
	
		
			
				|  |  | -                       e.addSuppressed(e1);
 | 
	
		
			
				|  |  | +                        e.addSuppressed(e1);
 | 
	
		
			
				|  |  |                      }
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |                  throw new EngineCreationFailureException(shardId, "failed to create engine", e);
 | 
	
	
		
			
				|  | @@ -343,13 +304,23 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    @Override
 | 
	
		
			
				|  |  | +    public void stop() throws EngineException {
 | 
	
		
			
				|  |  | +        throw new UnsupportedOperationException("stop() is not supported by InternalEngineImpl. Use InternalEngine.");
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      private void readLastCommittedSegmentsInfo() throws IOException {
 | 
	
		
			
				|  |  |          lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo();
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    @Override
 | 
	
		
			
				|  |  | +    public ShardId shardId() {
 | 
	
		
			
				|  |  | +        return shardId;
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      @Override
 | 
	
		
			
				|  |  |      public TimeValue defaultRefreshInterval() {
 | 
	
		
			
				|  |  | -        return new TimeValue(1, TimeUnit.SECONDS);
 | 
	
		
			
				|  |  | +        return InternalEngineHolder.DEFAULT_REFRESH_ITERVAL;
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      /** return the current indexing buffer size setting * */
 | 
	
	
		
			
				|  | @@ -362,6 +333,46 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |          this.enableGcDeletes = enableGcDeletes;
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    public void updateSettings(final long gcDeletesInMillis, final boolean compoundOnFlush, boolean failEngineOnCorruption, final int indexConcurrency, final String codecName) {
 | 
	
		
			
				|  |  | +        ensureOpen();
 | 
	
		
			
				|  |  | +        if (this.gcDeletesInMillis != gcDeletesInMillis) {
 | 
	
		
			
				|  |  | +            logger.trace("[impl] updating gcDeletesInMillis from [{}] to [{}]", this.gcDeletesInMillis, gcDeletesInMillis);
 | 
	
		
			
				|  |  | +            this.gcDeletesInMillis = gcDeletesInMillis;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +        if (this.compoundOnFlush != compoundOnFlush) {
 | 
	
		
			
				|  |  | +            this.compoundOnFlush = compoundOnFlush;
 | 
	
		
			
				|  |  | +            logger.trace("[impl] updating compoundOnFlush from [{}] to [{}]", this.compoundOnFlush, compoundOnFlush);
 | 
	
		
			
				|  |  | +            indexWriter.getConfig().setUseCompoundFile(compoundOnFlush);
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +        if (this.failEngineOnCorruption != failEngineOnCorruption) {
 | 
	
		
			
				|  |  | +            logger.trace("[impl] updating failEngineOnCorruption from [{}] to [{}]", this.failEngineOnCorruption, failEngineOnCorruption);
 | 
	
		
			
				|  |  | +            this.failEngineOnCorruption = failEngineOnCorruption;
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +        if (indexConcurrency != this.indexConcurrency || !codecName.equals(this.codecName)) {
 | 
	
		
			
				|  |  | +            boolean requiresFlushing = false;
 | 
	
		
			
				|  |  | +            try (InternalLock _ = readLock.acquire()) {
 | 
	
		
			
				|  |  | +                if (indexConcurrency != this.indexConcurrency) {
 | 
	
		
			
				|  |  | +                    logger.trace("[impl] updating indexConcurrency from [{}] to [{}]", this.indexConcurrency, indexConcurrency);
 | 
	
		
			
				|  |  | +                    this.indexConcurrency = indexConcurrency;
 | 
	
		
			
				|  |  | +                    // we have to flush in this case, since it only applies on a new index writer
 | 
	
		
			
				|  |  | +                    requiresFlushing = true;
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +                if (!codecName.equals(this.codecName)) {
 | 
	
		
			
				|  |  | +                    logger.trace("[impl] updating codecName from [{}] to [{}]", this.codecName, codecName);
 | 
	
		
			
				|  |  | +                    this.codecName = codecName;
 | 
	
		
			
				|  |  | +                    // we want to flush in this case, so the new codec will be reflected right away...
 | 
	
		
			
				|  |  | +                    requiresFlushing = true;
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +            } finally {
 | 
	
		
			
				|  |  | +                if (requiresFlushing) {
 | 
	
		
			
				|  |  | +                    flush(new Flush().type(Flush.Type.NEW_WRITER));
 | 
	
		
			
				|  |  | +                }
 | 
	
		
			
				|  |  | +            }
 | 
	
		
			
				|  |  | +        }
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  | +    @Override
 | 
	
		
			
				|  |  |      public GetResult get(Get get) throws EngineException {
 | 
	
		
			
				|  |  |          try (InternalLock _ = readLock.acquire()) {
 | 
	
		
			
				|  |  |              if (get.realtime()) {
 | 
	
	
		
			
				|  | @@ -863,10 +874,6 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                              indexWriter.rollback();
 | 
	
		
			
				|  |  |                          }
 | 
	
		
			
				|  |  |                          indexWriter = createWriter();
 | 
	
		
			
				|  |  | -                        mergeScheduler.removeListener(this.throttle);
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -                        this.throttle = new IndexThrottle(mergeScheduler, this.logger, indexingService);
 | 
	
		
			
				|  |  | -                        mergeScheduler.addListener(throttle);
 | 
	
		
			
				|  |  |                          // 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
 | 
	
		
			
				|  |  |                          if (flushNeeded || flush.force()) {
 | 
	
	
		
			
				|  | @@ -1035,7 +1042,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |              throw new OptimizeFailedEngineException(shardId, t);
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  | -    
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |      private void waitForMerges(boolean flushAfter) {
 | 
	
		
			
				|  |  |          try {
 | 
	
		
			
				|  |  |              currentIndexWriter().waitForMerges();
 | 
	
	
		
			
				|  | @@ -1063,9 +1070,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                  MergePolicy mp = writer.getConfig().getMergePolicy();
 | 
	
		
			
				|  |  |                  assert mp instanceof ElasticsearchMergePolicy : "MergePolicy is " + mp.getClass().getName();
 | 
	
		
			
				|  |  |                  if (optimize.upgrade()) {
 | 
	
		
			
				|  |  | -                    ((ElasticsearchMergePolicy)mp).setUpgradeInProgress(true);
 | 
	
		
			
				|  |  | +                    ((ElasticsearchMergePolicy) mp).setUpgradeInProgress(true);
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  | -                
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |                  if (optimize.onlyExpungeDeletes()) {
 | 
	
		
			
				|  |  |                      writer.forceMergeDeletes(false);
 | 
	
		
			
				|  |  |                  } else if (optimize.maxNumSegments() <= 0) {
 | 
	
	
		
			
				|  | @@ -1081,7 +1088,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                  optimizeMutex.set(false);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | -        
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |          // wait for the merges outside of the read lock
 | 
	
		
			
				|  |  |          if (optimize.waitForMerge()) {
 | 
	
		
			
				|  |  |              waitForMerges(optimize.flush());
 | 
	
	
		
			
				|  | @@ -1092,6 +1099,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                  public void onFailure(Throwable t) {
 | 
	
		
			
				|  |  |                      logger.error("Exception while waiting for merges asynchronously after optimize", t);
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  | +
 | 
	
		
			
				|  |  |                  @Override
 | 
	
		
			
				|  |  |                  protected void doRun() throws Exception {
 | 
	
		
			
				|  |  |                      waitForMerges(true);
 | 
	
	
		
			
				|  | @@ -1180,9 +1188,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                  failEngine("corrupt file detected source: [" + source + "]", t);
 | 
	
		
			
				|  |  |                  return true;
 | 
	
		
			
				|  |  |              } else {
 | 
	
		
			
				|  |  | -                logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, INDEX_FAIL_ON_CORRUPTION, this.failEngineOnCorruption);
 | 
	
		
			
				|  |  | +                logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, InternalEngineHolder.INDEX_FAIL_ON_CORRUPTION, this.failEngineOnCorruption);
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  | -        }else if (ExceptionsHelper.isOOM(t)) {
 | 
	
		
			
				|  |  | +        } else if (ExceptionsHelper.isOOM(t)) {
 | 
	
		
			
				|  |  |              failEngine("out of memory", t);
 | 
	
		
			
				|  |  |              return true;
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -1211,7 +1219,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |                  stats.addVersionMapMemoryInBytes(versionMap.ramBytesUsed());
 | 
	
		
			
				|  |  |                  stats.addIndexWriterMemoryInBytes(indexWriter.ramBytesUsed());
 | 
	
		
			
				|  |  | -                stats.addIndexWriterMaxMemoryInBytes((long) (indexWriter.getConfig().getRAMBufferSizeMB()*1024*1024));
 | 
	
		
			
				|  |  | +                stats.addIndexWriterMaxMemoryInBytes((long) (indexWriter.getConfig().getRAMBufferSizeMB() * 1024 * 1024));
 | 
	
		
			
				|  |  |                  return stats;
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -1305,9 +1313,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |              if (!closed) {
 | 
	
		
			
				|  |  |                  try {
 | 
	
		
			
				|  |  |                      closed = true;
 | 
	
		
			
				|  |  | -                    indexSettingsService.removeListener(applySettings);
 | 
	
		
			
				|  |  |                      this.versionMap.clear();
 | 
	
		
			
				|  |  | -                    this.failedEngineListeners.clear();
 | 
	
		
			
				|  |  |                      logger.debug("close searcherManager");
 | 
	
		
			
				|  |  |                      try {
 | 
	
		
			
				|  |  |                          IOUtils.close(searcherManager);
 | 
	
	
		
			
				|  | @@ -1327,8 +1333,8 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                  } catch (Throwable e) {
 | 
	
		
			
				|  |  |                      logger.warn("failed to rollback writer on close", e);
 | 
	
		
			
				|  |  |                  } finally {
 | 
	
		
			
				|  |  | -                    indexWriter = null;
 | 
	
		
			
				|  |  |                      store.decRef();
 | 
	
		
			
				|  |  | +                    indexWriter = null;
 | 
	
		
			
				|  |  |                  }
 | 
	
		
			
				|  |  |              }
 | 
	
		
			
				|  |  |          }
 | 
	
	
		
			
				|  | @@ -1339,21 +1345,6 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |          return this.indexWriter.getConfig();
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    class FailEngineOnMergeFailure implements MergeSchedulerProvider.FailureListener {
 | 
	
		
			
				|  |  | -        @Override
 | 
	
		
			
				|  |  | -        public void onFailedMerge(MergePolicy.MergeException e) {
 | 
	
		
			
				|  |  | -            if (Lucene.isCorruptionException(e)) {
 | 
	
		
			
				|  |  | -                if (failEngineOnCorruption) {
 | 
	
		
			
				|  |  | -                    failEngine("corrupt file detected source: [merge]", e);
 | 
	
		
			
				|  |  | -                } else {
 | 
	
		
			
				|  |  | -                    logger.warn("corrupt file detected source: [merge] but [{}] is set to [{}]", e, INDEX_FAIL_ON_CORRUPTION, failEngineOnCorruption);
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -            } else {
 | 
	
		
			
				|  |  | -                failEngine("merge exception", e);
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |      @Override
 | 
	
		
			
				|  |  |      public void failEngine(String reason, Throwable failure) {
 | 
	
		
			
				|  |  |          assert failure != null;
 | 
	
	
		
			
				|  | @@ -1380,9 +1371,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |                      logger.warn("failed engine [{}]", failure, reason);
 | 
	
		
			
				|  |  |                      // we must set a failure exception, generate one if not supplied
 | 
	
		
			
				|  |  |                      failedEngine = failure;
 | 
	
		
			
				|  |  | -                    for (FailedEngineListener listener : failedEngineListeners) {
 | 
	
		
			
				|  |  | -                        listener.onFailedEngine(shardId, reason, failure);
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | +                    failedEngineListener.onFailedEngine(shardId, reason, failure);
 | 
	
		
			
				|  |  |                  } finally {
 | 
	
		
			
				|  |  |                      close();
 | 
	
		
			
				|  |  |                  }
 | 
	
	
		
			
				|  | @@ -1425,7 +1414,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |              config.setCommitOnClose(false); // we by default don't commit on close
 | 
	
		
			
				|  |  |              config.setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND);
 | 
	
		
			
				|  |  |              config.setIndexDeletionPolicy(deletionPolicy);
 | 
	
		
			
				|  |  | -            config.setInfoStream(new LoggerInfoStream(indexSettings, shardId));
 | 
	
		
			
				|  |  | +            config.setInfoStream(new LoggerInfoStream(logger));
 | 
	
		
			
				|  |  |              config.setMergeScheduler(mergeScheduler.newMergeScheduler());
 | 
	
		
			
				|  |  |              MergePolicy mergePolicy = mergePolicyProvider.getMergePolicy();
 | 
	
		
			
				|  |  |              // Give us the opportunity to upgrade old segments while performing
 | 
	
	
		
			
				|  | @@ -1475,63 +1464,6 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    public static final String INDEX_INDEX_CONCURRENCY = "index.index_concurrency";
 | 
	
		
			
				|  |  | -    public static final String INDEX_COMPOUND_ON_FLUSH = "index.compound_on_flush";
 | 
	
		
			
				|  |  | -    public static final String INDEX_GC_DELETES = "index.gc_deletes";
 | 
	
		
			
				|  |  | -    public static final String INDEX_FAIL_ON_MERGE_FAILURE = "index.fail_on_merge_failure";
 | 
	
		
			
				|  |  | -    public static final String INDEX_FAIL_ON_CORRUPTION = "index.fail_on_corruption";
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    class ApplySettings implements IndexSettingsService.Listener {
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -        @Override
 | 
	
		
			
				|  |  | -        public void onRefreshSettings(Settings settings) {
 | 
	
		
			
				|  |  | -            long gcDeletesInMillis = settings.getAsTime(INDEX_GC_DELETES, TimeValue.timeValueMillis(InternalEngine.this.gcDeletesInMillis)).millis();
 | 
	
		
			
				|  |  | -            if (gcDeletesInMillis != InternalEngine.this.gcDeletesInMillis) {
 | 
	
		
			
				|  |  | -                logger.info("updating index.gc_deletes from [{}] to [{}]", TimeValue.timeValueMillis(InternalEngine.this.gcDeletesInMillis), TimeValue.timeValueMillis(gcDeletesInMillis));
 | 
	
		
			
				|  |  | -                InternalEngine.this.gcDeletesInMillis = gcDeletesInMillis;
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -            final boolean compoundOnFlush = settings.getAsBoolean(INDEX_COMPOUND_ON_FLUSH, InternalEngine.this.compoundOnFlush);
 | 
	
		
			
				|  |  | -            if (compoundOnFlush != InternalEngine.this.compoundOnFlush) {
 | 
	
		
			
				|  |  | -                logger.info("updating {} from [{}] to [{}]", InternalEngine.INDEX_COMPOUND_ON_FLUSH, InternalEngine.this.compoundOnFlush, compoundOnFlush);
 | 
	
		
			
				|  |  | -                InternalEngine.this.compoundOnFlush = compoundOnFlush;
 | 
	
		
			
				|  |  | -                indexWriter.getConfig().setUseCompoundFile(compoundOnFlush);
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -            InternalEngine.this.failEngineOnCorruption = settings.getAsBoolean(INDEX_FAIL_ON_CORRUPTION, InternalEngine.this.failEngineOnCorruption);
 | 
	
		
			
				|  |  | -            int indexConcurrency = settings.getAsInt(INDEX_INDEX_CONCURRENCY, InternalEngine.this.indexConcurrency);
 | 
	
		
			
				|  |  | -            boolean failOnMergeFailure = settings.getAsBoolean(INDEX_FAIL_ON_MERGE_FAILURE, InternalEngine.this.failOnMergeFailure);
 | 
	
		
			
				|  |  | -            String codecName = settings.get(INDEX_CODEC, InternalEngine.this.codecName);
 | 
	
		
			
				|  |  | -            boolean requiresFlushing = false;
 | 
	
		
			
				|  |  | -            if (indexConcurrency != InternalEngine.this.indexConcurrency ||
 | 
	
		
			
				|  |  | -                    !codecName.equals(InternalEngine.this.codecName) ||
 | 
	
		
			
				|  |  | -                    failOnMergeFailure != InternalEngine.this.failOnMergeFailure) {
 | 
	
		
			
				|  |  | -                try (InternalLock _ = readLock.acquire()) {
 | 
	
		
			
				|  |  | -                    if (indexConcurrency != InternalEngine.this.indexConcurrency) {
 | 
	
		
			
				|  |  | -                        logger.info("updating index.index_concurrency from [{}] to [{}]", InternalEngine.this.indexConcurrency, indexConcurrency);
 | 
	
		
			
				|  |  | -                        InternalEngine.this.indexConcurrency = indexConcurrency;
 | 
	
		
			
				|  |  | -                        // we have to flush in this case, since it only applies on a new index writer
 | 
	
		
			
				|  |  | -                        requiresFlushing = true;
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | -                    if (!codecName.equals(InternalEngine.this.codecName)) {
 | 
	
		
			
				|  |  | -                        logger.info("updating index.codec from [{}] to [{}]", InternalEngine.this.codecName, codecName);
 | 
	
		
			
				|  |  | -                        InternalEngine.this.codecName = codecName;
 | 
	
		
			
				|  |  | -                        // we want to flush in this case, so the new codec will be reflected right away...
 | 
	
		
			
				|  |  | -                        requiresFlushing = true;
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | -                    if (failOnMergeFailure != InternalEngine.this.failOnMergeFailure) {
 | 
	
		
			
				|  |  | -                        logger.info("updating {} from [{}] to [{}]", InternalEngine.INDEX_FAIL_ON_MERGE_FAILURE, InternalEngine.this.failOnMergeFailure, failOnMergeFailure);
 | 
	
		
			
				|  |  | -                        InternalEngine.this.failOnMergeFailure = failOnMergeFailure;
 | 
	
		
			
				|  |  | -                    }
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -                if (requiresFlushing) {
 | 
	
		
			
				|  |  | -                    flush(new Flush().type(Flush.Type.NEW_WRITER));
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  | -    }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |      private SearcherManager buildSearchManager(IndexWriter indexWriter) throws IOException {
 | 
	
		
			
				|  |  |          final DirectoryReader directoryReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(indexWriter, true), shardId);
 | 
	
		
			
				|  |  |          return new SearcherManager(directoryReader, searcherFactory);
 | 
	
	
		
			
				|  | @@ -1723,54 +1655,35 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    public void activateThrottling() {
 | 
	
		
			
				|  |  | +        throttle.activate();
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | +    public void deactivateThrottling() {
 | 
	
		
			
				|  |  | +        throttle.deactivate();
 | 
	
		
			
				|  |  | +    }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -    static final class IndexThrottle implements MergeSchedulerProvider.Listener {
 | 
	
		
			
				|  |  | +    static final class IndexThrottle {
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          private static final InternalLock NOOP_LOCK = new InternalLock(new NoOpLock());
 | 
	
		
			
				|  |  |          private final InternalLock lockReference = new InternalLock(new ReentrantLock());
 | 
	
		
			
				|  |  | -        private final AtomicInteger numMergesInFlight = new AtomicInteger(0);
 | 
	
		
			
				|  |  | -        private final AtomicBoolean isThrottling = new AtomicBoolean();
 | 
	
		
			
				|  |  | -        private final MergeSchedulerProvider mergeScheduler;
 | 
	
		
			
				|  |  | -        private final ESLogger logger;
 | 
	
		
			
				|  |  | -        private final ShardIndexingService indexingService;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          private volatile InternalLock lock = NOOP_LOCK;
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        public IndexThrottle(MergeSchedulerProvider mergeScheduler, ESLogger logger, ShardIndexingService indexingService) {
 | 
	
		
			
				|  |  | -            this.mergeScheduler = mergeScheduler;
 | 
	
		
			
				|  |  | -            this.logger = logger;
 | 
	
		
			
				|  |  | -            this.indexingService = indexingService;
 | 
	
		
			
				|  |  | -        }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |          public Releasable acquireThrottle() {
 | 
	
		
			
				|  |  |              return lock.acquire();
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        @Override
 | 
	
		
			
				|  |  | -        public synchronized void beforeMerge(OnGoingMerge merge) {
 | 
	
		
			
				|  |  | -            int maxNumMerges = mergeScheduler.getMaxMerges();
 | 
	
		
			
				|  |  | -            if (numMergesInFlight.incrementAndGet() > maxNumMerges) {
 | 
	
		
			
				|  |  | -                if (isThrottling.getAndSet(true) == false) {
 | 
	
		
			
				|  |  | -                    logger.info("now throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges);
 | 
	
		
			
				|  |  | -                    indexingService.throttlingActivated();
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -                lock = lockReference;
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  | +        public void activate() {
 | 
	
		
			
				|  |  | +            assert lock == NOOP_LOCK : "throttling activated while already active";
 | 
	
		
			
				|  |  | +            lock = lockReference;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -        @Override
 | 
	
		
			
				|  |  | -        public synchronized void afterMerge(OnGoingMerge merge) {
 | 
	
		
			
				|  |  | -            int maxNumMerges = mergeScheduler.getMaxMerges();
 | 
	
		
			
				|  |  | -            if (numMergesInFlight.decrementAndGet() < maxNumMerges) {
 | 
	
		
			
				|  |  | -                if (isThrottling.getAndSet(false)) {
 | 
	
		
			
				|  |  | -                    logger.info("stop throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges);
 | 
	
		
			
				|  |  | -                    indexingService.throttlingDeactivated();
 | 
	
		
			
				|  |  | -                }
 | 
	
		
			
				|  |  | -                lock = NOOP_LOCK;
 | 
	
		
			
				|  |  | -            }
 | 
	
		
			
				|  |  | +        public void deactivate() {
 | 
	
		
			
				|  |  | +            assert lock != NOOP_LOCK : "throttling deactivated but not active";
 | 
	
		
			
				|  |  | +            lock = NOOP_LOCK;
 | 
	
		
			
				|  |  |          }
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  |      }
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |      private static final class NoOpLock implements Lock {
 |