Improved warm-up of new segments.

* Merged segments are now warmed-up at the end of the merge operation instead
  of _refresh, so that _refresh doesn't pay the price for the warm-up of merged
  segments, which is often higher than flushed segments because of their size.
* Even when no _warmer is registered, some basic warm-up of the segments is
  performed: norms, doc values (_version). This should help a bit people who
  forget to register warmers.
* Eager loading support for the parent id cache and field data: when one
  can't predict what terms will be present in the index, it is tempting to use
  a match_all query in a warmer, but in that case, query execution might not be
  much faster than field data loading so having a warmer that only loads field
  data without running a query can be useful.

Closes #3819
This commit is contained in:
Adrien Grand 2013-10-04 21:40:37 +02:00
parent 88a2f54dfe
commit 97958ed02a
11 changed files with 296 additions and 61 deletions

View File

@ -24,6 +24,29 @@ field data after a certain time of inactivity. Defaults to `-1`. For
example, can be set to `5m` for a 5 minute expiry.
|=======================================================================
[float]
=== Fielddata loading
By default, field data is loaded lazily, on the first time that a query that
requires field data is fired. However, this can make the first requests that
follow a merge operation quite slow since fielddata loading is a heavy
operation.
It is possible to force field data to be loaded and cached eagerly through the
`loading` setting of fielddata:
[source,js]
--------------------------------------------------
{
category: {
type: "string",
fielddata: {
loading: "eager"
}
}
}
--------------------------------------------------
[float]
[[field-data-filtering]]
=== Filtering fielddata

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.engine.robin;
import com.google.common.collect.Lists;
import org.apache.lucene.index.*;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.SearcherFactory;
@ -1283,6 +1284,15 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
}
}
/** Returns whether a leaf reader comes from a merge (versus flush or addIndexes). */
private static boolean isMergedSegment(AtomicReader reader) {
// We expect leaves to be segment readers
final Map<String, String> diagnostics = ((SegmentReader) reader).getSegmentInfo().info.getDiagnostics();
final String source = diagnostics.get(IndexWriter.SOURCE);
assert Arrays.asList(IndexWriter.SOURCE_ADDINDEXES_READERS, IndexWriter.SOURCE_FLUSH, IndexWriter.SOURCE_MERGE).contains(source) : "Unknown source " + source;
return IndexWriter.SOURCE_MERGE.equals(source);
}
private IndexWriter createWriter() throws IOException {
try {
// release locks when started
@ -1313,6 +1323,28 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
* in combination with the default writelock timeout*/
config.setWriteLockTimeout(5000);
config.setUseCompoundFile(this.compoundOnFlush);
// Warm-up hook for newly-merged segments. Warming up segments here is better since it will be performed at the end
// of the merge operation and won't slow down _refresh
config.setMergedSegmentWarmer(new IndexReaderWarmer() {
@Override
public void warm(AtomicReader reader) throws IOException {
try {
assert isMergedSegment(reader);
final Engine.Searcher searcher = new SimpleSearcher("warmer", new IndexSearcher(reader));
final IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId, searcher);
warmer.warm(context);
} catch (Throwable t) {
// Don't fail a merge if the warm-up failed
if (!closed) {
logger.warn("Warm-up failed", t);
}
if (t instanceof Error) {
// assertion/out-of-memory error, don't ignore those
throw (Error) t;
}
}
}
});
return new IndexWriter(store.directory(), config);
} catch (LockObtainFailedException ex) {
boolean isLocked = IndexWriter.isLocked(store.directory());
@ -1485,6 +1517,10 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
// figure out the newSearcher, with only the new readers that are relevant for us
List<IndexReader> readers = Lists.newArrayList();
for (AtomicReaderContext newReaderContext : searcher.getIndexReader().leaves()) {
if (isMergedSegment(newReaderContext.reader())) {
// merged segments are already handled by IndexWriterConfig.setMergedSegmentWarmer
continue;
}
boolean found = false;
for (AtomicReaderContext currentReaderContext : currentSearcher.reader().leaves()) {
if (currentReaderContext.reader().getCoreCacheKey().equals(newReaderContext.reader().getCoreCacheKey())) {
@ -1505,7 +1541,6 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
if (newSearcher != null) {
IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId,
new SimpleSearcher("warmer", searcher),
new SimpleSearcher("warmer", newSearcher));
warmer.warm(context);
}

View File

@ -21,12 +21,22 @@ package org.elasticsearch.index.fielddata;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.MapperParsingException;
/**
*/
public class FieldDataType {
private static final String LOADING_KEY = "loading";
private static final String EAGER_LOADING_VALUE = "eager";
private static final String LAZY_LOADING_VALUE = "lazy";
public static enum Loading {
LAZY, EAGER;
}
private final String type;
private final Loading loading;
private final Settings settings;
public FieldDataType(String type) {
@ -40,6 +50,14 @@ public class FieldDataType {
public FieldDataType(String type, Settings settings) {
this.type = type;
this.settings = settings;
final String loading = settings.get(LOADING_KEY);
if (loading == null || loading.equals(LAZY_LOADING_VALUE)) {
this.loading = Loading.LAZY;
} else if (loading.equals(EAGER_LOADING_VALUE)) {
this.loading = Loading.EAGER;
} else {
throw new MapperParsingException("Unknown [" + LOADING_KEY + "] value: [" + loading + "]");
}
}
public String getType() {
@ -50,6 +68,10 @@ public class FieldDataType {
return this.settings;
}
public Loading getLoading() {
return loading;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;

View File

@ -30,12 +30,14 @@ import org.elasticsearch.index.cache.id.ShardIdCache;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.fielddata.ShardFieldData;
import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.get.ShardGetService;
import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.merge.MergeStats;
@ -44,6 +46,7 @@ import org.elasticsearch.index.percolator.stats.ShardPercolateService;
import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.search.stats.ShardSearchService;
import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.DocsStats;
import org.elasticsearch.index.shard.IndexShardComponent;
import org.elasticsearch.index.shard.IndexShardState;
@ -106,6 +109,12 @@ public interface IndexShard extends IndexShardComponent {
ShardTermVectorService termVectorService();
MapperService mapperService();
IndexFieldDataService indexFieldDataService();
IndexService indexService();
IndexShardState state();
Engine.Create prepareCreate(SourceToParse source) throws ElasticSearchException;

View File

@ -50,6 +50,7 @@ import org.elasticsearch.index.cache.id.ShardIdCache;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.*;
import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.fielddata.ShardFieldData;
import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
@ -66,6 +67,7 @@ import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.nested.NonNestedDocsFilter;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.search.stats.ShardSearchService;
import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.*;
@ -116,6 +118,8 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
private final ShardPercolateService shardPercolateService;
private final CodecService codecService;
private final ShardTermVectorService termVectorService;
private final IndexFieldDataService indexFieldDataService;
private final IndexService indexService;
private final Object mutex = new Object();
private final String checkIndexOnStartup;
@ -141,7 +145,7 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService,
ShardFilterCache shardFilterCache, ShardIdCache shardIdCache, ShardFieldData shardFieldData,
PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService,
ShardTermVectorService termVectorService) {
ShardTermVectorService termVectorService, IndexFieldDataService indexFieldDataService, IndexService indexService) {
super(shardId, indexSettings);
this.indicesLifecycle = (InternalIndicesLifecycle) indicesLifecycle;
this.indexSettingsService = indexSettingsService;
@ -164,6 +168,8 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
this.shardFieldData = shardFieldData;
this.percolatorQueriesRegistry = percolatorQueriesRegistry;
this.shardPercolateService = shardPercolateService;
this.indexFieldDataService = indexFieldDataService;
this.indexService = indexService;
this.codecService = codecService;
state = IndexShardState.CREATED;
@ -207,6 +213,20 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
return termVectorService;
}
@Override
public IndexFieldDataService indexFieldDataService() {
return indexFieldDataService;
}
@Override
public MapperService mapperService() {
return mapperService;
}
@Override
public IndexService indexService() {
return indexService;
}
@Override
public ShardSearchService searchService() {

View File

@ -35,19 +35,17 @@ public interface IndicesWarmer {
return ThreadPool.Names.WARMER;
}
public abstract void warm(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context);
public abstract void warm(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool);
}
public static class WarmerContext {
private final ShardId shardId;
private final Engine.Searcher fullSearcher;
private final Engine.Searcher newSearcher;
public WarmerContext(ShardId shardId, Engine.Searcher fullSearcher, Engine.Searcher newSearcher) {
public WarmerContext(ShardId shardId, Engine.Searcher newSearcher) {
this.shardId = shardId;
this.fullSearcher = fullSearcher;
this.newSearcher = newSearcher;
}
@ -55,10 +53,7 @@ public interface IndicesWarmer {
return shardId;
}
public Engine.Searcher fullSearcher() {
return fullSearcher;
}
/** Return a searcher instance that only wraps the segments to warm. */
public Engine.Searcher newSearcher() {
return newSearcher;
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
/**
@ -83,29 +82,12 @@ public class InternalIndicesWarmer extends AbstractComponent implements IndicesW
return;
}
if (logger.isTraceEnabled()) {
logger.trace("[{}][{}] warming [{}], new [{}]", context.shardId().index().name(), context.shardId().id(), context.fullSearcher().reader(), context.newSearcher().reader());
logger.trace("[{}][{}] warming [{}]", context.shardId().index().name(), context.shardId().id(), context.newSearcher().reader());
}
indexShard.warmerService().onPreWarm();
long time = System.nanoTime();
for (final Listener listener : listeners) {
final CountDownLatch latch = new CountDownLatch(1);
threadPool.executor(listener.executor()).execute(new Runnable() {
@Override
public void run() {
try {
listener.warm(indexShard, indexMetaData, context);
} catch (Throwable e) {
indexShard.warmerService().logger().warn("failed to warm [{}]", e, listener);
} finally {
latch.countDown();
}
}
});
try {
latch.await();
} catch (InterruptedException e) {
return;
}
listener.warm(indexShard, indexMetaData, context, threadPool);
}
long took = System.nanoTime() - time;
indexShard.warmerService().onPostWarm(took);

View File

@ -20,8 +20,12 @@
package org.elasticsearch.search;
import com.google.common.collect.ImmutableMap;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.SimpleMergedSegmentWarmer;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.util.InfoStream;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.ElasticSearchIllegalStateException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cache.recycler.CacheRecycler;
@ -39,12 +43,19 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.search.stats.StatsGroupsParseElement;
import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.warmer.IndicesWarmer;
import org.elasticsearch.indices.warmer.IndicesWarmer.WarmerContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.dfs.CachedDfSource;
import org.elasticsearch.search.dfs.DfsPhase;
@ -60,6 +71,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicLong;
@ -125,6 +137,8 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval);
this.indicesWarmer.addListener(new IndexReaderWarmer());
this.indicesWarmer.addListener(new FieldDataWarmer());
this.indicesWarmer.addListener(new SearchWarmer());
}
@ -618,36 +632,157 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
}
}
static class IndexReaderWarmer extends IndicesWarmer.Listener {
private final SimpleMergedSegmentWarmer warmer = new SimpleMergedSegmentWarmer(InfoStream.NO_OUTPUT);
@Override
public void warm(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool) {
long start = System.nanoTime();
try {
for (AtomicReaderContext ctx : context.newSearcher().reader().leaves()) {
warmer.warm(ctx.reader());
}
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed readers, took [{}]", TimeValue.timeValueNanos(System.nanoTime() - start));
}
} catch (Throwable t) {
throw new ElasticSearchIllegalStateException("Unexpected exception while warming-up segment", t);
}
}
}
static class FieldDataWarmer extends IndicesWarmer.Listener {
@Override
public void warm(final IndexShard indexShard, IndexMetaData indexMetaData, final WarmerContext context, ThreadPool threadPool) {
final MapperService mapperService = indexShard.mapperService();
final Map<String, FieldMapper<?>> warmUp = new HashMap<String, FieldMapper<?>>();
boolean parentChild = false;
for (DocumentMapper docMapper : mapperService) {
for (FieldMapper<?> fieldMapper : docMapper.mappers().mappers()) {
if (fieldMapper instanceof ParentFieldMapper) {
parentChild = true;
}
final FieldDataType fieldDataType = fieldMapper.fieldDataType();
if (fieldDataType == null) {
continue;
}
if (fieldDataType.getLoading() != FieldDataType.Loading.EAGER) {
continue;
}
final String indexName = fieldMapper.names().indexName();
if (warmUp.containsKey(indexName)) {
continue;
}
warmUp.put(indexName, fieldMapper);
}
}
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
final int numTasks = warmUp.size() * context.newSearcher().reader().leaves().size() + (parentChild ? 1 : 0);
final CountDownLatch latch = new CountDownLatch(numTasks);
for (final AtomicReaderContext ctx : context.newSearcher().reader().leaves()) {
for (final FieldMapper<?> fieldMapper : warmUp.values()) {
threadPool.executor(executor()).execute(new Runnable() {
@Override
public void run() {
try {
final long start = System.nanoTime();
indexFieldDataService.getForField(fieldMapper).load(ctx);
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed fielddata for [{}], took [{}]", fieldMapper.names().name(), TimeValue.timeValueNanos(System.nanoTime() - start));
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("failed to warm-up fielddata for [{}]", t, fieldMapper.names().name());
} finally {
latch.countDown();
}
}
});
}
}
if (parentChild) {
threadPool.executor(executor()).execute(new Runnable() {
@Override
public void run() {
try {
final long start = System.nanoTime();
indexShard.indexService().cache().idCache().refresh(context.newSearcher().reader().leaves());
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed id_cache, took [{}]", TimeValue.timeValueNanos(System.nanoTime() - start));
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("failed to warm-up id cache", t);
} finally {
latch.countDown();
}
}
});
}
try {
latch.await();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
}
class SearchWarmer extends IndicesWarmer.Listener {
@Override
public void warm(IndexShard indexShard, IndexMetaData indexMetaData, IndicesWarmer.WarmerContext warmerContext) {
public void warm(final IndexShard indexShard, final IndexMetaData indexMetaData, final IndicesWarmer.WarmerContext warmerContext, ThreadPool threadPool) {
IndexWarmersMetaData custom = indexMetaData.custom(IndexWarmersMetaData.TYPE);
if (custom == null) {
return;
}
for (IndexWarmersMetaData.Entry entry : custom.entries()) {
SearchContext context = null;
try {
long now = System.nanoTime();
ShardSearchRequest request = new ShardSearchRequest(indexShard.shardId().index().name(), indexShard.shardId().id(), indexMetaData.numberOfShards(),
SearchType.QUERY_THEN_FETCH /* we don't use COUNT so sorting will also kick in whatever warming logic*/)
.source(entry.source())
.types(entry.types());
context = createContext(request, warmerContext.newSearcher());
queryPhase.execute(context);
long took = System.nanoTime() - now;
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed [{}], took [{}]", entry.name(), TimeValue.timeValueNanos(took));
final CountDownLatch latch = new CountDownLatch(custom.entries().size());
for (final IndexWarmersMetaData.Entry entry : custom.entries()) {
threadPool.executor(executor()).execute(new Runnable() {
@Override
public void run() {
SearchContext context = null;
try {
long now = System.nanoTime();
ShardSearchRequest request = new ShardSearchRequest(indexShard.shardId().index().name(), indexShard.shardId().id(), indexMetaData.numberOfShards(),
SearchType.QUERY_THEN_FETCH /* we don't use COUNT so sorting will also kick in whatever warming logic*/)
.source(entry.source())
.types(entry.types());
context = createContext(request, warmerContext.newSearcher());
queryPhase.execute(context);
long took = System.nanoTime() - now;
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed [{}], took [{}]", entry.name(), TimeValue.timeValueNanos(took));
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("warmer [{}] failed", t, entry.name());
} finally {
try {
if (context != null) {
freeContext(context);
cleanContext(context);
}
} finally {
latch.countDown();
}
}
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("warmer [{}] failed", t, entry.name());
} finally {
if (context != null) {
freeContext(context);
cleanContext(context);
}
}
});
}
try {
latch.await();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
}

View File

@ -345,12 +345,22 @@ public class SimpleChildQuerySearchTests extends AbstractIntegrationTest {
.put("index.number_of_replicas", 0)
).execute().actionGet();
client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();
client().prepareIndex("test", "parent", "p0").setSource("p_field", "p_value0").execute().actionGet();
client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet();
// No _parent field yet, there shouldn't be anything in the parent id cache
IndicesStatsResponse indicesStatsResponse = client().admin().indices()
.prepareStats("test").setIdCache(true).execute().actionGet();
assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), equalTo(0l));
// Now add mapping + children
client().admin().indices().preparePutMapping("test").setType("child").setSource(jsonBuilder().startObject().startObject("type")
.startObject("_parent").field("type", "parent").endObject()
.endObject().endObject()).execute().actionGet();
// index simple data
client().prepareIndex("test", "parent", "p1").setSource("p_field", "p_value1").execute().actionGet();
client().prepareIndex("test", "child", "c1").setSource("c_field", "red").setParent("p1").execute().actionGet();
client().prepareIndex("test", "child", "c2").setSource("c_field", "yellow").setParent("p1").execute().actionGet();
client().prepareIndex("test", "parent", "p2").setSource("p_field", "p_value2").execute().actionGet();
@ -359,9 +369,10 @@ public class SimpleChildQuerySearchTests extends AbstractIntegrationTest {
client().admin().indices().prepareRefresh().execute().actionGet();
IndicesStatsResponse indicesStatsResponse = client().admin().indices()
indicesStatsResponse = client().admin().indices()
.prepareStats("test").setIdCache(true).execute().actionGet();
assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), equalTo(0l));
// automatic warm-up has populated the cache since it found a parent field mapper
assertThat(indicesStatsResponse.getTotal().getIdCache().getMemorySizeInBytes(), greaterThan(0l));
SearchResponse searchResponse = client().prepareSearch("test")
.setQuery(constantScoreQuery(hasChildFilter("child", termQuery("c_field", "blue"))))

View File

@ -52,6 +52,7 @@ public class ExtendedFacetsTests extends AbstractIntegrationTest {
.field("index", "not_analyzed")
.startObject("fielddata")
.field("format", "paged_bytes")
.field("loading", randomBoolean() ? "eager" : "lazy")
.endObject()
.endObject()
.startObject("field1_fst")
@ -59,6 +60,7 @@ public class ExtendedFacetsTests extends AbstractIntegrationTest {
.field("index", "not_analyzed")
.startObject("fielddata")
.field("format", "fst")
.field("loading", randomBoolean() ? "eager" : "lazy")
.endObject()
.endObject()
.startObject("field2")
@ -66,6 +68,7 @@ public class ExtendedFacetsTests extends AbstractIntegrationTest {
.field("index", "not_analyzed")
.startObject("fielddata")
.field("format", "fst")
.field("loading", randomBoolean() ? "eager" : "lazy")
.endObject()
.endObject()
.startObject("q_field")

View File

@ -485,30 +485,30 @@ public class SimpleFacetsTests extends AbstractIntegrationTest {
.addMapping("type", jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("name_paged")
.field("type", "string")
.startObject("fielddata").field("format", "paged_bytes").endObject()
.startObject("fielddata").field("format", "paged_bytes").field("loading", randomBoolean() ? "eager" : "lazy").endObject()
.endObject()
.startObject("name_fst")
.field("type", "string")
.startObject("fielddata").field("format", "fst").endObject()
.startObject("fielddata").field("format", "fst").field("loading", randomBoolean() ? "eager" : "lazy").endObject()
.endObject()
.startObject("name_paged_mv")
.field("type", "string")
.startObject("fielddata").field("format", "paged_bytes").endObject()
.startObject("fielddata").field("format", "paged_bytes").field("loading", randomBoolean() ? "eager" : "lazy").endObject()
.endObject()
.startObject("name_fst_mv")
.field("type", "string")
.startObject("fielddata").field("format", "fst").endObject()
.startObject("fielddata").field("format", "fst").field("loading", randomBoolean() ? "eager" : "lazy").endObject()
.endObject()
.startObject("filtered")
.field("type", "string")
.startObject("fielddata").field("format", "fst").startObject("filter")
.startObject("fielddata").field("format", "fst").field("loading", randomBoolean() ? "eager" : "lazy").startObject("filter")
.startObject("regex").field("pattern", "\\d{1,2}").endObject().endObject()
.endObject()
// only 1 or 2 digits
.endObject()
.startObject("filtered_mv")
.field("type", "string")
.startObject("fielddata").field("format", "fst").startObject("filter")
.startObject("fielddata").field("format", "fst").field("loading", randomBoolean() ? "eager" : "lazy").startObject("filter")
.startObject("regex").field("pattern", "\\d{1,2}").endObject().endObject()
.endObject()
.endObject().endObject().endObject())