Make IndexingMemoryController private to IndicesService

This commit detaches the IndexingMemoryController from guice and moves
it's creation and closing into IndicesService where it logically belongs.
This commit is contained in:
Simon Willnauer 2016-01-10 20:40:51 +01:00
parent 37f6b5ced7
commit 91e8d156f5
9 changed files with 33 additions and 45 deletions

View File

@ -35,7 +35,7 @@ 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.TranslogConfig; import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -192,7 +192,7 @@ public final class EngineConfig {
} }
/** /**
* Returns the initial index buffer size. This setting is only read on startup and otherwise controlled by {@link org.elasticsearch.indices.memory.IndexingMemoryController} * Returns the initial index buffer size. This setting is only read on startup and otherwise controlled by {@link IndexingMemoryController}
*/ */
public ByteSizeValue getIndexingBufferSize() { public ByteSizeValue getIndexingBufferSize() {
return indexingBufferSize; return indexingBufferSize;

View File

@ -111,7 +111,7 @@ import org.elasticsearch.index.warmer.ShardIndexWarmerService;
import org.elasticsearch.index.warmer.WarmerStats; import org.elasticsearch.index.warmer.WarmerStats;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryFailedException;
import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.percolator.PercolatorService; import org.elasticsearch.percolator.PercolatorService;

View File

@ -17,10 +17,9 @@
* under the License. * under the License.
*/ */
package org.elasticsearch.indices.memory; package org.elasticsearch.indices;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
@ -32,16 +31,16 @@ import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.monitor.jvm.JvmInfo; import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.List; import java.util.List;
import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledFuture;
public class IndexingMemoryController extends AbstractLifecycleComponent<IndexingMemoryController> implements IndexEventListener { public class IndexingMemoryController extends AbstractComponent implements IndexEventListener, Closeable {
/** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */ /** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */
public static final String INDEX_BUFFER_SIZE_SETTING = "indices.memory.index_buffer_size"; public static final String INDEX_BUFFER_SIZE_SETTING = "indices.memory.index_buffer_size";
@ -70,10 +69,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
/** Once a shard becomes inactive, we reduce the {@code IndexWriter} buffer to this value (500 KB) to let active shards use the heap instead. */ /** Once a shard becomes inactive, we reduce the {@code IndexWriter} buffer to this value (500 KB) to let active shards use the heap instead. */
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER"); public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
/** Once a shard becomes inactive, we reduce the {@code Translog} buffer to this value (1 KB) to let active shards use the heap instead. */
public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb", "INACTIVE_SHARD_TRANSLOG_BUFFER");
private final ThreadPool threadPool;
private final IndicesService indicesService; private final IndicesService indicesService;
private final ByteSizeValue indexingBuffer; private final ByteSizeValue indexingBuffer;
@ -81,22 +76,20 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
private final ByteSizeValue maxShardIndexBufferSize; private final ByteSizeValue maxShardIndexBufferSize;
private final TimeValue interval; private final TimeValue interval;
private volatile ScheduledFuture scheduler; private final ScheduledFuture scheduler;
private static final EnumSet<IndexShardState> CAN_UPDATE_INDEX_BUFFER_STATES = EnumSet.of( private static final EnumSet<IndexShardState> CAN_UPDATE_INDEX_BUFFER_STATES = EnumSet.of(
IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED, IndexShardState.RELOCATED); IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED, IndexShardState.RELOCATED);
private final ShardsIndicesStatusChecker statusChecker; private final ShardsIndicesStatusChecker statusChecker;
@Inject IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) {
public IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) {
this(settings, threadPool, indicesService, JvmInfo.jvmInfo().getMem().getHeapMax().bytes()); this(settings, threadPool, indicesService, JvmInfo.jvmInfo().getMem().getHeapMax().bytes());
} }
// for testing // for testing
protected IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService, long jvmMemoryInBytes) { IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService, long jvmMemoryInBytes) {
super(settings); super(settings);
this.threadPool = threadPool;
this.indicesService = indicesService; this.indicesService = indicesService;
ByteSizeValue indexingBuffer; ByteSizeValue indexingBuffer;
@ -131,29 +124,23 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
MIN_SHARD_INDEX_BUFFER_SIZE_SETTING, this.minShardIndexBufferSize, MIN_SHARD_INDEX_BUFFER_SIZE_SETTING, this.minShardIndexBufferSize,
MAX_SHARD_INDEX_BUFFER_SIZE_SETTING, this.maxShardIndexBufferSize, MAX_SHARD_INDEX_BUFFER_SIZE_SETTING, this.maxShardIndexBufferSize,
SHARD_INACTIVE_INTERVAL_TIME_SETTING, this.interval); SHARD_INACTIVE_INTERVAL_TIME_SETTING, this.interval);
this.scheduler = scheduleTask(threadPool);
} }
@Override protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
protected void doStart() {
// it's fine to run it on the scheduler thread, no busy work // it's fine to run it on the scheduler thread, no busy work
this.scheduler = threadPool.scheduleWithFixedDelay(statusChecker, interval); return threadPool.scheduleWithFixedDelay(statusChecker, interval);
} }
@Override public void close() {
protected void doStop() {
FutureUtils.cancel(scheduler); FutureUtils.cancel(scheduler);
scheduler = null;
}
@Override
protected void doClose() {
} }
/** /**
* returns the current budget for the total amount of indexing buffers of * returns the current budget for the total amount of indexing buffers of
* active shards on this node * active shards on this node
*/ */
public ByteSizeValue indexingBufferSize() { ByteSizeValue indexingBufferSize() {
return indexingBuffer; return indexingBuffer;
} }
@ -188,7 +175,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
} }
/** check if any shards active status changed, now. */ /** check if any shards active status changed, now. */
public void forceCheck() { void forceCheck() {
statusChecker.run(); statusChecker.run();
} }

