mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-09 14:34:43 +00:00
Move PageCacheRecycler into BigArrays (#18666)
PageCacheRecycler is really just an implementation detail of BigArrays. There is no need to leak this class anywhere outside of it.
This commit is contained in:
parent
93c5a9dacd
commit
88800e8e47
@ -28,7 +28,6 @@ import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
|
||||
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
|
||||
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockException;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
@ -43,7 +42,6 @@ import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.query.QueryShardException;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
@ -73,8 +71,6 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
|
||||
|
||||
private final ScriptService scriptService;
|
||||
|
||||
private final PageCacheRecycler pageCacheRecycler;
|
||||
|
||||
private final BigArrays bigArrays;
|
||||
|
||||
private final FetchPhase fetchPhase;
|
||||
@ -82,13 +78,12 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
|
||||
@Inject
|
||||
public TransportValidateQueryAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, IndicesService indicesService, ScriptService scriptService,
|
||||
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, ActionFilters actionFilters,
|
||||
BigArrays bigArrays, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, FetchPhase fetchPhase) {
|
||||
super(settings, ValidateQueryAction.NAME, threadPool, clusterService, transportService, actionFilters,
|
||||
indexNameExpressionResolver, ValidateQueryRequest::new, ShardValidateQueryRequest::new, ThreadPool.Names.SEARCH);
|
||||
this.indicesService = indicesService;
|
||||
this.scriptService = scriptService;
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
this.bigArrays = bigArrays;
|
||||
this.fetchPhase = fetchPhase;
|
||||
}
|
||||
@ -176,7 +171,7 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
|
||||
|
||||
DefaultSearchContext searchContext = new DefaultSearchContext(0,
|
||||
new ShardSearchLocalRequest(request.types(), request.nowInMillis(), request.filteringAliases()), null, searcher,
|
||||
indexService, indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(),
|
||||
indexService, indexShard, scriptService, bigArrays, threadPool.estimatedTimeInMillisCounter(),
|
||||
parseFieldMatcher, SearchService.NO_TIMEOUT, fetchPhase);
|
||||
SearchContext.setCurrent(searchContext);
|
||||
try {
|
||||
|
@ -26,7 +26,6 @@ import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.RoutingMissingException;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.routing.ShardIterator;
|
||||
@ -65,7 +64,6 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
|
||||
|
||||
private final ScriptService scriptService;
|
||||
|
||||
private final PageCacheRecycler pageCacheRecycler;
|
||||
|
||||
private final BigArrays bigArrays;
|
||||
|
||||
@ -74,13 +72,12 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
|
||||
@Inject
|
||||
public TransportExplainAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, IndicesService indicesService, ScriptService scriptService,
|
||||
PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver, FetchPhase fetchPhase) {
|
||||
BigArrays bigArrays, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
FetchPhase fetchPhase) {
|
||||
super(settings, ExplainAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
|
||||
ExplainRequest::new, ThreadPool.Names.GET);
|
||||
this.indicesService = indicesService;
|
||||
this.scriptService = scriptService;
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
this.bigArrays = bigArrays;
|
||||
this.fetchPhase = fetchPhase;
|
||||
}
|
||||
@ -117,7 +114,7 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
|
||||
|
||||
SearchContext context = new DefaultSearchContext(0,
|
||||
new ShardSearchLocalRequest(new String[] { request.type() }, request.nowInMillis, request.filteringAlias()), null,
|
||||
result.searcher(), indexService, indexShard, scriptService, pageCacheRecycler, bigArrays,
|
||||
result.searcher(), indexService, indexShard, scriptService, bigArrays,
|
||||
threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, SearchService.NO_TIMEOUT, fetchPhase);
|
||||
SearchContext.setCurrent(context);
|
||||
|
||||
|
@ -26,7 +26,6 @@ import org.elasticsearch.action.ActionModule;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.client.support.AbstractClient;
|
||||
import org.elasticsearch.client.transport.support.TransportProxyClient;
|
||||
import org.elasticsearch.cluster.ClusterNameModule;
|
||||
@ -41,6 +40,7 @@ import org.elasticsearch.common.network.NetworkService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsModule;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerModule;
|
||||
import org.elasticsearch.monitor.MonitorService;
|
||||
import org.elasticsearch.node.internal.InternalSettingsPreparer;
|
||||
@ -272,7 +272,7 @@ public class TransportClient extends AbstractClient {
|
||||
// ignore
|
||||
}
|
||||
|
||||
injector.getInstance(PageCacheRecycler.class).close();
|
||||
injector.getInstance(BigArrays.class).close();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -25,13 +25,13 @@ import org.elasticsearch.common.lease.Releasable;
|
||||
* A recycled object, note, implementations should support calling obtain and then recycle
|
||||
* on different threads.
|
||||
*/
|
||||
public interface Recycler<T> {
|
||||
public interface Recycler<T> extends Releasable {
|
||||
|
||||
public static interface Factory<T> {
|
||||
interface Factory<T> {
|
||||
Recycler<T> build();
|
||||
}
|
||||
|
||||
public static interface C<T> {
|
||||
interface C<T> {
|
||||
|
||||
/** Create a new empty instance of the given size. */
|
||||
T newInstance(int sizing);
|
||||
@ -43,7 +43,7 @@ public interface Recycler<T> {
|
||||
void destroy(T value);
|
||||
}
|
||||
|
||||
public static interface V<T> extends Releasable {
|
||||
interface V<T> extends Releasable {
|
||||
|
||||
/** Reference to the value. */
|
||||
T v();
|
||||
|
@ -24,7 +24,7 @@ import org.elasticsearch.action.support.AutoCreateIndex;
|
||||
import org.elasticsearch.action.support.DestructiveOperations;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeReadAction;
|
||||
import org.elasticsearch.bootstrap.BootstrapSettings;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.util.PageCacheRecycler;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.transport.TransportClientNodesService;
|
||||
import org.elasticsearch.cluster.ClusterModule;
|
||||
|
@ -21,7 +21,6 @@ package org.elasticsearch.common.util;
|
||||
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.recycler.Recycler;
|
||||
|
||||
|
@ -22,7 +22,6 @@ package org.elasticsearch.common.util;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.RamUsageEstimator;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.breaker.CircuitBreakingException;
|
||||
@ -30,14 +29,15 @@ import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.recycler.Recycler;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
/** Utility class to work with arrays. */
|
||||
public class BigArrays {
|
||||
public class BigArrays implements Releasable {
|
||||
|
||||
public static final BigArrays NON_RECYCLING_INSTANCE = new BigArrays(null, null);
|
||||
public static final BigArrays NON_RECYCLING_INSTANCE = new BigArrays(null, null, false);
|
||||
|
||||
/** Page size in bytes: 16KB */
|
||||
public static final int PAGE_SIZE_IN_BYTES = 1 << 14;
|
||||
@ -84,6 +84,11 @@ public class BigArrays {
|
||||
return index == (int) index;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
recycler.close();
|
||||
}
|
||||
|
||||
private static abstract class AbstractArrayWrapper extends AbstractArray implements BigArray {
|
||||
|
||||
protected static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(ByteArrayWrapper.class);
|
||||
@ -369,9 +374,9 @@ public class BigArrays {
|
||||
private final BigArrays circuitBreakingInstance;
|
||||
|
||||
@Inject
|
||||
public BigArrays(PageCacheRecycler recycler, @Nullable final CircuitBreakerService breakerService) {
|
||||
public BigArrays(Settings settings, @Nullable final CircuitBreakerService breakerService) {
|
||||
// Checking the breaker is disabled if not specified
|
||||
this(recycler, breakerService, false);
|
||||
this(new PageCacheRecycler(settings), breakerService, false);
|
||||
}
|
||||
|
||||
public BigArrays(PageCacheRecycler recycler, @Nullable final CircuitBreakerService breakerService, boolean checkBreaker) {
|
||||
|
@ -17,11 +17,12 @@
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.cache.recycler;
|
||||
package org.elasticsearch.common.util;
|
||||
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
import org.elasticsearch.common.recycler.AbstractRecyclerC;
|
||||
import org.elasticsearch.common.recycler.Recycler;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
@ -63,14 +64,10 @@ public class PageCacheRecycler extends AbstractComponent implements Releasable {
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
bytePage.close();
|
||||
intPage.close();
|
||||
longPage.close();
|
||||
objectPage.close();
|
||||
Releasables.close(true, bytePage, intPage, longPage, objectPage);
|
||||
}
|
||||
|
||||
@Inject
|
||||
public PageCacheRecycler(Settings settings) {
|
||||
protected PageCacheRecycler(Settings settings) {
|
||||
super(settings);
|
||||
final Type type = TYPE_SETTING .get(settings);
|
||||
final long limit = LIMIT_HEAP_SETTING .get(settings).bytes();
|
@ -26,7 +26,6 @@ import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.ElasticsearchTimeoutException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionModule;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.node.NodeClientModule;
|
||||
import org.elasticsearch.cluster.ClusterModule;
|
||||
@ -62,6 +61,7 @@ import org.elasticsearch.common.settings.SettingsModule;
|
||||
import org.elasticsearch.common.transport.BoundTransportAddress;
|
||||
import org.elasticsearch.common.transport.TransportAddress;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.discovery.Discovery;
|
||||
import org.elasticsearch.discovery.DiscoveryModule;
|
||||
@ -526,7 +526,7 @@ public class Node implements Closeable {
|
||||
|
||||
|
||||
toClose.add(injector.getInstance(NodeEnvironment.class));
|
||||
toClose.add(injector.getInstance(PageCacheRecycler.class));
|
||||
toClose.add(injector.getInstance(BigArrays.class));
|
||||
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("Close times for each service:\n{}", stopWatch.prettyPrint());
|
||||
|
@ -19,7 +19,6 @@
|
||||
|
||||
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.ingest.ProcessorsRegistry;
|
||||
@ -56,7 +55,6 @@ public class NodeModule extends AbstractModule {
|
||||
private final ProcessorsRegistry.Builder processorsRegistryBuilder;
|
||||
|
||||
// pkg private so tests can mock
|
||||
Class<? extends PageCacheRecycler> pageCacheRecyclerImpl = PageCacheRecycler.class;
|
||||
Class<? extends BigArrays> bigArraysImpl = BigArrays.class;
|
||||
|
||||
public NodeModule(Node node, MonitorService monitorService) {
|
||||
@ -84,11 +82,6 @@ public class NodeModule extends AbstractModule {
|
||||
|
||||
@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 {
|
||||
|
@ -33,10 +33,8 @@ import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
@ -134,8 +132,6 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
||||
|
||||
private final ScriptService scriptService;
|
||||
|
||||
private final PageCacheRecycler pageCacheRecycler;
|
||||
|
||||
private final BigArrays bigArrays;
|
||||
|
||||
private final DfsPhase dfsPhase;
|
||||
@ -162,7 +158,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
||||
|
||||
@Inject
|
||||
public SearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService, IndicesService indicesService,
|
||||
ThreadPool threadPool, ScriptService scriptService, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, DfsPhase dfsPhase,
|
||||
ThreadPool threadPool, ScriptService scriptService, BigArrays bigArrays, DfsPhase dfsPhase,
|
||||
QueryPhase queryPhase, FetchPhase fetchPhase, AggregatorParsers aggParsers, Suggesters suggesters) {
|
||||
super(settings);
|
||||
this.aggParsers = aggParsers;
|
||||
@ -172,7 +168,6 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
||||
this.clusterService = clusterService;
|
||||
this.indicesService = indicesService;
|
||||
this.scriptService = scriptService;
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
this.bigArrays = bigArrays;
|
||||
this.dfsPhase = dfsPhase;
|
||||
this.queryPhase = queryPhase;
|
||||
@ -548,7 +543,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
||||
|
||||
DefaultSearchContext context = new DefaultSearchContext(idGenerator.incrementAndGet(), request, shardTarget, engineSearcher,
|
||||
indexService,
|
||||
indexShard, scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
|
||||
indexShard, scriptService, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
|
||||
defaultSearchTimeout, fetchPhase);
|
||||
SearchContext.setCurrent(context);
|
||||
request.rewrite(context.getQueryShardContext());
|
||||
|
@ -19,7 +19,6 @@
|
||||
package org.elasticsearch.search.aggregations.support;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.geo.GeoPoint;
|
||||
import org.elasticsearch.common.geo.GeoUtils;
|
||||
@ -51,10 +50,6 @@ public class AggregationContext {
|
||||
return searchContext;
|
||||
}
|
||||
|
||||
public PageCacheRecycler pageCacheRecycler() {
|
||||
return searchContext.pageCacheRecycler();
|
||||
}
|
||||
|
||||
public BigArrays bigArrays() {
|
||||
return searchContext.bigArrays();
|
||||
}
|
||||
|
@ -29,7 +29,6 @@ import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.lease.Releasables;
|
||||
@ -91,7 +90,6 @@ public class DefaultSearchContext extends SearchContext {
|
||||
private SearchType searchType;
|
||||
private final Engine.Searcher engineSearcher;
|
||||
private final ScriptService scriptService;
|
||||
private final PageCacheRecycler pageCacheRecycler;
|
||||
private final BigArrays bigArrays;
|
||||
private final IndexShard indexShard;
|
||||
private final IndexService indexService;
|
||||
@ -153,7 +151,7 @@ public class DefaultSearchContext extends SearchContext {
|
||||
private FetchPhase fetchPhase;
|
||||
|
||||
public DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget, Engine.Searcher engineSearcher,
|
||||
IndexService indexService, IndexShard indexShard, ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
|
||||
IndexService indexService, IndexShard indexShard, ScriptService scriptService,
|
||||
BigArrays bigArrays, Counter timeEstimateCounter, ParseFieldMatcher parseFieldMatcher, TimeValue timeout,
|
||||
FetchPhase fetchPhase) {
|
||||
super(parseFieldMatcher);
|
||||
@ -164,7 +162,6 @@ public class DefaultSearchContext extends SearchContext {
|
||||
this.shardTarget = shardTarget;
|
||||
this.engineSearcher = engineSearcher;
|
||||
this.scriptService = scriptService;
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
// SearchContexts use a BigArrays that can circuit break
|
||||
this.bigArrays = bigArrays.withCircuitBreaking();
|
||||
this.dfsResult = new DfsSearchResult(id, shardTarget);
|
||||
@ -474,11 +471,6 @@ public class DefaultSearchContext extends SearchContext {
|
||||
return scriptService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PageCacheRecycler pageCacheRecycler() {
|
||||
return pageCacheRecycler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BigArrays bigArrays() {
|
||||
return bigArrays;
|
||||
|
@ -24,7 +24,6 @@ import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
@ -249,11 +248,6 @@ public abstract class FilteredSearchContext extends SearchContext {
|
||||
return in.scriptService();
|
||||
}
|
||||
|
||||
@Override
|
||||
public PageCacheRecycler pageCacheRecycler() {
|
||||
return in.pageCacheRecycler();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BigArrays bigArrays() {
|
||||
return in.bigArrays();
|
||||
|
@ -22,10 +22,8 @@ package org.elasticsearch.search.internal;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Sort;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
@ -222,8 +220,6 @@ public abstract class SearchContext implements Releasable {
|
||||
|
||||
public abstract ScriptService scriptService();
|
||||
|
||||
public abstract PageCacheRecycler pageCacheRecycler();
|
||||
|
||||
public abstract BigArrays bigArrays();
|
||||
|
||||
public abstract BitsetFilterCache bitsetFilterCache();
|
||||
|
@ -51,7 +51,7 @@ public class PagedBytesReferenceTests extends ESTestCase {
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
bigarrays = new BigArrays(null, new NoneCircuitBreakerService());
|
||||
bigarrays = new BigArrays(null, new NoneCircuitBreakerService(), false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -20,7 +20,6 @@
|
||||
package org.elasticsearch.common.util;
|
||||
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.breaker.CircuitBreakingException;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
@ -38,8 +37,7 @@ import java.util.Arrays;
|
||||
public class BigArraysTests extends ESSingleNodeTestCase {
|
||||
|
||||
private BigArrays randombigArrays() {
|
||||
final PageCacheRecycler recycler = randomBoolean() ? null : getInstanceFromNode(PageCacheRecycler.class);
|
||||
return new MockBigArrays(recycler, new NoneCircuitBreakerService());
|
||||
return new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
private BigArrays bigArrays;
|
||||
@ -339,7 +337,7 @@ public class BigArraysTests extends ESSingleNodeTestCase {
|
||||
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), size - 1, ByteSizeUnit.BYTES)
|
||||
.build(),
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
|
||||
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
|
||||
BigArrays bigArrays = new BigArrays(null, hcbs, false).withCircuitBreaking();
|
||||
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
|
||||
try {
|
||||
create.invoke(bigArrays, size);
|
||||
@ -359,7 +357,7 @@ public class BigArraysTests extends ESSingleNodeTestCase {
|
||||
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), maxSize, ByteSizeUnit.BYTES)
|
||||
.build(),
|
||||
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
|
||||
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
|
||||
BigArrays bigArrays = new BigArrays(null, hcbs, false).withCircuitBreaking();
|
||||
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
|
||||
final int size = scaledRandomIntBetween(1, 20);
|
||||
BigArray array = (BigArray) create.invoke(bigArrays, size);
|
||||
|
@ -25,7 +25,7 @@ import com.carrotsearch.hppc.cursors.ObjectLongCursor;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefBuilder;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
|
||||
@ -41,8 +41,7 @@ public class BytesRefHashTests extends ESSingleNodeTestCase {
|
||||
BytesRefHash hash;
|
||||
|
||||
private BigArrays randombigArrays() {
|
||||
final PageCacheRecycler recycler = randomBoolean() ? null : getInstanceFromNode(PageCacheRecycler.class);
|
||||
return new MockBigArrays(recycler, new NoneCircuitBreakerService());
|
||||
return new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
private void newHash() {
|
||||
|
@ -22,7 +22,7 @@ package org.elasticsearch.common.util;
|
||||
import com.carrotsearch.hppc.LongLongHashMap;
|
||||
import com.carrotsearch.hppc.LongLongMap;
|
||||
import com.carrotsearch.hppc.cursors.LongLongCursor;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
|
||||
@ -36,8 +36,7 @@ public class LongHashTests extends ESSingleNodeTestCase {
|
||||
LongHash hash;
|
||||
|
||||
private BigArrays randombigArrays() {
|
||||
final PageCacheRecycler recycler = randomBoolean() ? null : getInstanceFromNode(PageCacheRecycler.class);
|
||||
return new MockBigArrays(recycler, new NoneCircuitBreakerService());
|
||||
return new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
private void newHash() {
|
||||
|
@ -20,15 +20,14 @@
|
||||
package org.elasticsearch.common.util;
|
||||
|
||||
import com.carrotsearch.hppc.LongObjectHashMap;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
|
||||
public class LongObjectHashMapTests extends ESSingleNodeTestCase {
|
||||
|
||||
private BigArrays randombigArrays() {
|
||||
final PageCacheRecycler recycler = randomBoolean() ? null : getInstanceFromNode(PageCacheRecycler.class);
|
||||
return new MockBigArrays(recycler, new NoneCircuitBreakerService());
|
||||
return new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
public void testDuel() {
|
||||
|
@ -19,7 +19,6 @@
|
||||
|
||||
package org.elasticsearch.http.netty;
|
||||
|
||||
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.network.NetworkService;
|
||||
@ -72,8 +71,7 @@ public class NettyHttpChannelTests extends ESTestCase {
|
||||
public void setup() throws Exception {
|
||||
networkService = new NetworkService(Settings.EMPTY);
|
||||
threadPool = new ThreadPool("test");
|
||||
MockPageCacheRecycler mockPageCacheRecycler = new MockPageCacheRecycler(Settings.EMPTY);
|
||||
bigArrays = new MockBigArrays(mockPageCacheRecycler, new NoneCircuitBreakerService());
|
||||
bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -18,7 +18,6 @@
|
||||
*/
|
||||
package org.elasticsearch.http.netty;
|
||||
|
||||
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
|
||||
import org.elasticsearch.common.network.NetworkService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.InetSocketTransportAddress;
|
||||
@ -68,7 +67,6 @@ import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
|
||||
public class NettyHttpServerPipeliningTests extends ESTestCase {
|
||||
private NetworkService networkService;
|
||||
private ThreadPool threadPool;
|
||||
private MockPageCacheRecycler mockPageCacheRecycler;
|
||||
private MockBigArrays bigArrays;
|
||||
private CustomNettyHttpServerTransport httpServerTransport;
|
||||
|
||||
@ -76,8 +74,7 @@ public class NettyHttpServerPipeliningTests extends ESTestCase {
|
||||
public void setup() throws Exception {
|
||||
networkService = new NetworkService(Settings.EMPTY);
|
||||
threadPool = new ThreadPool("test");
|
||||
mockPageCacheRecycler = new MockPageCacheRecycler(Settings.EMPTY);
|
||||
bigArrays = new MockBigArrays(mockPageCacheRecycler, new NoneCircuitBreakerService());
|
||||
bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
@After
|
||||
|
@ -19,7 +19,6 @@
|
||||
|
||||
package org.elasticsearch.http.netty;
|
||||
|
||||
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.network.NetworkService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
@ -50,15 +49,13 @@ import static org.hamcrest.Matchers.equalTo;
|
||||
public class NettyHttpServerTransportTests extends ESTestCase {
|
||||
private NetworkService networkService;
|
||||
private ThreadPool threadPool;
|
||||
private MockPageCacheRecycler mockPageCacheRecycler;
|
||||
private MockBigArrays bigArrays;
|
||||
|
||||
@Before
|
||||
public void setup() throws Exception {
|
||||
networkService = new NetworkService(Settings.EMPTY);
|
||||
threadPool = new ThreadPool("test");
|
||||
mockPageCacheRecycler = new MockPageCacheRecycler(Settings.EMPTY);
|
||||
bigArrays = new MockBigArrays(mockPageCacheRecycler, new NoneCircuitBreakerService());
|
||||
bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
}
|
||||
|
||||
@After
|
||||
@ -68,7 +65,6 @@ public class NettyHttpServerTransportTests extends ESTestCase {
|
||||
}
|
||||
threadPool = null;
|
||||
networkService = null;
|
||||
mockPageCacheRecycler = null;
|
||||
bigArrays = null;
|
||||
}
|
||||
|
||||
|
@ -32,7 +32,6 @@ import org.apache.lucene.search.similarities.BM25Similarity;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.util.SetOnce.AlreadySetException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
@ -113,8 +112,7 @@ public class IndexModuleTests extends ESTestCase {
|
||||
// TODO this can be used in other place too - lets first refactor the IndicesQueriesRegistry
|
||||
ThreadPool threadPool = new ThreadPool("test");
|
||||
CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
|
||||
PageCacheRecycler recycler = new PageCacheRecycler(settings);
|
||||
BigArrays bigArrays = new BigArrays(recycler, circuitBreakerService);
|
||||
BigArrays bigArrays = new BigArrays(settings, circuitBreakerService);
|
||||
Set<ScriptEngineService> scriptEngines = Collections.emptySet();
|
||||
scriptEngines.addAll(Arrays.asList(scriptEngineServices));
|
||||
ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.emptyList());
|
||||
|
@ -21,7 +21,6 @@ package org.elasticsearch.index;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
@ -49,9 +48,8 @@ public class SearchSlowLogTests extends ESSingleNodeTestCase {
|
||||
protected SearchContext createSearchContext(IndexService indexService) {
|
||||
BigArrays bigArrays = indexService.getBigArrays();
|
||||
ThreadPool threadPool = indexService.getThreadPool();
|
||||
PageCacheRecycler pageCacheRecycler = node().injector().getInstance(PageCacheRecycler.class);
|
||||
ScriptService scriptService = node().injector().getInstance(ScriptService.class);
|
||||
return new TestSearchContext(threadPool, pageCacheRecycler, bigArrays, scriptService, indexService) {
|
||||
return new TestSearchContext(threadPool, bigArrays, scriptService, indexService) {
|
||||
@Override
|
||||
public ShardSearchRequest request() {
|
||||
return new ShardSearchRequest() {
|
||||
|
@ -20,7 +20,6 @@
|
||||
package org.elasticsearch.transport;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.network.NetworkService;
|
||||
@ -67,7 +66,7 @@ public class NettySizeHeaderFrameDecoderTests extends ESTestCase {
|
||||
threadPool = new ThreadPool(settings);
|
||||
threadPool.setClusterSettings(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
|
||||
NetworkService networkService = new NetworkService(settings);
|
||||
BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(settings), new NoneCircuitBreakerService());
|
||||
BigArrays bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
nettyTransport = new NettyTransport(settings, threadPool, networkService, bigArrays, Version.CURRENT, new NamedWriteableRegistry(),
|
||||
new NoneCircuitBreakerService());
|
||||
nettyTransport.start();
|
||||
|
@ -19,7 +19,6 @@
|
||||
package org.elasticsearch.transport.netty;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.component.Lifecycle;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.network.NetworkService;
|
||||
@ -134,7 +133,7 @@ public class NettyTransportMultiPortTests extends ESTestCase {
|
||||
}
|
||||
|
||||
private NettyTransport startNettyTransport(Settings settings, ThreadPool threadPool) {
|
||||
BigArrays bigArrays = new MockBigArrays(new PageCacheRecycler(settings), new NoneCircuitBreakerService());
|
||||
BigArrays bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService());
|
||||
|
||||
NettyTransport nettyTransport = new NettyTransport(settings, threadPool, new NetworkService(settings), bigArrays, Version.CURRENT,
|
||||
new NamedWriteableRegistry(), new NoneCircuitBreakerService());
|
||||
|
@ -24,8 +24,8 @@ import com.carrotsearch.randomizedtesting.SeedUtils;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
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.settings.Settings;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
@ -75,11 +75,11 @@ public class MockBigArrays extends BigArrays {
|
||||
private final CircuitBreakerService breakerService;
|
||||
|
||||
@Inject
|
||||
public MockBigArrays(PageCacheRecycler recycler, CircuitBreakerService breakerService) {
|
||||
this(recycler, breakerService, false);
|
||||
public MockBigArrays(Settings settings, CircuitBreakerService breakerService) {
|
||||
this(new MockPageCacheRecycler(settings), breakerService, false);
|
||||
}
|
||||
|
||||
public MockBigArrays(PageCacheRecycler recycler, CircuitBreakerService breakerService, boolean checkBreaker) {
|
||||
private MockBigArrays(PageCacheRecycler recycler, CircuitBreakerService breakerService, boolean checkBreaker) {
|
||||
super(recycler, breakerService, checkBreaker);
|
||||
this.recycler = recycler;
|
||||
this.breakerService = breakerService;
|
||||
@ -371,7 +371,7 @@ public class MockBigArrays extends BigArrays {
|
||||
public void fill(long fromIndex, long toIndex, int value) {
|
||||
in.fill(fromIndex, toIndex, value);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("delegate", in));
|
||||
@ -416,7 +416,7 @@ public class MockBigArrays extends BigArrays {
|
||||
public void fill(long fromIndex, long toIndex, long value) {
|
||||
in.fill(fromIndex, toIndex, value);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Collection<Accountable> getChildResources() {
|
||||
return Collections.singleton(Accountables.namedAccountable("delegate", in));
|
||||
|
@ -17,9 +17,8 @@
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.cache.recycler;
|
||||
package org.elasticsearch.common.util;
|
||||
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.recycler.Recycler.V;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
@ -58,8 +57,7 @@ public class MockPageCacheRecycler extends PageCacheRecycler {
|
||||
|
||||
private final Random random;
|
||||
|
||||
@Inject
|
||||
public MockPageCacheRecycler(Settings settings) {
|
||||
MockPageCacheRecycler(Settings settings) {
|
||||
super(settings);
|
||||
// we always initialize with 0 here since we really only wanna have some random bytes / ints / longs
|
||||
// and given the fact that it's called concurrently it won't reproduces anyway the same order other than in a unittest
|
@ -18,7 +18,6 @@
|
||||
*/
|
||||
package org.elasticsearch.node;
|
||||
|
||||
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
|
||||
import org.elasticsearch.common.util.MockBigArrays;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
|
||||
@ -35,7 +34,6 @@ public class NodeMocksPlugin extends Plugin {
|
||||
}
|
||||
|
||||
public void onModule(NodeModule module) {
|
||||
module.pageCacheRecyclerImpl = MockPageCacheRecycler.class;
|
||||
module.bigArraysImpl = MockBigArrays.class;
|
||||
}
|
||||
}
|
||||
|
@ -19,7 +19,6 @@
|
||||
|
||||
package org.elasticsearch.search;
|
||||
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
@ -84,10 +83,10 @@ public class MockSearchService extends SearchService {
|
||||
|
||||
@Inject
|
||||
public MockSearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService,
|
||||
IndicesService indicesService, ThreadPool threadPool, ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
|
||||
IndicesService indicesService, ThreadPool threadPool, ScriptService scriptService,
|
||||
BigArrays bigArrays, DfsPhase dfsPhase, QueryPhase queryPhase, FetchPhase fetchPhase,
|
||||
AggregatorParsers aggParsers, Suggesters suggesters) {
|
||||
super(settings, clusterSettings, clusterService, indicesService, threadPool, scriptService, pageCacheRecycler, bigArrays, dfsPhase,
|
||||
super(settings, clusterSettings, clusterService, indicesService, threadPool, scriptService, bigArrays, dfsPhase,
|
||||
queryPhase, fetchPhase, aggParsers, suggesters);
|
||||
}
|
||||
|
||||
|
@ -23,7 +23,6 @@ import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
|
||||
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
@ -35,6 +34,7 @@ import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.common.util.PageCacheRecycler;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.env.Environment;
|
||||
@ -282,9 +282,8 @@ public abstract class ESSingleNodeTestCase extends ESTestCase {
|
||||
protected SearchContext createSearchContext(IndexService indexService) {
|
||||
BigArrays bigArrays = indexService.getBigArrays();
|
||||
ThreadPool threadPool = indexService.getThreadPool();
|
||||
PageCacheRecycler pageCacheRecycler = node().injector().getInstance(PageCacheRecycler.class);
|
||||
ScriptService scriptService = node().injector().getInstance(ScriptService.class);
|
||||
return new TestSearchContext(threadPool, pageCacheRecycler, bigArrays, scriptService, indexService);
|
||||
return new TestSearchContext(threadPool, bigArrays, scriptService, indexService);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -37,7 +37,7 @@ import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.TimeUnits;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.bootstrap.BootstrapForTesting;
|
||||
import org.elasticsearch.cache.recycler.MockPageCacheRecycler;
|
||||
import org.elasticsearch.common.util.MockPageCacheRecycler;
|
||||
import org.elasticsearch.client.Requests;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
@ -82,9 +82,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.BooleanSupplier;
|
||||
import java.util.function.Consumer;
|
||||
|
@ -31,7 +31,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.action.admin.indices.stats.CommonStatsFlags.Flag;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.transport.TransportClient;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
@ -60,6 +59,7 @@ 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.PageCacheRecycler;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
|
@ -27,7 +27,6 @@ import org.apache.lucene.search.FieldDoc;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.cache.recycler.PageCacheRecycler;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
@ -67,7 +66,6 @@ import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
public class TestSearchContext extends SearchContext {
|
||||
|
||||
final PageCacheRecycler pageCacheRecycler;
|
||||
final BigArrays bigArrays;
|
||||
final IndexService indexService;
|
||||
final IndexFieldDataService indexFieldDataService;
|
||||
@ -92,9 +90,8 @@ public class TestSearchContext extends SearchContext {
|
||||
private final long originNanoTime = System.nanoTime();
|
||||
private final Map<String, FetchSubPhaseContext> subPhaseContexts = new HashMap<>();
|
||||
|
||||
public TestSearchContext(ThreadPool threadPool,PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, ScriptService scriptService, IndexService indexService) {
|
||||
public TestSearchContext(ThreadPool threadPool, BigArrays bigArrays, ScriptService scriptService, IndexService indexService) {
|
||||
super(ParseFieldMatcher.STRICT);
|
||||
this.pageCacheRecycler = pageCacheRecycler;
|
||||
this.bigArrays = bigArrays.withCircuitBreaking();
|
||||
this.indexService = indexService;
|
||||
this.indexFieldDataService = indexService.fieldData();
|
||||
@ -107,7 +104,6 @@ public class TestSearchContext extends SearchContext {
|
||||
|
||||
public TestSearchContext(QueryShardContext queryShardContext) {
|
||||
super(ParseFieldMatcher.STRICT);
|
||||
this.pageCacheRecycler = null;
|
||||
this.bigArrays = null;
|
||||
this.indexService = null;
|
||||
this.indexFieldDataService = null;
|
||||
@ -299,11 +295,6 @@ public class TestSearchContext extends SearchContext {
|
||||
return scriptService;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PageCacheRecycler pageCacheRecycler() {
|
||||
return pageCacheRecycler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BigArrays bigArrays() {
|
||||
return bigArrays;
|
||||
|
Loading…
x
Reference in New Issue
Block a user