Merge pull request #12863 from rjernst/fix/12784

Refactor classes only plugged in by tests to use package private extension points
This commit is contained in:
Ryan Ernst 2015-08-13 14:36:36 -07:00
commit 71a3bdb8e8
34 changed files with 372 additions and 338 deletions

View File

@ -1,48 +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.cache.recycler;
import org.elasticsearch.common.Classes;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
/**
*/
public class PageCacheRecyclerModule extends AbstractModule {
public static final String CACHE_IMPL = "cache.recycler.page_cache_impl";
private final Settings settings;
public PageCacheRecyclerModule(Settings settings) {
this.settings = settings;
}
@Override
protected void configure() {
String impl = settings.get(CACHE_IMPL);
if (impl == null) {
bind(PageCacheRecycler.class).asEagerSingleton();
} else {
Class<? extends PageCacheRecycler> implClass = Classes.loadClass(getClass().getClassLoader(), impl);
bind(PageCacheRecycler.class).to(implClass).asEagerSingleton();
}
}
}

View File

@ -52,6 +52,9 @@ public class ClusterModule extends AbstractModule implements SpawnModules {
private Set<Class<? extends IndexTemplateFilter>> indexTemplateFilters = new HashSet<>();
// pkg private so tests can mock
Class<? extends ClusterInfoService> clusterInfoServiceImpl = InternalClusterInfoService.class;
public ClusterModule(Settings settings) {
this.settings = settings;
}
@ -88,13 +91,7 @@ public class ClusterModule extends AbstractModule implements SpawnModules {
bind(NodeIndexDeletedAction.class).asEagerSingleton();
bind(NodeMappingRefreshAction.class).asEagerSingleton();
bind(MappingUpdatedAction.class).asEagerSingleton();
String impl = settings.get(CLUSTER_SERVICE_IMPL);
Class<? extends ClusterInfoService> implClass = InternalClusterInfoService.class;
if (impl != null) {
implClass = Classes.loadClass(getClass().getClassLoader(), impl);
}
bind(ClusterInfoService.class).to(implClass).asEagerSingleton();
bind(ClusterInfoService.class).to(clusterInfoServiceImpl).asEagerSingleton();
Multibinder<IndexTemplateFilter> mbinder = Multibinder.newSetBinder(binder(), IndexTemplateFilter.class);
for (Class<? extends IndexTemplateFilter> indexTemplateFilter : indexTemplateFilters) {

View File

@ -1,50 +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.common.util;
import org.elasticsearch.common.Classes;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import static org.elasticsearch.common.inject.Modules.createModule;
/**
*/
public class BigArraysModule extends AbstractModule {
public static final String IMPL = "common.util.big_arrays_impl";
private final Settings settings;
public BigArraysModule(Settings settings) {
this.settings = settings;
}
@Override
protected void configure() {
String impl = settings.get(IMPL);
if (impl == null) {
bind(BigArrays.class).asEagerSingleton();
} else {
Class<? extends BigArrays> implClass = Classes.loadClass(getClass().getClassLoader(), impl);
bind(BigArrays.class).to(implClass).asEagerSingleton();
}
}
}

View File

@ -20,15 +20,21 @@
package org.elasticsearch.index.cache;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Scopes;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.bitset.BitsetFilterCacheModule;
import org.elasticsearch.index.cache.query.QueryCacheModule;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
/**
*
*/
public class IndexCacheModule extends AbstractModule {
public static final String INDEX_QUERY_CACHE = "index";
public static final String NONE_QUERY_CACHE = "none";
public static final String QUERY_CACHE_TYPE = "index.queries.cache.type";
// for test purposes only
public static final String QUERY_CACHE_EVERYTHING = "index.queries.cache.everything";
private final Settings settings;
public IndexCacheModule(Settings settings) {
@ -37,9 +43,17 @@ public class IndexCacheModule extends AbstractModule {
@Override
protected void configure() {
new QueryCacheModule(settings).configure(binder());
new BitsetFilterCacheModule(settings).configure(binder());
String queryCacheType = settings.get(QUERY_CACHE_TYPE, INDEX_QUERY_CACHE);
Class<? extends QueryCache> queryCacheImpl;
if (queryCacheType.equals(INDEX_QUERY_CACHE)) {
queryCacheImpl = IndexQueryCache.class;
} else if (queryCacheType.equals(NONE_QUERY_CACHE)) {
queryCacheImpl = NoneQueryCache.class;
} else {
throw new IllegalArgumentException("Unknown QueryCache type [" + queryCacheType + "]");
}
bind(QueryCache.class).to(queryCacheImpl).in(Scopes.SINGLETON);
bind(BitsetFilterCache.class).asEagerSingleton();
bind(IndexCache.class).asEagerSingleton();
}
}

View File

@ -1,56 +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.cache.query;
import org.elasticsearch.cluster.metadata.AliasOrIndex;
import org.elasticsearch.common.Classes;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Scopes;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
/**
*
*/
public class QueryCacheModule extends AbstractModule {
public static final class QueryCacheSettings {
public static final String QUERY_CACHE_TYPE = "index.queries.cache.type";
// for test purposes only
public static final String QUERY_CACHE_EVERYTHING = "index.queries.cache.everything";
}
private final Settings settings;
public QueryCacheModule(Settings settings) {
this.settings = settings;
}
@Override
protected void configure() {
Class<? extends IndexQueryCache> queryCacheClass = IndexQueryCache.class;
String customQueryCache = settings.get(QueryCacheSettings.QUERY_CACHE_TYPE);
if (customQueryCache != null) {
// TODO: make this only useable from tests
queryCacheClass = Classes.loadClass(getClass().getClassLoader(), customQueryCache);
}
bind(QueryCache.class).to(queryCacheClass).in(Scopes.SINGLETON);
}
}

View File

@ -58,8 +58,8 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.aliases.IndexAliasesService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache;
import org.elasticsearch.index.cache.query.QueryCacheModule.QueryCacheSettings;
import org.elasticsearch.index.cache.query.QueryCacheStats;
import org.elasticsearch.index.cache.request.ShardRequestCache;
import org.elasticsearch.index.codec.CodecService;
@ -249,7 +249,7 @@ public class IndexShard extends AbstractIndexShardComponent {
final QueryCachingPolicy cachingPolicy;
// the query cache is a node-level thing, however we want the most popular filters
// to be computed on a per-shard basis
if (indexSettings.getAsBoolean(QueryCacheSettings.QUERY_CACHE_EVERYTHING, false)) {
if (indexSettings.getAsBoolean(IndexCacheModule.QUERY_CACHE_EVERYTHING, false)) {
cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE;
} else {
cachingPolicy = new UsageTrackingQueryCachingPolicy();

View File

@ -40,12 +40,13 @@ import org.elasticsearch.index.translog.TranslogService;
*/
public class IndexShardModule extends AbstractModule {
public static final String ENGINE_FACTORY = "index.engine.factory";
private final ShardId shardId;
private final Settings settings;
private final boolean primary;
// pkg private so tests can mock
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class;
public IndexShardModule(ShardId shardId, boolean primary, Settings settings) {
this.settings = settings;
this.shardId = shardId;
@ -70,13 +71,7 @@ public class IndexShardModule extends AbstractModule {
bind(TranslogService.class).asEagerSingleton();
}
Class<? extends InternalEngineFactory> engineFactoryClass = InternalEngineFactory.class;
String customEngineFactory = settings.get(ENGINE_FACTORY);
if (customEngineFactory != null) {
// TODO: make this only useable from tests
engineFactoryClass = Classes.loadClass(getClass().getClassLoader(), customEngineFactory);
}
bind(EngineFactory.class).to(engineFactoryClass);
bind(EngineFactory.class).to(engineFactoryImpl);
bind(StoreRecoveryService.class).asEagerSingleton();
bind(ShardPercolateService.class).asEagerSingleton();
bind(ShardTermVectorsService.class).asEagerSingleton();

View File

@ -23,7 +23,6 @@ import org.elasticsearch.Build;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionModule;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.cache.recycler.PageCacheRecyclerModule;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.node.NodeClientModule;
import org.elasticsearch.cluster.ClusterModule;
@ -44,7 +43,6 @@ import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.util.BigArraysModule;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.discovery.DiscoveryService;
@ -71,7 +69,6 @@ import org.elasticsearch.monitor.MonitorModule;
import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.node.internal.NodeModule;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.percolator.PercolatorModule;
import org.elasticsearch.percolator.PercolatorService;
@ -161,9 +158,7 @@ public class Node implements Releasable {
try {
ModulesBuilder modules = new ModulesBuilder();
modules.add(new Version.Module(version));
modules.add(new PageCacheRecyclerModule(settings));
modules.add(new CircuitBreakerModule(settings));
modules.add(new BigArraysModule(settings));
modules.add(new PluginsModule(settings, pluginsService));
modules.add(new SettingsModule(settings));
modules.add(new NodeModule(this));

View File

@ -17,9 +17,11 @@
* under the License.
*/
package org.elasticsearch.node.internal;
package org.elasticsearch.node;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.node.settings.NodeSettingsService;
@ -31,12 +33,27 @@ public class NodeModule extends AbstractModule {
private final Node node;
// pkg private so tests can mock
Class<? extends PageCacheRecycler> pageCacheRecyclerImpl = PageCacheRecycler.class;
Class<? extends BigArrays> bigArraysImpl = BigArrays.class;
public NodeModule(Node node) {
this.node = node;
}
@Override
protected void configure() {
if (pageCacheRecyclerImpl == PageCacheRecycler.class) {
bind(PageCacheRecycler.class).asEagerSingleton();
} else {
bind(PageCacheRecycler.class).to(pageCacheRecyclerImpl).asEagerSingleton();
}
if (bigArraysImpl == BigArrays.class) {
bind(BigArrays.class).asEagerSingleton();
} else {
bind(BigArrays.class).to(bigArraysImpl).asEagerSingleton();
}
bind(Node.class).toInstance(node);
bind(NodeSettingsService.class).asEagerSingleton();
bind(NodeService.class).asEagerSingleton();

View File

@ -155,8 +155,6 @@ import java.util.*;
*/
public class SearchModule extends AbstractModule {
public static final String SEARCH_SERVICE_IMPL = "search.service_impl";
private final Settings settings;
private final Set<Class<? extends Aggregator.Parser>> aggParsers = new HashSet<>();
private final Set<Class<? extends PipelineAggregator.Parser>> pipelineAggParsers = new HashSet<>();
@ -167,6 +165,9 @@ public class SearchModule extends AbstractModule {
private final Set<Class<? extends SignificanceHeuristicParser>> heuristicParsers = new HashSet<>();
private final Set<Class<? extends MovAvgModel.AbstractModelParser>> modelParsers = new HashSet<>();
// pkg private so tests can mock
Class<? extends SearchService> searchServiceImpl = SearchService.class;
public SearchModule(Settings settings) {
this.settings = settings;
}
@ -333,13 +334,11 @@ public class SearchModule extends AbstractModule {
bind(FetchPhase.class).asEagerSingleton();
bind(SearchServiceTransportAction.class).asEagerSingleton();
bind(MoreLikeThisFetchService.class).asEagerSingleton();
// search service -- testing only!
String impl = settings.get(SEARCH_SERVICE_IMPL);
if (impl == null) {
if (searchServiceImpl == SearchService.class) {
bind(SearchService.class).asEagerSingleton();
} else {
Class<? extends SearchService> implClass = Classes.loadClass(getClass().getClassLoader(), impl);
bind(SearchService.class).to(implClass).asEagerSingleton();
bind(SearchService.class).to(searchServiceImpl).asEagerSingleton();
}
}

View File

@ -17,15 +17,16 @@
* under the License.
*/
package org.elasticsearch.test.cache.recycler;
package org.elasticsearch.cache.recycler;
import com.google.common.base.Predicate;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.recycler.Recycler.V;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.NodeModule;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.threadpool.ThreadPool;

View File

@ -0,0 +1,95 @@
/*
* 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.cluster;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction;
import org.elasticsearch.cluster.routing.allocation.decider.MockDiskUsagesIT;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.concurrent.CountDownLatch;
/**
* Fake ClusterInfoService class that allows updating the nodes stats disk
* usage with fake values
*/
public class MockInternalClusterInfoService extends InternalClusterInfoService {
public static class Plugin extends AbstractPlugin {
@Override
public String name() {
return "mock-cluster-info-service";
}
@Override
public String description() {
return "a mock cluster info service for testing";
}
public void onModule(ClusterModule module) {
module.clusterInfoServiceImpl = MockInternalClusterInfoService.class;
}
}
private final ClusterName clusterName;
private volatile NodeStats[] stats = new NodeStats[3];
@Inject
public MockInternalClusterInfoService(Settings settings, NodeSettingsService nodeSettingsService,
TransportNodesStatsAction transportNodesStatsAction,
TransportIndicesStatsAction transportIndicesStatsAction,
ClusterService clusterService, ThreadPool threadPool) {
super(settings, nodeSettingsService, transportNodesStatsAction, transportIndicesStatsAction, clusterService, threadPool);
this.clusterName = ClusterName.clusterNameFromSettings(settings);
stats[0] = MockDiskUsagesIT.makeStats("node_t1", new DiskUsage("node_t1", "n1", 100, 100));
stats[1] = MockDiskUsagesIT.makeStats("node_t2", new DiskUsage("node_t2", "n2", 100, 100));
stats[2] = MockDiskUsagesIT.makeStats("node_t3", new DiskUsage("node_t3", "n3", 100, 100));
}
public void setN1Usage(String nodeName, DiskUsage newUsage) {
stats[0] = MockDiskUsagesIT.makeStats(nodeName, newUsage);
}
public void setN2Usage(String nodeName, DiskUsage newUsage) {
stats[1] = MockDiskUsagesIT.makeStats(nodeName, newUsage);
}
public void setN3Usage(String nodeName, DiskUsage newUsage) {
stats[2] = MockDiskUsagesIT.makeStats(nodeName, newUsage);
}
@Override
public CountDownLatch updateNodeStats(final ActionListener<NodesStatsResponse> listener) {
NodesStatsResponse response = new NodesStatsResponse(clusterName, stats);
listener.onResponse(response);
return new CountDownLatch(0);
}
@Override
public CountDownLatch updateIndicesStats(final ActionListener<IndicesStatsResponse> listener) {
// Not used, so noop
return new CountDownLatch(0);
}
}

View File

@ -20,28 +20,20 @@
package org.elasticsearch.cluster.routing.allocation.decider;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction;
import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.Test;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import static com.google.common.collect.Lists.newArrayList;
import static com.google.common.collect.Maps.newHashMap;
@ -57,8 +49,8 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
// Use the mock internal cluster info service, which has fake-able disk usages
.put(ClusterModule.CLUSTER_SERVICE_IMPL, MockInternalClusterInfoService.class.getName())
// Use the mock internal cluster info service, which has fake-able disk usages
.extendArray("plugin.types", MockInternalClusterInfoService.Plugin.class.getName())
// Update more frequently
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "1s")
.build();
@ -183,50 +175,4 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
null);
}
/**
* Fake ClusterInfoService class that allows updating the nodes stats disk
* usage with fake values
*/
public static class MockInternalClusterInfoService extends InternalClusterInfoService {
private final ClusterName clusterName;
private volatile NodeStats[] stats = new NodeStats[3];
@Inject
public MockInternalClusterInfoService(Settings settings, NodeSettingsService nodeSettingsService,
TransportNodesStatsAction transportNodesStatsAction,
TransportIndicesStatsAction transportIndicesStatsAction,
ClusterService clusterService, ThreadPool threadPool) {
super(settings, nodeSettingsService, transportNodesStatsAction, transportIndicesStatsAction, clusterService, threadPool);
this.clusterName = ClusterName.clusterNameFromSettings(settings);
stats[0] = makeStats("node_t1", new DiskUsage("node_t1", "n1", 100, 100));
stats[1] = makeStats("node_t2", new DiskUsage("node_t2", "n2", 100, 100));
stats[2] = makeStats("node_t3", new DiskUsage("node_t3", "n3", 100, 100));
}
public void setN1Usage(String nodeName, DiskUsage newUsage) {
stats[0] = makeStats(nodeName, newUsage);
}
public void setN2Usage(String nodeName, DiskUsage newUsage) {
stats[1] = makeStats(nodeName, newUsage);
}
public void setN3Usage(String nodeName, DiskUsage newUsage) {
stats[2] = makeStats(nodeName, newUsage);
}
@Override
public CountDownLatch updateNodeStats(final ActionListener<NodesStatsResponse> listener) {
NodesStatsResponse response = new NodesStatsResponse(clusterName, stats);
listener.onResponse(response);
return new CountDownLatch(0);
}
@Override
public CountDownLatch updateIndicesStats(final ActionListener<IndicesStatsResponse> listener) {
// Not used, so noop
return new CountDownLatch(0);
}
}
}

View File

@ -29,7 +29,6 @@ import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.junit.Before;
import java.lang.reflect.InvocationTargetException;

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.test.cache.recycler;
package org.elasticsearch.common.util;
import com.carrotsearch.randomizedtesting.RandomizedContext;
import com.carrotsearch.randomizedtesting.SeedUtils;
@ -30,15 +30,8 @@ import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.BigArray;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ByteArray;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.common.util.FloatArray;
import org.elasticsearch.common.util.IntArray;
import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.test.ESTestCase;
import java.util.Collection;

View File

@ -27,8 +27,8 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.threadpool.ThreadPool;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;

View File

@ -29,8 +29,8 @@ import org.elasticsearch.http.netty.pipelining.OrderedDownstreamChannelEvent;
import org.elasticsearch.http.netty.pipelining.OrderedUpstreamMessageEvent;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.threadpool.ThreadPool;
import org.jboss.netty.buffer.ChannelBuffer;
import org.jboss.netty.buffer.ChannelBuffers;

View File

@ -0,0 +1,49 @@
/*
* 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.shard;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.test.engine.MockEngineFactory;
import org.elasticsearch.test.engine.MockEngineSupportModule;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
// this must exist in the same package as IndexShardModule to allow access to setting the impl
public class MockEngineFactoryPlugin extends AbstractPlugin {
@Override
public String name() {
return "mock-engine-factory";
}
@Override
public String description() {
return "a mock engine factory for testing";
}
@Override
public Collection<Class<? extends Module>> indexModules() {
List<Class<? extends Module>> modules = new ArrayList<>();
modules.add(MockEngineSupportModule.class);
return modules;
}
public void onModule(IndexShardModule module) {
module.engineFactoryImpl = MockEngineFactory.class;
}
}

View File

@ -35,9 +35,11 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.engine.MockEngineSupport;
import org.elasticsearch.test.engine.MockEngineSupportModule;
import org.elasticsearch.test.engine.ThrowingLeafReaderWrapper;
import org.junit.Test;
@ -105,7 +107,7 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
Settings.Builder settings = settingsBuilder()
.put(indexSettings())
.put(MockEngineSupport.READER_WRAPPER_TYPE, RandomExceptionDirectoryReaderWrapper.class.getName())
.extendArray("plugin.types", RandomExceptionDirectoryReaderWrapper.Plugin.class.getName())
.put(EXCEPTION_TOP_LEVEL_RATIO_KEY, topLevelRate)
.put(EXCEPTION_LOW_LEVEL_RATIO_KEY, lowLevelRate)
.put(MockEngineSupport.WRAP_READER_RATIO, 1.0d);
@ -199,6 +201,21 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
// TODO: Generalize this class and add it as a utility
public static class RandomExceptionDirectoryReaderWrapper extends MockEngineSupport.DirectoryReaderWrapper {
public static class Plugin extends AbstractPlugin {
@Override
public String name() {
return "random-exception-reader-wrapper";
}
@Override
public String description() {
return "a mock reader wrapper that throws random exceptions for testing";
}
public void onModule(MockEngineSupportModule module) {
module.wrapperImpl = RandomExceptionDirectoryReaderWrapper.class;
}
}
private final Settings settings;
static class ThrowingSubReaderWrapper extends SubReaderWrapper implements ThrowingLeafReaderWrapper.Thrower {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.indices.stats;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.Version;
@ -39,9 +40,7 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.query.QueryCacheModule;
import org.elasticsearch.index.cache.query.QueryCacheStats;
import org.elasticsearch.index.cache.query.QueryCacheModule.QueryCacheSettings;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.shard.MergePolicyConfig;
@ -79,8 +78,8 @@ public class IndexStatsIT extends ESIntegTestCase {
//Filter/Query cache is cleaned periodically, default is 60s, so make sure it runs often. Thread.sleep for 60s is bad
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
.put(IndicesRequestCache.INDICES_CACHE_REQUEST_CLEAN_INTERVAL, "1ms")
.put(QueryCacheSettings.QUERY_CACHE_EVERYTHING, true)
.put(QueryCacheModule.QueryCacheSettings.QUERY_CACHE_TYPE, IndexQueryCache.class)
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true)
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE)
.build();
}

View File

@ -0,0 +1,41 @@
/*
* 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.node;
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.plugins.AbstractPlugin;
public class NodeMocksPlugin extends AbstractPlugin {
@Override
public String name() {
return "node-mocks";
}
@Override
public String description() {
return "a plugin to setup mocks for node level classes";
}
public void onModule(NodeModule module) {
module.pageCacheRecyclerImpl = MockPageCacheRecycler.class;
module.bigArraysImpl = MockBigArrays.class;
}
}

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.test.search;
package org.elasticsearch.search;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.cluster.ClusterService;
@ -28,6 +28,7 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.cache.request.IndicesRequestCache;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.dfs.DfsPhase;
@ -42,6 +43,20 @@ import java.util.concurrent.ConcurrentHashMap;
public class MockSearchService extends SearchService {
public static class Plugin extends AbstractPlugin {
@Override
public String name() {
return "mock-search-service";
}
@Override
public String description() {
return "a mock search service for testing";
}
public void onModule(SearchModule module) {
module.searchServiceImpl = MockSearchService.class;
}
}
private static final Map<SearchContext, Throwable> ACTIVE_SEARCH_CONTEXTS = new ConcurrentHashMap<>();
/** Throw an {@link AssertionError} if there are still in-flight contexts. */

View File

@ -35,10 +35,12 @@ import org.elasticsearch.common.settings.Settings.Builder;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.engine.MockEngineSupport;
import org.elasticsearch.test.engine.MockEngineSupportModule;
import org.elasticsearch.test.engine.ThrowingLeafReaderWrapper;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.store.MockFSDirectoryService;
@ -250,7 +252,7 @@ public class SearchWithRandomExceptionsIT extends ESIntegTestCase {
Builder settings = settingsBuilder()
.put(indexSettings())
.put(MockEngineSupport.READER_WRAPPER_TYPE, RandomExceptionDirectoryReaderWrapper.class.getName())
.extendArray("plugin.types", RandomExceptionDirectoryReaderWrapper.Plugin.class.getName())
.put(EXCEPTION_TOP_LEVEL_RATIO_KEY, topLevelRate)
.put(EXCEPTION_LOW_LEVEL_RATIO_KEY, lowLevelRate)
.put(MockEngineSupport.WRAP_READER_RATIO, 1.0d);
@ -310,6 +312,21 @@ public class SearchWithRandomExceptionsIT extends ESIntegTestCase {
public static class RandomExceptionDirectoryReaderWrapper extends MockEngineSupport.DirectoryReaderWrapper {
public static class Plugin extends AbstractPlugin {
@Override
public String name() {
return "random-exception-reader-wrapper";
}
@Override
public String description() {
return "a mock reader wrapper that throws random exceptions for testing";
}
public void onModule(MockEngineSupportModule module) {
module.wrapperImpl = RandomExceptionDirectoryReaderWrapper.class;
}
}
private final Settings settings;
static class ThrowingSubReaderWrapper extends FilterDirectoryReader.SubReaderWrapper implements ThrowingLeafReaderWrapper.Thrower {

View File

@ -29,8 +29,7 @@ import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.common.lucene.search.function.CombineFunction;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.cache.query.QueryCacheModule;
import org.elasticsearch.index.cache.query.QueryCacheModule.QueryCacheSettings;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.query.HasChildQueryBuilder;
@ -74,8 +73,8 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
// aggressive filter caching so that we can assert on the filter cache size
.put(QueryCacheModule.QueryCacheSettings.QUERY_CACHE_TYPE, IndexQueryCache.class)
.put(QueryCacheSettings.QUERY_CACHE_EVERYTHING, true)
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE)
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true)
.build();
}

View File

@ -21,8 +21,7 @@ package org.elasticsearch.search.scriptfilter;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.query.QueryCacheModule;
import org.elasticsearch.index.cache.query.QueryCacheModule.QueryCacheSettings;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
@ -50,8 +49,8 @@ public class ScriptQuerySearchIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
// aggressive filter caching so that we can assert on the number of iterations of the script filters
.put(QueryCacheModule.QueryCacheSettings.QUERY_CACHE_TYPE, IndexQueryCache.class)
.put(QueryCacheSettings.QUERY_CACHE_EVERYTHING, true)
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE)
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true)
.build();
}

View File

@ -46,18 +46,17 @@ import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsAbortPolicy;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.test.junit.listeners.AssertionErrorThreadDumpPrinter;
import org.elasticsearch.test.junit.listeners.LoggingListener;
import org.elasticsearch.test.junit.listeners.ReproduceInfoPrinter;
import org.elasticsearch.test.search.MockSearchService;
import org.elasticsearch.search.MockSearchService;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.*;
import org.junit.rules.RuleChain;

View File

@ -42,7 +42,6 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.cache.recycler.PageCacheRecyclerModule;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.cluster.ClusterName;
@ -72,20 +71,16 @@ import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArraysModule;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.cache.query.QueryCacheModule;
import org.elasticsearch.index.cache.query.QueryCacheModule.QueryCacheSettings;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.engine.CommitStats;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineClosedException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardModule;
import org.elasticsearch.index.shard.MockEngineFactoryPlugin;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
@ -95,16 +90,14 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.NodeMocksPlugin;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.test.disruption.ServiceDisruptionScheme;
import org.elasticsearch.test.engine.MockEngineFactory;
import org.elasticsearch.test.search.MockSearchService;
import org.elasticsearch.search.MockSearchService;
import org.elasticsearch.test.store.MockFSIndexStore;
import org.elasticsearch.test.transport.AssertingLocalTransport;
import org.elasticsearch.test.transport.MockTransportService;
@ -390,11 +383,12 @@ public final class InternalTestCluster extends TestCluster {
Builder builder = Settings.settingsBuilder()
.put(SETTING_CLUSTER_NODE_SEED, seed);
if (ENABLE_MOCK_MODULES && usually(random)) {
builder.extendArray("plugin.types", MockTransportService.Plugin.class.getName(), MockFSIndexStore.Plugin.class.getName());
builder.put(IndexShardModule.ENGINE_FACTORY, MockEngineFactory.class);
builder.put(PageCacheRecyclerModule.CACHE_IMPL, MockPageCacheRecycler.class.getName());
builder.put(BigArraysModule.IMPL, MockBigArrays.class.getName());
builder.put(SearchModule.SEARCH_SERVICE_IMPL, MockSearchService.class.getName());
builder.extendArray("plugin.types",
MockTransportService.Plugin.class.getName(),
MockFSIndexStore.Plugin.class.getName(),
NodeMocksPlugin.class.getName(),
MockEngineFactoryPlugin.class.getName(),
MockSearchService.Plugin.class.getName());
}
if (isLocalTransportConfigured()) {
builder.extendArray("plugin.types", AssertingLocalTransport.Plugin.class.getName());
@ -457,11 +451,11 @@ public final class InternalTestCluster extends TestCluster {
}
if (random.nextBoolean()) {
builder.put(QueryCacheModule.QueryCacheSettings.QUERY_CACHE_TYPE, random.nextBoolean() ? IndexQueryCache.class : NoneQueryCache.class);
builder.put(IndexCacheModule.QUERY_CACHE_TYPE, random.nextBoolean() ? IndexCacheModule.INDEX_QUERY_CACHE : IndexCacheModule.NONE_QUERY_CACHE);
}
if (random.nextBoolean()) {
builder.put(QueryCacheSettings.QUERY_CACHE_EVERYTHING, random.nextBoolean());
builder.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, random.nextBoolean());
}
if (random.nextBoolean()) {

View File

@ -18,25 +18,41 @@
*/
package org.elasticsearch.test.engine;
import org.elasticsearch.common.settings.Settings;
import org.apache.lucene.index.FilterDirectoryReader;
import org.elasticsearch.common.inject.BindingAnnotation;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.plugins.AbstractPlugin;
import org.elasticsearch.transport.TransportModule;
/**
*
*/
import java.lang.annotation.Retention;
import java.lang.annotation.Target;
import static java.lang.annotation.ElementType.FIELD;
import static java.lang.annotation.ElementType.PARAMETER;
import static java.lang.annotation.RetentionPolicy.RUNTIME;
public final class MockEngineFactory implements EngineFactory {
@BindingAnnotation
@Target({FIELD, PARAMETER})
@Retention(RUNTIME)
public @interface MockReaderType {
}
private Class<? extends FilterDirectoryReader> wrapper;
@Inject
public MockEngineFactory(@MockReaderType Class wrapper) {
this.wrapper = wrapper;
}
@Override
public Engine newReadWriteEngine(EngineConfig config, boolean skipTranslogRecovery) {
return new MockInternalEngine(config, skipTranslogRecovery);
return new MockInternalEngine(config, skipTranslogRecovery, wrapper);
}
@Override
public Engine newReadOnlyEngine(EngineConfig config) {
return new MockShadowEngine(config);
return new MockShadowEngine(config, wrapper);
}
}

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.test.engine;
import org.apache.lucene.index.AssertingDirectoryReader;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.index.IndexReader;
@ -29,7 +28,6 @@ import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Classes;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
@ -80,7 +78,7 @@ public final class MockEngineSupport {
}
}
public MockEngineSupport(EngineConfig config) {
public MockEngineSupport(EngineConfig config, Class<? extends FilterDirectoryReader> wrapper) {
Settings indexSettings = config.getIndexSettings();
shardId = config.getShardId();
filterCache = config.getQueryCache();
@ -88,13 +86,6 @@ public final class MockEngineSupport {
final long seed = indexSettings.getAsLong(ESIntegTestCase.SETTING_INDEX_SEED, 0l);
Random random = new Random(seed);
final double ratio = indexSettings.getAsDouble(WRAP_READER_RATIO, 0.0d); // DISABLED by default - AssertingDR is crazy slow
String readerWrapperType = indexSettings.get(READER_WRAPPER_TYPE);
Class<? extends AssertingDirectoryReader > wrapper;
if (readerWrapperType == null) {
wrapper = AssertingDirectoryReader.class;
} else {
wrapper = Classes.loadClass(getClass().getClassLoader(), readerWrapperType);
}
boolean wrapReader = random.nextDouble() < ratio;
if (logger.isTraceEnabled()) {
logger.trace("Using [{}] for shard [{}] seed: [{}] wrapReader: [{}]", this.getClass().getName(), shardId, seed, wrapReader);

View File

@ -16,21 +16,17 @@
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test.engine;
package org.elasticsearch.index.cache.bitset;
import org.apache.lucene.index.AssertingDirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
/**
*/
public class BitsetFilterCacheModule extends AbstractModule {
public BitsetFilterCacheModule(Settings settings) {
}
public class MockEngineSupportModule extends AbstractModule {
public Class<? extends FilterDirectoryReader> wrapperImpl = AssertingDirectoryReader.class;
@Override
protected void configure() {
bind(BitsetFilterCache.class).asEagerSingleton();
bind(Class.class).annotatedWith(MockEngineFactory.MockReaderType.class).toInstance(wrapperImpl);
}
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.test.engine;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherManager;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -30,17 +31,19 @@ import java.io.IOException;
final class MockInternalEngine extends InternalEngine {
private MockEngineSupport support;
private final boolean randomizeFlushOnClose;
private Class<? extends FilterDirectoryReader> wrapperClass;
MockInternalEngine(EngineConfig config, boolean skipInitialTranslogRecovery) throws EngineException {
MockInternalEngine(EngineConfig config, boolean skipInitialTranslogRecovery, Class<? extends FilterDirectoryReader> wrapper) throws EngineException {
super(config, skipInitialTranslogRecovery);
randomizeFlushOnClose = IndexMetaData.isOnSharedFilesystem(config.getIndexSettings()) == false;
wrapperClass = wrapper;
}
private synchronized MockEngineSupport support() {
// lazy initialized since we need it already on super() ctor execution :(
if (support == null) {
support = new MockEngineSupport(config());
support = new MockEngineSupport(config(), wrapperClass);
}
return support;
}

View File

@ -19,6 +19,8 @@
package org.elasticsearch.test.engine;
import org.apache.lucene.index.AssertingDirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.search.AssertingIndexSearcher;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherManager;
@ -32,9 +34,9 @@ import java.util.Map;
final class MockShadowEngine extends ShadowEngine {
private final MockEngineSupport support;
MockShadowEngine(EngineConfig config) {
MockShadowEngine(EngineConfig config, Class<? extends FilterDirectoryReader> wrapper) {
super(config);
this.support = new MockEngineSupport(config);
this.support = new MockEngineSupport(config, wrapper);
}
@Override

View File

@ -29,8 +29,8 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.test.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.netty.NettyTransport;
import org.junit.After;

View File

@ -31,7 +31,7 @@ import org.elasticsearch.common.transport.InetSocketTransportAddress;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cache.recycler.MockBigArrays;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.test.junit.rule.RepeatOnExceptionRule;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BindTransportException;