From 00be9e58f27a025adbb3c28e692e5e8032271666 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 5 Feb 2016 09:43:33 +0100 Subject: [PATCH 01/19] Simplify IndicesFieldDataCache and detach from guice Indices level field data cacheing belongs into IndicesService and doesn't need to be wired by guice. This commit also moves the async cache refresh out of the class into IndicesService such that threadpool dependencies are removed and testing / creation becomes simpler. --- .../common/settings/ClusterSettings.java | 3 +- .../org/elasticsearch/index/IndexModule.java | 5 +- .../org/elasticsearch/index/IndexService.java | 4 +- .../index/NodeServicesProvider.java | 9 +- .../index/fielddata/IndexFieldDataCache.java | 4 +- .../elasticsearch/indices/IndicesModule.java | 5 -- .../elasticsearch/indices/IndicesService.java | 86 ++++++++++++++++++- .../cache/IndicesFieldDataCache.java | 61 +------------ .../java/org/elasticsearch/node/Node.java | 1 - .../elasticsearch/index/IndexModuleTests.java | 31 ++++--- .../fielddata/AbstractFieldDataTestCase.java | 3 +- .../fielddata/IndexFieldDataServiceTests.java | 4 +- .../index/query/AbstractQueryTestCase.java | 6 +- .../test/InternalTestCluster.java | 2 +- 14 files changed, 125 insertions(+), 99 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index a6babce961d..e4b92d82d20 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -60,6 +60,7 @@ import org.elasticsearch.gateway.PrimaryShardAllocator; import org.elasticsearch.http.HttpTransportSettings; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.store.IndexStoreConfig; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.analysis.HunspellService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; @@ -290,7 +291,7 @@ public final class ClusterSettings extends AbstractScopedSettings { ScriptService.SCRIPT_CACHE_SIZE_SETTING, ScriptService.SCRIPT_CACHE_EXPIRE_SETTING, ScriptService.SCRIPT_AUTO_RELOAD_ENABLED_SETTING, - IndicesFieldDataCache.INDICES_FIELDDATA_CLEAN_INTERVAL_SETTING, + IndicesService.INDICES_FIELDDATA_CLEAN_INTERVAL_SETTING, IndicesFieldDataCache.INDICES_FIELDDATA_CACHE_SIZE_KEY, IndicesRequestCache.INDICES_CACHE_QUERY_SIZE, IndicesRequestCache.INDICES_CACHE_QUERY_EXPIRE, diff --git a/core/src/main/java/org/elasticsearch/index/IndexModule.java b/core/src/main/java/org/elasticsearch/index/IndexModule.java index 4688fba5034..fada5f42830 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/core/src/main/java/org/elasticsearch/index/IndexModule.java @@ -38,6 +38,7 @@ import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStoreConfig; import org.elasticsearch.indices.cache.query.IndicesQueryCache; +import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.mapper.MapperRegistry; import java.io.IOException; @@ -240,7 +241,7 @@ public final class IndexModule { IndexSearcherWrapper newWrapper(final IndexService indexService); } - public IndexService newIndexService(NodeEnvironment environment, IndexService.ShardStoreDeleter shardStoreDeleter, NodeServicesProvider servicesProvider, MapperRegistry mapperRegistry, + public IndexService newIndexService(NodeEnvironment environment, IndexService.ShardStoreDeleter shardStoreDeleter, NodeServicesProvider servicesProvider, MapperRegistry mapperRegistry, IndicesFieldDataCache indicesFieldDataCache, IndexingOperationListener... listeners) throws IOException { IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get(); IndexEventListener eventListener = freeze(); @@ -264,7 +265,7 @@ public final class IndexModule { final BiFunction queryCacheProvider = queryCaches.get(queryCacheType); final QueryCache queryCache = queryCacheProvider.apply(indexSettings, servicesProvider.getIndicesQueryCache()); return new IndexService(indexSettings, environment, new SimilarityService(indexSettings, similarities), shardStoreDeleter, analysisRegistry, engineFactory.get(), - servicesProvider, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, listeners); + servicesProvider, queryCache, store, eventListener, searcherWrapperFactory, mapperRegistry, indicesFieldDataCache, listeners); } } diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 8c87b2b5606..af800caf1c8 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -76,6 +76,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.AliasFilterParsingException; import org.elasticsearch.indices.InvalidAliasNameException; +import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.threadpool.ThreadPool; @@ -122,13 +123,14 @@ public final class IndexService extends AbstractIndexComponent implements IndexC IndexEventListener eventListener, IndexModule.IndexSearcherWrapperFactory wrapperFactory, MapperRegistry mapperRegistry, + IndicesFieldDataCache indicesFieldDataCache, IndexingOperationListener... listenersIn) throws IOException { super(indexSettings); this.indexSettings = indexSettings; this.analysisService = registry.build(indexSettings); this.similarityService = similarityService; this.mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, IndexService.this::getQueryShardContext); - this.indexFieldData = new IndexFieldDataService(indexSettings, nodeServicesProvider.getIndicesFieldDataCache(), nodeServicesProvider.getCircuitBreakerService(), mapperService); + this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, nodeServicesProvider.getCircuitBreakerService(), mapperService); this.shardStoreDeleter = shardStoreDeleter; this.eventListener = eventListener; this.nodeEnv = nodeEnv; diff --git a/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java b/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java index 36d3fdc5dce..d2b4894a5e6 100644 --- a/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java +++ b/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; -import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.ScriptService; import org.elasticsearch.threadpool.ThreadPool; @@ -45,11 +44,10 @@ public final class NodeServicesProvider { private final Client client; private final IndicesQueriesRegistry indicesQueriesRegistry; private final ScriptService scriptService; - private final IndicesFieldDataCache indicesFieldDataCache; private final CircuitBreakerService circuitBreakerService; @Inject - public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, IndicesFieldDataCache indicesFieldDataCache, CircuitBreakerService circuitBreakerService) { + public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, CircuitBreakerService circuitBreakerService) { this.threadPool = threadPool; this.indicesQueryCache = indicesQueryCache; this.warmer = warmer; @@ -57,7 +55,6 @@ public final class NodeServicesProvider { this.client = client; this.indicesQueriesRegistry = indicesQueriesRegistry; this.scriptService = scriptService; - this.indicesFieldDataCache = indicesFieldDataCache; this.circuitBreakerService = circuitBreakerService; } @@ -87,10 +84,6 @@ public final class NodeServicesProvider { return scriptService; } - public IndicesFieldDataCache getIndicesFieldDataCache() { - return indicesFieldDataCache; - } - public CircuitBreakerService getCircuitBreakerService() { return circuitBreakerService; } diff --git a/core/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataCache.java b/core/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataCache.java index 7640a9be200..68c05aba3fe 100644 --- a/core/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataCache.java +++ b/core/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataCache.java @@ -48,12 +48,12 @@ public interface IndexFieldDataCache { /** * Called after the fielddata is loaded during the cache phase */ - void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage); + default void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage){} /** * Called after the fielddata is unloaded */ - void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes); + default void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes){} } class None implements IndexFieldDataCache { diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java index 256aa72a105..e66cd3e721c 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -55,12 +55,9 @@ import org.elasticsearch.index.mapper.internal.VersionFieldMapper; import org.elasticsearch.index.mapper.ip.IpFieldMapper; import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.index.percolator.PercolatorFieldMapper; -import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.request.IndicesRequestCache; import org.elasticsearch.indices.cluster.IndicesClusterStateService; -import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; -import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener; import org.elasticsearch.indices.flush.SyncedFlushService; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.indices.recovery.RecoverySettings; @@ -165,13 +162,11 @@ public class IndicesModule extends AbstractModule { bind(SyncedFlushService.class).asEagerSingleton(); bind(IndicesQueryCache.class).asEagerSingleton(); bind(IndicesRequestCache.class).asEagerSingleton(); - bind(IndicesFieldDataCache.class).asEagerSingleton(); bind(TransportNodesListShardStoreMetaData.class).asEagerSingleton(); bind(IndicesTTLService.class).asEagerSingleton(); bind(IndicesWarmer.class).asEagerSingleton(); bind(UpdateHelper.class).asEagerSingleton(); bind(MetaDataIndexUpgradeService.class).asEagerSingleton(); - bind(IndicesFieldDataCacheListener.class).asEagerSingleton(); bind(NodeServicesProvider.class).asEagerSingleton(); } diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index bb61fed4362..e3685e47076 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices; import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.util.Accountable; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; @@ -33,11 +34,15 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.FileSystemUtils; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -51,6 +56,8 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.NodeServicesProvider; import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.merge.MergeStats; @@ -63,6 +70,8 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexingStats; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.IndexStoreConfig; +import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.plugins.PluginsService; @@ -81,6 +90,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; import static java.util.Collections.emptyMap; @@ -94,6 +104,7 @@ import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList; public class IndicesService extends AbstractLifecycleComponent implements Iterable, IndexService.ShardStoreDeleter { public static final String INDICES_SHARDS_CLOSED_TIMEOUT = "indices.shards_closed_timeout"; + public static final Setting INDICES_FIELDDATA_CLEAN_INTERVAL_SETTING = Setting.positiveTimeSetting("indices.fielddata.cache.cleanup_interval", TimeValue.timeValueMinutes(1), false, Setting.Scope.CLUSTER); private final PluginsService pluginsService; private final NodeEnvironment nodeEnv; private final TimeValue shardsClosedTimeout; @@ -102,23 +113,30 @@ public class IndicesService extends AbstractLifecycleComponent i private final ClusterService clusterService; private final IndexNameExpressionResolver indexNameExpressionResolver; private final IndexScopedSettings indexScopeSetting; + private final IndicesFieldDataCache indicesFieldDataCache; + private final FieldDataCacheCleaner fieldDataCacheCleaner; + private final ThreadPool threadPool; private volatile Map indices = emptyMap(); private final Map> pendingDeletes = new HashMap<>(); private final OldShardsStats oldShardsStats = new OldShardsStats(); private final IndexStoreConfig indexStoreConfig; private final MapperRegistry mapperRegistry; private final IndexingMemoryController indexingMemoryController; + private final TimeValue cleanInterval; @Override protected void doStart() { + // Start thread that will manage cleaning the field data cache periodically + threadPool.schedule(this.cleanInterval, ThreadPool.Names.SAME, this.fieldDataCacheCleaner); } @Inject public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry, IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver, - ClusterService clusterService, MapperRegistry mapperRegistry, ThreadPool threadPool, IndexScopedSettings indexScopedSettings) { + ClusterService clusterService, MapperRegistry mapperRegistry, ThreadPool threadPool, IndexScopedSettings indexScopedSettings, CircuitBreakerService circuitBreakerService) { super(settings); + this.threadPool = threadPool; this.pluginsService = pluginsService; this.nodeEnv = nodeEnv; this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS)); @@ -132,6 +150,17 @@ public class IndicesService extends AbstractLifecycleComponent i clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, indexStoreConfig::setRateLimitingThrottle); indexingMemoryController = new IndexingMemoryController(settings, threadPool, this); this.indexScopeSetting = indexScopedSettings; + this.indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() { + @Override + public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) { + assert sizeInBytes >= 0 : "When reducing circuit breaker, it should be adjusted with a number higher or equal to 0 and not [" + sizeInBytes + "]"; + circuitBreakerService.getBreaker(CircuitBreaker.FIELDDATA).addWithoutBreaking(-sizeInBytes); + } + }); + this.cleanInterval = INDICES_FIELDDATA_CLEAN_INTERVAL_SETTING.get(settings); + this.fieldDataCacheCleaner = new FieldDataCacheCleaner(indicesFieldDataCache, logger, threadPool, this.cleanInterval); + + } @Override @@ -165,7 +194,7 @@ public class IndicesService extends AbstractLifecycleComponent i @Override protected void doClose() { - IOUtils.closeWhileHandlingException(analysisRegistry, indexingMemoryController); + IOUtils.closeWhileHandlingException(analysisRegistry, indexingMemoryController, indicesFieldDataCache, fieldDataCacheCleaner); } /** @@ -320,7 +349,7 @@ public class IndicesService extends AbstractLifecycleComponent i indexModule.addIndexEventListener(oldShardsStats); final IndexEventListener listener = indexModule.freeze(); listener.beforeIndexCreated(index, idxSettings.getSettings()); - final IndexService indexService = indexModule.newIndexService(nodeEnv, this, nodeServicesProvider, mapperRegistry, indexingMemoryController); + final IndexService indexService = indexModule.newIndexService(nodeEnv, this, nodeServicesProvider, mapperRegistry, indicesFieldDataCache, indexingMemoryController); boolean success = false; try { assert indexService.getIndexEventListener() == listener; @@ -381,6 +410,10 @@ public class IndicesService extends AbstractLifecycleComponent i } } + public IndicesFieldDataCache getIndicesFieldDataCache() { + return indicesFieldDataCache; + } + static class OldShardsStats implements IndexEventListener { final SearchStats searchStats = new SearchStats(); @@ -760,4 +793,51 @@ public class IndicesService extends AbstractLifecycleComponent i public AnalysisRegistry getAnalysis() { return analysisRegistry; } + + /** + * FieldDataCacheCleaner is a scheduled Runnable used to clean a Guava cache + * periodically. In this case it is the field data cache, because a cache that + * has an entry invalidated may not clean up the entry if it is not read from + * or written to after invalidation. + */ + private final static class FieldDataCacheCleaner implements Runnable, Releasable { + + private final IndicesFieldDataCache cache; + private final ESLogger logger; + private final ThreadPool threadPool; + private final TimeValue interval; + private final AtomicBoolean closed = new AtomicBoolean(false); + + public FieldDataCacheCleaner(IndicesFieldDataCache cache, ESLogger logger, ThreadPool threadPool, TimeValue interval) { + this.cache = cache; + this.logger = logger; + this.threadPool = threadPool; + this.interval = interval; + } + + @Override + public void run() { + long startTimeNS = System.nanoTime(); + if (logger.isTraceEnabled()) { + logger.trace("running periodic field data cache cleanup"); + } + try { + this.cache.getCache().refresh(); + } catch (Exception e) { + logger.warn("Exception during periodic field data cache cleanup:", e); + } + if (logger.isTraceEnabled()) { + logger.trace("periodic field data cache cleanup finished in {} milliseconds", TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)); + } + // Reschedule itself to run again if not closed + if (closed.get() == false) { + threadPool.schedule(interval, ThreadPool.Names.SAME, this); + } + } + + @Override + public void close() { + closed.compareAndSet(false, true); + } + } } diff --git a/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java b/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java index 144f8b7f775..0a3f063dfcc 100644 --- a/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java +++ b/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java @@ -30,14 +30,12 @@ import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.cache.RemovalListener; import org.elasticsearch.common.cache.RemovalNotification; import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; import org.elasticsearch.index.fielddata.AtomicFieldData; import org.elasticsearch.index.fielddata.FieldDataType; @@ -45,7 +43,6 @@ import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; -import org.elasticsearch.threadpool.ThreadPool; import java.util.ArrayList; import java.util.List; @@ -55,20 +52,12 @@ import java.util.function.ToLongBiFunction; */ public class IndicesFieldDataCache extends AbstractComponent implements RemovalListener, Releasable{ - public static final Setting INDICES_FIELDDATA_CLEAN_INTERVAL_SETTING = Setting.positiveTimeSetting("indices.fielddata.cache.cleanup_interval", TimeValue.timeValueMinutes(1), false, Setting.Scope.CLUSTER); public static final Setting INDICES_FIELDDATA_CACHE_SIZE_KEY = Setting.byteSizeSetting("indices.fielddata.cache.size", new ByteSizeValue(-1), false, Setting.Scope.CLUSTER); - - - private final IndicesFieldDataCacheListener indicesFieldDataCacheListener; + private final IndexFieldDataCache.Listener indicesFieldDataCacheListener; private final Cache cache; - private final TimeValue cleanInterval; - private final ThreadPool threadPool; - private volatile boolean closed = false; - @Inject - public IndicesFieldDataCache(Settings settings, IndicesFieldDataCacheListener indicesFieldDataCacheListener, ThreadPool threadPool) { + public IndicesFieldDataCache(Settings settings, IndexFieldDataCache.Listener indicesFieldDataCacheListener) { super(settings); - this.threadPool = threadPool; this.indicesFieldDataCacheListener = indicesFieldDataCacheListener; final long sizeInBytes = INDICES_FIELDDATA_CACHE_SIZE_KEY.get(settings).bytes(); CacheBuilder cacheBuilder = CacheBuilder.builder() @@ -76,19 +65,12 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL if (sizeInBytes > 0) { cacheBuilder.setMaximumWeight(sizeInBytes).weigher(new FieldDataWeigher()); } - cache = cacheBuilder.build(); - - this.cleanInterval = INDICES_FIELDDATA_CLEAN_INTERVAL_SETTING.get(settings); - // Start thread that will manage cleaning the field data cache periodically - threadPool.schedule(this.cleanInterval, ThreadPool.Names.SAME, - new FieldDataCacheCleaner(this.cache, this.logger, this.threadPool, this.cleanInterval)); } @Override public void close() { cache.invalidateAll(); - this.closed = true; } public IndexFieldDataCache buildIndexFieldDataCache(IndexFieldDataCache.Listener listener, Index index, String fieldName, FieldDataType fieldDataType) { @@ -260,44 +242,5 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL } } - /** - * FieldDataCacheCleaner is a scheduled Runnable used to clean a Guava cache - * periodically. In this case it is the field data cache, because a cache that - * has an entry invalidated may not clean up the entry if it is not read from - * or written to after invalidation. - */ - public class FieldDataCacheCleaner implements Runnable { - private final Cache cache; - private final ESLogger logger; - private final ThreadPool threadPool; - private final TimeValue interval; - - public FieldDataCacheCleaner(Cache cache, ESLogger logger, ThreadPool threadPool, TimeValue interval) { - this.cache = cache; - this.logger = logger; - this.threadPool = threadPool; - this.interval = interval; - } - - @Override - public void run() { - long startTimeNS = System.nanoTime(); - if (logger.isTraceEnabled()) { - logger.trace("running periodic field data cache cleanup"); - } - try { - this.cache.refresh(); - } catch (Exception e) { - logger.warn("Exception during periodic field data cache cleanup:", e); - } - if (logger.isTraceEnabled()) { - logger.trace("periodic field data cache cleanup finished in {} milliseconds", TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)); - } - // Reschedule itself to run again if not closed - if (closed == false) { - threadPool.schedule(interval, ThreadPool.Names.SAME, this); - } - } - } } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index db942ffd284..88d21764abb 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -392,7 +392,6 @@ public class Node implements Closeable { toClose.add(injector.getInstance(IndicesService.class)); // close filter/fielddata caches after indices toClose.add(injector.getInstance(IndicesQueryCache.class)); - toClose.add(injector.getInstance(IndicesFieldDataCache.class)); toClose.add(injector.getInstance(IndicesStore.class)); toClose.add(() ->stopWatch.stop().start("routing")); toClose.add(injector.getInstance(RoutingService.class)); diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 4fdbf651a6f..f2245cc7de0 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -44,6 +44,7 @@ import org.elasticsearch.index.cache.query.QueryCache; import org.elasticsearch.index.cache.query.index.IndexQueryCache; import org.elasticsearch.index.cache.query.none.NoneQueryCache; import org.elasticsearch.index.engine.EngineException; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.ShardId; @@ -96,6 +97,8 @@ public class IndexModuleTests extends ESTestCase { public void addPendingDelete(ShardId shardId, IndexSettings indexSettings) { } }; + + private final IndexFieldDataCache.Listener listener = new IndexFieldDataCache.Listener() {}; private MapperRegistry mapperRegistry; static NodeServicesProvider newNodeServiceProvider(Settings settings, Environment environment, Client client, ScriptEngineService... scriptEngineServices) throws IOException { @@ -106,7 +109,6 @@ public class IndexModuleTests extends ESTestCase { CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService(); PageCacheRecycler recycler = new PageCacheRecycler(settings, threadPool); BigArrays bigArrays = new BigArrays(recycler, circuitBreakerService); - IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndicesFieldDataCacheListener(circuitBreakerService), threadPool); Set scriptEngines = Collections.emptySet(); scriptEngines.addAll(Arrays.asList(scriptEngineServices)); ScriptEngineRegistry scriptEngineRegistry = new ScriptEngineRegistry(Collections.emptyList()); @@ -114,7 +116,7 @@ public class IndexModuleTests extends ESTestCase { ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry, scriptContextRegistry, scriptSettings); IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, emptyMap()); - return new NodeServicesProvider(threadPool, indicesQueryCache, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService); + return new NodeServicesProvider(threadPool, indicesQueryCache, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, circuitBreakerService); } @Override @@ -143,7 +145,7 @@ public class IndexModuleTests extends ESTestCase { IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); module.setSearcherWrapper((s) -> new Wrapper()); module.engineFactory.set(new MockEngineFactory(AssertingDirectoryReader.class)); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, new IndicesFieldDataCache(settings, listener)); assertTrue(indexService.getSearcherWrapper() instanceof Wrapper); assertSame(indexService.getEngineFactory(), module.engineFactory.get()); indexService.close("simon says", false); @@ -161,7 +163,7 @@ public class IndexModuleTests extends ESTestCase { final Index index = indexSettings.getIndex(); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); module.addIndexStore("foo_store", FooStore::new); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, new IndicesFieldDataCache(settings, listener)); assertTrue(indexService.getIndexStore() instanceof FooStore); try { module.addIndexStore("foo_store", FooStore::new); @@ -184,7 +186,8 @@ public class IndexModuleTests extends ESTestCase { IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment)); Consumer listener = (s) -> {}; module.addIndexEventListener(eventListener); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, this.listener)); IndexSettings x = indexService.getIndexSettings(); assertEquals(x.getSettings().getAsMap(), indexSettings.getSettings().getAsMap()); assertEquals(x.getIndex(), index); @@ -208,7 +211,8 @@ public class IndexModuleTests extends ESTestCase { } - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, listener)); assertSame(booleanSetting, indexService.getIndexSettings().getScopedSettings().get(booleanSetting.getKey())); indexService.close("simon says", false); @@ -234,7 +238,8 @@ public class IndexModuleTests extends ESTestCase { } }); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, listener)); SimilarityService similarityService = indexService.similarityService(); assertNotNull(similarityService.getSimilarity("my_similarity")); assertTrue(similarityService.getSimilarity("my_similarity").get() instanceof TestSimilarity); @@ -251,7 +256,8 @@ public class IndexModuleTests extends ESTestCase { .build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), null, new AnalysisRegistry(null, environment)); try { - module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, listener)); } catch (IllegalArgumentException ex) { assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage()); } @@ -265,7 +271,8 @@ public class IndexModuleTests extends ESTestCase { .build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), null, new AnalysisRegistry(null, environment)); try { - module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, listener)); } catch (IllegalArgumentException ex) { assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage()); } @@ -312,7 +319,8 @@ public class IndexModuleTests extends ESTestCase { assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]"); } - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, listener)); assertTrue(indexService.cache().query() instanceof CustomQueryCache); indexService.close("simon says", false); } @@ -322,7 +330,8 @@ public class IndexModuleTests extends ESTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings("foo", indexSettings), null, new AnalysisRegistry(null, environment)); - IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry); + IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider, mapperRegistry, + new IndicesFieldDataCache(settings, listener)); assertTrue(indexService.cache().query() instanceof IndexQueryCache); indexService.close("simon says", false); } diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java b/core/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java index 0bdbfb58722..6b3f0ddbd3f 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/AbstractFieldDataTestCase.java @@ -48,6 +48,7 @@ import org.elasticsearch.index.mapper.geo.GeoPointFieldMapper; import org.elasticsearch.index.mapper.geo.GeoPointFieldMapperLegacy; import org.elasticsearch.index.mapper.internal.ParentFieldMapper; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -132,7 +133,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); indexService = createIndex("test", settings); mapperService = indexService.mapperService(); - indicesFieldDataCache = getInstanceFromNode(IndicesFieldDataCache.class); + indicesFieldDataCache = getInstanceFromNode(IndicesService.class).getIndicesFieldDataCache(); ifdService = indexService.fieldData(); // LogByteSizeMP to preserve doc ID order writer = new IndexWriter(new RAMDirectory(), new IndexWriterConfig(new StandardAnalyzer()).setMergePolicy(new LogByteSizeMergePolicy())); diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java index da2899bf2c3..3a98cac6134 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java @@ -205,7 +205,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase { private void doTestRequireDocValues(MappedFieldType ft) { ThreadPool threadPool = new ThreadPool("random_threadpool_name"); try { - IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null, threadPool); + IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null); IndexFieldDataService ifds = new IndexFieldDataService(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), cache, null, null); ft.setName("some_long"); ft.setHasDocValues(true); @@ -238,7 +238,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase { ThreadPool threadPool = new ThreadPool("random_threadpool_name"); StringFieldMapper.StringFieldType ft = new StringFieldMapper.StringFieldType(); try { - IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null, threadPool); + IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null); IndexFieldDataService ifds = new IndexFieldDataService(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), cache, null, null); ft.setName("some_str"); ft.setFieldDataType(new FieldDataType("string", Settings.builder().put(FieldDataType.FORMAT_KEY, "disabled").build())); diff --git a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java index d8d018284e9..e84e655fe2e 100644 --- a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java @@ -56,7 +56,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.ToXContent; @@ -71,6 +70,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.support.QueryParsers; @@ -271,7 +271,9 @@ public abstract class AbstractQueryTestCase> SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap()); MapperRegistry mapperRegistry = injector.getInstance(MapperRegistry.class); MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry, () -> queryShardContext); - indexFieldDataService = new IndexFieldDataService(idxSettings, injector.getInstance(IndicesFieldDataCache.class), injector.getInstance(CircuitBreakerService.class), mapperService); + IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() { + }); + indexFieldDataService = new IndexFieldDataService(idxSettings, indicesFieldDataCache, injector.getInstance(CircuitBreakerService.class), mapperService); BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 61a58c59b1e..cfc820460f8 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1819,7 +1819,7 @@ public final class InternalTestCluster extends TestCluster { // network request, because a network request can increment one // of the breakers for (NodeAndClient nodeAndClient : nodes.values()) { - final IndicesFieldDataCache fdCache = getInstanceFromNode(IndicesFieldDataCache.class, nodeAndClient.node); + final IndicesFieldDataCache fdCache = getInstanceFromNode(IndicesService.class, nodeAndClient.node).getIndicesFieldDataCache(); // Clean up the cache, ensuring that entries' listeners have been called fdCache.getCache().refresh(); From 9d07501c96f0c9963eec163ea8259631ec941e5e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 5 Feb 2016 09:54:55 +0100 Subject: [PATCH 02/19] fix test --- .../main/java/org/elasticsearch/indices/IndicesService.java | 6 ++++++ .../index/fielddata/IndexFieldDataServiceTests.java | 5 +++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index e3685e47076..876411964b4 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -116,6 +116,7 @@ public class IndicesService extends AbstractLifecycleComponent i private final IndicesFieldDataCache indicesFieldDataCache; private final FieldDataCacheCleaner fieldDataCacheCleaner; private final ThreadPool threadPool; + private final CircuitBreakerService circuitBreakerService; private volatile Map indices = emptyMap(); private final Map> pendingDeletes = new HashMap<>(); private final OldShardsStats oldShardsStats = new OldShardsStats(); @@ -150,6 +151,7 @@ public class IndicesService extends AbstractLifecycleComponent i clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, indexStoreConfig::setRateLimitingThrottle); indexingMemoryController = new IndexingMemoryController(settings, threadPool, this); this.indexScopeSetting = indexScopedSettings; + this.circuitBreakerService = circuitBreakerService; this.indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() { @Override public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) { @@ -414,6 +416,10 @@ public class IndicesService extends AbstractLifecycleComponent i return indicesFieldDataCache; } + public CircuitBreakerService getCircuitBreakerService() { + return circuitBreakerService; + } + static class OldShardsStats implements IndexEventListener { final SearchStats searchStats = new SearchStats(); diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java index 3a98cac6134..0a2a3c4d79c 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java @@ -51,6 +51,7 @@ import org.elasticsearch.index.mapper.core.LongFieldMapper; import org.elasticsearch.index.mapper.core.ShortFieldMapper; import org.elasticsearch.index.mapper.core.StringFieldMapper; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -131,10 +132,10 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase { public void testFieldDataCacheListener() throws Exception { final IndexService indexService = createIndex("test"); - IndexFieldDataService shardPrivateService = indexService.fieldData(); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); // copy the ifdService since we can set the listener only once. final IndexFieldDataService ifdService = new IndexFieldDataService(indexService.getIndexSettings(), - getInstanceFromNode(IndicesFieldDataCache.class), getInstanceFromNode(CircuitBreakerService.class), indexService.mapperService()); + indicesService.getIndicesFieldDataCache(), indicesService.getCircuitBreakerService(), indexService.mapperService()); final BuilderContext ctx = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1)); final MappedFieldType mapper1 = MapperBuilders.stringField("s").tokenized(false).docValues(true).fieldDataSettings(Settings.builder().put(FieldDataType.FORMAT_KEY, "paged_bytes").build()).build(ctx).fieldType(); From 7d0181b5d4111b4db9edc63aef86d2656726c407 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Thu, 4 Feb 2016 16:00:55 +0100 Subject: [PATCH 03/19] Rename bin/plugin in bin/elasticsearch-plugin --- .../gradle/test/ClusterFormationTasks.groovy | 2 +- dev-tools/create_bwc_index.py | 2 +- dev-tools/prepare_release_candidate.py | 10 ++--- dev-tools/smoke_test_rc.py | 2 +- distribution/build.gradle | 4 +- .../bin/{plugin => elasticsearch-plugin} | 0 .../{plugin.bat => elasticsearch-plugin.bat} | 0 docs/plugins/analysis-icu.asciidoc | 4 +- docs/plugins/analysis-kuromoji.asciidoc | 4 +- docs/plugins/analysis-phonetic.asciidoc | 4 +- docs/plugins/analysis-smartcn.asciidoc | 4 +- docs/plugins/analysis-stempel.asciidoc | 4 +- docs/plugins/authors.asciidoc | 2 +- docs/plugins/delete-by-query.asciidoc | 4 +- docs/plugins/discovery-azure.asciidoc | 6 +-- docs/plugins/discovery-ec2.asciidoc | 4 +- docs/plugins/discovery-gce.asciidoc | 6 +-- docs/plugins/lang-javascript.asciidoc | 4 +- docs/plugins/lang-python.asciidoc | 4 +- docs/plugins/mapper-attachments.asciidoc | 4 +- docs/plugins/mapper-murmur3.asciidoc | 4 +- docs/plugins/mapper-size.asciidoc | 4 +- docs/plugins/plugin-script.asciidoc | 40 +++++++++---------- docs/plugins/repository-azure.asciidoc | 4 +- docs/plugins/repository-hdfs.asciidoc | 4 +- docs/plugins/repository-s3.asciidoc | 4 +- docs/plugins/store-smb.asciidoc | 4 +- .../migration/migrate_2_0/packaging.asciidoc | 4 +- .../migration/migrate_2_0/removals.asciidoc | 8 ++-- .../migration/migrate_2_0/settings.asciidoc | 2 +- docs/reference/migration/migrate_3_0.asciidoc | 2 + .../packaging/scripts/plugin_test_cases.bash | 12 +++--- .../resources/packaging/scripts/plugins.bash | 4 +- .../test/resources/packaging/scripts/tar.bash | 2 +- 34 files changed, 87 insertions(+), 85 deletions(-) rename distribution/src/main/resources/bin/{plugin => elasticsearch-plugin} (100%) rename distribution/src/main/resources/bin/{plugin.bat => elasticsearch-plugin.bat} (100%) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 0d116a32712..9d205869881 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -340,7 +340,7 @@ class ClusterFormationTasks { } // delay reading the file location until execution time by wrapping in a closure within a GString String file = "${-> new File(node.pluginsTmpDir, pluginZip.singleFile.getName()).toURI().toURL().toString()}" - Object[] args = [new File(node.homeDir, 'bin/plugin'), 'install', file] + Object[] args = [new File(node.homeDir, 'bin/elasticsearch-plugin'), 'install', file] return configureExecTask(name, project, setup, node, args) } diff --git a/dev-tools/create_bwc_index.py b/dev-tools/create_bwc_index.py index af5945a418e..da3aa6c8f4a 100644 --- a/dev-tools/create_bwc_index.py +++ b/dev-tools/create_bwc_index.py @@ -156,7 +156,7 @@ def remove_plugin(version, release_dir, plugin_name): run_plugin(version, release_dir, 'remove', [plugin_name]) def run_plugin(version, release_dir, plugin_cmd, args): - cmd = [os.path.join(release_dir, 'bin/plugin'), plugin_cmd] + args + cmd = [os.path.join(release_dir, 'bin/elasticsearch-plugin'), plugin_cmd] + args subprocess.check_call(cmd) def create_client(http_port=DEFAULT_HTTP_TCP_PORT, timeout=30): diff --git a/dev-tools/prepare_release_candidate.py b/dev-tools/prepare_release_candidate.py index e0baa754d0c..544ec72d876 100644 --- a/dev-tools/prepare_release_candidate.py +++ b/dev-tools/prepare_release_candidate.py @@ -56,14 +56,14 @@ The packages may be downloaded from the following URLs: Plugins can be installed as follows: - bin/plugin -Des.plugins.staging=true install cloud-aws + bin/elasticsearch-plugin -Des.plugins.staging=true install cloud-aws The same goes for the x-plugins: - bin/plugin -Des.plugins.staging=true install license - bin/plugin -Des.plugins.staging=true install marvel-agent - bin/plugin -Des.plugins.staging=true install shield - bin/plugin -Des.plugins.staging=true install watcher + bin/elasticsearch-plugin -Des.plugins.staging=true install license + bin/elasticsearch-plugin -Des.plugins.staging=true install marvel-agent + bin/elasticsearch-plugin -Des.plugins.staging=true install shield + bin/elasticsearch-plugin -Des.plugins.staging=true install watcher To install the deb from an APT repo: diff --git a/dev-tools/smoke_test_rc.py b/dev-tools/smoke_test_rc.py index f32a6b80f52..0ad2d4075ec 100644 --- a/dev-tools/smoke_test_rc.py +++ b/dev-tools/smoke_test_rc.py @@ -178,7 +178,7 @@ def smoke_test_release(release, files, expected_hash, plugins): continue # nothing to do here es_run_path = os.path.join(tmp_dir, 'elasticsearch-%s' % (release), 'bin/elasticsearch') print(' Smoke testing package [%s]' % release_file) - es_plugin_path = os.path.join(tmp_dir, 'elasticsearch-%s' % (release), 'bin/plugin') + es_plugin_path = os.path.join(tmp_dir, 'elasticsearch-%s' % (release), 'bin/elasticsearch-plugin') plugin_names = {} for plugin in plugins: print(' Install plugin [%s]' % (plugin)) diff --git a/distribution/build.gradle b/distribution/build.gradle index fcf22ff48dc..dffb4ef12b2 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -372,7 +372,7 @@ task run(type: RunTask) {} /** * Build some variables that are replaced in the packages. This includes both - * scripts like bin/elasticsearch and bin/plugin that a user might run and also + * scripts like bin/elasticsearch and bin/elasticsearch-plugin that a user might run and also * scripts like postinst which are run as part of the installation. * *
@@ -384,7 +384,7 @@ task run(type: RunTask) {} *
path.conf
*
The default directory from which to load configuration. This is used in * the packaging scripts, but in that context it is always - * /etc/elasticsearch. Its also used in bin/plugin, where it is + * /etc/elasticsearch. Its also used in bin/elasticsearch-plugin, where it is * /etc/elasticsearch for the os packages but $ESHOME/config otherwise.
*
path.env
*
The env file sourced before bin/elasticsearch to set environment diff --git a/distribution/src/main/resources/bin/plugin b/distribution/src/main/resources/bin/elasticsearch-plugin similarity index 100% rename from distribution/src/main/resources/bin/plugin rename to distribution/src/main/resources/bin/elasticsearch-plugin diff --git a/distribution/src/main/resources/bin/plugin.bat b/distribution/src/main/resources/bin/elasticsearch-plugin.bat similarity index 100% rename from distribution/src/main/resources/bin/plugin.bat rename to distribution/src/main/resources/bin/elasticsearch-plugin.bat diff --git a/docs/plugins/analysis-icu.asciidoc b/docs/plugins/analysis-icu.asciidoc index dcd73fe48d1..7973c8fbc1a 100644 --- a/docs/plugins/analysis-icu.asciidoc +++ b/docs/plugins/analysis-icu.asciidoc @@ -15,7 +15,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install analysis-icu +sudo bin/elasticsearch-plugin install analysis-icu ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -29,7 +29,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove analysis-icu +sudo bin/elasticsearch-plugin remove analysis-icu ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/analysis-kuromoji.asciidoc b/docs/plugins/analysis-kuromoji.asciidoc index 3575457aacb..8d1d6ac2d98 100644 --- a/docs/plugins/analysis-kuromoji.asciidoc +++ b/docs/plugins/analysis-kuromoji.asciidoc @@ -12,7 +12,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install analysis-kuromoji +sudo bin/elasticsearch-plugin install analysis-kuromoji ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -26,7 +26,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove analysis-kuromoji +sudo bin/elasticsearch-plugin remove analysis-kuromoji ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/analysis-phonetic.asciidoc b/docs/plugins/analysis-phonetic.asciidoc index 29fe9b4c7d1..f4a9684519d 100644 --- a/docs/plugins/analysis-phonetic.asciidoc +++ b/docs/plugins/analysis-phonetic.asciidoc @@ -13,7 +13,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install analysis-phonetic +sudo bin/elasticsearch-plugin install analysis-phonetic ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -27,7 +27,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove analysis-phonetic +sudo bin/elasticsearch-plugin remove analysis-phonetic ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/analysis-smartcn.asciidoc b/docs/plugins/analysis-smartcn.asciidoc index 33c755b40b2..fb1eebb4518 100644 --- a/docs/plugins/analysis-smartcn.asciidoc +++ b/docs/plugins/analysis-smartcn.asciidoc @@ -18,7 +18,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install analysis-smartcn +sudo bin/elasticsearch-plugin install analysis-smartcn ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -32,7 +32,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove analysis-smartcn +sudo bin/elasticsearch-plugin remove analysis-smartcn ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/analysis-stempel.asciidoc b/docs/plugins/analysis-stempel.asciidoc index a0b5c26a4db..26aad5248ec 100644 --- a/docs/plugins/analysis-stempel.asciidoc +++ b/docs/plugins/analysis-stempel.asciidoc @@ -15,7 +15,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install analysis-stempel +sudo bin/elasticsearch-plugin install analysis-stempel ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -29,7 +29,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove analysis-stempel +sudo bin/elasticsearch-plugin remove analysis-stempel ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/authors.asciidoc b/docs/plugins/authors.asciidoc index 6f63eab83ca..7d5ce489c17 100644 --- a/docs/plugins/authors.asciidoc +++ b/docs/plugins/authors.asciidoc @@ -68,7 +68,7 @@ in the presence of plugins with the incorrect `elasticsearch.version`. === Testing your plugin When testing a Java plugin, it will only be auto-loaded if it is in the -`plugins/` directory. Use `bin/plugin install file:///path/to/your/plugin` +`plugins/` directory. Use `bin/elasticsearch-plugin install file:///path/to/your/plugin` to install your plugin for testing. You may also load your plugin within the test framework for integration tests. diff --git a/docs/plugins/delete-by-query.asciidoc b/docs/plugins/delete-by-query.asciidoc index a207ae560e5..643dbfed8b3 100644 --- a/docs/plugins/delete-by-query.asciidoc +++ b/docs/plugins/delete-by-query.asciidoc @@ -30,7 +30,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install delete-by-query +sudo bin/elasticsearch-plugin install delete-by-query ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -43,7 +43,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove delete-by-query +sudo bin/elasticsearch-plugin remove delete-by-query ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/discovery-azure.asciidoc b/docs/plugins/discovery-azure.asciidoc index cedfca70f9d..e3bdc83e470 100644 --- a/docs/plugins/discovery-azure.asciidoc +++ b/docs/plugins/discovery-azure.asciidoc @@ -11,7 +11,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install discovery-azure +sudo bin/elasticsearch-plugin install discovery-azure ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -25,7 +25,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove discovery-azure +sudo bin/elasticsearch-plugin remove discovery-azure ---------------------------------------------------------------- The node must be stopped before removing the plugin. @@ -403,7 +403,7 @@ This command should give you a JSON result: sudo service elasticsearch stop # Install the plugin -sudo /usr/share/elasticsearch/bin/plugin install discovery-azure +sudo /usr/share/elasticsearch/bin/elasticsearch-plugin install discovery-azure # Configure it sudo vi /etc/elasticsearch/elasticsearch.yml diff --git a/docs/plugins/discovery-ec2.asciidoc b/docs/plugins/discovery-ec2.asciidoc index 27d6d08a4ec..2ca0b8d9314 100644 --- a/docs/plugins/discovery-ec2.asciidoc +++ b/docs/plugins/discovery-ec2.asciidoc @@ -11,7 +11,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install discovery-ec2 +sudo bin/elasticsearch-plugin install discovery-ec2 ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -25,7 +25,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove discovery-ec2 +sudo bin/elasticsearch-plugin remove discovery-ec2 ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/discovery-gce.asciidoc b/docs/plugins/discovery-gce.asciidoc index b9bac91f423..b455fc689db 100644 --- a/docs/plugins/discovery-gce.asciidoc +++ b/docs/plugins/discovery-gce.asciidoc @@ -11,7 +11,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install discovery-gce +sudo bin/elasticsearch-plugin install discovery-gce ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -25,7 +25,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove discovery-gce +sudo bin/elasticsearch-plugin remove discovery-gce ---------------------------------------------------------------- The node must be stopped before removing the plugin. @@ -216,7 +216,7 @@ Install the plugin: [source,sh] -------------------------------------------------- # Use Plugin Manager to install it -sudo bin/plugin install discovery-gce +sudo bin/elasticsearch-plugin install discovery-gce -------------------------------------------------- Open the `elasticsearch.yml` file: diff --git a/docs/plugins/lang-javascript.asciidoc b/docs/plugins/lang-javascript.asciidoc index fb699d1202b..b3e811209b2 100644 --- a/docs/plugins/lang-javascript.asciidoc +++ b/docs/plugins/lang-javascript.asciidoc @@ -13,7 +13,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install lang-javascript +sudo bin/elasticsearch-plugin install lang-javascript ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -27,7 +27,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove lang-javascript +sudo bin/elasticsearch-plugin remove lang-javascript ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/lang-python.asciidoc b/docs/plugins/lang-python.asciidoc index 956a4b7ed34..7c5595254ee 100644 --- a/docs/plugins/lang-python.asciidoc +++ b/docs/plugins/lang-python.asciidoc @@ -12,7 +12,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install lang-python +sudo bin/elasticsearch-plugin install lang-python ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -26,7 +26,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove lang-python +sudo bin/elasticsearch-plugin remove lang-python ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/mapper-attachments.asciidoc b/docs/plugins/mapper-attachments.asciidoc index c13d8ee0b4e..2a53a1a4a99 100644 --- a/docs/plugins/mapper-attachments.asciidoc +++ b/docs/plugins/mapper-attachments.asciidoc @@ -15,7 +15,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install mapper-attachments +sudo bin/elasticsearch-plugin install mapper-attachments ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -29,7 +29,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove mapper-attachments +sudo bin/elasticsearch-plugin remove mapper-attachments ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/mapper-murmur3.asciidoc b/docs/plugins/mapper-murmur3.asciidoc index 2889eb4aa68..3bb04454cd1 100644 --- a/docs/plugins/mapper-murmur3.asciidoc +++ b/docs/plugins/mapper-murmur3.asciidoc @@ -13,7 +13,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install mapper-murmur3 +sudo bin/elasticsearch-plugin install mapper-murmur3 ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -27,7 +27,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove mapper-murmur3 +sudo bin/elasticsearch-plugin remove mapper-murmur3 ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/mapper-size.asciidoc b/docs/plugins/mapper-size.asciidoc index 51f0c3972d6..3213d502792 100644 --- a/docs/plugins/mapper-size.asciidoc +++ b/docs/plugins/mapper-size.asciidoc @@ -13,7 +13,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install mapper-size +sudo bin/elasticsearch-plugin install mapper-size ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -27,7 +27,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove mapper-size +sudo bin/elasticsearch-plugin remove mapper-size ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/plugin-script.asciidoc b/docs/plugins/plugin-script.asciidoc index 58ba1e53d71..d2e57b2efc8 100644 --- a/docs/plugins/plugin-script.asciidoc +++ b/docs/plugins/plugin-script.asciidoc @@ -10,15 +10,15 @@ Run the following command to get usage instructions: [source,shell] ----------------------------------- -sudo bin/plugin -h +sudo bin/elasticsearch-plugin -h ----------------------------------- [IMPORTANT] .Running as root ===================== If Elasticsearch was installed using the deb or rpm package then run -`bin/plugin` as `root` so it can write to the appropriate files on disk. -Otherwise run `bin/plugin` as the user that owns all of the Elasticsearch +`bin/elasticsearch-plugin` as `root` so it can write to the appropriate files on disk. +Otherwise run `bin/elasticsearch-plugin` as the user that owns all of the Elasticsearch files. ===================== @@ -36,7 +36,7 @@ Core Elasticsearch plugins can be installed as follows: [source,shell] ----------------------------------- -sudo bin/plugin install [plugin_name] +sudo bin/elasticsearch-plugin install [plugin_name] ----------------------------------- For instance, to install the core <>, just run the @@ -44,7 +44,7 @@ following command: [source,shell] ----------------------------------- -sudo bin/plugin install analysis-icu +sudo bin/elasticsearch-plugin install analysis-icu ----------------------------------- This command will install the version of the plugin that matches your @@ -59,7 +59,7 @@ and Sonatype), or from GitHub. In this case, the command is as follows: [source,shell] ----------------------------------- -sudo bin/plugin install [org]/[user|component]/[version] +sudo bin/elasticsearch-plugin install [org]/[user|component]/[version] ----------------------------------- For instance, to install the https://github.com/lmenezes/elasticsearch-kopf[Kopf] @@ -67,8 +67,8 @@ plugin from GitHub, run one of the following commands: [source,shell] ----------------------------------- -sudo bin/plugin install lmenezes/elasticsearch-kopf <1> -sudo bin/plugin install lmenezes/elasticsearch-kopf/2.x <2> +sudo bin/elasticsearch-plugin install lmenezes/elasticsearch-kopf <1> +sudo bin/elasticsearch-plugin install lmenezes/elasticsearch-kopf/2.x <2> ----------------------------------- <1> Installs the latest version from GitHub. <2> Installs the 1.x version from GitHub. @@ -80,7 +80,7 @@ plugin from Sonatype, run: [source,shell] ----------------------------------- -sudo bin/plugin install org.elasticsearch.plugin/mapper-attachments/3.0.0 <1> +sudo bin/elasticsearch-plugin install org.elasticsearch.plugin/mapper-attachments/3.0.0 <1> ----------------------------------- <1> When installing from `download.elastic.co` or from Maven Central/Sonatype, the version is required. @@ -92,7 +92,7 @@ A plugin can also be downloaded directly from a custom location by specifying th [source,shell] ----------------------------------- -sudo bin/plugin install [url] <1> +sudo bin/elasticsearch-plugin install [url] <1> ----------------------------------- <1> must be a valid URL, the plugin name is determined from its descriptor. @@ -100,7 +100,7 @@ For instance, to install a plugin from your local file system, you could run: [source,shell] ----------------------------------- -sudo bin/plugin install file:///path/to/plugin.zip +sudo bin/elasticsearch-plugin install file:///path/to/plugin.zip ----------------------------------- The plugin script will refuse to talk to an HTTPS URL with an untrusted @@ -109,7 +109,7 @@ to a local Java truststore and pass the location to the script as follows: [source,shell] ----------------------------------- -sudo bin/plugin -Djavax.net.ssl.trustStore=/path/to/trustStore.jks install https://.... +sudo bin/elasticsearch-plugin -Djavax.net.ssl.trustStore=/path/to/trustStore.jks install https://.... ----------------------------------- [[listing-removing]] @@ -122,7 +122,7 @@ A list of the currently loaded plugins can be retrieved with the `list` option: [source,shell] ----------------------------------- -sudo bin/plugin list +sudo bin/elasticsearch-plugin list ----------------------------------- Alternatively, use the {ref}/cluster-nodes-info.html[node-info API] to find @@ -136,7 +136,7 @@ Plugins can be removed manually, by deleting the appropriate directory under [source,shell] ----------------------------------- -sudo bin/plugin remove [pluginname] +sudo bin/elasticsearch-plugin remove [pluginname] ----------------------------------- After a Java plugin has been removed, you will need to restart the node to complete the removal process. @@ -167,7 +167,7 @@ can do this as follows: [source,sh] --------------------- -sudo bin/plugin -Des.path.conf=/path/to/custom/config/dir install +sudo bin/elasticsearch-plugin -Des.path.conf=/path/to/custom/config/dir install --------------------- You can also set the `CONF_DIR` environment variable to the custom config @@ -183,13 +183,13 @@ waits. Here is some examples of setting it to different values: [source,shell] ----------------------------------- # Wait for 30 seconds before failing -sudo bin/plugin install mobz/elasticsearch-head --timeout 30s +sudo bin/elasticsearch-plugin install mobz/elasticsearch-head --timeout 30s # Wait for 1 minute before failing -sudo bin/plugin install mobz/elasticsearch-head --timeout 1m +sudo bin/elasticsearch-plugin install mobz/elasticsearch-head --timeout 1m # Wait forever (default) -sudo bin/plugin install mobz/elasticsearch-head --timeout 0 +sudo bin/elasticsearch-plugin install mobz/elasticsearch-head --timeout 0 ----------------------------------- [float] @@ -201,7 +201,7 @@ options can be set on the command line: [source,shell] ----------------------------------- -sudo bin/plugin install mobz/elasticsearch-head -DproxyHost=host_name -DproxyPort=port_number +sudo bin/elasticsearch-plugin install mobz/elasticsearch-head -DproxyHost=host_name -DproxyPort=port_number ----------------------------------- On Windows, they need to be added to the `JAVA_OPTS` environment variable: @@ -209,7 +209,7 @@ On Windows, they need to be added to the `JAVA_OPTS` environment variable: [source,shell] ----------------------------------- set JAVA_OPTS="-DproxyHost=host_name -DproxyPort=port_number" -bin/plugin install mobz/elasticsearch-head +bin/elasticsearch-plugin install mobz/elasticsearch-head ----------------------------------- === Settings related to plugins diff --git a/docs/plugins/repository-azure.asciidoc b/docs/plugins/repository-azure.asciidoc index c93419de260..1c7edd50625 100644 --- a/docs/plugins/repository-azure.asciidoc +++ b/docs/plugins/repository-azure.asciidoc @@ -12,7 +12,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install repository-azure +sudo bin/elasticsearch-plugin install repository-azure ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -26,7 +26,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove repository-azure +sudo bin/elasticsearch-plugin remove repository-azure ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/repository-hdfs.asciidoc b/docs/plugins/repository-hdfs.asciidoc index 28abaf78f66..e91532e8af5 100644 --- a/docs/plugins/repository-hdfs.asciidoc +++ b/docs/plugins/repository-hdfs.asciidoc @@ -12,7 +12,7 @@ This plugin can be installed through the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install repository-hdfs +sudo bin/elasticsearch-plugin install repository-hdfs ---------------------------------------------------------------- The plugin must be installed on _every_ node in the cluster, and each node must @@ -26,7 +26,7 @@ The plugin can be removed by specifying the _installed_ package: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove repository-hdfs +sudo bin/elasticsearch-plugin remove repository-hdfs ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/repository-s3.asciidoc b/docs/plugins/repository-s3.asciidoc index 7c89b0e29e6..007018a6abc 100644 --- a/docs/plugins/repository-s3.asciidoc +++ b/docs/plugins/repository-s3.asciidoc @@ -12,7 +12,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install repository-s3 +sudo bin/elasticsearch-plugin install repository-s3 ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -26,7 +26,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove repository-s3 +sudo bin/elasticsearch-plugin remove repository-s3 ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/plugins/store-smb.asciidoc b/docs/plugins/store-smb.asciidoc index b8cbe57ed3b..1c0b5f4ec18 100644 --- a/docs/plugins/store-smb.asciidoc +++ b/docs/plugins/store-smb.asciidoc @@ -11,7 +11,7 @@ This plugin can be installed using the plugin manager: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin install store-smb +sudo bin/elasticsearch-plugin install store-smb ---------------------------------------------------------------- The plugin must be installed on every node in the cluster, and each node must @@ -25,7 +25,7 @@ The plugin can be removed with the following command: [source,sh] ---------------------------------------------------------------- -sudo bin/plugin remove store-smb +sudo bin/elasticsearch-plugin remove store-smb ---------------------------------------------------------------- The node must be stopped before removing the plugin. diff --git a/docs/reference/migration/migrate_2_0/packaging.asciidoc b/docs/reference/migration/migrate_2_0/packaging.asciidoc index 15b7b51d385..dae87187ba4 100644 --- a/docs/reference/migration/migrate_2_0/packaging.asciidoc +++ b/docs/reference/migration/migrate_2_0/packaging.asciidoc @@ -32,7 +32,7 @@ the foreground, was deprecated in 1.0 and removed in 2.0. ==== `V` for version -The `-v` parameter now means `--verbose` for both `bin/plugin` and +The `-v` parameter now means `--verbose` for both `bin/elasticsearch-plugin` and `bin/elasticsearch` (although it has no effect on the latter). To output the version, use `-V` or `--version` instead. @@ -52,7 +52,7 @@ Official plugins can be installed as follows: [source,sh] --------------- -sudo bin/plugin install analysis-icu +sudo bin/elasticsearch-plugin install analysis-icu --------------- Community-provided plugins can be installed as before. diff --git a/docs/reference/migration/migrate_2_0/removals.asciidoc b/docs/reference/migration/migrate_2_0/removals.asciidoc index 82e1cd923f6..31693c3d3ac 100644 --- a/docs/reference/migration/migrate_2_0/removals.asciidoc +++ b/docs/reference/migration/migrate_2_0/removals.asciidoc @@ -38,7 +38,7 @@ can install the plugin with: [source,sh] ------------------ -./bin/plugin install delete-by-query +./bin/elasticsearch-plugin install delete-by-query ------------------ See {plugins}/plugins-delete-by-query.html for more information. @@ -55,7 +55,7 @@ still need to use multicast discovery, you can install the plugin with: [source,sh] ------------------ -./bin/plugin install discovery-multicast +./bin/elasticsearch-plugin install discovery-multicast ------------------ ==== `_shutdown` API @@ -70,7 +70,7 @@ out of core and is available as a plugin. It can be installed as: [source,sh] ------------------ -./bin/plugin install mapper-murmur3 +./bin/elasticsearch-plugin install mapper-murmur3 ------------------ ==== `_size` is now a plugin @@ -81,7 +81,7 @@ can be installed as: [source,sh] ------------------ -./bin/plugin install mapper-size +./bin/elasticsearch-plugin install mapper-size ------------------ ==== Thrift and memcached transport diff --git a/docs/reference/migration/migrate_2_0/settings.asciidoc b/docs/reference/migration/migrate_2_0/settings.asciidoc index 5e840ac3653..8695b7ba7b2 100644 --- a/docs/reference/migration/migrate_2_0/settings.asciidoc +++ b/docs/reference/migration/migrate_2_0/settings.asciidoc @@ -182,7 +182,7 @@ The location of a custom config directory may be specified as follows: [source,sh] -------------- ./bin/elasticsearch --path.conf=/path/to/conf/dir -./bin/plugin -Des.path.conf=/path/to/conf/dir install analysis-icu +./bin/elasticsearch-plugin -Des.path.conf=/path/to/conf/dir install analysis-icu -------------- When using the RPM or debian packages, the plugin script and the diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index c69060aee8d..3ee709624f3 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -331,6 +331,8 @@ disable doc values is by using the `doc_values` property of mappings. [[breaking_30_plugins]] === Plugin changes +The command `bin/plugin` has been renamed to `bin/elasticsearch-plugin`. + ==== Site plugins removed Site plugins have been removed. It is recommended to migrate site plugins to Kibana plugins. diff --git a/qa/vagrant/src/test/resources/packaging/scripts/plugin_test_cases.bash b/qa/vagrant/src/test/resources/packaging/scripts/plugin_test_cases.bash index 18a93711ff9..c81d850d94d 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/plugin_test_cases.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/plugin_test_cases.bash @@ -85,10 +85,10 @@ fi @test "[$GROUP] install jvm-example plugin with a custom CONFIG_FILE and check failure" { local relativePath=${1:-$(readlink -m jvm-example-*.zip)} - CONF_FILE="$ESCONFIG/elasticsearch.yml" run sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/plugin" install "file://$relativePath" + CONF_FILE="$ESCONFIG/elasticsearch.yml" run sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" install "file://$relativePath" # this should fail because CONF_FILE is no longer supported [ $status = 1 ] - CONF_FILE="$ESCONFIG/elasticsearch.yml" run sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/plugin" remove jvm-example + CONF_FILE="$ESCONFIG/elasticsearch.yml" run sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" remove jvm-example echo "status is $status" [ $status = 1 ] } @@ -170,7 +170,7 @@ fi local JAVA=$(which java) sudo chmod -x $JAVA - run "$ESHOME/bin/plugin" + run "$ESHOME/bin/elasticsearch-plugin" sudo chmod +x $JAVA [ "$status" -eq 1 ] @@ -264,7 +264,7 @@ fi } @test "[$GROUP] check the installed plugins can be listed with 'plugins list' and result matches the list of plugins in plugins pom" { - "$ESHOME/bin/plugin" list | tail -n +2 | sed 's/^......//' > /tmp/installed + "$ESHOME/bin/elasticsearch-plugin" list | tail -n +2 | sed 's/^......//' > /tmp/installed compare_plugins_list "/tmp/installed" "'plugins list'" } @@ -389,7 +389,7 @@ fi @test "[$GROUP] install jvm-example with different logging modes and check output" { local relativePath=${1:-$(readlink -m jvm-example-*.zip)} - sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/plugin" install "file://$relativePath" > /tmp/plugin-cli-output + sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" install "file://$relativePath" > /tmp/plugin-cli-output local loglines=$(cat /tmp/plugin-cli-output | wc -l) if [ "$GROUP" == "TAR PLUGINS" ]; then # tar extraction does not create the plugins directory so the plugin tool will print an additional line that the directory will be created @@ -408,7 +408,7 @@ fi remove_jvm_example local relativePath=${1:-$(readlink -m jvm-example-*.zip)} - sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/plugin" install "file://$relativePath" -Des.logger.level=DEBUG > /tmp/plugin-cli-output + sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" install "file://$relativePath" -Des.logger.level=DEBUG > /tmp/plugin-cli-output local loglines=$(cat /tmp/plugin-cli-output | wc -l) if [ "$GROUP" == "TAR PLUGINS" ]; then [ "$loglines" -gt "7" ] || { diff --git a/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash b/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash index 7200f28b139..8a32c982af1 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/plugins.bash @@ -32,7 +32,7 @@ install_plugin() { assert_file_exist "$path" - sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/plugin" install "file://$path" + sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" install "file://$path" assert_file_exist "$ESPLUGINS/$name" assert_file_exist "$ESPLUGINS/$name/plugin-descriptor.properties" @@ -60,7 +60,7 @@ remove_plugin() { local name=$1 echo "Removing $name...." - sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/plugin" remove $name + sudo -E -u $ESPLUGIN_COMMAND_USER "$ESHOME/bin/elasticsearch-plugin" remove $name assert_file_not_exist "$ESPLUGINS/$name" diff --git a/qa/vagrant/src/test/resources/packaging/scripts/tar.bash b/qa/vagrant/src/test/resources/packaging/scripts/tar.bash index d447b1a0425..f9bcc10525b 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/tar.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/tar.bash @@ -82,7 +82,7 @@ verify_archive_installation() { assert_file "$ESHOME/bin" d assert_file "$ESHOME/bin/elasticsearch" f assert_file "$ESHOME/bin/elasticsearch.in.sh" f - assert_file "$ESHOME/bin/plugin" f + assert_file "$ESHOME/bin/elasticsearch-plugin" f assert_file "$ESCONFIG" d assert_file "$ESCONFIG/elasticsearch.yml" f assert_file "$ESCONFIG/logging.yml" f From 89a5eadfea5158a7f228a8acb5534f46c2e2ae52 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 5 Feb 2016 10:13:17 +0100 Subject: [PATCH 04/19] Change path separator for Checkstyle suppressions to be Windows compatible --- .../gradle/precommit/PrecommitTasks.groovy | 4 +- .../resources/checkstyle_suppressions.xml | 3408 +++++++++-------- 2 files changed, 1708 insertions(+), 1704 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy index 1cdeec762d2..dd77c38e679 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy @@ -91,11 +91,12 @@ class PrecommitTasks { // on them. But we want `precommit` to depend on `checkstyle` which depends on them so // we have to swap them. project.pluginManager.apply('checkstyle') + URL checkstyleSuppressions = PrecommitTasks.getResource('/checkstyle_suppressions.xml') project.checkstyle { config = project.resources.text.fromFile( PrecommitTasks.getResource('/checkstyle.xml'), 'UTF-8') configProperties = [ - suppressions: PrecommitTasks.getResource('/checkstyle_suppressions.xml') + suppressions: checkstyleSuppressions ] } for (String taskName : ['checkstyleMain', 'checkstyleTest']) { @@ -103,6 +104,7 @@ class PrecommitTasks { if (task != null) { project.tasks['check'].dependsOn.remove(task) checkstyleTask.dependsOn(task) + task.inputs.file(checkstyleSuppressions) } } return checkstyleTask diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 3b3f91bc233..476d1269884 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -4,1712 +4,1714 @@ "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd"> + + - - + + - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + - - - - - - - - - - - - + + + + + + + + + + + + From b5ef48ce576f43be9c526dfae4c1eb6d53ef9e56 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 5 Feb 2016 10:36:02 +0100 Subject: [PATCH 05/19] fix more tests --- .../memory/breaker/RandomExceptionCircuitBreakerIT.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java b/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java index 4807cf61dea..a406106f110 100644 --- a/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java +++ b/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.MockEngineFactoryPlugin; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.sort.SortOrder; @@ -176,7 +177,7 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase { // Since .cleanUp() is no longer called on cache clear, we need to call it on each node manually for (String node : internalCluster().getNodeNames()) { - final IndicesFieldDataCache fdCache = internalCluster().getInstance(IndicesFieldDataCache.class, node); + final IndicesFieldDataCache fdCache = internalCluster().getInstance(IndicesService.class, node).getIndicesFieldDataCache(); // Clean up the cache, ensuring that entries' listeners have been called fdCache.getCache().refresh(); } From 8685a003d97b3b49db6f5e283bc0641f4c33983e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 5 Feb 2016 11:17:53 +0100 Subject: [PATCH 06/19] fix another test --- .../elasticsearch/messy/tests/TemplateQueryParserTests.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java index 2e19f660047..e0d8fc3a506 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java @@ -43,6 +43,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; +import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryShardContext; @@ -138,7 +139,8 @@ public class TemplateQueryParserTests extends ESTestCase { SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap()); MapperRegistry mapperRegistry = new IndicesModule().getMapperRegistry(); MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry, () -> context); - IndexFieldDataService indexFieldDataService =new IndexFieldDataService(idxSettings, injector.getInstance(IndicesFieldDataCache.class), injector.getInstance(CircuitBreakerService.class), mapperService); + IndicesFieldDataCache cache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() {}); + IndexFieldDataService indexFieldDataService =new IndexFieldDataService(idxSettings, cache, injector.getInstance(CircuitBreakerService.class), mapperService); BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { From 1e18c9a7b6623db0100d93e92919a4a7d29e7231 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 5 Feb 2016 11:21:00 +0100 Subject: [PATCH 07/19] Apply system properties after all arguemnts are parsed in BootstrapCLIParser One of our tests leaked a system property here since we failed after appling some system properties in BootstrapCLIParser. This is not a huge deal in production since we exit the JVM if we fail on that. Yet for correctnes we should only apply them if we manage to parse them all. This also caused a test failure lately on CI but on an unrelated test: https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+periodic/314/console --- .../elasticsearch/bootstrap/BootstrapCLIParser.java | 11 ++++++++--- .../bootstrap/BootstrapCliParserTests.java | 12 ++++++++++++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCLIParser.java b/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCLIParser.java index 9cae3b8cb15..ca67fc91132 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCLIParser.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCLIParser.java @@ -32,10 +32,12 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.monitor.jvm.JvmInfo; +import java.util.HashMap; import java.util.Iterator; import java.util.Locale; import java.util.Map; import java.util.Properties; +import java.util.Set; import static org.elasticsearch.common.cli.CliToolConfig.Builder.cmd; import static org.elasticsearch.common.cli.CliToolConfig.Builder.optionBuilder; @@ -131,6 +133,7 @@ final class BootstrapCLIParser extends CliTool { // hacky way to extract all the fancy extra args, there is no CLI tool helper for this Iterator iterator = cli.getArgList().iterator(); + final Map properties = new HashMap<>(); while (iterator.hasNext()) { String arg = iterator.next(); if (!arg.startsWith("--")) { @@ -148,20 +151,22 @@ final class BootstrapCLIParser extends CliTool { String[] splitArg = arg.split("=", 2); String key = splitArg[0]; String value = splitArg[1]; - System.setProperty("es." + key, value); + properties.put("es." + key, value); } else { if (iterator.hasNext()) { String value = iterator.next(); if (value.startsWith("--")) { throw new UserError(ExitStatus.USAGE, "Parameter [" + arg + "] needs value"); } - System.setProperty("es." + arg, value); + properties.put("es." + arg, value); } else { throw new UserError(ExitStatus.USAGE, "Parameter [" + arg + "] needs value"); } } } - + for (Map.Entry entry : properties.entrySet()) { + System.setProperty(entry.getKey(), entry.getValue()); + } return new Start(terminal); } diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/BootstrapCliParserTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/BootstrapCliParserTests.java index f417fe70d8b..012af99cef0 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/BootstrapCliParserTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/BootstrapCliParserTests.java @@ -29,11 +29,15 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.monitor.jvm.JvmInfo; import org.hamcrest.Matcher; import org.junit.After; +import org.junit.Before; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Locale; +import java.util.Map; +import java.util.Properties; import static org.elasticsearch.common.cli.CliTool.ExitStatus.OK; import static org.elasticsearch.common.cli.CliTool.ExitStatus.OK_AND_EXIT; @@ -48,6 +52,12 @@ public class BootstrapCliParserTests extends CliToolTestCase { private CaptureOutputTerminal terminal = new CaptureOutputTerminal(); private List propertiesToClear = new ArrayList<>(); + private Map properties; + + @Before + public void before() { + this.properties = new HashMap<>(System.getProperties()); + } @After public void clearProperties() { @@ -55,6 +65,7 @@ public class BootstrapCliParserTests extends CliToolTestCase { System.clearProperty(property); } propertiesToClear.clear(); + assertEquals("properties leaked", properties, new HashMap<>(System.getProperties())); } public void testThatVersionIsReturned() throws Exception { @@ -235,6 +246,7 @@ public class BootstrapCliParserTests extends CliToolTestCase { parser.parse("start", new String[]{"--foo=bar", "-Dbaz=qux"}); }); assertThat(e.getMessage(), containsString("must be before any parameters starting with --")); + assertNull(System.getProperty("es.foo")); } private void registerProperties(String ... systemProperties) { From 7f2b369dfdcd224d8a82a62fbda9060e066ba5fc Mon Sep 17 00:00:00 2001 From: Andrej Kazakov Date: Thu, 4 Feb 2016 17:47:08 +0000 Subject: [PATCH 08/19] Use Accept header field in cat API The cat API previously used the Content-Type header field for determining the media type of the response. This is in opposition to the HTTP spec which specifies the Accept header field for this purpose. This commit replaces the use of the Content-Type header field with the Accept header field in the cat API. Closes #14421 --- .../common/xcontent/XContentType.java | 51 ++++++----- .../gateway/MetaStateService.java | 2 +- .../elasticsearch/rest/BytesRestResponse.java | 4 +- .../org/elasticsearch/rest/RestChannel.java | 2 +- .../rest/action/support/RestTable.java | 2 +- .../common/xcontent/XContentTypeTests.java | 87 +++++++++++++++++++ .../rest/action/support/RestTableTests.java | 84 ++++++++++++++++++ docs/reference/cat.asciidoc | 49 +++++++++++ docs/reference/migration/migrate_3_0.asciidoc | 9 ++ .../rest-api-spec/api/cat.aliases.json | 4 + .../rest-api-spec/api/cat.allocation.json | 4 + .../rest-api-spec/api/cat.count.json | 4 + .../rest-api-spec/api/cat.fielddata.json | 4 + .../rest-api-spec/api/cat.health.json | 4 + .../rest-api-spec/api/cat.indices.json | 4 + .../rest-api-spec/api/cat.master.json | 4 + .../rest-api-spec/api/cat.nodeattrs.json | 4 + .../rest-api-spec/api/cat.nodes.json | 4 + .../rest-api-spec/api/cat.pending_tasks.json | 4 + .../rest-api-spec/api/cat.plugins.json | 4 + .../rest-api-spec/api/cat.recovery.json | 4 + .../rest-api-spec/api/cat.repositories.json | 4 + .../rest-api-spec/api/cat.segments.json | 4 + .../rest-api-spec/api/cat.shards.json | 4 + .../rest-api-spec/api/cat.snapshots.json | 4 + .../rest-api-spec/api/cat.thread_pool.json | 4 + .../test/cat.aliases/10_basic.yaml | 50 +++++++++++ .../test/get/50_with_headers.yaml | 2 +- 28 files changed, 383 insertions(+), 27 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/common/xcontent/XContentTypeTests.java diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/XContentType.java b/core/src/main/java/org/elasticsearch/common/xcontent/XContentType.java index 329bad87265..296f9d2aedd 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/XContentType.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/XContentType.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.xcontent.smile.SmileXContent; import org.elasticsearch.common.xcontent.yaml.YamlXContent; import java.io.IOException; +import java.util.Locale; /** * The content type of {@link org.elasticsearch.common.xcontent.XContent}. @@ -38,7 +39,12 @@ public enum XContentType { */ JSON(0) { @Override - public String restContentType() { + protected String mediaTypeWithoutParameters() { + return "application/json"; + } + + @Override + public String mediaType() { return "application/json; charset=UTF-8"; } @@ -57,7 +63,7 @@ public enum XContentType { */ SMILE(1) { @Override - public String restContentType() { + protected String mediaTypeWithoutParameters() { return "application/smile"; } @@ -76,7 +82,7 @@ public enum XContentType { */ YAML(2) { @Override - public String restContentType() { + protected String mediaTypeWithoutParameters() { return "application/yaml"; } @@ -95,7 +101,7 @@ public enum XContentType { */ CBOR(3) { @Override - public String restContentType() { + protected String mediaTypeWithoutParameters() { return "application/cbor"; } @@ -108,31 +114,30 @@ public enum XContentType { public XContent xContent() { return CborXContent.cborXContent; } - },; + }; - public static XContentType fromRestContentType(String contentType) { - if (contentType == null) { + public static XContentType fromMediaTypeOrFormat(String mediaType) { + if (mediaType == null) { return null; } - if ("application/json".equals(contentType) || "json".equalsIgnoreCase(contentType)) { + for (XContentType type : values()) { + if (isSameMediaTypeAs(mediaType, type)) { + return type; + } + } + if(mediaType.toLowerCase(Locale.ROOT).startsWith("application/*")) { return JSON; } - if ("application/smile".equals(contentType) || "smile".equalsIgnoreCase(contentType)) { - return SMILE; - } - - if ("application/yaml".equals(contentType) || "yaml".equalsIgnoreCase(contentType)) { - return YAML; - } - - if ("application/cbor".equals(contentType) || "cbor".equalsIgnoreCase(contentType)) { - return CBOR; - } - return null; } + private static boolean isSameMediaTypeAs(String stringType, XContentType type) { + return type.mediaTypeWithoutParameters().equalsIgnoreCase(stringType) || + stringType.toLowerCase(Locale.ROOT).startsWith(type.mediaTypeWithoutParameters().toLowerCase(Locale.ROOT) + ";") || + type.shortName().equalsIgnoreCase(stringType); + } + private int index; XContentType(int index) { @@ -143,12 +148,16 @@ public enum XContentType { return index; } - public abstract String restContentType(); + public String mediaType() { + return mediaTypeWithoutParameters(); + } public abstract String shortName(); public abstract XContent xContent(); + protected abstract String mediaTypeWithoutParameters(); + public static XContentType readFrom(StreamInput in) throws IOException { int index = in.readVInt(); for (XContentType contentType : values()) { diff --git a/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java b/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java index 6f38b039fec..9ef09753c43 100644 --- a/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java +++ b/core/src/main/java/org/elasticsearch/gateway/MetaStateService.java @@ -59,7 +59,7 @@ public class MetaStateService extends AbstractComponent { public MetaStateService(Settings settings, NodeEnvironment nodeEnv) { super(settings); this.nodeEnv = nodeEnv; - this.format = XContentType.fromRestContentType(settings.get(FORMAT_SETTING, "smile")); + this.format = XContentType.fromMediaTypeOrFormat(settings.get(FORMAT_SETTING, "smile")); if (this.format == XContentType.SMILE) { Map params = new HashMap<>(); params.put("binary", "true"); diff --git a/core/src/main/java/org/elasticsearch/rest/BytesRestResponse.java b/core/src/main/java/org/elasticsearch/rest/BytesRestResponse.java index ba7e54c123f..ac8eadade0b 100644 --- a/core/src/main/java/org/elasticsearch/rest/BytesRestResponse.java +++ b/core/src/main/java/org/elasticsearch/rest/BytesRestResponse.java @@ -48,7 +48,7 @@ public class BytesRestResponse extends RestResponse { * Creates a new response based on {@link XContentBuilder}. */ public BytesRestResponse(RestStatus status, XContentBuilder builder) { - this(status, builder.contentType().restContentType(), builder.bytes()); + this(status, builder.contentType().mediaType(), builder.bytes()); } /** @@ -93,7 +93,7 @@ public class BytesRestResponse extends RestResponse { } else { XContentBuilder builder = convert(channel, status, t); this.content = builder.bytes(); - this.contentType = builder.contentType().restContentType(); + this.contentType = builder.contentType().mediaType(); } if (t instanceof ElasticsearchException) { copyHeaders(((ElasticsearchException) t)); diff --git a/core/src/main/java/org/elasticsearch/rest/RestChannel.java b/core/src/main/java/org/elasticsearch/rest/RestChannel.java index cd185b07958..ed6abe56258 100644 --- a/core/src/main/java/org/elasticsearch/rest/RestChannel.java +++ b/core/src/main/java/org/elasticsearch/rest/RestChannel.java @@ -53,7 +53,7 @@ public abstract class RestChannel { } public XContentBuilder newBuilder(@Nullable BytesReference autoDetectSource, boolean useFiltering) throws IOException { - XContentType contentType = XContentType.fromRestContentType(request.param("format", request.header("Content-Type"))); + XContentType contentType = XContentType.fromMediaTypeOrFormat(request.param("format", request.header("Accept"))); if (contentType == null) { // try and guess it from the auto detect source if (autoDetectSource != null) { diff --git a/core/src/main/java/org/elasticsearch/rest/action/support/RestTable.java b/core/src/main/java/org/elasticsearch/rest/action/support/RestTable.java index 13b638910ae..ae721707e97 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/support/RestTable.java +++ b/core/src/main/java/org/elasticsearch/rest/action/support/RestTable.java @@ -48,7 +48,7 @@ public class RestTable { public static RestResponse buildResponse(Table table, RestChannel channel) throws Exception { RestRequest request = channel.request(); - XContentType xContentType = XContentType.fromRestContentType(request.param("format", request.header("Content-Type"))); + XContentType xContentType = XContentType.fromMediaTypeOrFormat(request.param("format", request.header("Accept"))); if (xContentType != null) { return buildXContentBuilder(table, channel); } diff --git a/core/src/test/java/org/elasticsearch/common/xcontent/XContentTypeTests.java b/core/src/test/java/org/elasticsearch/common/xcontent/XContentTypeTests.java new file mode 100644 index 00000000000..47a470e2cea --- /dev/null +++ b/core/src/test/java/org/elasticsearch/common/xcontent/XContentTypeTests.java @@ -0,0 +1,87 @@ +/* + * 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.xcontent; + +import org.elasticsearch.test.ESTestCase; + +import java.util.Locale; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; + +public class XContentTypeTests extends ESTestCase { + public void testFromJson() throws Exception { + String mediaType = "application/json"; + XContentType expectedXContentType = XContentType.JSON; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + "; charset=UTF-8"), equalTo(expectedXContentType)); + } + + public void testFromJsonUppercase() throws Exception { + String mediaType = "application/json".toUpperCase(Locale.ROOT); + XContentType expectedXContentType = XContentType.JSON; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + "; charset=UTF-8"), equalTo(expectedXContentType)); + } + + public void testFromYaml() throws Exception { + String mediaType = "application/yaml"; + XContentType expectedXContentType = XContentType.YAML; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + "; charset=UTF-8"), equalTo(expectedXContentType)); + } + + public void testFromSmile() throws Exception { + String mediaType = "application/smile"; + XContentType expectedXContentType = XContentType.SMILE; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + } + + public void testFromCbor() throws Exception { + String mediaType = "application/cbor"; + XContentType expectedXContentType = XContentType.CBOR; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + } + + public void testFromWildcard() throws Exception { + String mediaType = "application/*"; + XContentType expectedXContentType = XContentType.JSON; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + } + + public void testFromWildcardUppercase() throws Exception { + String mediaType = "APPLICATION/*"; + XContentType expectedXContentType = XContentType.JSON; + assertThat(XContentType.fromMediaTypeOrFormat(mediaType), equalTo(expectedXContentType)); + assertThat(XContentType.fromMediaTypeOrFormat(mediaType + ";"), equalTo(expectedXContentType)); + } + + public void testFromRubbish() throws Exception { + assertThat(XContentType.fromMediaTypeOrFormat(null), nullValue()); + assertThat(XContentType.fromMediaTypeOrFormat(""), nullValue()); + assertThat(XContentType.fromMediaTypeOrFormat("text/plain"), nullValue()); + assertThat(XContentType.fromMediaTypeOrFormat("gobbly;goop"), nullValue()); + } +} diff --git a/core/src/test/java/org/elasticsearch/rest/action/support/RestTableTests.java b/core/src/test/java/org/elasticsearch/rest/action/support/RestTableTests.java index 161668dba04..662bd7b4490 100644 --- a/core/src/test/java/org/elasticsearch/rest/action/support/RestTableTests.java +++ b/core/src/test/java/org/elasticsearch/rest/action/support/RestTableTests.java @@ -20,20 +20,45 @@ package org.elasticsearch.rest.action.support; import org.elasticsearch.common.Table; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestResponse; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.FakeRestRequest; import org.junit.Before; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import static org.elasticsearch.rest.action.support.RestTable.buildDisplayHeaders; +import static org.elasticsearch.rest.action.support.RestTable.buildResponse; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.not; public class RestTableTests extends ESTestCase { + private static final String APPLICATION_JSON = XContentType.JSON.mediaType(); + private static final String APPLICATION_YAML = XContentType.YAML.mediaType(); + private static final String APPLICATION_SMILE = XContentType.SMILE.mediaType(); + private static final String APPLICATION_CBOR = XContentType.CBOR.mediaType(); + private static final String CONTENT_TYPE = "Content-Type"; + private static final String ACCEPT = "Accept"; + private static final String TEXT_PLAIN = "text/plain; charset=UTF-8"; + private static final String TEXT_TABLE_BODY = "foo foo foo foo foo foo\n"; + private static final String JSON_TABLE_BODY = "[{\"bulk.foo\":\"foo\",\"bulk.bar\":\"foo\",\"aliasedBulk\":\"foo\"," + + "\"aliasedSecondBulk\":\"foo\",\"unmatched\":\"foo\"," + + "\"invalidAliasesBulk\":\"foo\"}]"; + private static final String YAML_TABLE_BODY = "---\n" + + "- bulk.foo: \"foo\"\n" + + " bulk.bar: \"foo\"\n" + + " aliasedBulk: \"foo\"\n" + + " aliasedSecondBulk: \"foo\"\n" + + " unmatched: \"foo\"\n" + + " invalidAliasesBulk: \"foo\"\n"; private Table table = new Table(); private FakeRestRequest restRequest = new FakeRestRequest(); @@ -70,6 +95,65 @@ public class RestTableTests extends ESTestCase { assertThat(headerNames, not(hasItem("unmatched"))); } + public void testThatWeUseTheAcceptHeaderJson() throws Exception { + assertResponse(Collections.singletonMap(ACCEPT, APPLICATION_JSON), + APPLICATION_JSON, + JSON_TABLE_BODY); + } + + public void testThatWeUseTheAcceptHeaderYaml() throws Exception { + assertResponse(Collections.singletonMap(ACCEPT, APPLICATION_YAML), + APPLICATION_YAML, + YAML_TABLE_BODY); + } + + public void testThatWeUseTheAcceptHeaderSmile() throws Exception { + assertResponseContentType(Collections.singletonMap(ACCEPT, APPLICATION_SMILE), + APPLICATION_SMILE); + } + + public void testThatWeUseTheAcceptHeaderCbor() throws Exception { + assertResponseContentType(Collections.singletonMap(ACCEPT, APPLICATION_CBOR), + APPLICATION_CBOR); + } + + public void testThatWeUseTheAcceptHeaderText() throws Exception { + assertResponse(Collections.singletonMap(ACCEPT, TEXT_PLAIN), + TEXT_PLAIN, + TEXT_TABLE_BODY); + } + + public void testIgnoreContentType() throws Exception { + assertResponse(Collections.singletonMap(CONTENT_TYPE, APPLICATION_JSON), + TEXT_PLAIN, + TEXT_TABLE_BODY); + } + + private RestResponse assertResponseContentType(Map headers, String mediaType) throws Exception { + FakeRestRequest requestWithAcceptHeader = new FakeRestRequest(headers); + table.startRow(); + table.addCell("foo"); + table.addCell("foo"); + table.addCell("foo"); + table.addCell("foo"); + table.addCell("foo"); + table.addCell("foo"); + table.endRow(); + RestResponse response = buildResponse(table, new RestChannel(requestWithAcceptHeader, true) { + @Override + public void sendResponse(RestResponse response) { + } + }); + + assertThat(response.contentType(), equalTo(mediaType)); + return response; + } + + private void assertResponse(Map headers, String mediaType, String body) throws Exception { + RestResponse response = assertResponseContentType(headers, mediaType); + assertThat(response.content().toUtf8(), equalTo(body)); + } + private List getHeaderNames(List headers) { List headerNames = new ArrayList<>(); for (RestTable.DisplayHeader header : headers) { diff --git a/docs/reference/cat.asciidoc b/docs/reference/cat.asciidoc index ed1f09b0f60..41d2ec85f15 100644 --- a/docs/reference/cat.asciidoc +++ b/docs/reference/cat.asciidoc @@ -96,6 +96,55 @@ green foo 1 0 227 0 2065131 2065131 -------------------------------------------------- +=== Response as text, json, smile, yaml or cbor + +[source,sh] +-------------------------------------------------- +% curl '192.168.56.10:9200/_cat/indices?format=json' | jq . +[ + { + "pri.store.size": "650b", + "health": "yellow", + "status": "open", + "index": "twitter", + "pri": "5", + "rep": "1", + "docs.count": "0", + "docs.deleted": "0", + "store.size": "650b" + } +] +-------------------------------------------------- + +Currently supported formats (for the ?format parameter): +- text (default) +- json +- smile +- yaml +- cbor + +alternatively you can set the "Accept" HTTP header to the appropriate media format. +All formats above are supported, the GET parameter takes precedence over the header. + +For example +[source,sh] +-------------------------------------------------- +% curl '192.168.56.10:9200/_cat/indices' -H "Accept: application/json" | jq . +[ + { + "pri.store.size": "650b", + "health": "yellow", + "status": "open", + "index": "twitter", + "pri": "5", + "rep": "1", + "docs.count": "0", + "docs.deleted": "0", + "store.size": "650b" + } +] +-------------------------------------------------- + -- include::cat/alias.asciidoc[] diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index 3ee709624f3..3a40f57bbd5 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -6,6 +6,7 @@ your application to Elasticsearch 3.0. * <> * <> +* <> * <> * <> * <> @@ -158,6 +159,14 @@ Removed support for the undocumented `query_binary` and `filter_binary` sections Payloads are now loaded when needed. +[[breaking_30_cat_api]] +=== REST API changes + +==== Replace use of Content-Type header in _cat API in favour of the Accept header + +Before we would mistakenly use the Content-Type of the request to establish the response media type, that's the +role of the Accept header as per HTTP spec. + [[breaking_30_parent_child_changes]] === Parent/Child changes diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json index 07c72a2ac98..4bc93460b75 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.aliases.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json index 9de5a729faa..bca547ce6a7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.allocation.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "bytes": { "type": "enum", "description" : "The unit in which to display byte values", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json index 2661af347d0..73803bac867 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.count.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json index a380b558e89..d2b207d4a49 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.fielddata.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "bytes": { "type": "enum", "description" : "The unit in which to display byte values", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json index bd0cfcedd53..349bf37aa06 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.health.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json index b5c487b9521..7858fc2ee9e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.indices.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "bytes": { "type": "enum", "description" : "The unit in which to display byte values", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json index d3e35330e1d..bdf474c7a02 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.master.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json index cdbe75ac936..4e44eb820f1 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodeattrs.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json index 1d8bcda3c01..f0fcf390102 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.nodes.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json index 88afa9b0c40..74fc54b9604 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.pending_tasks.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json index 55ee66a2792..5822f64c8e2 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.plugins.json @@ -6,6 +6,10 @@ "path": "/_cat/plugins", "paths": ["/_cat/plugins"], "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json index c235406221a..e6a80552492 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.recovery.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "bytes": { "type": "enum", "description" : "The unit in which to display byte values", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json index 2dcbac5cac9..ace0a1ea3c9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.repositories.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json index ebb989b4cf6..eed4a627736 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.segments.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "h": { "type": "list", "description" : "Comma-separated list of column names to display" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json index 6d941f17d10..a9a1a30770f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.shards.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json index 4fd5520ecb1..b54bee38c24 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.snapshots.json @@ -12,6 +12,10 @@ } }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "ignore_unavailable": { "type": "boolean", "description": "Set to true to ignore unavailable snapshots", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json index cb8e5e13632..a942cd0ca66 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/cat.thread_pool.json @@ -8,6 +8,10 @@ "parts": { }, "params": { + "format": { + "type" : "string", + "description" : "a short version of the Accept header, e.g. json, yaml" + }, "local": { "type" : "boolean", "description" : "Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml index deb1307cec8..4ce21978840 100755 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yaml @@ -49,6 +49,56 @@ - \s+ $/ +--- +"Simple alias with yaml body through Accept header": + + - do: + indices.create: + index: test + + - do: + indices.put_alias: + index: test + name: test_alias + + - do: + cat.aliases: {} + headers: + Accept: application/yaml + + - match: + $body: | + /^---\n + -\s+alias:\s+"test_alias"\s+ + index:\s+"test"\s+ + filter:\s+"-"\s+ + routing.index:\s+"-"\s+ + routing.search:\s+"-"\s+$/ +--- +"Simple alias with yaml body through format argument": + + - do: + indices.create: + index: test + + - do: + indices.put_alias: + index: test + name: test_alias + + - do: + cat.aliases: + format: yaml + + - match: + $body: | + /^---\n + -\s+alias:\s+"test_alias"\s+ + index:\s+"test"\s+ + filter:\s+"-"\s+ + routing.index:\s+"-"\s+ + routing.search:\s+"-"\s+$/ + --- "Complex alias": diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/get/50_with_headers.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/get/50_with_headers.yaml index 71229686eed..1bb031f0878 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/get/50_with_headers.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/get/50_with_headers.yaml @@ -12,7 +12,7 @@ - do: headers: - Content-Type: application/yaml + Accept: application/yaml get: index: test_1 type: _all From 21e22a2ddc5c4f696ba8c5740a417a16a07a68a2 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 5 Feb 2016 06:22:01 -0500 Subject: [PATCH 09/19] Reword cat API breaking changes --- docs/reference/migration/migrate_3_0.asciidoc | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index 3a40f57bbd5..4f9cf68f0b4 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -160,12 +160,16 @@ Removed support for the undocumented `query_binary` and `filter_binary` sections Payloads are now loaded when needed. [[breaking_30_cat_api]] -=== REST API changes +=== CAT API changes -==== Replace use of Content-Type header in _cat API in favour of the Accept header +==== Use Accept header for specifying response media type -Before we would mistakenly use the Content-Type of the request to establish the response media type, that's the -role of the Accept header as per HTTP spec. +Previous versions of Elasticsearch accepted the Content-type header +field for controlling the media type of the response in the cat API. +This is in opposition to the HTTP spec which specifies the Accept +header field for this purpose. Elasticsearch now uses the Accept header +field and support for using the Content-Type header field for this +purpose has been removed. [[breaking_30_parent_child_changes]] === Parent/Child changes From d885fe62ea680989f1c27a9206f616cc5bae25ce Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 4 Feb 2016 21:56:43 +0100 Subject: [PATCH 10/19] Add test for minimum_should_match, one term and multiple fields This adds a test case similar to the issue in #13884 which was fixed in #16155. --- .../resources/checkstyle_suppressions.xml | 2 -- .../index/query/AbstractQueryTestCase.java | 4 +-- .../query/SimpleQueryStringBuilderTests.java | 36 ++++++++++++++++++- .../search/query/SimpleQueryStringIT.java | 28 ++++++++++----- 4 files changed, 57 insertions(+), 13 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 476d1269884..69ecb69d32a 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -1273,7 +1273,6 @@ - @@ -1447,7 +1446,6 @@ - diff --git a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java index d8d018284e9..d9f70978ee2 100644 --- a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java @@ -22,6 +22,7 @@ package org.elasticsearch.index.query; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.io.JsonStringEncoder; + import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; @@ -56,7 +57,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.ToXContent; @@ -89,9 +89,9 @@ import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptContextRegistry; import org.elasticsearch.script.ScriptEngineRegistry; import org.elasticsearch.script.ScriptEngineService; -import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.ScriptSettings; import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ESTestCase; diff --git a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java index 190824a1ced..3da15049672 100644 --- a/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java @@ -297,7 +297,8 @@ public class SimpleQueryStringBuilderTests extends AbstractQueryTestCase 1) { + expectedMinimumShouldMatch = 0; + } + assertEquals(expectedMinimumShouldMatch, boolQuery.getMinimumNumberShouldMatch()); + } + } } diff --git a/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java b/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java index 923153a7dd0..1a585db3934 100644 --- a/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/SimpleQueryStringIT.java @@ -116,12 +116,21 @@ public class SimpleQueryStringIT extends ESIntegTestCase { assertSearchHits(searchResponse, "3", "4"); logger.info("--> query 2"); - searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar").field("body").field("body2").minimumShouldMatch("2")).get(); + searchResponse = client().prepareSearch() + .setQuery(simpleQueryStringQuery("foo bar").field("body").field("body2").minimumShouldMatch("2")).get(); assertHitCount(searchResponse, 2L); assertSearchHits(searchResponse, "3", "4"); + // test case from #13884 logger.info("--> query 3"); - searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar baz").field("body").field("body2").minimumShouldMatch("70%")).get(); + searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo") + .field("body").field("body2").field("body3").minimumShouldMatch("-50%")).get(); + assertHitCount(searchResponse, 3L); + assertSearchHits(searchResponse, "1", "3", "4"); + + logger.info("--> query 4"); + searchResponse = client().prepareSearch() + .setQuery(simpleQueryStringQuery("foo bar baz").field("body").field("body2").minimumShouldMatch("70%")).get(); assertHitCount(searchResponse, 2L); assertSearchHits(searchResponse, "3", "4"); @@ -131,18 +140,20 @@ public class SimpleQueryStringIT extends ESIntegTestCase { client().prepareIndex("test", "type1", "7").setSource("body2", "foo bar", "other", "foo"), client().prepareIndex("test", "type1", "8").setSource("body2", "foo baz bar", "other", "foo")); - logger.info("--> query 4"); - searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar").field("body").field("body2").minimumShouldMatch("2")).get(); + logger.info("--> query 5"); + searchResponse = client().prepareSearch() + .setQuery(simpleQueryStringQuery("foo bar").field("body").field("body2").minimumShouldMatch("2")).get(); assertHitCount(searchResponse, 4L); assertSearchHits(searchResponse, "3", "4", "7", "8"); - logger.info("--> query 5"); + logger.info("--> query 6"); searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar").minimumShouldMatch("2")).get(); assertHitCount(searchResponse, 5L); assertSearchHits(searchResponse, "3", "4", "6", "7", "8"); - logger.info("--> query 6"); - searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("foo bar baz").field("body2").field("other").minimumShouldMatch("70%")).get(); + logger.info("--> query 7"); + searchResponse = client().prepareSearch() + .setQuery(simpleQueryStringQuery("foo bar baz").field("body2").field("other").minimumShouldMatch("70%")).get(); assertHitCount(searchResponse, 3L); assertSearchHits(searchResponse, "6", "7", "8"); } @@ -330,7 +341,8 @@ public class SimpleQueryStringIT extends ESIntegTestCase { indexRandom(true, client().prepareIndex("test1", "type1", "1").setSource("location", "Köln")); refresh(); - SearchResponse searchResponse = client().prepareSearch().setQuery(simpleQueryStringQuery("Köln*").analyzeWildcard(true).field("location")).get(); + SearchResponse searchResponse = client().prepareSearch() + .setQuery(simpleQueryStringQuery("Köln*").analyzeWildcard(true).field("location")).get(); assertNoFailures(searchResponse); assertHitCount(searchResponse, 1L); assertSearchHits(searchResponse, "1"); From 0c41a686904b499b47ed4f62128e1827dd4d8ae6 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Fri, 5 Feb 2016 14:47:27 +0100 Subject: [PATCH 11/19] Fixed asciidoc issues --- docs/reference/cat.asciidoc | 8 ++++---- docs/reference/migration/migrate_3_0.asciidoc | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/reference/cat.asciidoc b/docs/reference/cat.asciidoc index 41d2ec85f15..7c4125a7f36 100644 --- a/docs/reference/cat.asciidoc +++ b/docs/reference/cat.asciidoc @@ -95,7 +95,7 @@ green wiki1 3 0 10000 413 103776272 103776272 green foo 1 0 227 0 2065131 2065131 -------------------------------------------------- - +[float] === Response as text, json, smile, yaml or cbor [source,sh] @@ -116,17 +116,17 @@ green foo 1 0 227 0 2065131 2065131 ] -------------------------------------------------- -Currently supported formats (for the ?format parameter): +Currently supported formats (for the `?format=` parameter): - text (default) - json - smile - yaml - cbor -alternatively you can set the "Accept" HTTP header to the appropriate media format. +Alternatively you can set the "Accept" HTTP header to the appropriate media format. All formats above are supported, the GET parameter takes precedence over the header. +For example: -For example [source,sh] -------------------------------------------------- % curl '192.168.56.10:9200/_cat/indices' -H "Accept: application/json" | jq . diff --git a/docs/reference/migration/migrate_3_0.asciidoc b/docs/reference/migration/migrate_3_0.asciidoc index 4f9cf68f0b4..bae484f4a13 100644 --- a/docs/reference/migration/migrate_3_0.asciidoc +++ b/docs/reference/migration/migrate_3_0.asciidoc @@ -314,7 +314,7 @@ On all types but `string`, the `index` property now only accepts `true`/`false` instead of `not_analyzed`/`no`. The `string` field still accepts `analyzed`/`not_analyzed`/`no`. -==== `_source`'s `format` option +==== ++_source++'s `format` option The `_source` mapping does not support the `format` option anymore. This option will still be accepted for indices created before the upgrade to 3.0 for backward From b185209b63aa5e824d54209cc4a1c5eba38d7776 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 5 Feb 2016 10:31:17 +0100 Subject: [PATCH 12/19] Make IndicesWarmer a private class of IndexService There is no need for IndicesWarmer to be a global accessible class. All it needs access to is inside IndexService. It also doesn't need to be mutable once it's not a per node instance. This commit move IndicesWarmer to IndexWarmer and makes the default impls like field data and norms warming an impl detail. Also the IndexShard doesn't depend on this class anymore, instead it accepts an Engine.Warmer as a ctor argument which delegates to the actual warmer from the index. --- .../common/settings/IndexScopedSettings.java | 4 +- .../org/elasticsearch/index/IndexService.java | 16 +- .../org/elasticsearch/index/IndexWarmer.java | 292 ++++++++++++++++++ .../index/NodeServicesProvider.java | 10 +- .../index/cache/bitset/BitsetFilterCache.java | 35 +-- .../elasticsearch/index/shard/IndexShard.java | 10 +- .../index/shard/ShadowIndexShard.java | 4 +- .../elasticsearch/indices/IndicesModule.java | 1 - .../elasticsearch/indices/IndicesWarmer.java | 131 -------- .../elasticsearch/search/SearchService.java | 206 +----------- .../elasticsearch/index/IndexModuleTests.java | 4 +- .../cache/bitset/BitSetFilterCacheTests.java | 18 +- .../index/query/AbstractQueryTestCase.java | 3 +- .../index/shard/IndexShardTests.java | 2 +- .../messy/tests/TemplateQueryParserTests.java | 3 +- .../search/MockSearchService.java | 5 +- .../elasticsearch/test/ESIntegTestCase.java | 6 +- 17 files changed, 346 insertions(+), 404 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/index/IndexWarmer.java delete mode 100644 core/src/main/java/org/elasticsearch/indices/IndicesWarmer.java diff --git a/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 86ead8c7ff9..4197da980bc 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -38,8 +38,8 @@ import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; import org.elasticsearch.index.store.FsDirectoryService; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.IndexWarmer; import org.elasticsearch.indices.cache.request.IndicesRequestCache; -import org.elasticsearch.search.SearchService; import java.util.Arrays; import java.util.Collections; @@ -132,7 +132,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS_SETTING, FsDirectoryService.INDEX_LOCK_FACTOR_SETTING, EngineConfig.INDEX_CODEC_SETTING, - SearchService.INDEX_NORMS_LOADING_SETTING, + IndexWarmer.INDEX_NORMS_LOADING_SETTING, // this sucks but we can't really validate all the analyzers/similarity in here Setting.groupSetting("index.similarity.", false, Setting.Scope.INDEX), // this allows similarity settings to be passed Setting.groupSetting("index.analysis.", false, Setting.Scope.INDEX) // this allows analysis settings to be passed diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index af800caf1c8..86925c35a18 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -54,6 +54,7 @@ import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.cache.query.QueryCache; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineClosedException; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.fielddata.FieldDataType; @@ -102,6 +103,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC private final MapperService mapperService; private final SimilarityService similarityService; private final EngineFactory engineFactory; + private final IndexWarmer warmer; private volatile Map shards = emptyMap(); private final AtomicBoolean closed = new AtomicBoolean(false); private final AtomicBoolean deleted = new AtomicBoolean(false); @@ -137,7 +139,8 @@ public final class IndexService extends AbstractIndexComponent implements IndexC this.nodeServicesProvider = nodeServicesProvider; this.indexStore = indexStore; indexFieldData.setListener(new FieldDataCacheListener(this)); - this.bitsetFilterCache = new BitsetFilterCache(indexSettings, nodeServicesProvider.getWarmer(), new BitsetCacheListener(this)); + this.bitsetFilterCache = new BitsetFilterCache(indexSettings, new BitsetCacheListener(this)); + this.warmer = new IndexWarmer(indexSettings.getSettings(), nodeServicesProvider.getThreadPool(), bitsetFilterCache.createListener(nodeServicesProvider.getThreadPool())); this.indexCache = new IndexCache(indexSettings, queryCache, bitsetFilterCache); this.engineFactory = engineFactory; // initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE @@ -312,11 +315,18 @@ public final class IndexService extends AbstractIndexComponent implements IndexC // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary. final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false || (primary && IndexMetaData.isOnSharedFilesystem(indexSettings)); + final Engine.Warmer engineWarmer = (searcher, toLevel) -> { + IndexShard shard = getShardOrNull(shardId.getId()); + if (shard != null) { + warmer.warm(searcher, shard, IndexService.this.indexSettings, toLevel); + } + }; + store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> nodeServicesProvider.getIndicesQueryCache().onClose(shardId))); if (useShadowEngine(primary, indexSettings)) { - indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, searchSlowLog); // no indexing listeners - shadow engines don't index + indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, searchSlowLog, engineWarmer); // no indexing listeners - shadow engines don't index } else { - indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, searchSlowLog, listeners); + indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, searchSlowLog, engineWarmer, listeners); } eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); diff --git a/core/src/main/java/org/elasticsearch/index/IndexWarmer.java b/core/src/main/java/org/elasticsearch/index/IndexWarmer.java new file mode 100644 index 00000000000..9fabc8efc40 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/IndexWarmer.java @@ -0,0 +1,292 @@ +/* + * 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; + +import com.carrotsearch.hppc.ObjectHashSet; +import com.carrotsearch.hppc.ObjectSet; +import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataService; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.FieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardState; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; + +/** + */ +public final class IndexWarmer extends AbstractComponent { + + public static final Setting INDEX_NORMS_LOADING_SETTING = new Setting<>("index.norms.loading", + MappedFieldType.Loading.LAZY.toString(), (s) -> MappedFieldType.Loading.parse(s, MappedFieldType.Loading.LAZY), + false, Setting.Scope.INDEX); + private final List listeners; + + IndexWarmer(Settings settings, ThreadPool threadPool, Listener... listeners) { + super(settings); + ArrayList list = new ArrayList<>(); + final Executor executor = threadPool.executor(ThreadPool.Names.WARMER); + list.add(new NormsWarmer(executor)); + list.add(new FieldDataWarmer(executor)); + for (Listener listener : listeners) { + list.add(listener); + } + this.listeners = Collections.unmodifiableList(list); + } + + void warm(Engine.Searcher searcher, IndexShard shard, IndexSettings settings, boolean isTopReader) { + if (shard.state() == IndexShardState.CLOSED) { + return; + } + if (settings.isWarmerEnabled() == false) { + return; + } + if (logger.isTraceEnabled()) { + if (isTopReader) { + logger.trace("{} top warming [{}]", shard.shardId(), searcher.reader()); + } else { + logger.trace("{} warming [{}]", shard.shardId(), searcher.reader()); + } + } + shard.warmerService().onPreWarm(); + long time = System.nanoTime(); + final List terminationHandles = new ArrayList<>(); + // get a handle on pending tasks + for (final Listener listener : listeners) { + if (isTopReader) { + terminationHandles.add(listener.warmTopReader(shard, searcher)); + } else { + terminationHandles.add(listener.warmNewReaders(shard, searcher)); + } + } + // wait for termination + for (TerminationHandle terminationHandle : terminationHandles) { + try { + terminationHandle.awaitTermination(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + if (isTopReader) { + logger.warn("top warming has been interrupted", e); + } else { + logger.warn("warming has been interrupted", e); + } + break; + } + } + long took = System.nanoTime() - time; + shard.warmerService().onPostWarm(took); + if (shard.warmerService().logger().isTraceEnabled()) { + if (isTopReader) { + shard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS)); + } else { + shard.warmerService().logger().trace("warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS)); + } + } + } + + /** A handle on the execution of warm-up action. */ + public interface TerminationHandle { + + TerminationHandle NO_WAIT = () -> {}; + + /** Wait until execution of the warm-up action completes. */ + void awaitTermination() throws InterruptedException; + } + public interface Listener { + /** Queue tasks to warm-up the given segments and return handles that allow to wait for termination of the + * execution of those tasks. */ + TerminationHandle warmNewReaders(IndexShard indexShard, Engine.Searcher searcher); + + TerminationHandle warmTopReader(IndexShard indexShard, Engine.Searcher searcher); + } + + private static class NormsWarmer implements IndexWarmer.Listener { + private final Executor executor; + public NormsWarmer(Executor executor) { + this.executor = executor; + } + @Override + public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) { + final MappedFieldType.Loading defaultLoading = indexShard.indexSettings().getValue(INDEX_NORMS_LOADING_SETTING); + final MapperService mapperService = indexShard.mapperService(); + final ObjectSet warmUp = new ObjectHashSet<>(); + for (DocumentMapper docMapper : mapperService.docMappers(false)) { + for (FieldMapper fieldMapper : docMapper.mappers()) { + final String indexName = fieldMapper.fieldType().name(); + MappedFieldType.Loading normsLoading = fieldMapper.fieldType().normsLoading(); + if (normsLoading == null) { + normsLoading = defaultLoading; + } + if (fieldMapper.fieldType().indexOptions() != IndexOptions.NONE && !fieldMapper.fieldType().omitNorms() + && normsLoading == MappedFieldType.Loading.EAGER) { + warmUp.add(indexName); + } + } + } + + final CountDownLatch latch = new CountDownLatch(1); + // Norms loading may be I/O intensive but is not CPU intensive, so we execute it in a single task + executor.execute(() -> { + try { + for (ObjectCursor stringObjectCursor : warmUp) { + final String indexName = stringObjectCursor.value; + final long start = System.nanoTime(); + for (final LeafReaderContext ctx : searcher.reader().leaves()) { + final NumericDocValues values = ctx.reader().getNormValues(indexName); + if (values != null) { + values.get(0); + } + } + if (indexShard.warmerService().logger().isTraceEnabled()) { + indexShard.warmerService().logger().trace("warmed norms for [{}], took [{}]", indexName, + TimeValue.timeValueNanos(System.nanoTime() - start)); + } + } + } catch (Throwable t) { + indexShard.warmerService().logger().warn("failed to warm-up norms", t); + } finally { + latch.countDown(); + } + }); + + return () -> latch.await(); + } + + @Override + public TerminationHandle warmTopReader(IndexShard indexShard, final Engine.Searcher searcher) { + return TerminationHandle.NO_WAIT; + } + } + + private static class FieldDataWarmer implements IndexWarmer.Listener { + + private final Executor executor; + public FieldDataWarmer(Executor executor) { + this.executor = executor; + } + + @Override + public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) { + final MapperService mapperService = indexShard.mapperService(); + final Map warmUp = new HashMap<>(); + for (DocumentMapper docMapper : mapperService.docMappers(false)) { + for (FieldMapper fieldMapper : docMapper.mappers()) { + final FieldDataType fieldDataType = fieldMapper.fieldType().fieldDataType(); + final String indexName = fieldMapper.fieldType().name(); + if (fieldDataType == null) { + continue; + } + if (fieldDataType.getLoading() == MappedFieldType.Loading.LAZY) { + continue; + } + + if (warmUp.containsKey(indexName)) { + continue; + } + warmUp.put(indexName, fieldMapper.fieldType()); + } + } + final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService(); + final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size()); + for (final LeafReaderContext ctx : searcher.reader().leaves()) { + for (final MappedFieldType fieldType : warmUp.values()) { + executor.execute(() -> { + try { + final long start = System.nanoTime(); + indexFieldDataService.getForField(fieldType).load(ctx); + if (indexShard.warmerService().logger().isTraceEnabled()) { + indexShard.warmerService().logger().trace("warmed fielddata for [{}], took [{}]", fieldType.name(), + TimeValue.timeValueNanos(System.nanoTime() - start)); + } + } catch (Throwable t) { + indexShard.warmerService().logger().warn("failed to warm-up fielddata for [{}]", t, fieldType.name()); + } finally { + latch.countDown(); + } + }); + } + } + return () -> latch.await(); + } + + @Override + public TerminationHandle warmTopReader(final IndexShard indexShard, final Engine.Searcher searcher) { + final MapperService mapperService = indexShard.mapperService(); + final Map warmUpGlobalOrdinals = new HashMap<>(); + for (DocumentMapper docMapper : mapperService.docMappers(false)) { + for (FieldMapper fieldMapper : docMapper.mappers()) { + final FieldDataType fieldDataType = fieldMapper.fieldType().fieldDataType(); + final String indexName = fieldMapper.fieldType().name(); + if (fieldDataType == null) { + continue; + } + if (fieldDataType.getLoading() != MappedFieldType.Loading.EAGER_GLOBAL_ORDINALS) { + continue; + } + if (warmUpGlobalOrdinals.containsKey(indexName)) { + continue; + } + warmUpGlobalOrdinals.put(indexName, fieldMapper.fieldType()); + } + } + final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService(); + final CountDownLatch latch = new CountDownLatch(warmUpGlobalOrdinals.size()); + for (final MappedFieldType fieldType : warmUpGlobalOrdinals.values()) { + executor.execute(() -> { + try { + final long start = System.nanoTime(); + IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType); + ifd.loadGlobal(searcher.getDirectoryReader()); + if (indexShard.warmerService().logger().isTraceEnabled()) { + indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.name(), + TimeValue.timeValueNanos(System.nanoTime() - start)); + } + } catch (Throwable t) { + indexShard.warmerService().logger().warn("failed to warm-up global ordinals for [{}]", t, fieldType.name()); + } finally { + latch.countDown(); + } + }); + } + return () -> latch.await(); + } + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java b/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java index d2b4894a5e6..4bb25214708 100644 --- a/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java +++ b/core/src/main/java/org/elasticsearch/index/NodeServicesProvider.java @@ -20,10 +20,8 @@ package org.elasticsearch.index; import org.elasticsearch.client.Client; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.query.IndicesQueriesRegistry; @@ -39,7 +37,6 @@ public final class NodeServicesProvider { private final ThreadPool threadPool; private final IndicesQueryCache indicesQueryCache; - private final IndicesWarmer warmer; private final BigArrays bigArrays; private final Client client; private final IndicesQueriesRegistry indicesQueriesRegistry; @@ -47,10 +44,9 @@ public final class NodeServicesProvider { private final CircuitBreakerService circuitBreakerService; @Inject - public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, @Nullable IndicesWarmer warmer, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, CircuitBreakerService circuitBreakerService) { + public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, BigArrays bigArrays, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, CircuitBreakerService circuitBreakerService) { this.threadPool = threadPool; this.indicesQueryCache = indicesQueryCache; - this.warmer = warmer; this.bigArrays = bigArrays; this.client = client; this.indicesQueriesRegistry = indicesQueriesRegistry; @@ -66,10 +62,6 @@ public final class NodeServicesProvider { return indicesQueryCache; } - public IndicesWarmer getWarmer() { - return warmer; - } - public BigArrays getBigArrays() { return bigArrays; } public Client getClient() { diff --git a/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java b/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java index 52858416ee0..7d5540b6224 100644 --- a/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java +++ b/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java @@ -48,8 +48,9 @@ import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; -import org.elasticsearch.indices.IndicesWarmer; -import org.elasticsearch.indices.IndicesWarmer.TerminationHandle; +import org.elasticsearch.index.IndexWarmer; +import org.elasticsearch.index.IndexWarmer.TerminationHandle; +import org.elasticsearch.threadpool.ThreadPool; import java.io.Closeable; import java.io.IOException; @@ -74,22 +75,20 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L private final boolean loadRandomAccessFiltersEagerly; private final Cache> loadedFilters; private final Listener listener; - private final BitSetProducerWarmer warmer; - private final IndicesWarmer indicesWarmer; - public BitsetFilterCache(IndexSettings indexSettings, IndicesWarmer indicesWarmer, Listener listener) { + public BitsetFilterCache(IndexSettings indexSettings, Listener listener) { super(indexSettings); if (listener == null) { throw new IllegalArgumentException("listener must not be null"); } this.loadRandomAccessFiltersEagerly = this.indexSettings.getValue(INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING); this.loadedFilters = CacheBuilder.>builder().removalListener(this).build(); - this.warmer = new BitSetProducerWarmer(); - this.indicesWarmer = indicesWarmer; - indicesWarmer.addListener(warmer); this.listener = listener; } + public IndexWarmer.Listener createListener(ThreadPool threadPool) { + return new BitSetProducerWarmer(threadPool); + } public BitSetProducer getBitSetProducer(Query query) { @@ -103,11 +102,7 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L @Override public void close() { - try { - indicesWarmer.removeListener(warmer); - } finally { - clear("close"); - } + clear("close"); } public void clear(String reason) { @@ -210,10 +205,16 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L } } - final class BitSetProducerWarmer implements IndicesWarmer.Listener { + final class BitSetProducerWarmer implements IndexWarmer.Listener { + + private final Executor executor; + + BitSetProducerWarmer(ThreadPool threadPool) { + this.executor = threadPool.executor(ThreadPool.Names.WARMER); + } @Override - public IndicesWarmer.TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) { + public IndexWarmer.TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) { if (indexSettings.getIndex().equals(indexShard.getIndexSettings().getIndex()) == false) { // this is from a different index return TerminationHandle.NO_WAIT; @@ -244,7 +245,6 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L warmUp.add(Queries.newNonNestedFilter()); } - final Executor executor = indicesWarmer.getExecutor(); final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size()); for (final LeafReaderContext ctx : searcher.reader().leaves()) { for (final Query filterToWarm : warmUp) { @@ -277,7 +277,6 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L return loadedFilters; } - /** * A listener interface that is executed for each onCache / onRemoval event */ @@ -295,6 +294,4 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L */ void onRemoval(ShardId shardId, Accountable accountable); } - - } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 89b6efe7076..705d4d5aa5a 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -105,7 +105,6 @@ import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.index.warmer.ShardIndexWarmerService; import org.elasticsearch.index.warmer.WarmerStats; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.IndexingMemoryController; import org.elasticsearch.indices.recovery.RecoveryFailedException; @@ -151,7 +150,7 @@ public class IndexShard extends AbstractIndexShardComponent { private final Object mutex = new Object(); private final String checkIndexOnStartup; private final CodecService codecService; - private final IndicesWarmer warmer; + private final Engine.Warmer warmer; private final SnapshotDeletionPolicy deletionPolicy; private final SimilarityService similarityService; private final EngineConfig engineConfig; @@ -208,12 +207,12 @@ public class IndexShard extends AbstractIndexShardComponent { public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory, - IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider, SearchSlowLog slowLog, IndexingOperationListener... listeners) { + IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider, SearchSlowLog slowLog, Engine.Warmer warmer, IndexingOperationListener... listeners) { super(shardId, indexSettings); final Settings settings = indexSettings.getSettings(); this.idxSettings = indexSettings; this.codecService = new CodecService(mapperService, logger); - this.warmer = provider.getWarmer(); + this.warmer = warmer; this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); this.similarityService = similarityService; Objects.requireNonNull(store, "Store must be provided to the index shard"); @@ -1389,9 +1388,8 @@ public class IndexShard extends AbstractIndexShardComponent { recoveryState.getTranslog().incrementRecoveredOperations(); } }; - final Engine.Warmer engineWarmer = (searcher, toLevel) -> warmer.warm(searcher, this, idxSettings, toLevel); return new EngineConfig(shardId, - threadPool, indexSettings, engineWarmer, store, deletionPolicy, indexSettings.getMergePolicy(), + threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(), mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, idxSettings.getSettings().getAsTime(IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING, IndexingMemoryController.SHARD_DEFAULT_INACTIVE_TIME)); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index e3600291318..355e4ee0cad 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -45,8 +45,8 @@ import java.io.IOException; public final class ShadowIndexShard extends IndexShard { public ShadowIndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory, - IndexEventListener indexEventListener, IndexSearcherWrapper wrapper, NodeServicesProvider provider, SearchSlowLog searchSlowLog) throws IOException { - super(shardId, indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldDataService, engineFactory, indexEventListener, wrapper, provider, searchSlowLog); + IndexEventListener indexEventListener, IndexSearcherWrapper wrapper, NodeServicesProvider provider, SearchSlowLog searchSlowLog, Engine.Warmer engineWarmer) throws IOException { + super(shardId, indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldDataService, engineFactory, indexEventListener, wrapper, provider, searchSlowLog, engineWarmer); } /** diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java index e66cd3e721c..f14d708c0f9 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -164,7 +164,6 @@ public class IndicesModule extends AbstractModule { bind(IndicesRequestCache.class).asEagerSingleton(); bind(TransportNodesListShardStoreMetaData.class).asEagerSingleton(); bind(IndicesTTLService.class).asEagerSingleton(); - bind(IndicesWarmer.class).asEagerSingleton(); bind(UpdateHelper.class).asEagerSingleton(); bind(MetaDataIndexUpgradeService.class).asEagerSingleton(); bind(NodeServicesProvider.class).asEagerSingleton(); diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesWarmer.java b/core/src/main/java/org/elasticsearch/indices/IndicesWarmer.java deleted file mode 100644 index c3364de9d5b..00000000000 --- a/core/src/main/java/org/elasticsearch/indices/IndicesWarmer.java +++ /dev/null @@ -1,131 +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.indices; - -import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardState; -import org.elasticsearch.threadpool.ThreadPool; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.Executor; -import java.util.concurrent.TimeUnit; - -/** - */ -public final class IndicesWarmer extends AbstractComponent { - - private final ThreadPool threadPool; - - private final CopyOnWriteArrayList listeners = new CopyOnWriteArrayList<>(); - - @Inject - public IndicesWarmer(Settings settings, ThreadPool threadPool) { - super(settings); - this.threadPool = threadPool; - } - - public void addListener(Listener listener) { - listeners.add(listener); - } - public void removeListener(Listener listener) { - listeners.remove(listener); - } - - public void warm(Engine.Searcher searcher, IndexShard shard, IndexSettings settings, boolean isTopReader) { - if (shard.state() == IndexShardState.CLOSED) { - return; - } - if (settings.isWarmerEnabled() == false) { - return; - } - if (logger.isTraceEnabled()) { - if (isTopReader) { - logger.trace("{} top warming [{}]", shard.shardId(), searcher.reader()); - } else { - logger.trace("{} warming [{}]", shard.shardId(), searcher.reader()); - } - } - shard.warmerService().onPreWarm(); - long time = System.nanoTime(); - final List terminationHandles = new ArrayList<>(); - // get a handle on pending tasks - for (final Listener listener : listeners) { - if (isTopReader) { - terminationHandles.add(listener.warmTopReader(shard, searcher)); - } else { - terminationHandles.add(listener.warmNewReaders(shard, searcher)); - } - } - // wait for termination - for (TerminationHandle terminationHandle : terminationHandles) { - try { - terminationHandle.awaitTermination(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - if (isTopReader) { - logger.warn("top warming has been interrupted", e); - } else { - logger.warn("warming has been interrupted", e); - } - break; - } - } - long took = System.nanoTime() - time; - shard.warmerService().onPostWarm(took); - if (shard.warmerService().logger().isTraceEnabled()) { - if (isTopReader) { - shard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS)); - } else { - shard.warmerService().logger().trace("warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS)); - } - } - } - - /** - * Returns an executor for async warmer tasks - */ - public Executor getExecutor() { - return threadPool.executor(ThreadPool.Names.WARMER); - } - - /** A handle on the execution of warm-up action. */ - public interface TerminationHandle { - - TerminationHandle NO_WAIT = () -> {}; - - /** Wait until execution of the warm-up action completes. */ - void awaitTermination() throws InterruptedException; - } - public interface Listener { - /** Queue tasks to warm-up the given segments and return handles that allow to wait for termination of the execution of those tasks. */ - TerminationHandle warmNewReaders(IndexShard indexShard, Engine.Searcher searcher); - - TerminationHandle warmTopReader(IndexShard indexShard, Engine.Searcher searcher); - } - -} diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 4c47de2c9bc..0da838a799b 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -20,13 +20,6 @@ package org.elasticsearch.search; import com.carrotsearch.hppc.ObjectFloatHashMap; -import com.carrotsearch.hppc.ObjectHashSet; -import com.carrotsearch.hppc.ObjectSet; -import com.carrotsearch.hppc.cursors.ObjectCursor; - -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.TopDocs; import org.elasticsearch.ExceptionsHelper; @@ -54,14 +47,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.fielddata.FieldDataType; -import org.elasticsearch.index.fielddata.IndexFieldData; -import org.elasticsearch.index.fielddata.IndexFieldDataService; -import org.elasticsearch.index.mapper.DocumentMapper; -import org.elasticsearch.index.mapper.FieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.MappedFieldType.Loading; -import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.search.stats.ShardSearchStats; @@ -69,8 +54,6 @@ import org.elasticsearch.index.search.stats.StatsGroupsParseElement; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.IndicesWarmer; -import org.elasticsearch.indices.IndicesWarmer.TerminationHandle; import org.elasticsearch.indices.cache.request.IndicesRequestCache; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.ScriptContext; @@ -109,9 +92,7 @@ import java.io.IOException; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Executor; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicLong; @@ -124,7 +105,6 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; */ public class SearchService extends AbstractLifecycleComponent implements IndexEventListener { - public static final Setting INDEX_NORMS_LOADING_SETTING = new Setting<>("index.norms.loading", Loading.LAZY.toString(), (s) -> Loading.parse(s, Loading.LAZY), false, Setting.Scope.INDEX); // we can have 5 minutes here, since we make sure to clean with search requests and when shard/index closes public static final Setting DEFAULT_KEEPALIVE_SETTING = Setting.positiveTimeSetting("search.default_keep_alive", timeValueMinutes(5), false, Setting.Scope.CLUSTER); public static final Setting KEEPALIVE_INTERVAL_SETTING = Setting.positiveTimeSetting("search.keep_alive_interval", timeValueMinutes(1), false, Setting.Scope.CLUSTER); @@ -139,8 +119,6 @@ public class SearchService extends AbstractLifecycleComponent imp private final IndicesService indicesService; - private final IndicesWarmer indicesWarmer; - private final ScriptService scriptService; private final PageCacheRecycler pageCacheRecycler; @@ -170,7 +148,7 @@ public class SearchService extends AbstractLifecycleComponent imp private final ParseFieldMatcher parseFieldMatcher; @Inject - public SearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService, IndicesService indicesService, IndicesWarmer indicesWarmer, ThreadPool threadPool, + public SearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ScriptService scriptService, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, DfsPhase dfsPhase, QueryPhase queryPhase, FetchPhase fetchPhase, IndicesRequestCache indicesQueryCache) { super(settings); @@ -178,7 +156,6 @@ public class SearchService extends AbstractLifecycleComponent imp this.threadPool = threadPool; this.clusterService = clusterService; this.indicesService = indicesService; - this.indicesWarmer = indicesWarmer; this.scriptService = scriptService; this.pageCacheRecycler = pageCacheRecycler; this.bigArrays = bigArrays; @@ -199,9 +176,6 @@ public class SearchService extends AbstractLifecycleComponent imp this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval); - this.indicesWarmer.addListener(new NormsWarmer(indicesWarmer)); - this.indicesWarmer.addListener(new FieldDataWarmer(indicesWarmer)); - defaultSearchTimeout = DEFAULT_SEARCH_TIMEOUT_SETTING.get(settings); clusterSettings.addSettingsUpdateConsumer(DEFAULT_SEARCH_TIMEOUT_SETTING, this::setDefaultSearchTimeout); } @@ -946,184 +920,6 @@ public class SearchService extends AbstractLifecycleComponent imp return this.activeContexts.size(); } - static class NormsWarmer implements IndicesWarmer.Listener { - private final IndicesWarmer indicesWarmer; - - public NormsWarmer(IndicesWarmer indicesWarmer) { - this.indicesWarmer = indicesWarmer; - } - @Override - public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) { - final Loading defaultLoading = indexShard.indexSettings().getValue(INDEX_NORMS_LOADING_SETTING); - final MapperService mapperService = indexShard.mapperService(); - final ObjectSet warmUp = new ObjectHashSet<>(); - for (DocumentMapper docMapper : mapperService.docMappers(false)) { - for (FieldMapper fieldMapper : docMapper.mappers()) { - final String indexName = fieldMapper.fieldType().name(); - Loading normsLoading = fieldMapper.fieldType().normsLoading(); - if (normsLoading == null) { - normsLoading = defaultLoading; - } - if (fieldMapper.fieldType().indexOptions() != IndexOptions.NONE && !fieldMapper.fieldType().omitNorms() && normsLoading == Loading.EAGER) { - warmUp.add(indexName); - } - } - } - - final CountDownLatch latch = new CountDownLatch(1); - // Norms loading may be I/O intensive but is not CPU intensive, so we execute it in a single task - indicesWarmer.getExecutor().execute(new Runnable() { - @Override - public void run() { - try { - for (ObjectCursor stringObjectCursor : warmUp) { - final String indexName = stringObjectCursor.value; - final long start = System.nanoTime(); - for (final LeafReaderContext ctx : searcher.reader().leaves()) { - final NumericDocValues values = ctx.reader().getNormValues(indexName); - if (values != null) { - values.get(0); - } - } - if (indexShard.warmerService().logger().isTraceEnabled()) { - indexShard.warmerService().logger().trace("warmed norms for [{}], took [{}]", indexName, TimeValue.timeValueNanos(System.nanoTime() - start)); - } - } - } catch (Throwable t) { - indexShard.warmerService().logger().warn("failed to warm-up norms", t); - } finally { - latch.countDown(); - } - } - }); - - return new TerminationHandle() { - @Override - public void awaitTermination() throws InterruptedException { - latch.await(); - } - }; - } - - @Override - public TerminationHandle warmTopReader(IndexShard indexShard, final Engine.Searcher searcher) { - return TerminationHandle.NO_WAIT; - } - } - - static class FieldDataWarmer implements IndicesWarmer.Listener { - - private final IndicesWarmer indicesWarmer; - - public FieldDataWarmer(IndicesWarmer indicesWarmer) { - this.indicesWarmer = indicesWarmer; - } - - @Override - public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) { - final MapperService mapperService = indexShard.mapperService(); - final Map warmUp = new HashMap<>(); - for (DocumentMapper docMapper : mapperService.docMappers(false)) { - for (FieldMapper fieldMapper : docMapper.mappers()) { - final FieldDataType fieldDataType = fieldMapper.fieldType().fieldDataType(); - final String indexName = fieldMapper.fieldType().name(); - if (fieldDataType == null) { - continue; - } - if (fieldDataType.getLoading() == Loading.LAZY) { - continue; - } - - if (warmUp.containsKey(indexName)) { - continue; - } - warmUp.put(indexName, fieldMapper.fieldType()); - } - } - final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService(); - final Executor executor = indicesWarmer.getExecutor(); - final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size()); - for (final LeafReaderContext ctx : searcher.reader().leaves()) { - for (final MappedFieldType fieldType : warmUp.values()) { - executor.execute(new Runnable() { - - @Override - public void run() { - try { - final long start = System.nanoTime(); - indexFieldDataService.getForField(fieldType).load(ctx); - if (indexShard.warmerService().logger().isTraceEnabled()) { - indexShard.warmerService().logger().trace("warmed fielddata for [{}], took [{}]", fieldType.name(), TimeValue.timeValueNanos(System.nanoTime() - start)); - } - } catch (Throwable t) { - indexShard.warmerService().logger().warn("failed to warm-up fielddata for [{}]", t, fieldType.name()); - } finally { - latch.countDown(); - } - } - - }); - } - } - return new TerminationHandle() { - @Override - public void awaitTermination() throws InterruptedException { - latch.await(); - } - }; - } - - @Override - public TerminationHandle warmTopReader(final IndexShard indexShard, final Engine.Searcher searcher) { - final MapperService mapperService = indexShard.mapperService(); - final Map warmUpGlobalOrdinals = new HashMap<>(); - for (DocumentMapper docMapper : mapperService.docMappers(false)) { - for (FieldMapper fieldMapper : docMapper.mappers()) { - final FieldDataType fieldDataType = fieldMapper.fieldType().fieldDataType(); - final String indexName = fieldMapper.fieldType().name(); - if (fieldDataType == null) { - continue; - } - if (fieldDataType.getLoading() != Loading.EAGER_GLOBAL_ORDINALS) { - continue; - } - if (warmUpGlobalOrdinals.containsKey(indexName)) { - continue; - } - warmUpGlobalOrdinals.put(indexName, fieldMapper.fieldType()); - } - } - final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService(); - final Executor executor = indicesWarmer.getExecutor(); - final CountDownLatch latch = new CountDownLatch(warmUpGlobalOrdinals.size()); - for (final MappedFieldType fieldType : warmUpGlobalOrdinals.values()) { - executor.execute(new Runnable() { - @Override - public void run() { - try { - final long start = System.nanoTime(); - IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType); - ifd.loadGlobal(searcher.getDirectoryReader()); - if (indexShard.warmerService().logger().isTraceEnabled()) { - indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.name(), TimeValue.timeValueNanos(System.nanoTime() - start)); - } - } catch (Throwable t) { - indexShard.warmerService().logger().warn("failed to warm-up global ordinals for [{}]", t, fieldType.name()); - } finally { - latch.countDown(); - } - } - }); - } - return new TerminationHandle() { - @Override - public void awaitTermination() throws InterruptedException { - latch.await(); - } - }; - } - } - class Reaper implements Runnable { @Override public void run() { diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index f2245cc7de0..0b0691bc588 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -53,7 +53,6 @@ import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStoreConfig; import org.elasticsearch.indices.IndicesModule; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.indices.cache.query.IndicesQueryCache; @@ -104,7 +103,6 @@ public class IndexModuleTests extends ESTestCase { static NodeServicesProvider newNodeServiceProvider(Settings settings, Environment environment, Client client, ScriptEngineService... scriptEngineServices) throws IOException { // TODO this can be used in other place too - lets first refactor the IndicesQueriesRegistry ThreadPool threadPool = new ThreadPool("test"); - IndicesWarmer warmer = new IndicesWarmer(settings, threadPool); IndicesQueryCache indicesQueryCache = new IndicesQueryCache(settings); CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService(); PageCacheRecycler recycler = new PageCacheRecycler(settings, threadPool); @@ -116,7 +114,7 @@ public class IndexModuleTests extends ESTestCase { ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry); ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), scriptEngineRegistry, scriptContextRegistry, scriptSettings); IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, emptyMap()); - return new NodeServicesProvider(threadPool, indicesQueryCache, warmer, bigArrays, client, scriptService, indicesQueriesRegistry, circuitBreakerService); + return new NodeServicesProvider(threadPool, indicesQueryCache, bigArrays, client, scriptService, indicesQueriesRegistry, circuitBreakerService); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java b/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java index 4f2f12e6cbb..18714fe61ef 100644 --- a/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java +++ b/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java @@ -41,10 +41,8 @@ import org.apache.lucene.util.BitSet; import org.apache.lucene.util.IOUtils; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; @@ -57,8 +55,6 @@ import static org.hamcrest.Matchers.equalTo; public class BitSetFilterCacheTests extends ESTestCase { private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings("test", Settings.EMPTY); - private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null); - private static int matchCount(BitSetProducer producer, IndexReader reader) throws IOException { int count = 0; @@ -95,7 +91,7 @@ public class BitSetFilterCacheTests extends ESTestCase { reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId("test", "_na_", 0)); IndexSearcher searcher = new IndexSearcher(reader); - BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() { + BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { @@ -149,7 +145,7 @@ public class BitSetFilterCacheTests extends ESTestCase { final AtomicInteger onCacheCalls = new AtomicInteger(); final AtomicInteger onRemoveCalls = new AtomicInteger(); - final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() { + final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { onCacheCalls.incrementAndGet(); @@ -188,7 +184,7 @@ public class BitSetFilterCacheTests extends ESTestCase { public void testSetNullListener() { try { - new BitsetFilterCache(INDEX_SETTINGS, warmer, null); + new BitsetFilterCache(INDEX_SETTINGS, null); fail("listener can't be null"); } catch (IllegalArgumentException ex) { assertEquals("listener must not be null", ex.getMessage()); @@ -197,7 +193,7 @@ public class BitSetFilterCacheTests extends ESTestCase { } public void testRejectOtherIndex() throws IOException { - BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() { + BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { @@ -208,7 +204,7 @@ public class BitSetFilterCacheTests extends ESTestCase { } }); - + Directory dir = newDirectory(); IndexWriter writer = new IndexWriter( dir, @@ -218,9 +214,9 @@ public class BitSetFilterCacheTests extends ESTestCase { DirectoryReader reader = DirectoryReader.open(writer, true); writer.close(); reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId("test2", "_na_", 0)); - + BitSetProducer producer = cache.getBitSetProducer(new MatchAllDocsQuery()); - + try { producer.getBitSet(reader.leaves().get(0)); fail(); diff --git a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java index 29d556c9a8a..2435a34080d 100644 --- a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java @@ -78,7 +78,6 @@ import org.elasticsearch.index.query.support.QueryParsers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; @@ -275,7 +274,7 @@ public abstract class AbstractQueryTestCase> IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() { }); indexFieldDataService = new IndexFieldDataService(idxSettings, indicesFieldDataCache, injector.getInstance(CircuitBreakerService.class), mapperService); - BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() { + BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index e1c0d6c3bed..ec2f8f32a20 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -1154,7 +1154,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { ShardRouting routing = new ShardRouting(shard.routingEntry()); shard.close("simon says", true); NodeServicesProvider indexServices = indexService.getIndexServices(); - IndexShard newShard = new IndexShard(shard.shardId(), indexService.getIndexSettings(), shard.shardPath(), shard.store(), indexService.cache(), indexService.mapperService(), indexService.similarityService(), indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, indexServices, indexService.getSearchSlowLog(), listeners); + IndexShard newShard = new IndexShard(shard.shardId(), indexService.getIndexSettings(), shard.shardPath(), shard.store(), indexService.cache(), indexService.mapperService(), indexService.similarityService(), indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, indexServices, indexService.getSearchSlowLog(), null, listeners); ShardRoutingHelper.reinit(routing); newShard.updateRoutingEntry(routing, false); DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT); diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java index e0d8fc3a506..86667515bd6 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java @@ -52,7 +52,6 @@ import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.IndicesModule; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; @@ -141,7 +140,7 @@ public class TemplateQueryParserTests extends ESTestCase { MapperService mapperService = new MapperService(idxSettings, analysisService, similarityService, mapperRegistry, () -> context); IndicesFieldDataCache cache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() {}); IndexFieldDataService indexFieldDataService =new IndexFieldDataService(idxSettings, cache, injector.getInstance(CircuitBreakerService.class), mapperService); - BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() { + BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new BitsetFilterCache.Listener() { @Override public void onCache(ShardId shardId, Accountable accountable) { diff --git a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java index 98b5181636d..7cc583273b9 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java +++ b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.cache.request.IndicesRequestCache; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; @@ -67,10 +66,10 @@ public class MockSearchService extends SearchService { } @Inject - public MockSearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService, IndicesService indicesService, IndicesWarmer indicesWarmer, + public MockSearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ScriptService scriptService, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, DfsPhase dfsPhase, QueryPhase queryPhase, FetchPhase fetchPhase, IndicesRequestCache indicesQueryCache) { - super(settings, clusterSettings, clusterService, indicesService, indicesWarmer, threadPool, scriptService, pageCacheRecycler, bigArrays, dfsPhase, + super(settings, clusterSettings, clusterService, indicesService, threadPool, scriptService, pageCacheRecycler, bigArrays, dfsPhase, queryPhase, fetchPhase, indicesQueryCache); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 39508644361..d24baeb6c10 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -47,7 +47,6 @@ import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse; import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; @@ -108,6 +107,7 @@ import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.MergeSchedulerConfig; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.index.IndexWarmer; import org.elasticsearch.indices.cache.request.IndicesRequestCache; import org.elasticsearch.indices.store.IndicesStore; import org.elasticsearch.node.NodeMocksPlugin; @@ -115,7 +115,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.MockSearchService; import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchService; import org.elasticsearch.test.client.RandomizingClient; import org.elasticsearch.test.disruption.ServiceDisruptionScheme; import org.elasticsearch.test.rest.client.http.HttpRequestBuilder; @@ -129,7 +128,6 @@ import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; -import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.lang.annotation.Annotation; @@ -510,7 +508,7 @@ public abstract class ESIntegTestCase extends ESTestCase { private static Settings.Builder setRandomIndexNormsLoading(Random random, Settings.Builder builder) { if (random.nextBoolean()) { - builder.put(SearchService.INDEX_NORMS_LOADING_SETTING.getKey(), RandomPicks.randomFrom(random, Arrays.asList(MappedFieldType.Loading.EAGER, MappedFieldType.Loading.LAZY))); + builder.put(IndexWarmer.INDEX_NORMS_LOADING_SETTING.getKey(), RandomPicks.randomFrom(random, Arrays.asList(MappedFieldType.Loading.EAGER, MappedFieldType.Loading.LAZY))); } return builder; } From f6f2d40fd5c8fdad27871d4d537999f1d425aa95 Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Fri, 5 Feb 2016 11:59:40 -0800 Subject: [PATCH 13/19] Minor clean up. * Minor clean up of Writer constants. * Removed synthetic attribute from the generated constructor and method. * Added a safeguard for maximum script length. Closes #16457 --- .../org/elasticsearch/painless/Compiler.java | 11 + .../org/elasticsearch/painless/Utility.java | 4 +- .../org/elasticsearch/painless/Writer.java | 332 ++++++++---------- .../painless/WhenThingsGoWrongTests.java | 18 + 4 files changed, 178 insertions(+), 187 deletions(-) diff --git a/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java b/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java index efb6b22f298..d1b0e2dc6fe 100644 --- a/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java +++ b/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java @@ -37,6 +37,11 @@ import java.security.cert.Certificate; * the {@link Writer}. */ final class Compiler { + /** + * The maximum number of characters allowed in the script source. + */ + static int MAXIMUM_SOURCE_LENGTH = 16384; + /** * The default language API to be used with Painless. The second construction is used * to finalize all the variables, so there is no mistake of modification afterwards. @@ -92,6 +97,12 @@ final class Compiler { */ static Executable compile(final Loader loader, final String name, final String source, final Definition custom, final CompilerSettings settings) { + if (source.length() > MAXIMUM_SOURCE_LENGTH) { + throw new IllegalArgumentException("Scripts may be no longer than " + MAXIMUM_SOURCE_LENGTH + + " characters. The passed in script is " + source.length() + " characters. Consider using a" + + " plugin if a script longer than this length is a requirement."); + } + final Definition definition = custom != null ? new Definition(custom) : DEFAULT_DEFINITION; final ParserRuleContext root = createParseTree(source, definition); final Metadata metadata = new Metadata(definition, source, root, settings); diff --git a/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Utility.java b/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Utility.java index 6eed17ff983..45c507b42ba 100644 --- a/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Utility.java +++ b/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Utility.java @@ -628,7 +628,7 @@ public class Utility { /** * Like {@link Math#toIntExact(long)} but for long range. */ - public static long toLongExactWithoutOverflow(float value) { + public static long toLongWithoutOverflow(float value) { if (value < Long.MIN_VALUE || value > Long.MAX_VALUE) { throw new ArithmeticException("long overflow"); } @@ -638,7 +638,7 @@ public class Utility { /** * Like {@link Math#toIntExact(long)} but for long range. */ - public static float toLongExactWithoutOverflow(double value) { + public static float toLongWithoutOverflow(double value) { if (value < Long.MIN_VALUE || value > Long.MAX_VALUE) { throw new ArithmeticException("long overflow"); } diff --git a/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java b/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java index 1abc2894a13..431e724127f 100644 --- a/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java +++ b/plugins/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java @@ -28,12 +28,67 @@ import org.elasticsearch.painless.Definition.Method; import org.elasticsearch.painless.Definition.Sort; import org.elasticsearch.painless.Definition.Transform; import org.elasticsearch.painless.Definition.Type; +import org.elasticsearch.painless.PainlessParser.AfterthoughtContext; +import org.elasticsearch.painless.PainlessParser.ArgumentsContext; +import org.elasticsearch.painless.PainlessParser.AssignmentContext; +import org.elasticsearch.painless.PainlessParser.BinaryContext; +import org.elasticsearch.painless.PainlessParser.BlockContext; +import org.elasticsearch.painless.PainlessParser.BoolContext; +import org.elasticsearch.painless.PainlessParser.BreakContext; +import org.elasticsearch.painless.PainlessParser.CastContext; +import org.elasticsearch.painless.PainlessParser.CharContext; +import org.elasticsearch.painless.PainlessParser.CompContext; +import org.elasticsearch.painless.PainlessParser.ConditionalContext; +import org.elasticsearch.painless.PainlessParser.ContinueContext; +import org.elasticsearch.painless.PainlessParser.DeclContext; +import org.elasticsearch.painless.PainlessParser.DeclarationContext; +import org.elasticsearch.painless.PainlessParser.DecltypeContext; +import org.elasticsearch.painless.PainlessParser.DeclvarContext; +import org.elasticsearch.painless.PainlessParser.DoContext; +import org.elasticsearch.painless.PainlessParser.EmptyContext; +import org.elasticsearch.painless.PainlessParser.EmptyscopeContext; +import org.elasticsearch.painless.PainlessParser.ExprContext; +import org.elasticsearch.painless.PainlessParser.ExpressionContext; +import org.elasticsearch.painless.PainlessParser.ExtbraceContext; +import org.elasticsearch.painless.PainlessParser.ExtcallContext; +import org.elasticsearch.painless.PainlessParser.ExtcastContext; +import org.elasticsearch.painless.PainlessParser.ExtdotContext; +import org.elasticsearch.painless.PainlessParser.ExternalContext; +import org.elasticsearch.painless.PainlessParser.ExtfieldContext; +import org.elasticsearch.painless.PainlessParser.ExtnewContext; +import org.elasticsearch.painless.PainlessParser.ExtprecContext; +import org.elasticsearch.painless.PainlessParser.ExtstartContext; +import org.elasticsearch.painless.PainlessParser.ExtstringContext; +import org.elasticsearch.painless.PainlessParser.ExttypeContext; +import org.elasticsearch.painless.PainlessParser.ExtvarContext; +import org.elasticsearch.painless.PainlessParser.FalseContext; +import org.elasticsearch.painless.PainlessParser.ForContext; +import org.elasticsearch.painless.PainlessParser.IfContext; +import org.elasticsearch.painless.PainlessParser.IncrementContext; +import org.elasticsearch.painless.PainlessParser.InitializerContext; +import org.elasticsearch.painless.PainlessParser.MultipleContext; +import org.elasticsearch.painless.PainlessParser.NullContext; +import org.elasticsearch.painless.PainlessParser.NumericContext; +import org.elasticsearch.painless.PainlessParser.PostincContext; +import org.elasticsearch.painless.PainlessParser.PrecedenceContext; +import org.elasticsearch.painless.PainlessParser.PreincContext; +import org.elasticsearch.painless.PainlessParser.ReturnContext; +import org.elasticsearch.painless.PainlessParser.SingleContext; +import org.elasticsearch.painless.PainlessParser.SourceContext; +import org.elasticsearch.painless.PainlessParser.StatementContext; +import org.elasticsearch.painless.PainlessParser.ThrowContext; +import org.elasticsearch.painless.PainlessParser.TrapContext; +import org.elasticsearch.painless.PainlessParser.TrueContext; +import org.elasticsearch.painless.PainlessParser.TryContext; +import org.elasticsearch.painless.PainlessParser.UnaryContext; +import org.elasticsearch.painless.PainlessParser.WhileContext; import org.elasticsearch.script.ScoreAccessor; import org.objectweb.asm.ClassWriter; import org.objectweb.asm.Label; import org.objectweb.asm.Opcodes; import org.objectweb.asm.commons.GeneratorAdapter; +import java.lang.invoke.MethodType; import java.util.ArrayDeque; import java.util.Deque; import java.util.HashMap; @@ -43,70 +98,16 @@ import java.util.Map; import java.util.Set; import static org.elasticsearch.painless.PainlessParser.ADD; -import static org.elasticsearch.painless.PainlessParser.AfterthoughtContext; -import static org.elasticsearch.painless.PainlessParser.ArgumentsContext; -import static org.elasticsearch.painless.PainlessParser.AssignmentContext; import static org.elasticsearch.painless.PainlessParser.BWAND; import static org.elasticsearch.painless.PainlessParser.BWOR; import static org.elasticsearch.painless.PainlessParser.BWXOR; -import static org.elasticsearch.painless.PainlessParser.BinaryContext; -import static org.elasticsearch.painless.PainlessParser.BlockContext; -import static org.elasticsearch.painless.PainlessParser.BoolContext; -import static org.elasticsearch.painless.PainlessParser.BreakContext; -import static org.elasticsearch.painless.PainlessParser.CastContext; -import static org.elasticsearch.painless.PainlessParser.CharContext; -import static org.elasticsearch.painless.PainlessParser.CompContext; -import static org.elasticsearch.painless.PainlessParser.ConditionalContext; -import static org.elasticsearch.painless.PainlessParser.ContinueContext; import static org.elasticsearch.painless.PainlessParser.DIV; -import static org.elasticsearch.painless.PainlessParser.DeclContext; -import static org.elasticsearch.painless.PainlessParser.DeclarationContext; -import static org.elasticsearch.painless.PainlessParser.DecltypeContext; -import static org.elasticsearch.painless.PainlessParser.DeclvarContext; -import static org.elasticsearch.painless.PainlessParser.DoContext; -import static org.elasticsearch.painless.PainlessParser.EmptyContext; -import static org.elasticsearch.painless.PainlessParser.EmptyscopeContext; -import static org.elasticsearch.painless.PainlessParser.ExprContext; -import static org.elasticsearch.painless.PainlessParser.ExpressionContext; -import static org.elasticsearch.painless.PainlessParser.ExtbraceContext; -import static org.elasticsearch.painless.PainlessParser.ExtcallContext; -import static org.elasticsearch.painless.PainlessParser.ExtcastContext; -import static org.elasticsearch.painless.PainlessParser.ExtdotContext; -import static org.elasticsearch.painless.PainlessParser.ExternalContext; -import static org.elasticsearch.painless.PainlessParser.ExtfieldContext; -import static org.elasticsearch.painless.PainlessParser.ExtnewContext; -import static org.elasticsearch.painless.PainlessParser.ExtprecContext; -import static org.elasticsearch.painless.PainlessParser.ExtstartContext; -import static org.elasticsearch.painless.PainlessParser.ExtstringContext; -import static org.elasticsearch.painless.PainlessParser.ExttypeContext; -import static org.elasticsearch.painless.PainlessParser.ExtvarContext; -import static org.elasticsearch.painless.PainlessParser.FalseContext; -import static org.elasticsearch.painless.PainlessParser.ForContext; -import static org.elasticsearch.painless.PainlessParser.IfContext; -import static org.elasticsearch.painless.PainlessParser.IncrementContext; -import static org.elasticsearch.painless.PainlessParser.InitializerContext; import static org.elasticsearch.painless.PainlessParser.LSH; import static org.elasticsearch.painless.PainlessParser.MUL; -import static org.elasticsearch.painless.PainlessParser.MultipleContext; -import static org.elasticsearch.painless.PainlessParser.NullContext; -import static org.elasticsearch.painless.PainlessParser.NumericContext; -import static org.elasticsearch.painless.PainlessParser.PostincContext; -import static org.elasticsearch.painless.PainlessParser.PrecedenceContext; -import static org.elasticsearch.painless.PainlessParser.PreincContext; import static org.elasticsearch.painless.PainlessParser.REM; import static org.elasticsearch.painless.PainlessParser.RSH; -import static org.elasticsearch.painless.PainlessParser.ReturnContext; import static org.elasticsearch.painless.PainlessParser.SUB; -import static org.elasticsearch.painless.PainlessParser.SingleContext; -import static org.elasticsearch.painless.PainlessParser.SourceContext; -import static org.elasticsearch.painless.PainlessParser.StatementContext; -import static org.elasticsearch.painless.PainlessParser.ThrowContext; -import static org.elasticsearch.painless.PainlessParser.TrapContext; -import static org.elasticsearch.painless.PainlessParser.TrueContext; -import static org.elasticsearch.painless.PainlessParser.TryContext; import static org.elasticsearch.painless.PainlessParser.USH; -import static org.elasticsearch.painless.PainlessParser.UnaryContext; -import static org.elasticsearch.painless.PainlessParser.WhileContext; class Writer extends PainlessParserBaseVisitor { private static class Branch { @@ -125,181 +126,142 @@ class Writer extends PainlessParserBaseVisitor { final static String BASE_CLASS_NAME = Executable.class.getName(); final static String CLASS_NAME = BASE_CLASS_NAME + "$CompiledPainlessExecutable"; private final static org.objectweb.asm.Type BASE_CLASS_TYPE = org.objectweb.asm.Type.getType(Executable.class); - private final static org.objectweb.asm.Type CLASS_TYPE = - org.objectweb.asm.Type.getType("L" + CLASS_NAME.replace(".", "/") + ";"); + private final static org.objectweb.asm.Type CLASS_TYPE = org.objectweb.asm.Type.getType("L" + CLASS_NAME.replace(".", "/") + ";"); + + private final static org.objectweb.asm.commons.Method CONSTRUCTOR = + getAsmMethod(void.class, "", Definition.class, String.class, String.class); + private final static org.objectweb.asm.commons.Method EXECUTE = getAsmMethod(Object.class, "execute", Map.class); + private final static String SIGNATURE = "(Ljava/util/Map;)Ljava/lang/Object;"; private final static org.objectweb.asm.Type PAINLESS_ERROR_TYPE = org.objectweb.asm.Type.getType(PainlessError.class); - private final static org.objectweb.asm.commons.Method CONSTRUCTOR = org.objectweb.asm.commons.Method.getMethod( - "void (org.elasticsearch.painless.Definition, java.lang.String, java.lang.String)"); - private final static org.objectweb.asm.commons.Method EXECUTE = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object execute(java.util.Map)"); - private final static String SIGNATURE = "(Ljava/util/Map;)Ljava/lang/Object;"; - private final static org.objectweb.asm.Type DEFINITION_TYPE = org.objectweb.asm.Type.getType(Definition.class); private final static org.objectweb.asm.Type MAP_TYPE = org.objectweb.asm.Type.getType(Map.class); - private final static org.objectweb.asm.commons.Method MAP_GET = - org.objectweb.asm.commons.Method.getMethod("Object get(Object)"); + private final static org.objectweb.asm.commons.Method MAP_GET = getAsmMethod(Object.class, "get", Object.class); private final static org.objectweb.asm.Type SCORE_ACCESSOR_TYPE = org.objectweb.asm.Type.getType(ScoreAccessor.class); - private final static org.objectweb.asm.commons.Method SCORE_ACCESSOR_FLOAT = - org.objectweb.asm.commons.Method.getMethod("float floatValue()"); + private final static org.objectweb.asm.commons.Method SCORE_ACCESSOR_FLOAT = getAsmMethod(float.class, "floatValue"); - private final static org.objectweb.asm.commons.Method DEF_METHOD_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object methodCall(java.lang.Object, java.lang.String, " + - "org.elasticsearch.painless.Definition, java.lang.Object[], boolean[])"); - private final static org.objectweb.asm.commons.Method DEF_ARRAY_STORE = org.objectweb.asm.commons.Method.getMethod( - "void arrayStore(java.lang.Object, java.lang.Object, java.lang.Object, " + - "org.elasticsearch.painless.Definition, boolean, boolean)"); - private final static org.objectweb.asm.commons.Method DEF_ARRAY_LOAD = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object arrayLoad(java.lang.Object, java.lang.Object, " + - "org.elasticsearch.painless.Definition, boolean)"); - private final static org.objectweb.asm.commons.Method DEF_FIELD_STORE = org.objectweb.asm.commons.Method.getMethod( - "void fieldStore(java.lang.Object, java.lang.Object, java.lang.String, " + - "org.elasticsearch.painless.Definition, boolean)"); - private final static org.objectweb.asm.commons.Method DEF_FIELD_LOAD = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object fieldLoad(java.lang.Object, java.lang.String, org.elasticsearch.painless.Definition)"); + private final static org.objectweb.asm.commons.Method DEF_METHOD_CALL = getAsmMethod( + Object.class, "methodCall", Object.class, String.class, Definition.class, Object[].class, boolean[].class); + private final static org.objectweb.asm.commons.Method DEF_ARRAY_STORE = getAsmMethod( + void.class, "arrayStore", Object.class, Object.class, Object.class, Definition.class, boolean.class, boolean.class); + private final static org.objectweb.asm.commons.Method DEF_ARRAY_LOAD = getAsmMethod( + Object.class, "arrayLoad", Object.class, Object.class, Definition.class, boolean.class); + private final static org.objectweb.asm.commons.Method DEF_FIELD_STORE = getAsmMethod( + void.class, "fieldStore", Object.class, Object.class, String.class, Definition.class, boolean.class); + private final static org.objectweb.asm.commons.Method DEF_FIELD_LOAD = getAsmMethod( + Object.class, "fieldLoad", Object.class, String.class, Definition.class); - private final static org.objectweb.asm.commons.Method DEF_NOT_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object not(java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_NEG_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object neg(java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_MUL_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object mul(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_DIV_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object div(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_REM_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object rem(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_ADD_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object add(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_SUB_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object sub(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_LSH_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object lsh(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_RSH_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object rsh(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_USH_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object ush(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_AND_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object and(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_XOR_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object xor(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_OR_CALL = org.objectweb.asm.commons.Method.getMethod( - "java.lang.Object or(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_EQ_CALL = org.objectweb.asm.commons.Method.getMethod( - "boolean eq(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_LT_CALL = org.objectweb.asm.commons.Method.getMethod( - "boolean lt(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_LTE_CALL = org.objectweb.asm.commons.Method.getMethod( - "boolean lte(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_GT_CALL = org.objectweb.asm.commons.Method.getMethod( - "boolean gt(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method DEF_GTE_CALL = org.objectweb.asm.commons.Method.getMethod( - "boolean gte(java.lang.Object, java.lang.Object)"); + private final static org.objectweb.asm.commons.Method DEF_NOT_CALL = getAsmMethod(Object.class, "not", Object.class); + private final static org.objectweb.asm.commons.Method DEF_NEG_CALL = getAsmMethod(Object.class, "neg", Object.class); + private final static org.objectweb.asm.commons.Method DEF_MUL_CALL = getAsmMethod(Object.class, "mul", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_DIV_CALL = getAsmMethod(Object.class, "div", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_REM_CALL = getAsmMethod(Object.class, "rem", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_ADD_CALL = getAsmMethod(Object.class, "add", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_SUB_CALL = getAsmMethod(Object.class, "sub", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_LSH_CALL = getAsmMethod(Object.class, "lsh", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_RSH_CALL = getAsmMethod(Object.class, "rsh", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_USH_CALL = getAsmMethod(Object.class, "ush", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_AND_CALL = getAsmMethod(Object.class, "and", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_XOR_CALL = getAsmMethod(Object.class, "xor", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_OR_CALL = getAsmMethod(Object.class, "or" , Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_EQ_CALL = getAsmMethod(boolean.class, "eq" , Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_LT_CALL = getAsmMethod(boolean.class, "lt" , Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_LTE_CALL = getAsmMethod(boolean.class, "lte", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_GT_CALL = getAsmMethod(boolean.class, "gt" , Object.class, Object.class); + private final static org.objectweb.asm.commons.Method DEF_GTE_CALL = getAsmMethod(boolean.class, "gte", Object.class, Object.class); private final static org.objectweb.asm.Type STRINGBUILDER_TYPE = org.objectweb.asm.Type.getType(StringBuilder.class); - private final static org.objectweb.asm.commons.Method STRINGBUILDER_CONSTRUCTOR = - org.objectweb.asm.commons.Method.getMethod("void ()"); + private final static org.objectweb.asm.commons.Method STRINGBUILDER_CONSTRUCTOR = getAsmMethod(void.class, ""); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_BOOLEAN = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(boolean)"); + getAsmMethod(StringBuilder.class, "append", boolean.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_CHAR = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(char)"); + getAsmMethod(StringBuilder.class, "append", char.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_INT = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(int)"); + getAsmMethod(StringBuilder.class, "append", int.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_LONG = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(long)"); + getAsmMethod(StringBuilder.class, "append", long.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_FLOAT = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(float)"); + getAsmMethod(StringBuilder.class, "append", float.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(double)"); + getAsmMethod(StringBuilder.class, "append", double.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_STRING = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(java.lang.String)"); + getAsmMethod(StringBuilder.class, "append", String.class); private final static org.objectweb.asm.commons.Method STRINGBUILDER_APPEND_OBJECT = - org.objectweb.asm.commons.Method.getMethod("java.lang.StringBuilder append(java.lang.Object)"); - private final static org.objectweb.asm.commons.Method STRINGBUILDER_TOSTRING = - org.objectweb.asm.commons.Method.getMethod("java.lang.String toString()"); + getAsmMethod(StringBuilder.class, "append", Object.class); + private final static org.objectweb.asm.commons.Method STRINGBUILDER_TOSTRING = getAsmMethod(String.class, "toString"); - private final static org.objectweb.asm.commons.Method TOINTEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("int toIntExact(long)"); - private final static org.objectweb.asm.commons.Method NEGATEEXACT_INT = - org.objectweb.asm.commons.Method.getMethod("int negateExact(int)"); - private final static org.objectweb.asm.commons.Method NEGATEEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("long negateExact(long)"); - private final static org.objectweb.asm.commons.Method MULEXACT_INT = - org.objectweb.asm.commons.Method.getMethod("int multiplyExact(int, int)"); - private final static org.objectweb.asm.commons.Method MULEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("long multiplyExact(long, long)"); - private final static org.objectweb.asm.commons.Method ADDEXACT_INT = - org.objectweb.asm.commons.Method.getMethod("int addExact(int, int)"); - private final static org.objectweb.asm.commons.Method ADDEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("long addExact(long, long)"); - private final static org.objectweb.asm.commons.Method SUBEXACT_INT = - org.objectweb.asm.commons.Method.getMethod("int subtractExact(int, int)"); - private final static org.objectweb.asm.commons.Method SUBEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("long subtractExact(long, long)"); + private final static org.objectweb.asm.commons.Method TOINTEXACT_LONG = getAsmMethod(int.class, "toIntExact", long.class); + private final static org.objectweb.asm.commons.Method NEGATEEXACT_INT = getAsmMethod(int.class, "negateExact", int.class); + private final static org.objectweb.asm.commons.Method NEGATEEXACT_LONG = getAsmMethod(long.class, "negateExact", long.class); + private final static org.objectweb.asm.commons.Method MULEXACT_INT = getAsmMethod(int.class, "multiplyExact", int.class, int.class); + private final static org.objectweb.asm.commons.Method MULEXACT_LONG = getAsmMethod(long.class, "multiplyExact", long.class, long.class); + private final static org.objectweb.asm.commons.Method ADDEXACT_INT = getAsmMethod(int.class, "addExact", int.class, int.class); + private final static org.objectweb.asm.commons.Method ADDEXACT_LONG = getAsmMethod(long.class, "addExact", long.class, long.class); + private final static org.objectweb.asm.commons.Method SUBEXACT_INT = getAsmMethod(int.class, "subtractExact", int.class, int.class); + private final static org.objectweb.asm.commons.Method SUBEXACT_LONG = getAsmMethod(long.class, "subtractExact", long.class, long.class); private final static org.objectweb.asm.commons.Method CHECKEQUALS = - org.objectweb.asm.commons.Method.getMethod("boolean checkEquals(java.lang.Object, java.lang.Object)"); - private final static org.objectweb.asm.commons.Method TOBYTEEXACT_INT = - org.objectweb.asm.commons.Method.getMethod("byte toByteExact(int)"); - private final static org.objectweb.asm.commons.Method TOBYTEEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("byte toByteExact(long)"); + getAsmMethod(boolean.class, "checkEquals", Object.class, Object.class); + private final static org.objectweb.asm.commons.Method TOBYTEEXACT_INT = getAsmMethod(byte.class, "toByteExact", int.class); + private final static org.objectweb.asm.commons.Method TOBYTEEXACT_LONG = getAsmMethod(byte.class, "toByteExact", long.class); private final static org.objectweb.asm.commons.Method TOBYTEWOOVERFLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("byte toByteWithoutOverflow(float)"); + getAsmMethod(byte.class, "toByteWithoutOverflow", float.class); private final static org.objectweb.asm.commons.Method TOBYTEWOOVERFLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("byte toByteWithoutOverflow(double)"); - private final static org.objectweb.asm.commons.Method TOSHORTEXACT_INT = - org.objectweb.asm.commons.Method.getMethod("short toShortExact(int)"); - private final static org.objectweb.asm.commons.Method TOSHORTEXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("short toShortExact(long)"); + getAsmMethod(byte.class, "toByteWithoutOverflow", double.class); + private final static org.objectweb.asm.commons.Method TOSHORTEXACT_INT = getAsmMethod(short.class, "toShortExact", int.class); + private final static org.objectweb.asm.commons.Method TOSHORTEXACT_LONG = getAsmMethod(short.class, "toShortExact", long.class); private final static org.objectweb.asm.commons.Method TOSHORTWOOVERFLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("short toShortWithoutOverflow(float)"); + getAsmMethod(short.class, "toShortWithoutOverflow", float.class); private final static org.objectweb.asm.commons.Method TOSHORTWOOVERFLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("short toShortWithoutOverflow(double)"); - private final static org.objectweb.asm.commons.Method TOCHAREXACT_INT = - org.objectweb.asm.commons.Method.getMethod("char toCharExact(int)"); - private final static org.objectweb.asm.commons.Method TOCHAREXACT_LONG = - org.objectweb.asm.commons.Method.getMethod("char toCharExact(long)"); + getAsmMethod(short.class, "toShortWihtoutOverflow", double.class); + private final static org.objectweb.asm.commons.Method TOCHAREXACT_INT = getAsmMethod(char.class, "toCharExact", int.class); + private final static org.objectweb.asm.commons.Method TOCHAREXACT_LONG = getAsmMethod(char.class, "toCharExact", long.class); private final static org.objectweb.asm.commons.Method TOCHARWOOVERFLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("char toCharWithoutOverflow(float)"); + getAsmMethod(char.class, "toCharWithoutOverflow", float.class); private final static org.objectweb.asm.commons.Method TOCHARWOOVERFLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("char toCharWithoutOverflow(double)"); + getAsmMethod(char.class, "toCharWithoutOverflow", double.class); private final static org.objectweb.asm.commons.Method TOINTWOOVERFLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("int toIntWithoutOverflow(float)"); + getAsmMethod(int.class, "toIntWithoutOverflow", float.class); private final static org.objectweb.asm.commons.Method TOINTWOOVERFLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("int toIntWithoutOverflow(double)"); + getAsmMethod(int.class, "toIntWithoutOverflow", double.class); private final static org.objectweb.asm.commons.Method TOLONGWOOVERFLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("long toLongExactWithoutOverflow(float)"); + getAsmMethod(long.class, "toLongWithoutOverflow", float.class); private final static org.objectweb.asm.commons.Method TOLONGWOOVERFLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("long toLongExactWithoutOverflow(double)"); + getAsmMethod(long.class, "toLongWithoutOverflow", double.class); private final static org.objectweb.asm.commons.Method TOFLOATWOOVERFLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("float toFloatWithoutOverflow(double)"); + getAsmMethod(float.class , "toFloatWihtoutOverflow", double.class); private final static org.objectweb.asm.commons.Method MULWOOVERLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("float multiplyWithoutOverflow(float, float)"); + getAsmMethod(float.class, "multiplyWithoutOverflow", float.class, float.class); private final static org.objectweb.asm.commons.Method MULWOOVERLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("double multiplyWithoutOverflow(double, double)"); + getAsmMethod(double.class, "multiplyWithoutOverflow", double.class, double.class); private final static org.objectweb.asm.commons.Method DIVWOOVERLOW_INT = - org.objectweb.asm.commons.Method.getMethod("int divideWithoutOverflow(int, int)"); + getAsmMethod(int.class, "divideWithoutOverflow", int.class, int.class); private final static org.objectweb.asm.commons.Method DIVWOOVERLOW_LONG = - org.objectweb.asm.commons.Method.getMethod("long divideWithoutOverflow(long, long)"); + getAsmMethod(long.class, "divideWithoutOverflow", long.class, long.class); private final static org.objectweb.asm.commons.Method DIVWOOVERLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("float divideWithoutOverflow(float, float)"); + getAsmMethod(float.class, "divideWithoutOverflow", float.class, float.class); private final static org.objectweb.asm.commons.Method DIVWOOVERLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("double divideWithoutOverflow(double, double)"); + getAsmMethod(double.class, "divideWithoutOverflow", double.class, double.class); private final static org.objectweb.asm.commons.Method REMWOOVERLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("float remainderWithoutOverflow(float, float)"); + getAsmMethod(float.class, "remainderWithoutOverflow", float.class, float.class); private final static org.objectweb.asm.commons.Method REMWOOVERLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("double remainderWithoutOverflow(double, double)"); + getAsmMethod(double.class, "remainderWithoutOverflow", double.class, double.class); private final static org.objectweb.asm.commons.Method ADDWOOVERLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("float addWithoutOverflow(float, float)"); + getAsmMethod(float.class, "addWithoutOverflow", float.class, float.class); private final static org.objectweb.asm.commons.Method ADDWOOVERLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("double addWithoutOverflow(double, double)"); + getAsmMethod(double.class, "addWithoutOverflow", double.class, double.class); private final static org.objectweb.asm.commons.Method SUBWOOVERLOW_FLOAT = - org.objectweb.asm.commons.Method.getMethod("float subtractWithoutOverflow(float, float)"); + getAsmMethod(float.class, "subtractWithoutOverflow", float.class, float.class); private final static org.objectweb.asm.commons.Method SUBWOOVERLOW_DOUBLE = - org.objectweb.asm.commons.Method.getMethod("double subtractWithoutOverflow(double, double)"); + getAsmMethod(double.class, "subtractWithoutOverflow", double.class, double.class); + + private static org.objectweb.asm.commons.Method getAsmMethod(final Class rtype, final String name, final Class... ptypes) { + return new org.objectweb.asm.commons.Method(name, MethodType.methodType(rtype, ptypes).toMethodDescriptorString()); + } static byte[] write(Metadata metadata) { Writer writer = new Writer(metadata); @@ -356,7 +318,7 @@ class Writer extends PainlessParserBaseVisitor { private void writeBegin() { final int compute = ClassWriter.COMPUTE_FRAMES | ClassWriter.COMPUTE_MAXS; final int version = Opcodes.V1_7; - final int access = Opcodes.ACC_PUBLIC | Opcodes.ACC_SUPER | Opcodes.ACC_FINAL | Opcodes.ACC_SYNTHETIC; + final int access = Opcodes.ACC_PUBLIC | Opcodes.ACC_SUPER | Opcodes.ACC_FINAL; final String base = BASE_CLASS_TYPE.getInternalName(); final String name = CLASS_TYPE.getInternalName(); @@ -366,7 +328,7 @@ class Writer extends PainlessParserBaseVisitor { } private void writeConstructor() { - final int access = Opcodes.ACC_PUBLIC | Opcodes.ACC_SYNTHETIC; + final int access = Opcodes.ACC_PUBLIC; final GeneratorAdapter constructor = new GeneratorAdapter(access, CONSTRUCTOR, null, null, writer); constructor.loadThis(); constructor.loadArgs(); @@ -376,7 +338,7 @@ class Writer extends PainlessParserBaseVisitor { } private void writeExecute() { - final int access = Opcodes.ACC_PUBLIC | Opcodes.ACC_SYNTHETIC; + final int access = Opcodes.ACC_PUBLIC; execute = new GeneratorAdapter(access, EXECUTE, SIGNATURE, null, writer); final Label fals = new Label(); diff --git a/plugins/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java b/plugins/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java index 8d4fa9a066f..e1467d9ebb3 100644 --- a/plugins/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java +++ b/plugins/lang-painless/src/test/java/org/elasticsearch/painless/WhenThingsGoWrongTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.painless; +import java.util.Arrays; import java.util.Collections; public class WhenThingsGoWrongTests extends ScriptTestCase { @@ -128,4 +129,21 @@ public class WhenThingsGoWrongTests extends ScriptTestCase { "The maximum number of statements that can be executed in a loop has been reached.")); } } + + public void testSourceLimits() { + char[] chars = new char[Compiler.MAXIMUM_SOURCE_LENGTH + 1]; + Arrays.fill(chars, '0'); + + try { + exec(new String(chars)); + fail("should have hit IllegalArgumentException"); + } catch (IllegalArgumentException expected) { + assertTrue(expected.getMessage().contains("Scripts may be no longer than")); + } + + chars = new char[Compiler.MAXIMUM_SOURCE_LENGTH]; + Arrays.fill(chars, '0'); + + assertEquals(0, exec(new String(chars))); + } } From 542373a7794ab82e5ccde2038e02be31269c798e Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 5 Feb 2016 11:00:45 -0500 Subject: [PATCH 14/19] Avoid cloning MessageDigest instances This commit modifies the MessageDigests message digest provider to return a thread local instance of MessageDigest instances instead of using clone since some providers do not support clone. Closes #16479 --- .../common/hash/MessageDigests.java | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java b/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java index 7b3a108cc45..9e05e77044e 100644 --- a/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java +++ b/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java @@ -26,41 +26,40 @@ import java.security.NoSuchAlgorithmException; public class MessageDigests { - private static final MessageDigest MD5_DIGEST; - private static final MessageDigest SHA_1_DIGEST; - private static final MessageDigest SHA_256_DIGEST; - - static { - try { - MD5_DIGEST = MessageDigest.getInstance("MD5"); - SHA_1_DIGEST = MessageDigest.getInstance("SHA-1"); - SHA_256_DIGEST = MessageDigest.getInstance("SHA-256"); - } catch (NoSuchAlgorithmException e) { - throw new ElasticsearchException("Unexpected exception creating MessageDigest instance", e); - } + private static ThreadLocal createThreadLocalMessageDigest(String digest) { + return ThreadLocal.withInitial(() -> { + try { + return MessageDigest.getInstance(digest); + } catch (NoSuchAlgorithmException e) { + throw new ElasticsearchException("unexpected exception creating MessageDigest instance for [" + digest + "]", e); + } + }); } + private static final ThreadLocal MD5_DIGEST = createThreadLocalMessageDigest("MD5"); + private static final ThreadLocal SHA_1_DIGEST = createThreadLocalMessageDigest("SHA-1"); + private static final ThreadLocal SHA_256_DIGEST = createThreadLocalMessageDigest("SHA-256"); + public static MessageDigest md5() { - return clone(MD5_DIGEST); + return get(MD5_DIGEST); } public static MessageDigest sha1() { - return clone(SHA_1_DIGEST); + return get(SHA_1_DIGEST); } public static MessageDigest sha256() { - return clone(SHA_256_DIGEST); + return get(SHA_256_DIGEST); } - private static MessageDigest clone(MessageDigest messageDigest) { - try { - return (MessageDigest) messageDigest.clone(); - } catch (CloneNotSupportedException e) { - throw new ElasticsearchException("Unexpected exception cloning MessageDigest instance", e); - } + private static MessageDigest get(ThreadLocal messageDigest) { + MessageDigest instance = messageDigest.get(); + instance.reset(); + return instance; } private static final char[] HEX_DIGITS = "0123456789abcdef".toCharArray(); + public static String toHexString(byte[] bytes) { if (bytes == null) { throw new NullPointerException("bytes"); @@ -74,4 +73,5 @@ public class MessageDigests { return sb.toString(); } + } From d0c03b26f639f3bff8985210f757e7a3bf99317d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 6 Feb 2016 09:16:28 -0500 Subject: [PATCH 15/19] Fix trace logging statement in ZenDiscovery --- .../main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 488938919db..ce083147117 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -910,7 +910,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent implemen activeNodes.add(localNode); long joinsCounter = clusterJoinsCounter.get(); if (joinsCounter > 0) { - logger.trace("adding local node to the list of active nodes who has previously joined the cluster (joins counter is [{}})", joinsCounter); + logger.trace("adding local node to the list of active nodes that have previously joined the cluster (joins counter is [{}])", joinsCounter); joinedOnceActiveNodes.add(localNode); } } From 16fe948a1529a680da6104d1599300e6cfec1156 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 7 Feb 2016 11:20:53 -0500 Subject: [PATCH 16/19] Objects#requireNonNull guard in MessageDigests --- .../java/org/elasticsearch/common/hash/MessageDigests.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java b/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java index 9e05e77044e..c7488c0f12b 100644 --- a/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java +++ b/core/src/main/java/org/elasticsearch/common/hash/MessageDigests.java @@ -23,6 +23,7 @@ import org.elasticsearch.ElasticsearchException; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.util.Objects; public class MessageDigests { @@ -61,9 +62,7 @@ public class MessageDigests { private static final char[] HEX_DIGITS = "0123456789abcdef".toCharArray(); public static String toHexString(byte[] bytes) { - if (bytes == null) { - throw new NullPointerException("bytes"); - } + Objects.requireNonNull(bytes); StringBuilder sb = new StringBuilder(2 * bytes.length); for (int i = 0; i < bytes.length; i++) { From 7bfb1dfcc136ec2d0290d006b63e112df9c90d7e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Sun, 7 Feb 2016 19:58:03 +0100 Subject: [PATCH 17/19] [TEST] Don't assert on null value. It's fine to not always see an exception in this part of the test. --- .../java/org/elasticsearch/index/translog/TranslogTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index c888c884549..337d91356b9 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -1460,8 +1460,8 @@ public class TranslogTests extends ESTestCase { } boolean atLeastOneFailed = false; for (Throwable ex : threadExceptions) { - assertTrue(ex.toString(), ex instanceof IOException || ex instanceof AlreadyClosedException); if (ex != null) { + assertTrue(ex.toString(), ex instanceof IOException || ex instanceof AlreadyClosedException); atLeastOneFailed = true; } } From 4decc72da6abc426b97789ad149b31eef9aad583 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 6 Feb 2016 13:40:38 +0100 Subject: [PATCH 18/19] Fix recovery translog stats totals when recovering from store Recovery from store fails to correctly set the translog recovery stats. This fixes it and tightens up the logic bringing it all to IndexShard (previously it was set by the recovery logic). Closes #15974 Closes #16493 --- .../cluster/routing/ShardRouting.java | 2 +- .../index/engine/InternalEngine.java | 17 +------- .../elasticsearch/index/shard/IndexShard.java | 21 ++++++++-- .../index/shard/StoreRecovery.java | 5 --- .../shard/TranslogRecoveryPerformer.java | 21 ++++++++++ .../cluster/routing/ShardRoutingHelper.java | 5 +++ .../index/shard/IndexShardTests.java | 41 +++++++++++++++++-- 7 files changed, 84 insertions(+), 28 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java b/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java index 7aaf6969948..7535aa1226e 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java @@ -451,7 +451,7 @@ public final class ShardRouting implements Streamable, ToXContent { } /** - * Moves the shard from started to initializing and bumps the version + * Moves the shard from started to initializing */ void reinitializeShard() { ensureNotFrozen(); diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index d9ee2f4177a..ff738c0140b 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -43,7 +43,6 @@ import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.InfoStream; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.routing.Murmur3HashFunction; import org.elasticsearch.common.Nullable; @@ -68,7 +67,6 @@ import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogCorruptedException; -import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; @@ -233,20 +231,7 @@ public class InternalEngine extends Engine { final TranslogRecoveryPerformer handler = engineConfig.getTranslogRecoveryPerformer(); try { Translog.Snapshot snapshot = translog.newSnapshot(); - Translog.Operation operation; - while ((operation = snapshot.next()) != null) { - try { - handler.performRecoveryOperation(this, operation, true); - opsRecovered++; - } catch (ElasticsearchException e) { - if (e.status() == RestStatus.BAD_REQUEST) { - // mainly for MapperParsingException and Failure to detect xcontent - logger.info("ignoring recovery of a corrupt translog entry", e); - } else { - throw e; - } - } - } + opsRecovered = handler.recoveryFromSnapshot(this, snapshot); } catch (Throwable e) { throw new EngineException(shardId, "failed to recover from translog", e); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 705d4d5aa5a..bbd00951277 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -55,6 +55,7 @@ import org.elasticsearch.gateway.MetaDataStateFormat; import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.NodeServicesProvider; +import org.elasticsearch.index.SearchSlowLog; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; @@ -89,13 +90,12 @@ import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.recovery.RecoveryStats; import org.elasticsearch.index.refresh.RefreshStats; -import org.elasticsearch.index.SearchSlowLog; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.snapshots.IndexShardRepository; -import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreStats; import org.elasticsearch.index.suggest.stats.ShardSuggestMetric; @@ -105,8 +105,8 @@ import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.index.warmer.ShardIndexWarmerService; import org.elasticsearch.index.warmer.WarmerStats; -import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.IndexingMemoryController; +import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.percolator.PercolatorService; @@ -874,6 +874,12 @@ public class IndexShard extends AbstractIndexShardComponent { * After the store has been recovered, we need to start the engine in order to apply operations */ public void performTranslogRecovery(boolean indexExists) { + if (indexExists == false) { + // note: these are set when recovering from the translog + final RecoveryState.Translog translogStats = recoveryState().getTranslog(); + translogStats.totalOperations(0); + translogStats.totalOperationsOnStart(0); + } internalPerformTranslogRecovery(false, indexExists); assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage(); } @@ -1387,6 +1393,15 @@ public class IndexShard extends AbstractIndexShardComponent { assert recoveryState != null; recoveryState.getTranslog().incrementRecoveredOperations(); } + + @Override + public int recoveryFromSnapshot(Engine engine, Translog.Snapshot snapshot) throws IOException { + assert recoveryState != null; + RecoveryState.Translog translogStats = recoveryState.getTranslog(); + translogStats.totalOperations(snapshot.totalOperations()); + translogStats.totalOperationsOnStart(snapshot.totalOperations()); + return super.recoveryFromSnapshot(engine, snapshot); + } }; return new EngineConfig(shardId, threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(), diff --git a/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 5f5aa95a994..aaa30c147c0 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/core/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -203,7 +203,6 @@ final class StoreRecovery { logger.trace("cleaning existing shard, shouldn't exists"); IndexWriter writer = new IndexWriter(store.directory(), new IndexWriterConfig(Lucene.STANDARD_ANALYZER).setOpenMode(IndexWriterConfig.OpenMode.CREATE)); writer.close(); - recoveryState.getTranslog().totalOperations(0); } } } catch (Throwable e) { @@ -224,10 +223,6 @@ final class StoreRecovery { } catch (IOException e) { logger.debug("failed to list file details", e); } - if (indexShouldExists == false) { - recoveryState.getTranslog().totalOperations(0); - recoveryState.getTranslog().totalOperationsOnStart(0); - } indexShard.performTranslogRecovery(indexShouldExists); indexShard.finalizeRecovery(); indexShard.postRecovery("post recovery from shard_store"); diff --git a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java index 4811ff1a275..f3f8f3c14cf 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java +++ b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.rest.RestStatus; import java.io.IOException; import java.util.HashMap; @@ -77,6 +78,25 @@ public class TranslogRecoveryPerformer { return numOps; } + public int recoveryFromSnapshot(Engine engine, Translog.Snapshot snapshot) throws IOException { + Translog.Operation operation; + int opsRecovered = 0; + while ((operation = snapshot.next()) != null) { + try { + performRecoveryOperation(engine, operation, true); + opsRecovered++; + } catch (ElasticsearchException e) { + if (e.status() == RestStatus.BAD_REQUEST) { + // mainly for MapperParsingException and Failure to detect xcontent + logger.info("ignoring recovery of a corrupt translog entry", e); + } else { + throw e; + } + } + } + return opsRecovered; + } + public static class BatchOperationException extends ElasticsearchException { private final int completedOperations; @@ -182,6 +202,7 @@ public class TranslogRecoveryPerformer { // noop } + /** * Returns the recovered types modifying the mapping during the recovery */ diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java index fe7938f23b9..5d3466b5e43 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java @@ -48,6 +48,11 @@ public class ShardRoutingHelper { routing.reinitializeShard(); } + public static void reinit(ShardRouting routing, UnassignedInfo.Reason reason) { + routing.reinitializeShard(); + routing.updateUnassignedInfo(new UnassignedInfo(reason, "test_reinit")); + } + public static void moveToUnassigned(ShardRouting routing, UnassignedInfo info) { routing.moveToUnassigned(info); } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index ec2f8f32a20..778831245d1 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -70,7 +70,6 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; -import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.NodeServicesProvider; @@ -865,10 +864,11 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService("test"); final IndexShard shard = test.getShardOrNull(0); - + int translogOps = 1; client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); if (randomBoolean()) { client().admin().indices().prepareFlush().get(); + translogOps = 0; } ShardRouting routing = new ShardRouting(shard.routingEntry()); test.removeShard(0, "b/c simon says so"); @@ -878,6 +878,10 @@ public class IndexShardTests extends ESSingleNodeTestCase { DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode, localNode)); assertTrue(newShard.recoverFromStore(localNode)); + assertEquals(translogOps, newShard.recoveryState().getTranslog().recoveredOperations()); + assertEquals(translogOps, newShard.recoveryState().getTranslog().totalOperations()); + assertEquals(translogOps, newShard.recoveryState().getTranslog().totalOperationsOnStart()); + assertEquals(100.0f, newShard.recoveryState().getTranslog().recoveredPercent(), 0.01f); routing = new ShardRouting(routing); ShardRoutingHelper.moveToStarted(routing); newShard.updateRoutingEntry(routing, true); @@ -885,6 +889,36 @@ public class IndexShardTests extends ESSingleNodeTestCase { assertHitCount(response, 1); } + public void testRecoverFromCleanStore() throws IOException { + createIndex("test"); + ensureGreen(); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService test = indicesService.indexService("test"); + final IndexShard shard = test.getShardOrNull(0); + client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); + if (randomBoolean()) { + client().admin().indices().prepareFlush().get(); + } + ShardRouting routing = new ShardRouting(shard.routingEntry()); + test.removeShard(0, "b/c simon says so"); + ShardRoutingHelper.reinit(routing, UnassignedInfo.Reason.INDEX_CREATED); + IndexShard newShard = test.createShard(routing); + newShard.updateRoutingEntry(routing, false); + DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT); + newShard.markAsRecovering("store", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.STORE, localNode, + localNode)); + assertTrue(newShard.recoverFromStore(localNode)); + assertEquals(0, newShard.recoveryState().getTranslog().recoveredOperations()); + assertEquals(0, newShard.recoveryState().getTranslog().totalOperations()); + assertEquals(0, newShard.recoveryState().getTranslog().totalOperationsOnStart()); + assertEquals(100.0f, newShard.recoveryState().getTranslog().recoveredPercent(), 0.01f); + routing = new ShardRouting(routing); + ShardRoutingHelper.moveToStarted(routing); + newShard.updateRoutingEntry(routing, true); + SearchResponse response = client().prepareSearch().get(); + assertHitCount(response, 0); + } + public void testFailIfIndexNotPresentInRecoverFromStore() throws IOException { createIndex("test"); ensureGreen(); @@ -1187,7 +1221,8 @@ public class IndexShardTests extends ESSingleNodeTestCase { List operations = new ArrayList<>(); operations.add(new Translog.Index("testtype", "1", jsonBuilder().startObject().field("foo", "bar").endObject().bytes().toBytes())); newShard.prepareForIndexRecovery(); - newShard.performTranslogRecovery(true); + newShard.recoveryState().getTranslog().totalOperations(operations.size()); + newShard.skipTranslogRecovery(); newShard.performBatchRecovery(operations); assertFalse(newShard.getTranslog().syncNeeded()); } From 0b3cab6da3048d0e6f5aae474d4fe5bce7f3fa9b Mon Sep 17 00:00:00 2001 From: Isabel Drost-Fromm Date: Mon, 8 Feb 2016 13:45:38 +0100 Subject: [PATCH 19/19] Move sorting tests w/o scripting back to core In preparation of #16127 this moves sorting tests back to core that don't actually need scripting or groovy to work. --- .../search/sort/FieldSortIT.java | 1526 +++++++++++++++++ .../messy/tests/SimpleSortTests.java | 1401 +-------------- 2 files changed, 1537 insertions(+), 1390 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java diff --git a/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java b/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java new file mode 100644 index 00000000000..2b3093563a3 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/sort/FieldSortIT.java @@ -0,0 +1,1526 @@ +/* + * 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.search.sort; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.fieldValueFactorFunction; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFirstHit; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertOrderedSearchHits; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSecondHit; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId; +import static org.hamcrest.Matchers.closeTo; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.Map.Entry; +import java.util.concurrent.ExecutionException; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; +import org.apache.lucene.util.UnicodeUtil; +import org.elasticsearch.action.admin.indices.alias.Alias; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHitField; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalSettingsPlugin; +import org.hamcrest.Matchers; + +public class FieldSortIT extends ESIntegTestCase { + @Override + protected Collection> nodePlugins() { + return pluginList(InternalSettingsPlugin.class); + } + + @LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elasticsearch/elasticsearch/issues/9421") + public void testIssue8226() { + int numIndices = between(5, 10); + final boolean useMapping = randomBoolean(); + for (int i = 0; i < numIndices; i++) { + if (useMapping) { + assertAcked(prepareCreate("test_" + i).addAlias(new Alias("test")).addMapping("foo", "entry", "type=long")); + } else { + assertAcked(prepareCreate("test_" + i).addAlias(new Alias("test"))); + } + if (i > 0) { + client().prepareIndex("test_" + i, "foo", "" + i).setSource("{\"entry\": " + i + "}").get(); + } + } + ensureYellow(); + refresh(); + // sort DESC + SearchResponse searchResponse = client().prepareSearch() + .addSort(new FieldSortBuilder("entry").order(SortOrder.DESC).unmappedType(useMapping ? null : "long")) + .setSize(10).get(); + logClusterState(); + assertSearchResponse(searchResponse); + + for (int j = 1; j < searchResponse.getHits().hits().length; j++) { + Number current = (Number) searchResponse.getHits().hits()[j].getSource().get("entry"); + Number previous = (Number) searchResponse.getHits().hits()[j-1].getSource().get("entry"); + assertThat(searchResponse.toString(), current.intValue(), lessThan(previous.intValue())); + } + + // sort ASC + searchResponse = client().prepareSearch() + .addSort(new FieldSortBuilder("entry").order(SortOrder.ASC).unmappedType(useMapping ? null : "long")) + .setSize(10).get(); + logClusterState(); + assertSearchResponse(searchResponse); + + for (int j = 1; j < searchResponse.getHits().hits().length; j++) { + Number current = (Number) searchResponse.getHits().hits()[j].getSource().get("entry"); + Number previous = (Number) searchResponse.getHits().hits()[j-1].getSource().get("entry"); + assertThat(searchResponse.toString(), current.intValue(), greaterThan(previous.intValue())); + } + } + + @LuceneTestCase.BadApple(bugUrl = "simon is working on this") + public void testIssue6614() throws ExecutionException, InterruptedException { + List builders = new ArrayList<>(); + boolean strictTimeBasedIndices = randomBoolean(); + final int numIndices = randomIntBetween(2, 25); // at most 25 days in the month + for (int i = 0; i < numIndices; i++) { + final String indexId = strictTimeBasedIndices ? "idx_" + i : "idx"; + if (strictTimeBasedIndices || i == 0) { + createIndex(indexId); + } + final int numDocs = randomIntBetween(1, 23); // hour of the day + for (int j = 0; j < numDocs; j++) { + builders.add( + client().prepareIndex(indexId, "type").setSource( + "foo", "bar", "timeUpdated", "2014/07/" + + String.format(Locale.ROOT, "%02d", i+1)+ + " " + + String.format(Locale.ROOT, "%02d", j+1) + + ":00:00")); + } + } + int docs = builders.size(); + indexRandom(true, builders); + ensureYellow(); + SearchResponse allDocsResponse = client().prepareSearch().setQuery( + QueryBuilders.boolQuery().must(QueryBuilders.termQuery("foo", "bar")).must( + QueryBuilders.rangeQuery("timeUpdated").gte("2014/0" + randomIntBetween(1, 7) + "/01"))) + .addSort(new FieldSortBuilder("timeUpdated").order(SortOrder.ASC).unmappedType("date")) + .setSize(docs).get(); + assertSearchResponse(allDocsResponse); + + final int numiters = randomIntBetween(1, 20); + for (int i = 0; i < numiters; i++) { + SearchResponse searchResponse = client().prepareSearch().setQuery( + QueryBuilders.boolQuery().must(QueryBuilders.termQuery("foo", "bar")).must( + QueryBuilders.rangeQuery("timeUpdated").gte( + "2014/" + String.format(Locale.ROOT, "%02d", randomIntBetween(1, 7)) + "/01"))) + .addSort(new FieldSortBuilder("timeUpdated").order(SortOrder.ASC).unmappedType("date")) + .setSize(scaledRandomIntBetween(1, docs)).get(); + assertSearchResponse(searchResponse); + for (int j = 0; j < searchResponse.getHits().hits().length; j++) { + assertThat(searchResponse.toString() + + "\n vs. \n" + + allDocsResponse.toString(), + searchResponse.getHits().hits()[j].getId(), + equalTo(allDocsResponse.getHits().hits()[j].getId())); + } + } + + } + + public void testIssue6639() throws ExecutionException, InterruptedException { + assertAcked(prepareCreate("$index") + .addMapping( + "$type", + "{\"$type\": " + + " {\"properties\": " + + " {\"grantee\": " + + " {\"index\": " + + " \"not_analyzed\", " + + " \"term_vector\": \"with_positions_offsets\", " + + " \"type\": \"string\", " + + " \"analyzer\": \"snowball\", " + + " \"boost\": 1.0, " + + " \"store\": true}}}}")); + indexRandom(true, + client().prepareIndex( + "$index", + "$type", + "data.activity.5").setSource("{\"django_ct\": \"data.activity\", \"grantee\": \"Grantee 1\"}"), + client().prepareIndex( + "$index", + "$type", + "data.activity.6").setSource("{\"django_ct\": \"data.activity\", \"grantee\": \"Grantee 2\"}")); + ensureYellow(); + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort("grantee", SortOrder.ASC) + .execute().actionGet(); + assertOrderedSearchHits(searchResponse, "data.activity.5", "data.activity.6"); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort("grantee", SortOrder.DESC) + .execute().actionGet(); + assertOrderedSearchHits(searchResponse, "data.activity.6", "data.activity.5"); + } + + public void testTrackScores() throws Exception { + createIndex("test"); + ensureGreen(); + index("test", "type1", jsonBuilder().startObject() + .field("id", "1") + .field("svalue", "aaa") + .field("ivalue", 100) + .field("dvalue", 0.1) + .endObject()); + index("test", "type1", jsonBuilder().startObject() + .field("id", "2") + .field("svalue", "bbb") + .field("ivalue", 200) + .field("dvalue", 0.2) + .endObject()); + refresh(); + + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort("svalue", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getMaxScore(), equalTo(Float.NaN)); + for (SearchHit hit : searchResponse.getHits()) { + assertThat(hit.getScore(), equalTo(Float.NaN)); + } + + // now check with score tracking + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort("svalue", SortOrder.ASC) + .setTrackScores(true) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getMaxScore(), not(equalTo(Float.NaN))); + for (SearchHit hit : searchResponse.getHits()) { + assertThat(hit.getScore(), not(equalTo(Float.NaN))); + } + } + + public void testRandomSorting() throws IOException, InterruptedException, ExecutionException { + Random random = getRandom(); + assertAcked(prepareCreate("test") + .addMapping("type", + XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("properties") + .startObject("sparse_bytes") + .field("type", "string") + .field("index", "not_analyzed") + .endObject() + .startObject("dense_bytes") + .field("type", "string") + .field("index", "not_analyzed") + .endObject() + .endObject() + .endObject() + .endObject())); + ensureGreen(); + + TreeMap sparseBytes = new TreeMap<>(); + TreeMap denseBytes = new TreeMap<>(); + int numDocs = randomIntBetween(200, 300); + IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; i++) { + String docId = Integer.toString(i); + BytesRef ref = null; + do { + ref = new BytesRef(TestUtil.randomRealisticUnicodeString(random)); + } while (denseBytes.containsKey(ref)); + denseBytes.put(ref, docId); + XContentBuilder src = jsonBuilder().startObject().field("dense_bytes", ref.utf8ToString()); + if (rarely()) { + src.field("sparse_bytes", ref.utf8ToString()); + sparseBytes.put(ref, docId); + } + src.endObject(); + builders[i] = client().prepareIndex("test", "type", docId).setSource(src); + } + indexRandom(true, builders); + { + int size = between(1, denseBytes.size()); + SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).setSize(size) + .addSort("dense_bytes", SortOrder.ASC).execute().actionGet(); + assertNoFailures(searchResponse); + assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) numDocs)); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + Set> entrySet = denseBytes.entrySet(); + Iterator> iterator = entrySet.iterator(); + for (int i = 0; i < size; i++) { + assertThat(iterator.hasNext(), equalTo(true)); + Entry next = iterator.next(); + assertThat("pos: " + i, searchResponse.getHits().getAt(i).id(), equalTo(next.getValue())); + assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); + } + } + if (!sparseBytes.isEmpty()) { + int size = between(1, sparseBytes.size()); + SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()) + .setPostFilter(QueryBuilders.existsQuery("sparse_bytes")).setSize(size).addSort("sparse_bytes", SortOrder.ASC).execute() + .actionGet(); + assertNoFailures(searchResponse); + assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) sparseBytes.size())); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + Set> entrySet = sparseBytes.entrySet(); + Iterator> iterator = entrySet.iterator(); + for (int i = 0; i < size; i++) { + assertThat(iterator.hasNext(), equalTo(true)); + Entry next = iterator.next(); + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(next.getValue())); + assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); + } + } + } + + public void test3078() { + createIndex("test"); + ensureGreen(); + + for (int i = 1; i < 101; i++) { + client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", Integer.toString(i)).execute().actionGet(); + } + refresh(); + SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + + // reindex and refresh + client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); + refresh(); + + searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + + // reindex - no refresh + client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); + + searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + + // force merge + forceMerge(); + refresh(); + + client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); + searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + + refresh(); + searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); + assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); + } + + public void testScoreSortDirection() throws Exception { + createIndex("test"); + ensureGreen(); + + client().prepareIndex("test", "type", "1").setSource("field", 2).execute().actionGet(); + client().prepareIndex("test", "type", "2").setSource("field", 1).execute().actionGet(); + client().prepareIndex("test", "type", "3").setSource("field", 0).execute().actionGet(); + + refresh(); + + SearchResponse searchResponse = client() + .prepareSearch("test") + .setQuery( + QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) + .execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); + + searchResponse = client() + .prepareSearch("test") + .setQuery( + QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) + .addSort("_score", SortOrder.DESC).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); + + searchResponse = client() + .prepareSearch("test") + .setQuery( + QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) + .addSort("_score", SortOrder.DESC).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + } + + public void testScoreSortDirectionWithFunctionScore() throws Exception { + createIndex("test"); + ensureGreen(); + + client().prepareIndex("test", "type", "1").setSource("field", 2).execute().actionGet(); + client().prepareIndex("test", "type", "2").setSource("field", 1).execute().actionGet(); + client().prepareIndex("test", "type", "3").setSource("field", 0).execute().actionGet(); + + refresh(); + + SearchResponse searchResponse = client().prepareSearch("test") + .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); + + searchResponse = client().prepareSearch("test") + .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))) + .addSort("_score", SortOrder.DESC).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); + + searchResponse = client().prepareSearch("test") + .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))) + .addSort("_score", SortOrder.DESC).execute().actionGet(); + assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + } + + public void testIssue2986() { + createIndex("test"); + + client().prepareIndex("test", "post", "1").setSource("{\"field1\":\"value1\"}").execute().actionGet(); + client().prepareIndex("test", "post", "2").setSource("{\"field1\":\"value2\"}").execute().actionGet(); + client().prepareIndex("test", "post", "3").setSource("{\"field1\":\"value3\"}").execute().actionGet(); + refresh(); + SearchResponse result = client().prepareSearch("test").setQuery(matchAllQuery()).setTrackScores(true) + .addSort("field1", SortOrder.ASC).execute().actionGet(); + + for (SearchHit hit : result.getHits()) { + assertFalse(Float.isNaN(hit.getScore())); + } + } + + public void testIssue2991() { + for (int i = 1; i < 4; i++) { + try { + client().admin().indices().prepareDelete("test").execute().actionGet(); + } catch (Exception e) { + // ignore + } + createIndex("test"); + ensureGreen(); + client().prepareIndex("test", "type", "1").setSource("tag", "alpha").execute().actionGet(); + refresh(); + + client().prepareIndex("test", "type", "3").setSource("tag", "gamma").execute().actionGet(); + refresh(); + + client().prepareIndex("test", "type", "4").setSource("tag", "delta").execute().actionGet(); + + refresh(); + client().prepareIndex("test", "type", "2").setSource("tag", "beta").execute().actionGet(); + + refresh(); + SearchResponse resp = client().prepareSearch("test").setSize(2).setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("tag").order(SortOrder.ASC)).execute().actionGet(); + assertHitCount(resp, 4); + assertThat(resp.getHits().hits().length, equalTo(2)); + assertFirstHit(resp, hasId("1")); + assertSecondHit(resp, hasId("2")); + + resp = client().prepareSearch("test").setSize(2).setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("tag").order(SortOrder.DESC)).execute().actionGet(); + assertHitCount(resp, 4); + assertThat(resp.getHits().hits().length, equalTo(2)); + assertFirstHit(resp, hasId("3")); + assertSecondHit(resp, hasId("4")); + } + } + + public void testSimpleSorts() throws Exception { + Random random = random(); + assertAcked(prepareCreate("test") +.addMapping("type1", + XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties").startObject("str_value") + .field("type", "string").field("index", "not_analyzed").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("boolean_value") + .field("type", "boolean").endObject().startObject("byte_value").field("type", "byte").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("short_value") + .field("type", "short").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null) + .endObject().endObject().startObject("integer_value").field("type", "integer").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("long_value") + .field("type", "long").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null) + .endObject().endObject().startObject("float_value").field("type", "float").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("double_value") + .field("type", "double").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null) + .endObject().endObject().endObject().endObject().endObject())); + ensureGreen(); + List builders = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + IndexRequestBuilder builder = client().prepareIndex("test", "type1", Integer.toString(i)).setSource(jsonBuilder().startObject() + .field("str_value", new String(new char[]{(char) (97 + i), (char) (97 + i)})) + .field("boolean_value", true) + .field("byte_value", i) + .field("short_value", i) + .field("integer_value", i) + .field("long_value", i) + .field("float_value", 0.1 * i) + .field("double_value", 0.1 * i) + .endObject()); + builders.add(builder); + } + Collections.shuffle(builders, random); + for (IndexRequestBuilder builder : builders) { + builder.execute().actionGet(); + if (random.nextBoolean()) { + if (random.nextInt(5) != 0) { + refresh(); + } else { + client().admin().indices().prepareFlush().execute().actionGet(); + } + } + + } + refresh(); + + // STRING + int size = 1 + random.nextInt(10); + + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(size) + .addSort("str_value", SortOrder.ASC) + .execute().actionGet(); + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), + equalTo(new String(new char[] { (char) (97 + i), (char) (97 + i) }))); + } + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(size) + .addSort("str_value", SortOrder.DESC) + .execute().actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), + equalTo(new String(new char[] { (char) (97 + (9 - i)), (char) (97 + (9 - i)) }))); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + + + // BYTE + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("byte_value", SortOrder.ASC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).byteValue(), equalTo((byte) i)); + } + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("byte_value", SortOrder.DESC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).byteValue(), equalTo((byte) (9 - i))); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + + // SHORT + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("short_value", SortOrder.ASC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).shortValue(), equalTo((short) i)); + } + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("short_value", SortOrder.DESC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).shortValue(), equalTo((short) (9 - i))); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + + // INTEGER + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("integer_value", SortOrder.ASC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).intValue(), equalTo(i)); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("integer_value", SortOrder.DESC) + .execute().actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).intValue(), equalTo((9 - i))); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + + // LONG + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("long_value", SortOrder.ASC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).longValue(), equalTo((long) i)); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("long_value", SortOrder.DESC).execute() + .actionGet(); + assertHitCount(searchResponse, 10L); + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).longValue(), equalTo((long) (9 - i))); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + + // FLOAT + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("float_value", SortOrder.ASC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10L); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("float_value", SortOrder.DESC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + + // DOUBLE + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("double_value", SortOrder.ASC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10L); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); + } + + assertThat(searchResponse.toString(), not(containsString("error"))); + size = 1 + random.nextInt(10); + searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("double_value", SortOrder.DESC).execute() + .actionGet(); + + assertHitCount(searchResponse, 10L); + assertThat(searchResponse.getHits().hits().length, equalTo(size)); + for (int i = 0; i < size; i++) { + assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); + assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); + } + + assertNoFailures(searchResponse); + } + + public void testSortMissingNumbers() throws Exception { + assertAcked(prepareCreate("test").addMapping("type1", + XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("properties") + .startObject("i_value") + .field("type", "integer") + .startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject() + .endObject() + .startObject("d_value") + .field("type", "float") + .startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject() + .endObject() + .endObject() + .endObject() + .endObject())); + ensureGreen(); + client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() + .field("id", "1") + .field("i_value", -1) + .field("d_value", -1.1) + .endObject()).execute().actionGet(); + + client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() + .field("id", "2") + .endObject()).execute().actionGet(); + + client().prepareIndex("test", "type1", "3").setSource(jsonBuilder().startObject() + .field("id", "1") + .field("i_value", 2) + .field("d_value", 2.2) + .endObject()).execute().actionGet(); + + flush(); + refresh(); + + logger.info("--> sort with no missing (same as missing _last)"); + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("i_value").order(SortOrder.ASC)) + .execute().actionGet(); + assertNoFailures(searchResponse); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + + logger.info("--> sort with missing _last"); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("i_value").order(SortOrder.ASC).missing("_last")) + .execute().actionGet(); + assertNoFailures(searchResponse); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + + logger.info("--> sort with missing _first"); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("i_value").order(SortOrder.ASC).missing("_first")) + .execute().actionGet(); + assertNoFailures(searchResponse); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); + } + + public void testSortMissingStrings() throws IOException { + assertAcked(prepareCreate("test").addMapping("type1", + XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("properties") + .startObject("value") + .field("type", "string") + .field("index", "not_analyzed") + .endObject() + .endObject() + .endObject() + .endObject())); + ensureGreen(); + client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() + .field("id", "1") + .field("value", "a") + .endObject()).execute().actionGet(); + + client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() + .field("id", "2") + .endObject()).execute().actionGet(); + + client().prepareIndex("test", "type1", "3").setSource(jsonBuilder().startObject() + .field("id", "1") + .field("value", "c") + .endObject()).execute().actionGet(); + + flush(); + refresh(); + + // TODO: WTF? + try { + Thread.sleep(2000); + } catch (InterruptedException e) { + throw new RuntimeException(); + } + + logger.info("--> sort with no missing (same as missing _last)"); + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC)) + .execute().actionGet(); + assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + + logger.info("--> sort with missing _last"); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC).missing("_last")) + .execute().actionGet(); + assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); + + logger.info("--> sort with missing _first"); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC).missing("_first")) + .execute().actionGet(); + assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); + + logger.info("--> sort with missing b"); + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC).missing("b")) + .execute().actionGet(); + assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + assertThat(searchResponse.getHits().getAt(1).id(), equalTo("2")); + assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); + } + + public void testIgnoreUnmapped() throws Exception { + createIndex("test"); + ensureYellow(); + + client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() + .field("id", "1") + .field("i_value", -1) + .field("d_value", -1.1) + .endObject()).execute().actionGet(); + + logger.info("--> sort with an unmapped field, verify it fails"); + try { + SearchResponse result = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("kkk")) + .execute().actionGet(); + assertThat("Expected exception but returned with", result, nullValue()); + } catch (SearchPhaseExecutionException e) { + //we check that it's a parse failure rather than a different shard failure + for (ShardSearchFailure shardSearchFailure : e.shardFailures()) { + assertThat(shardSearchFailure.toString(), containsString("[No mapping found for [kkk] in order to sort on]")); + } + } + + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("kkk").unmappedType("string")) + .execute().actionGet(); + assertNoFailures(searchResponse); + } + + public void testSortMVField() throws Exception { + assertAcked(prepareCreate("test") +.addMapping("type1", + XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties").startObject("long_values") + .field("type", "long").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null) + .endObject().endObject().startObject("int_values").field("type", "integer").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("short_values") + .field("type", "short").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null) + .endObject().endObject().startObject("byte_values").field("type", "byte").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("float_values") + .field("type", "float").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null) + .endObject().endObject().startObject("double_values").field("type", "double").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().startObject("string_values") + .field("type", "string").field("index", "not_analyzed").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().endObject().endObject() + .endObject())); + ensureGreen(); + + client().prepareIndex("test", "type1", Integer.toString(1)).setSource(jsonBuilder().startObject() + .array("long_values", 1L, 5L, 10L, 8L) + .array("int_values", 1, 5, 10, 8) + .array("short_values", 1, 5, 10, 8) + .array("byte_values", 1, 5, 10, 8) + .array("float_values", 1f, 5f, 10f, 8f) + .array("double_values", 1d, 5d, 10d, 8d) + .array("string_values", "01", "05", "10", "08") + .endObject()).execute().actionGet(); + client().prepareIndex("test", "type1", Integer.toString(2)).setSource(jsonBuilder().startObject() + .array("long_values", 11L, 15L, 20L, 7L) + .array("int_values", 11, 15, 20, 7) + .array("short_values", 11, 15, 20, 7) + .array("byte_values", 11, 15, 20, 7) + .array("float_values", 11f, 15f, 20f, 7f) + .array("double_values", 11d, 15d, 20d, 7d) + .array("string_values", "11", "15", "20", "07") + .endObject()).execute().actionGet(); + client().prepareIndex("test", "type1", Integer.toString(3)).setSource(jsonBuilder().startObject() + .array("long_values", 2L, 1L, 3L, -4L) + .array("int_values", 2, 1, 3, -4) + .array("short_values", 2, 1, 3, -4) + .array("byte_values", 2, 1, 3, -4) + .array("float_values", 2f, 1f, 3f, -4f) + .array("double_values", 2d, 1d, 3d, -4d) + .array("string_values", "02", "01", "03", "!4") + .endObject()).execute().actionGet(); + + refresh(); + + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("long_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(-4L)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(1L)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(7L)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("long_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(20L)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(10L)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(3L)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort(SortBuilders.fieldSort("long_values").order(SortOrder.DESC).sortMode("sum")) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(53L)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(24L)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(2L)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("int_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("int_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("short_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("short_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("byte_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("byte_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("float_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).floatValue(), equalTo(-4f)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).floatValue(), equalTo(1f)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).floatValue(), equalTo(7f)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("float_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).floatValue(), equalTo(20f)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).floatValue(), equalTo(10f)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).floatValue(), equalTo(3f)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("double_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(-4d)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(1d)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(7d)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("double_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(20d)); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(10d)); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(3d)); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("string_values", SortOrder.ASC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); + assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("!4")); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("01")); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); + assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("07")); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(10) + .addSort("string_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("03")); + } + + public void testSortOnRareField() throws IOException { + assertAcked(prepareCreate("test").addMapping("type1", + XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties").startObject("string_values") + .field("type", "string").field("index", "not_analyzed").startObject("fielddata") + .field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject().endObject().endObject() + .endObject())); + ensureGreen(); + client().prepareIndex("test", "type1", Integer.toString(1)).setSource(jsonBuilder().startObject() + .array("string_values", "01", "05", "10", "08") + .endObject()).execute().actionGet(); + + + refresh(); + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(3) + .addSort("string_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().hits().length, equalTo(1)); + + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(1))); + assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("10")); + + client().prepareIndex("test", "type1", Integer.toString(2)).setSource(jsonBuilder().startObject() + .array("string_values", "11", "15", "20", "07") + .endObject()).execute().actionGet(); + for (int i = 0; i < 15; i++) { + client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() + .array("some_other_field", "foobar") + .endObject()).execute().actionGet(); + } + refresh(); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(2) + .addSort("string_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().hits().length, equalTo(2)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); + + + client().prepareIndex("test", "type1", Integer.toString(3)).setSource(jsonBuilder().startObject() + .array("string_values", "02", "01", "03", "!4") + .endObject()).execute().actionGet(); + for (int i = 0; i < 15; i++) { + client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() + .array("some_other_field", "foobar") + .endObject()).execute().actionGet(); + } + refresh(); + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(3) + .addSort("string_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("03")); + + for (int i = 0; i < 15; i++) { + client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() + .array("some_other_field", "foobar") + .endObject()).execute().actionGet(); + refresh(); + } + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(3) + .addSort("string_values", SortOrder.DESC) + .execute().actionGet(); + + assertThat(searchResponse.getHits().hits().length, equalTo(3)); + + assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); + assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); + + assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); + assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); + + assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); + assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("03")); + } + + public void testSortMetaField() throws Exception { + XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("_timestamp").field("enabled", true).endObject() + .endObject().endObject(); + assertAcked(prepareCreate("test") + .addMapping("type", mapping)); + ensureGreen(); + final int numDocs = randomIntBetween(10, 20); + IndexRequestBuilder[] indexReqs = new IndexRequestBuilder[numDocs]; + for (int i = 0; i < numDocs; ++i) { + indexReqs[i] = client().prepareIndex("test", "type", Integer.toString(i)).setTimestamp(Integer.toString(randomInt(1000))) + .setSource(); + } + indexRandom(true, indexReqs); + + SortOrder order = randomFrom(SortOrder.values()); + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(randomIntBetween(1, numDocs + 5)) + .addSort("_uid", order) + .execute().actionGet(); + assertNoFailures(searchResponse); + SearchHit[] hits = searchResponse.getHits().hits(); + BytesRef previous = order == SortOrder.ASC ? new BytesRef() : UnicodeUtil.BIG_TERM; + for (int i = 0; i < hits.length; ++i) { + final BytesRef uid = new BytesRef(Uid.createUid(hits[i].type(), hits[i].id())); + assertThat(previous, order == SortOrder.ASC ? lessThan(uid) : greaterThan(uid)); + previous = uid; + } + + /* + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(randomIntBetween(1, numDocs + 5)) + .addSort("_id", order) + .execute().actionGet(); + assertNoFailures(searchResponse); + hits = searchResponse.getHits().hits(); + previous = order == SortOrder.ASC ? new BytesRef() : UnicodeUtil.BIG_TERM; + for (int i = 0; i < hits.length; ++i) { + final BytesRef id = new BytesRef(Uid.createUid(hits[i].type(), hits[i].id())); + assertThat(previous, order == SortOrder.ASC ? lessThan(id) : greaterThan(id)); + previous = id; + }*/ + + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .setSize(randomIntBetween(1, numDocs + 5)) + .addSort("_timestamp", order) + .addField("_timestamp") + .execute().actionGet(); + assertNoFailures(searchResponse); + hits = searchResponse.getHits().hits(); + Long previousTs = order == SortOrder.ASC ? 0 : Long.MAX_VALUE; + for (int i = 0; i < hits.length; ++i) { + SearchHitField timestampField = hits[i].getFields().get("_timestamp"); + Long timestamp = timestampField.getValue(); + assertThat(previousTs, order == SortOrder.ASC ? lessThanOrEqualTo(timestamp) : greaterThanOrEqualTo(timestamp)); + previousTs = timestamp; + } + } + + /** + * Test case for issue 6150: https://github.com/elasticsearch/elasticsearch/issues/6150 + */ + public void testNestedSort() throws IOException, InterruptedException, ExecutionException { + assertAcked(prepareCreate("test") + .addMapping("type", + XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("properties") + .startObject("nested") + .field("type", "nested") + .startObject("properties") + .startObject("foo") + .field("type", "string") + .startObject("fields") + .startObject("sub") + .field("type", "string") + .field("index", "not_analyzed") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject())); + ensureGreen(); + + client().prepareIndex("test", "type", "1").setSource(jsonBuilder().startObject() + .startObject("nested") + .field("foo", "bar bar") + .endObject() + .endObject()).execute().actionGet(); + refresh(); + + // We sort on nested field + SearchResponse searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("nested.foo").setNestedPath("nested").order(SortOrder.DESC)) + .execute().actionGet(); + assertNoFailures(searchResponse); + SearchHit[] hits = searchResponse.getHits().hits(); + for (int i = 0; i < hits.length; ++i) { + assertThat(hits[i].getSortValues().length, is(1)); + Object o = hits[i].getSortValues()[0]; + assertThat(o, notNullValue()); + Text text = (Text) o; + assertThat(text.string(), is("bar")); + } + + + // We sort on nested sub field + searchResponse = client().prepareSearch() + .setQuery(matchAllQuery()) + .addSort(SortBuilders.fieldSort("nested.foo.sub").setNestedPath("nested").order(SortOrder.DESC)) + .execute().actionGet(); + assertNoFailures(searchResponse); + hits = searchResponse.getHits().hits(); + for (int i = 0; i < hits.length; ++i) { + assertThat(hits[i].getSortValues().length, is(1)); + Object o = hits[i].getSortValues()[0]; + assertThat(o, notNullValue()); + Text text = (Text) o; + assertThat(text.string(), is("bar bar")); + } + } + + public void testSortDuelBetweenSingleShardAndMultiShardIndex() throws Exception { + String sortField = "sortField"; + assertAcked(prepareCreate("test1") + .setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, between(2, maximumNumberOfShards())) + .addMapping("type", sortField, "type=long").get()); + assertAcked(prepareCreate("test2") + .setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .addMapping("type", sortField, "type=long").get()); + + for (String index : new String[]{"test1", "test2"}) { + List docs = new ArrayList<>(); + for (int i = 0; i < 256; i++) { + docs.add(client().prepareIndex(index, "type", Integer.toString(i)).setSource(sortField, i)); + } + indexRandom(true, docs); + } + + ensureSearchable("test1", "test2"); + SortOrder order = randomBoolean() ? SortOrder.ASC : SortOrder.DESC; + int from = between(0, 256); + int size = between(0, 256); + SearchResponse multiShardResponse = client().prepareSearch("test1").setFrom(from).setSize(size).addSort(sortField, order).get(); + assertNoFailures(multiShardResponse); + SearchResponse singleShardResponse = client().prepareSearch("test2").setFrom(from).setSize(size).addSort(sortField, order).get(); + assertNoFailures(singleShardResponse); + + assertThat(multiShardResponse.getHits().totalHits(), equalTo(singleShardResponse.getHits().totalHits())); + assertThat(multiShardResponse.getHits().getHits().length, equalTo(singleShardResponse.getHits().getHits().length)); + for (int i = 0; i < multiShardResponse.getHits().getHits().length; i++) { + assertThat(multiShardResponse.getHits().getAt(i).sortValues()[0], + equalTo(singleShardResponse.getHits().getAt(i).sortValues()[0])); + assertThat(multiShardResponse.getHits().getAt(i).id(), equalTo(singleShardResponse.getHits().getAt(i).id())); + } + } + +} diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SimpleSortTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SimpleSortTests.java index bae2eb552d1..096e30f40da 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SimpleSortTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SimpleSortTests.java @@ -20,74 +20,37 @@ package org.elasticsearch.messy.tests; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.LuceneTestCase; -import org.apache.lucene.util.TestUtil; -import org.apache.lucene.util.UnicodeUtil; -import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.text.Text; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.Script; import org.elasticsearch.script.groovy.GroovyPlugin; -import org.elasticsearch.search.SearchHit; -import org.elasticsearch.search.SearchHitField; -import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.ScriptSortBuilder; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalSettingsPlugin; -import org.hamcrest.Matchers; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Iterator; import java.util.List; -import java.util.Locale; -import java.util.Map.Entry; import java.util.Random; -import java.util.Set; -import java.util.TreeMap; -import java.util.concurrent.ExecutionException; import static org.apache.lucene.util.GeoUtils.TOLERANCE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; -import static org.elasticsearch.index.query.functionscore.ScoreFunctionBuilders.fieldValueFactorFunction; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFirstHit; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertOrderedSearchHits; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSecondHit; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.hasId; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.lessThan; -import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; /** * @@ -98,397 +61,6 @@ public class SimpleSortTests extends ESIntegTestCase { return pluginList(GroovyPlugin.class, InternalSettingsPlugin.class); } - @LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elasticsearch/elasticsearch/issues/9421") - public void testIssue8226() { - int numIndices = between(5, 10); - final boolean useMapping = randomBoolean(); - for (int i = 0; i < numIndices; i++) { - if (useMapping) { - assertAcked(prepareCreate("test_" + i).addAlias(new Alias("test")).addMapping("foo", "entry", "type=long")); - } else { - assertAcked(prepareCreate("test_" + i).addAlias(new Alias("test"))); - } - if (i > 0) { - client().prepareIndex("test_" + i, "foo", "" + i).setSource("{\"entry\": " + i + "}").get(); - } - } - ensureYellow(); - refresh(); - // sort DESC - SearchResponse searchResponse = client().prepareSearch() - .addSort(new FieldSortBuilder("entry").order(SortOrder.DESC).unmappedType(useMapping ? null : "long")) - .setSize(10).get(); - logClusterState(); - assertSearchResponse(searchResponse); - - for (int j = 1; j < searchResponse.getHits().hits().length; j++) { - Number current = (Number) searchResponse.getHits().hits()[j].getSource().get("entry"); - Number previous = (Number) searchResponse.getHits().hits()[j-1].getSource().get("entry"); - assertThat(searchResponse.toString(), current.intValue(), lessThan(previous.intValue())); - } - - // sort ASC - searchResponse = client().prepareSearch() - .addSort(new FieldSortBuilder("entry").order(SortOrder.ASC).unmappedType(useMapping ? null : "long")) - .setSize(10).get(); - logClusterState(); - assertSearchResponse(searchResponse); - - for (int j = 1; j < searchResponse.getHits().hits().length; j++) { - Number current = (Number) searchResponse.getHits().hits()[j].getSource().get("entry"); - Number previous = (Number) searchResponse.getHits().hits()[j-1].getSource().get("entry"); - assertThat(searchResponse.toString(), current.intValue(), greaterThan(previous.intValue())); - } - } - - @LuceneTestCase.BadApple(bugUrl = "simon is working on this") - public void testIssue6614() throws ExecutionException, InterruptedException { - List builders = new ArrayList<>(); - boolean strictTimeBasedIndices = randomBoolean(); - final int numIndices = randomIntBetween(2, 25); // at most 25 days in the month - for (int i = 0; i < numIndices; i++) { - final String indexId = strictTimeBasedIndices ? "idx_" + i : "idx"; - if (strictTimeBasedIndices || i == 0) { - createIndex(indexId); - } - final int numDocs = randomIntBetween(1, 23); // hour of the day - for (int j = 0; j < numDocs; j++) { - builders.add(client().prepareIndex(indexId, "type").setSource("foo", "bar", "timeUpdated", "2014/07/" + String.format(Locale.ROOT, "%02d", i+1)+" " + String.format(Locale.ROOT, "%02d", j+1) + ":00:00")); - } - } - int docs = builders.size(); - indexRandom(true, builders); - ensureYellow(); - SearchResponse allDocsResponse = client().prepareSearch().setQuery( - QueryBuilders.boolQuery().must(QueryBuilders.termQuery("foo", "bar")).must( - QueryBuilders.rangeQuery("timeUpdated").gte("2014/0" + randomIntBetween(1, 7) + "/01"))) - .addSort(new FieldSortBuilder("timeUpdated").order(SortOrder.ASC).unmappedType("date")) - .setSize(docs).get(); - assertSearchResponse(allDocsResponse); - - final int numiters = randomIntBetween(1, 20); - for (int i = 0; i < numiters; i++) { - SearchResponse searchResponse = client().prepareSearch().setQuery( - QueryBuilders.boolQuery().must(QueryBuilders.termQuery("foo", "bar")).must( - QueryBuilders.rangeQuery("timeUpdated").gte("2014/" + String.format(Locale.ROOT, "%02d", randomIntBetween(1, 7)) + "/01"))) - .addSort(new FieldSortBuilder("timeUpdated").order(SortOrder.ASC).unmappedType("date")) - .setSize(scaledRandomIntBetween(1, docs)).get(); - assertSearchResponse(searchResponse); - for (int j = 0; j < searchResponse.getHits().hits().length; j++) { - assertThat(searchResponse.toString() + "\n vs. \n" + allDocsResponse.toString(), searchResponse.getHits().hits()[j].getId(), equalTo(allDocsResponse.getHits().hits()[j].getId())); - } - } - - } - - public void testIssue6639() throws ExecutionException, InterruptedException { - assertAcked(prepareCreate("$index") - .addMapping("$type","{\"$type\": {\"properties\": {\"grantee\": {\"index\": \"not_analyzed\", \"term_vector\": \"with_positions_offsets\", \"type\": \"string\", \"analyzer\": \"snowball\", \"boost\": 1.0, \"store\": true}}}}")); - indexRandom(true, - client().prepareIndex("$index", "$type", "data.activity.5").setSource("{\"django_ct\": \"data.activity\", \"grantee\": \"Grantee 1\"}"), - client().prepareIndex("$index", "$type", "data.activity.6").setSource("{\"django_ct\": \"data.activity\", \"grantee\": \"Grantee 2\"}")); - ensureYellow(); - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort("grantee", SortOrder.ASC) - .execute().actionGet(); - assertOrderedSearchHits(searchResponse, "data.activity.5", "data.activity.6"); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort("grantee", SortOrder.DESC) - .execute().actionGet(); - assertOrderedSearchHits(searchResponse, "data.activity.6", "data.activity.5"); - } - - public void testTrackScores() throws Exception { - createIndex("test"); - ensureGreen(); - index("test", "type1", jsonBuilder().startObject() - .field("id", "1") - .field("svalue", "aaa") - .field("ivalue", 100) - .field("dvalue", 0.1) - .endObject()); - index("test", "type1", jsonBuilder().startObject() - .field("id", "2") - .field("svalue", "bbb") - .field("ivalue", 200) - .field("dvalue", 0.2) - .endObject()); - refresh(); - - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort("svalue", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getMaxScore(), equalTo(Float.NaN)); - for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.getScore(), equalTo(Float.NaN)); - } - - // now check with score tracking - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort("svalue", SortOrder.ASC) - .setTrackScores(true) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getMaxScore(), not(equalTo(Float.NaN))); - for (SearchHit hit : searchResponse.getHits()) { - assertThat(hit.getScore(), not(equalTo(Float.NaN))); - } - } - - public void testRandomSorting() throws IOException, InterruptedException, ExecutionException { - Random random = getRandom(); - assertAcked(prepareCreate("test") - .addMapping("type", - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("properties") - .startObject("sparse_bytes") - .field("type", "string") - .field("index", "not_analyzed") - .endObject() - .startObject("dense_bytes") - .field("type", "string") - .field("index", "not_analyzed") - .endObject() - .endObject() - .endObject() - .endObject())); - ensureGreen(); - - TreeMap sparseBytes = new TreeMap<>(); - TreeMap denseBytes = new TreeMap<>(); - int numDocs = randomIntBetween(200, 300); - IndexRequestBuilder[] builders = new IndexRequestBuilder[numDocs]; - for (int i = 0; i < numDocs; i++) { - String docId = Integer.toString(i); - BytesRef ref = null; - do { - ref = new BytesRef(TestUtil.randomRealisticUnicodeString(random)); - } while (denseBytes.containsKey(ref)); - denseBytes.put(ref, docId); - XContentBuilder src = jsonBuilder().startObject().field("dense_bytes", ref.utf8ToString()); - if (rarely()) { - src.field("sparse_bytes", ref.utf8ToString()); - sparseBytes.put(ref, docId); - } - src.endObject(); - builders[i] = client().prepareIndex("test", "type", docId).setSource(src); - } - indexRandom(true, builders); - { - int size = between(1, denseBytes.size()); - SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).setSize(size) - .addSort("dense_bytes", SortOrder.ASC).execute().actionGet(); - assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) numDocs)); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - Set> entrySet = denseBytes.entrySet(); - Iterator> iterator = entrySet.iterator(); - for (int i = 0; i < size; i++) { - assertThat(iterator.hasNext(), equalTo(true)); - Entry next = iterator.next(); - assertThat("pos: " + i, searchResponse.getHits().getAt(i).id(), equalTo(next.getValue())); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); - } - } - if (!sparseBytes.isEmpty()) { - int size = between(1, sparseBytes.size()); - SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()) - .setPostFilter(QueryBuilders.existsQuery("sparse_bytes")).setSize(size).addSort("sparse_bytes", SortOrder.ASC).execute() - .actionGet(); - assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().getTotalHits(), equalTo((long) sparseBytes.size())); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - Set> entrySet = sparseBytes.entrySet(); - Iterator> iterator = entrySet.iterator(); - for (int i = 0; i < size; i++) { - assertThat(iterator.hasNext(), equalTo(true)); - Entry next = iterator.next(); - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(next.getValue())); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(next.getKey().utf8ToString())); - } - } - } - - public void test3078() { - createIndex("test"); - ensureGreen(); - - for (int i = 1; i < 101; i++) { - client().prepareIndex("test", "type", Integer.toString(i)).setSource("field", Integer.toString(i)).execute().actionGet(); - } - refresh(); - SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); - - // reindex and refresh - client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); - refresh(); - - searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); - - // reindex - no refresh - client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); - - searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); - - // force merge - forceMerge(); - refresh(); - - client().prepareIndex("test", "type", Integer.toString(1)).setSource("field", Integer.toString(1)).execute().actionGet(); - searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); - - refresh(); - searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("field").order(SortOrder.ASC)).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).sortValues()[0].toString(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).sortValues()[0].toString(), equalTo("10")); - assertThat(searchResponse.getHits().getAt(2).sortValues()[0].toString(), equalTo("100")); - } - - public void testScoreSortDirection() throws Exception { - createIndex("test"); - ensureGreen(); - - client().prepareIndex("test", "type", "1").setSource("field", 2).execute().actionGet(); - client().prepareIndex("test", "type", "2").setSource("field", 1).execute().actionGet(); - client().prepareIndex("test", "type", "3").setSource("field", 0).execute().actionGet(); - - refresh(); - - SearchResponse searchResponse = client() - .prepareSearch("test") - .setQuery( - QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) - .execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); - assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); - assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); - - searchResponse = client() - .prepareSearch("test") - .setQuery( - QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) - .addSort("_score", SortOrder.DESC).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); - assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); - assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); - - searchResponse = client() - .prepareSearch("test") - .setQuery( - QueryBuilders.functionScoreQuery(matchAllQuery(), ScoreFunctionBuilders.fieldValueFactorFunction("field"))) - .addSort("_score", SortOrder.DESC).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - } - - public void testScoreSortDirectionWithFunctionScore() throws Exception { - createIndex("test"); - ensureGreen(); - - client().prepareIndex("test", "type", "1").setSource("field", 2).execute().actionGet(); - client().prepareIndex("test", "type", "2").setSource("field", 1).execute().actionGet(); - client().prepareIndex("test", "type", "3").setSource("field", 0).execute().actionGet(); - - refresh(); - - SearchResponse searchResponse = client().prepareSearch("test") - .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); - assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); - assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); - - searchResponse = client().prepareSearch("test") - .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))) - .addSort("_score", SortOrder.DESC).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).score(), Matchers.lessThan(searchResponse.getHits().getAt(0).score())); - assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).score(), Matchers.lessThan(searchResponse.getHits().getAt(1).score())); - assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); - - searchResponse = client().prepareSearch("test") - .setQuery(functionScoreQuery(matchAllQuery(), fieldValueFactorFunction("field"))) - .addSort("_score", SortOrder.DESC).execute().actionGet(); - assertThat(searchResponse.getHits().getAt(2).getId(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(1).getId(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); - } - - public void testIssue2986() { - createIndex("test"); - - client().prepareIndex("test", "post", "1").setSource("{\"field1\":\"value1\"}").execute().actionGet(); - client().prepareIndex("test", "post", "2").setSource("{\"field1\":\"value2\"}").execute().actionGet(); - client().prepareIndex("test", "post", "3").setSource("{\"field1\":\"value3\"}").execute().actionGet(); - refresh(); - SearchResponse result = client().prepareSearch("test").setQuery(matchAllQuery()).setTrackScores(true).addSort("field1", SortOrder.ASC).execute().actionGet(); - - for (SearchHit hit : result.getHits()) { - assertFalse(Float.isNaN(hit.getScore())); - } - } - - public void testIssue2991() { - for (int i = 1; i < 4; i++) { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - createIndex("test"); - ensureGreen(); - client().prepareIndex("test", "type", "1").setSource("tag", "alpha").execute().actionGet(); - refresh(); - - client().prepareIndex("test", "type", "3").setSource("tag", "gamma").execute().actionGet(); - refresh(); - - client().prepareIndex("test", "type", "4").setSource("tag", "delta").execute().actionGet(); - - refresh(); - client().prepareIndex("test", "type", "2").setSource("tag", "beta").execute().actionGet(); - - refresh(); - SearchResponse resp = client().prepareSearch("test").setSize(2).setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("tag").order(SortOrder.ASC)).execute().actionGet(); - assertHitCount(resp, 4); - assertThat(resp.getHits().hits().length, equalTo(2)); - assertFirstHit(resp, hasId("1")); - assertSecondHit(resp, hasId("2")); - - resp = client().prepareSearch("test").setSize(2).setQuery(matchAllQuery()).addSort(SortBuilders.fieldSort("tag").order(SortOrder.DESC)).execute().actionGet(); - assertHitCount(resp, 4); - assertThat(resp.getHits().hits().length, equalTo(2)); - assertFirstHit(resp, hasId("3")); - assertSecondHit(resp, hasId("4")); - } - } - public void testSimpleSorts() throws Exception { Random random = random(); assertAcked(prepareCreate("test") @@ -531,41 +103,10 @@ public class SimpleSortTests extends ESIntegTestCase { } refresh(); - // STRING + // STRING script int size = 1 + random.nextInt(10); SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(size) - .addSort("str_value", SortOrder.ASC) - .execute().actionGet(); - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(new String(new char[]{(char) (97 + i), (char) (97 + i)}))); - } - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(size) - .addSort("str_value", SortOrder.DESC) - .execute().actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(searchResponse.getHits().getAt(i).sortValues()[0].toString(), equalTo(new String(new char[]{(char) (97 + (9 - i)), (char) (97 + (9 - i))}))); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - - - // STRING script - size = 1 + random.nextInt(10); - - searchResponse = client().prepareSearch() .setQuery(matchAllQuery()) .setSize(size) .addSort(new ScriptSortBuilder(new Script("doc['str_value'].value"), "string")).execute().actionGet(); @@ -590,172 +131,6 @@ public class SimpleSortTests extends ESIntegTestCase { assertThat(searchResponse.toString(), not(containsString("error"))); - // BYTE - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("byte_value", SortOrder.ASC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).byteValue(), equalTo((byte) i)); - } - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("byte_value", SortOrder.DESC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).byteValue(), equalTo((byte) (9 - i))); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - - // SHORT - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("short_value", SortOrder.ASC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).shortValue(), equalTo((short) i)); - } - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("short_value", SortOrder.DESC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).shortValue(), equalTo((short) (9 - i))); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - - // INTEGER - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("integer_value", SortOrder.ASC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).intValue(), equalTo(i)); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("integer_value", SortOrder.DESC) - .execute().actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).intValue(), equalTo((9 - i))); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - - // LONG - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("long_value", SortOrder.ASC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).longValue(), equalTo((long) i)); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("long_value", SortOrder.DESC).execute() - .actionGet(); - assertHitCount(searchResponse, 10L); - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).longValue(), equalTo((long) (9 - i))); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - - // FLOAT - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("float_value", SortOrder.ASC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10L); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("float_value", SortOrder.DESC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - - // DOUBLE - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("double_value", SortOrder.ASC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10L); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * i, 0.000001d)); - } - - assertThat(searchResponse.toString(), not(containsString("error"))); - size = 1 + random.nextInt(10); - searchResponse = client().prepareSearch().setQuery(matchAllQuery()).setSize(size).addSort("double_value", SortOrder.DESC).execute() - .actionGet(); - - assertHitCount(searchResponse, 10L); - assertThat(searchResponse.getHits().hits().length, equalTo(size)); - for (int i = 0; i < size; i++) { - assertThat(searchResponse.getHits().getAt(i).id(), equalTo(Integer.toString(9 - i))); - assertThat(((Number) searchResponse.getHits().getAt(i).sortValues()[0]).doubleValue(), closeTo(0.1d * (9 - i), 0.000001d)); - } - - assertNoFailures(searchResponse); - } - - public void test2920() throws IOException { - assertAcked(prepareCreate("test").addMapping( - "test", - jsonBuilder().startObject().startObject("test").startObject("properties").startObject("value").field("type", "string") - .endObject().endObject().endObject().endObject())); - ensureGreen(); - for (int i = 0; i < 10; i++) { - client().prepareIndex("test", "test", Integer.toString(i)) - .setSource(jsonBuilder().startObject().field("value", "" + i).endObject()).execute().actionGet(); - } - refresh(); - SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()) - .addSort(SortBuilders.scriptSort(new Script("\u0027\u0027"), "string")).setSize(10).execute().actionGet(); assertNoFailures(searchResponse); } @@ -939,773 +314,19 @@ public class SimpleSortTests extends ESIntegTestCase { assertThat(searchResponse.getHits().getAt(0).field("id").value(), equalTo("2")); } - public void testSortMissingNumbers() throws Exception { - assertAcked(prepareCreate("test").addMapping("type1", - XContentFactory.jsonBuilder() - .startObject() - .startObject("type1") - .startObject("properties") - .startObject("i_value") - .field("type", "integer") - .startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject() - .endObject() - .startObject("d_value") - .field("type", "float") - .startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject() - .endObject() - .endObject() - .endObject() - .endObject())); + public void test2920() throws IOException { + assertAcked(prepareCreate("test").addMapping( + "test", + jsonBuilder().startObject().startObject("test").startObject("properties").startObject("value").field("type", "string") + .endObject().endObject().endObject().endObject())); ensureGreen(); - client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() - .field("id", "1") - .field("i_value", -1) - .field("d_value", -1.1) - .endObject()).execute().actionGet(); - - client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() - .field("id", "2") - .endObject()).execute().actionGet(); - - client().prepareIndex("test", "type1", "3").setSource(jsonBuilder().startObject() - .field("id", "1") - .field("i_value", 2) - .field("d_value", 2.2) - .endObject()).execute().actionGet(); - - flush(); - refresh(); - - logger.info("--> sort with no missing (same as missing _last)"); - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("i_value").order(SortOrder.ASC)) - .execute().actionGet(); - assertNoFailures(searchResponse); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); - - logger.info("--> sort with missing _last"); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("i_value").order(SortOrder.ASC).missing("_last")) - .execute().actionGet(); - assertNoFailures(searchResponse); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); - - logger.info("--> sort with missing _first"); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("i_value").order(SortOrder.ASC).missing("_first")) - .execute().actionGet(); - assertNoFailures(searchResponse); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); - } - - public void testSortMissingStrings() throws IOException { - assertAcked(prepareCreate("test").addMapping("type1", - XContentFactory.jsonBuilder() - .startObject() - .startObject("type1") - .startObject("properties") - .startObject("value") - .field("type", "string") - .field("index", "not_analyzed") - .endObject() - .endObject() - .endObject() - .endObject())); - ensureGreen(); - client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() - .field("id", "1") - .field("value", "a") - .endObject()).execute().actionGet(); - - client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() - .field("id", "2") - .endObject()).execute().actionGet(); - - client().prepareIndex("test", "type1", "3").setSource(jsonBuilder().startObject() - .field("id", "1") - .field("value", "c") - .endObject()).execute().actionGet(); - - flush(); - refresh(); - - // TODO: WTF? - try { - Thread.sleep(2000); - } catch (InterruptedException e) { - throw new RuntimeException(); - } - - logger.info("--> sort with no missing (same as missing _last)"); - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC)) - .execute().actionGet(); - assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); - - logger.info("--> sort with missing _last"); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC).missing("_last")) - .execute().actionGet(); - assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("3")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("2")); - - logger.info("--> sort with missing _first"); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC).missing("_first")) - .execute().actionGet(); - assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); - - logger.info("--> sort with missing b"); - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("value").order(SortOrder.ASC).missing("b")) - .execute().actionGet(); - assertThat(Arrays.toString(searchResponse.getShardFailures()), searchResponse.getFailedShards(), equalTo(0)); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); - assertThat(searchResponse.getHits().getAt(1).id(), equalTo("2")); - assertThat(searchResponse.getHits().getAt(2).id(), equalTo("3")); - } - - public void testIgnoreUnmapped() throws Exception { - createIndex("test"); - ensureYellow(); - - client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() - .field("id", "1") - .field("i_value", -1) - .field("d_value", -1.1) - .endObject()).execute().actionGet(); - - logger.info("--> sort with an unmapped field, verify it fails"); - try { - SearchResponse result = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("kkk")) - .execute().actionGet(); - assertThat("Expected exception but returned with", result, nullValue()); - } catch (SearchPhaseExecutionException e) { - //we check that it's a parse failure rather than a different shard failure - for (ShardSearchFailure shardSearchFailure : e.shardFailures()) { - assertThat(shardSearchFailure.toString(), containsString("[No mapping found for [kkk] in order to sort on]")); - } - } - - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("kkk").unmappedType("string")) - .execute().actionGet(); - assertNoFailures(searchResponse); - } - - public void testSortMVField() throws Exception { - assertAcked(prepareCreate("test") - .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties") - .startObject("long_values").field("type", "long").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .startObject("int_values").field("type", "integer").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .startObject("short_values").field("type", "short").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .startObject("byte_values").field("type", "byte").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .startObject("float_values").field("type", "float").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .startObject("double_values").field("type", "double").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .startObject("string_values").field("type", "string").field("index", "not_analyzed").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .endObject().endObject().endObject())); - ensureGreen(); - - client().prepareIndex("test", "type1", Integer.toString(1)).setSource(jsonBuilder().startObject() - .array("long_values", 1L, 5L, 10L, 8L) - .array("int_values", 1, 5, 10, 8) - .array("short_values", 1, 5, 10, 8) - .array("byte_values", 1, 5, 10, 8) - .array("float_values", 1f, 5f, 10f, 8f) - .array("double_values", 1d, 5d, 10d, 8d) - .array("string_values", "01", "05", "10", "08") - .endObject()).execute().actionGet(); - client().prepareIndex("test", "type1", Integer.toString(2)).setSource(jsonBuilder().startObject() - .array("long_values", 11L, 15L, 20L, 7L) - .array("int_values", 11, 15, 20, 7) - .array("short_values", 11, 15, 20, 7) - .array("byte_values", 11, 15, 20, 7) - .array("float_values", 11f, 15f, 20f, 7f) - .array("double_values", 11d, 15d, 20d, 7d) - .array("string_values", "11", "15", "20", "07") - .endObject()).execute().actionGet(); - client().prepareIndex("test", "type1", Integer.toString(3)).setSource(jsonBuilder().startObject() - .array("long_values", 2L, 1L, 3L, -4L) - .array("int_values", 2, 1, 3, -4) - .array("short_values", 2, 1, 3, -4) - .array("byte_values", 2, 1, 3, -4) - .array("float_values", 2f, 1f, 3f, -4f) - .array("double_values", 2d, 1d, 3d, -4d) - .array("string_values", "02", "01", "03", "!4") - .endObject()).execute().actionGet(); - - refresh(); - - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("long_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(-4L)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(1L)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(7L)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("long_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(20L)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(10L)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(3L)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort(SortBuilders.fieldSort("long_values").order(SortOrder.DESC).sortMode("sum")) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).longValue(), equalTo(53L)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).longValue(), equalTo(24L)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).longValue(), equalTo(2L)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("int_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("int_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("short_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("short_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("byte_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(-4)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(1)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(7)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("byte_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).intValue(), equalTo(20)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).intValue(), equalTo(10)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).intValue(), equalTo(3)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("float_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).floatValue(), equalTo(-4f)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).floatValue(), equalTo(1f)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).floatValue(), equalTo(7f)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("float_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).floatValue(), equalTo(20f)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).floatValue(), equalTo(10f)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).floatValue(), equalTo(3f)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("double_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(-4d)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(1d)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(7d)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("double_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Number) searchResponse.getHits().getAt(0).sortValues()[0]).doubleValue(), equalTo(20d)); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Number) searchResponse.getHits().getAt(1).sortValues()[0]).doubleValue(), equalTo(10d)); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Number) searchResponse.getHits().getAt(2).sortValues()[0]).doubleValue(), equalTo(3d)); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("string_values", SortOrder.ASC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(3))); - assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("!4")); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("01")); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(2))); - assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("07")); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(10) - .addSort("string_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().getTotalHits(), equalTo(3L)); - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("03")); - } - - public void testSortOnRareField() throws IOException { - assertAcked(prepareCreate("test") - .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties") - .startObject("string_values").field("type", "string").field("index", "not_analyzed").startObject("fielddata").field("format", random().nextBoolean() ? "doc_values" : null).endObject().endObject() - .endObject().endObject().endObject())); - ensureGreen(); - client().prepareIndex("test", "type1", Integer.toString(1)).setSource(jsonBuilder().startObject() - .array("string_values", "01", "05", "10", "08") - .endObject()).execute().actionGet(); - - - refresh(); - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(3) - .addSort("string_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().hits().length, equalTo(1)); - - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(1))); - assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("10")); - - client().prepareIndex("test", "type1", Integer.toString(2)).setSource(jsonBuilder().startObject() - .array("string_values", "11", "15", "20", "07") - .endObject()).execute().actionGet(); - for (int i = 0; i < 15; i++) { - client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() - .array("some_other_field", "foobar") - .endObject()).execute().actionGet(); + for (int i = 0; i < 10; i++) { + client().prepareIndex("test", "test", Integer.toString(i)) + .setSource(jsonBuilder().startObject().field("value", "" + i).endObject()).execute().actionGet(); } refresh(); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(2) - .addSort("string_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().hits().length, equalTo(2)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); - - - client().prepareIndex("test", "type1", Integer.toString(3)).setSource(jsonBuilder().startObject() - .array("string_values", "02", "01", "03", "!4") - .endObject()).execute().actionGet(); - for (int i = 0; i < 15; i++) { - client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() - .array("some_other_field", "foobar") - .endObject()).execute().actionGet(); - } - refresh(); - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(3) - .addSort("string_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("03")); - - for (int i = 0; i < 15; i++) { - client().prepareIndex("test", "type1", Integer.toString(300 + i)).setSource(jsonBuilder().startObject() - .array("some_other_field", "foobar") - .endObject()).execute().actionGet(); - refresh(); - } - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(3) - .addSort("string_values", SortOrder.DESC) - .execute().actionGet(); - - assertThat(searchResponse.getHits().hits().length, equalTo(3)); - - assertThat(searchResponse.getHits().getAt(0).id(), equalTo(Integer.toString(2))); - assertThat(((Text) searchResponse.getHits().getAt(0).sortValues()[0]).string(), equalTo("20")); - - assertThat(searchResponse.getHits().getAt(1).id(), equalTo(Integer.toString(1))); - assertThat(((Text) searchResponse.getHits().getAt(1).sortValues()[0]).string(), equalTo("10")); - - assertThat(searchResponse.getHits().getAt(2).id(), equalTo(Integer.toString(3))); - assertThat(((Text) searchResponse.getHits().getAt(2).sortValues()[0]).string(), equalTo("03")); - } - - public void testSortMetaField() throws Exception { - XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type") - .startObject("_timestamp").field("enabled", true).endObject() - .endObject().endObject(); - assertAcked(prepareCreate("test") - .addMapping("type", mapping)); - ensureGreen(); - final int numDocs = randomIntBetween(10, 20); - IndexRequestBuilder[] indexReqs = new IndexRequestBuilder[numDocs]; - for (int i = 0; i < numDocs; ++i) { - indexReqs[i] = client().prepareIndex("test", "type", Integer.toString(i)).setTimestamp(Integer.toString(randomInt(1000))).setSource(); - } - indexRandom(true, indexReqs); - - SortOrder order = randomFrom(SortOrder.values()); - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(randomIntBetween(1, numDocs + 5)) - .addSort("_uid", order) - .execute().actionGet(); + SearchResponse searchResponse = client().prepareSearch().setQuery(matchAllQuery()) + .addSort(SortBuilders.scriptSort(new Script("\u0027\u0027"), "string")).setSize(10).execute().actionGet(); assertNoFailures(searchResponse); - SearchHit[] hits = searchResponse.getHits().hits(); - BytesRef previous = order == SortOrder.ASC ? new BytesRef() : UnicodeUtil.BIG_TERM; - for (int i = 0; i < hits.length; ++i) { - final BytesRef uid = new BytesRef(Uid.createUid(hits[i].type(), hits[i].id())); - assertThat(previous, order == SortOrder.ASC ? lessThan(uid) : greaterThan(uid)); - previous = uid; - } - - /* - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(randomIntBetween(1, numDocs + 5)) - .addSort("_id", order) - .execute().actionGet(); - assertNoFailures(searchResponse); - hits = searchResponse.getHits().hits(); - previous = order == SortOrder.ASC ? new BytesRef() : UnicodeUtil.BIG_TERM; - for (int i = 0; i < hits.length; ++i) { - final BytesRef id = new BytesRef(Uid.createUid(hits[i].type(), hits[i].id())); - assertThat(previous, order == SortOrder.ASC ? lessThan(id) : greaterThan(id)); - previous = id; - }*/ - - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .setSize(randomIntBetween(1, numDocs + 5)) - .addSort("_timestamp", order) - .addField("_timestamp") - .execute().actionGet(); - assertNoFailures(searchResponse); - hits = searchResponse.getHits().hits(); - Long previousTs = order == SortOrder.ASC ? 0 : Long.MAX_VALUE; - for (int i = 0; i < hits.length; ++i) { - SearchHitField timestampField = hits[i].getFields().get("_timestamp"); - Long timestamp = timestampField.getValue(); - assertThat(previousTs, order == SortOrder.ASC ? lessThanOrEqualTo(timestamp) : greaterThanOrEqualTo(timestamp)); - previousTs = timestamp; - } - } - - /** - * Test case for issue 6150: https://github.com/elasticsearch/elasticsearch/issues/6150 - */ - public void testNestedSort() throws IOException, InterruptedException, ExecutionException { - assertAcked(prepareCreate("test") - .addMapping("type", - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("properties") - .startObject("nested") - .field("type", "nested") - .startObject("properties") - .startObject("foo") - .field("type", "string") - .startObject("fields") - .startObject("sub") - .field("type", "string") - .field("index", "not_analyzed") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - .endObject())); - ensureGreen(); - - client().prepareIndex("test", "type", "1").setSource(jsonBuilder().startObject() - .startObject("nested") - .field("foo", "bar bar") - .endObject() - .endObject()).execute().actionGet(); - refresh(); - - // We sort on nested field - SearchResponse searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("nested.foo").setNestedPath("nested").order(SortOrder.DESC)) - .execute().actionGet(); - assertNoFailures(searchResponse); - SearchHit[] hits = searchResponse.getHits().hits(); - for (int i = 0; i < hits.length; ++i) { - assertThat(hits[i].getSortValues().length, is(1)); - Object o = hits[i].getSortValues()[0]; - assertThat(o, notNullValue()); - Text text = (Text) o; - assertThat(text.string(), is("bar")); - } - - - // We sort on nested sub field - searchResponse = client().prepareSearch() - .setQuery(matchAllQuery()) - .addSort(SortBuilders.fieldSort("nested.foo.sub").setNestedPath("nested").order(SortOrder.DESC)) - .execute().actionGet(); - assertNoFailures(searchResponse); - hits = searchResponse.getHits().hits(); - for (int i = 0; i < hits.length; ++i) { - assertThat(hits[i].getSortValues().length, is(1)); - Object o = hits[i].getSortValues()[0]; - assertThat(o, notNullValue()); - Text text = (Text) o; - assertThat(text.string(), is("bar bar")); - } - } - - public void testSortDuelBetweenSingleShardAndMultiShardIndex() throws Exception { - String sortField = "sortField"; - assertAcked(prepareCreate("test1") - .setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, between(2, maximumNumberOfShards())) - .addMapping("type", sortField, "type=long").get()); - assertAcked(prepareCreate("test2") - .setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .addMapping("type", sortField, "type=long").get()); - - for (String index : new String[]{"test1", "test2"}) { - List docs = new ArrayList<>(); - for (int i = 0; i < 256; i++) { - docs.add(client().prepareIndex(index, "type", Integer.toString(i)).setSource(sortField, i)); - } - indexRandom(true, docs); - } - - ensureSearchable("test1", "test2"); - SortOrder order = randomBoolean() ? SortOrder.ASC : SortOrder.DESC; - int from = between(0, 256); - int size = between(0, 256); - SearchResponse multiShardResponse = client().prepareSearch("test1").setFrom(from).setSize(size).addSort(sortField, order).get(); - assertNoFailures(multiShardResponse); - SearchResponse singleShardResponse = client().prepareSearch("test2").setFrom(from).setSize(size).addSort(sortField, order).get(); - assertNoFailures(singleShardResponse); - - assertThat(multiShardResponse.getHits().totalHits(), equalTo(singleShardResponse.getHits().totalHits())); - assertThat(multiShardResponse.getHits().getHits().length, equalTo(singleShardResponse.getHits().getHits().length)); - for (int i = 0; i < multiShardResponse.getHits().getHits().length; i++) { - assertThat(multiShardResponse.getHits().getAt(i).sortValues()[0], equalTo(singleShardResponse.getHits().getAt(i).sortValues()[0])); - assertThat(multiShardResponse.getHits().getAt(i).id(), equalTo(singleShardResponse.getHits().getAt(i).id())); - } } }