View File

@ -111,7 +111,6 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener;
import org.elasticsearch.indices.flush.SyncedFlushService; import org.elasticsearch.indices.flush.SyncedFlushService;
import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.recovery.RecoverySource; import org.elasticsearch.indices.recovery.RecoverySource;
@ -273,7 +272,6 @@ public class IndicesModule extends AbstractModule {
bind(RecoverySource.class).asEagerSingleton(); bind(RecoverySource.class).asEagerSingleton();
bind(IndicesStore.class).asEagerSingleton(); bind(IndicesStore.class).asEagerSingleton();
bind(IndicesClusterStateService.class).asEagerSingleton(); bind(IndicesClusterStateService.class).asEagerSingleton();
bind(IndexingMemoryController.class).asEagerSingleton();
bind(SyncedFlushService.class).asEagerSingleton(); bind(SyncedFlushService.class).asEagerSingleton();
bind(IndicesQueryCache.class).asEagerSingleton(); bind(IndicesQueryCache.class).asEagerSingleton();
bind(IndicesRequestCache.class).asEagerSingleton(); bind(IndicesRequestCache.class).asEagerSingleton();

View File

@ -65,6 +65,7 @@ import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
@ -105,6 +106,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private final OldShardsStats oldShardsStats = new OldShardsStats(); private final OldShardsStats oldShardsStats = new OldShardsStats();
private final IndexStoreConfig indexStoreConfig; private final IndexStoreConfig indexStoreConfig;
private final MapperRegistry mapperRegistry; private final MapperRegistry mapperRegistry;
private final IndexingMemoryController indexingMemoryController;
@Override @Override
protected void doStart() { protected void doStart() {
@ -114,7 +116,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv,
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry, ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver, IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver,
ClusterService clusterService, MapperRegistry mapperRegistry) { ClusterService clusterService, MapperRegistry mapperRegistry, ThreadPool threadPool) {
super(settings); super(settings);
this.pluginsService = pluginsService; this.pluginsService = pluginsService;
this.nodeEnv = nodeEnv; this.nodeEnv = nodeEnv;
@ -127,7 +129,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
this.mapperRegistry = mapperRegistry; this.mapperRegistry = mapperRegistry;
clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE_SETTING, indexStoreConfig::setRateLimitingType); clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE_SETTING, indexStoreConfig::setRateLimitingType);
clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, indexStoreConfig::setRateLimitingThrottle); clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, indexStoreConfig::setRateLimitingThrottle);
indexingMemoryController = new IndexingMemoryController(settings, threadPool, this);
} }
@Override @Override
@ -161,7 +163,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
@Override @Override
protected void doClose() { protected void doClose() {
IOUtils.closeWhileHandlingException(analysisRegistry); IOUtils.closeWhileHandlingException(analysisRegistry, indexingMemoryController);
} }
/** /**
@ -291,6 +293,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, analysisRegistry); final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, analysisRegistry);
pluginsService.onIndexModule(indexModule); pluginsService.onIndexModule(indexModule);
indexModule.addIndexEventListener(indexingMemoryController);
for (IndexEventListener listener : builtInListeners) { for (IndexEventListener listener : builtInListeners) {
indexModule.addIndexEventListener(listener); indexModule.addIndexEventListener(listener);
} }

View File

@ -63,7 +63,6 @@ import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.index.snapshots.IndexShardRepository; import org.elasticsearch.index.snapshots.IndexShardRepository;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.flush.SyncedFlushService; import org.elasticsearch.indices.flush.SyncedFlushService;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryFailedException;
import org.elasticsearch.indices.recovery.RecoverySource; import org.elasticsearch.indices.recovery.RecoverySource;
import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryState;
@ -130,9 +129,9 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
NodeMappingRefreshAction nodeMappingRefreshAction, NodeMappingRefreshAction nodeMappingRefreshAction,
RepositoriesService repositoriesService, RestoreService restoreService, RepositoriesService repositoriesService, RestoreService restoreService,
SearchService searchService, SyncedFlushService syncedFlushService, SearchService searchService, SyncedFlushService syncedFlushService,
RecoverySource recoverySource, NodeServicesProvider nodeServicesProvider, IndexingMemoryController indexingMemoryController) { RecoverySource recoverySource, NodeServicesProvider nodeServicesProvider) {
super(settings); super(settings);
this.buildInIndexListener = Arrays.asList(recoverySource, recoveryTarget, searchService, syncedFlushService, indexingMemoryController); this.buildInIndexListener = Arrays.asList(recoverySource, recoveryTarget, searchService, syncedFlushService);
this.indicesService = indicesService; this.indicesService = indicesService;
this.clusterService = clusterService; this.clusterService = clusterService;
this.threadPool = threadPool; this.threadPool = threadPool;

View File

@ -69,8 +69,6 @@ import org.elasticsearch.indices.breaker.CircuitBreakerModule;
import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.indices.cluster.IndicesClusterStateService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore; import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService; import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.monitor.MonitorService; import org.elasticsearch.monitor.MonitorService;
@ -249,7 +247,6 @@ public class Node implements Releasable {
injector.getInstance(MappingUpdatedAction.class).setClient(client); injector.getInstance(MappingUpdatedAction.class).setClient(client);
injector.getInstance(IndicesService.class).start(); injector.getInstance(IndicesService.class).start();
injector.getInstance(IndexingMemoryController.class).start();
injector.getInstance(IndicesClusterStateService.class).start(); injector.getInstance(IndicesClusterStateService.class).start();
injector.getInstance(IndicesTTLService.class).start(); injector.getInstance(IndicesTTLService.class).start();
injector.getInstance(SnapshotsService.class).start(); injector.getInstance(SnapshotsService.class).start();
@ -308,7 +305,6 @@ public class Node implements Releasable {
// stop any changes happening as a result of cluster state changes // stop any changes happening as a result of cluster state changes
injector.getInstance(IndicesClusterStateService.class).stop(); injector.getInstance(IndicesClusterStateService.class).stop();
// we close indices first, so operations won't be allowed on it // we close indices first, so operations won't be allowed on it
injector.getInstance(IndexingMemoryController.class).stop();
injector.getInstance(IndicesTTLService.class).stop(); injector.getInstance(IndicesTTLService.class).stop();
injector.getInstance(RoutingService.class).stop(); injector.getInstance(RoutingService.class).stop();
injector.getInstance(ClusterService.class).stop(); injector.getInstance(ClusterService.class).stop();
@ -360,7 +356,6 @@ public class Node implements Releasable {
stopWatch.stop().start("indices_cluster"); stopWatch.stop().start("indices_cluster");
injector.getInstance(IndicesClusterStateService.class).close(); injector.getInstance(IndicesClusterStateService.class).close();
stopWatch.stop().start("indices"); stopWatch.stop().start("indices");
injector.getInstance(IndexingMemoryController.class).close();
injector.getInstance(IndicesTTLService.class).close(); injector.getInstance(IndicesTTLService.class).close();
injector.getInstance(IndicesService.class).close(); injector.getInstance(IndicesService.class).close();
// close filter/fielddata caches after indices // close filter/fielddata caches after indices

View File

@ -17,7 +17,7 @@
* under the License. * under the License.
*/ */
package org.elasticsearch.indices.memory; package org.elasticsearch.indices;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;

View File

@ -16,7 +16,7 @@
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package org.elasticsearch.indices.memory; package org.elasticsearch.indices;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
@ -24,8 +24,8 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
@ -33,6 +33,7 @@ import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
@ -120,6 +121,11 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase {
activeShards.add(shard); activeShards.add(shard);
forceCheck(); forceCheck();
} }
@Override
protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
return null;
}
} }
public void testShardAdditionAndRemoval() { public void testShardAdditionAndRemoval() {