Refactor fielddata mappings. #17148

The fielddata settings in mappings have been refatored so that:
 - text and string have a `fielddata` (boolean) setting that tells whether it
   is ok to load in-memory fielddata. It is true by default for now but the
   plan is to make it default to false for text fields.
 - text and string have a `fielddata_frequency_filter` which contains the same
   thing as `fielddata.filter.frequency` used to (but validated at parsing time
   instead of being unchecked settings)
 - regex fielddata filtering is not supported anymore and will be dropped from
   mappings automatically on upgrade.
 - text, string and _parent fields have an `eager_global_ordinals` (boolean)
   setting that tells whether to load global ordinals eagerly on refresh.
 - in-memory fielddata is not supported on keyword fields anymore at all.
 - the `fielddata` setting is not supported on other fields that text and string
   and will be dropped when upgrading if specified.
This commit is contained in:
Adrien Grand 2016-03-16 16:25:35 +01:00
parent 435558a5c0
commit e50eeeaffb
114 changed files with 1232 additions and 2021 deletions

View File

@ -135,7 +135,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS_SETTING,
FsDirectoryService.INDEX_LOCK_FACTOR_SETTING,
EngineConfig.INDEX_CODEC_SETTING,
IndexWarmer.INDEX_NORMS_LOADING_SETTING,
// validate that built-in similarities don't get redefined
Setting.groupSetting("index.similarity.", (s) -> {
Map<String, Settings> groups = s.getAsGroups();

View File

@ -58,7 +58,6 @@ 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;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
@ -329,10 +328,10 @@ 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) -> {
final Engine.Warmer engineWarmer = (searcher) -> {
IndexShard shard = getShardOrNull(shardId.getId());
if (shard != null) {
warmer.warm(searcher, shard, IndexService.this.indexSettings, toLevel);
warmer.warm(searcher, shard, IndexService.this.indexSettings);
}
};
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock,
@ -525,21 +524,21 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
}
@Override
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, Accountable ramUsage) {
if (shardId != null) {
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
shard.fieldData().onCache(shardId, fieldName, fieldDataType, ramUsage);
shard.fieldData().onCache(shardId, fieldName, ramUsage);
}
}
}
@Override
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes) {
if (shardId != null) {
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
shard.fieldData().onRemoval(shardId, fieldName, fieldDataType, wasEvicted, sizeInBytes);
shard.fieldData().onRemoval(shardId, fieldName, wasEvicted, sizeInBytes);
}
}
}

View File

@ -19,14 +19,11 @@
package org.elasticsearch.index;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
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;
@ -50,9 +47,6 @@ import java.util.concurrent.TimeUnit;
*/
public final class IndexWarmer extends AbstractComponent {
public static final Setting<MappedFieldType.Loading> INDEX_NORMS_LOADING_SETTING = new Setting<>("index.norms.loading",
MappedFieldType.Loading.LAZY.toString(), (s) -> MappedFieldType.Loading.parse(s, MappedFieldType.Loading.LAZY),
Property.IndexScope);
private final List<Listener> listeners;
IndexWarmer(Settings settings, ThreadPool threadPool, Listener... listeners) {
@ -66,7 +60,7 @@ public final class IndexWarmer extends AbstractComponent {
this.listeners = Collections.unmodifiableList(list);
}
void warm(Engine.Searcher searcher, IndexShard shard, IndexSettings settings, boolean isTopReader) {
void warm(Engine.Searcher searcher, IndexShard shard, IndexSettings settings) {
if (shard.state() == IndexShardState.CLOSED) {
return;
}
@ -74,22 +68,14 @@ public final class IndexWarmer extends AbstractComponent {
return;
}
if (logger.isTraceEnabled()) {
if (isTopReader) {
logger.trace("{} top warming [{}]", shard.shardId(), searcher.reader());
} else {
logger.trace("{} warming [{}]", shard.shardId(), searcher.reader());
}
logger.trace("{} top warming [{}]", shard.shardId(), searcher.reader());
}
shard.warmerService().onPreWarm();
long time = System.nanoTime();
final List<TerminationHandle> 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));
}
terminationHandles.add(listener.warmReader(shard, searcher));
}
// wait for termination
for (TerminationHandle terminationHandle : terminationHandles) {
@ -97,22 +83,14 @@ public final class IndexWarmer extends AbstractComponent {
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);
}
logger.warn("top 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));
}
shard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS));
}
}
@ -127,9 +105,7 @@ public final class IndexWarmer extends AbstractComponent {
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);
TerminationHandle warmReader(IndexShard indexShard, Engine.Searcher searcher);
}
private static class FieldDataWarmer implements IndexWarmer.Listener {
@ -140,67 +116,17 @@ public final class IndexWarmer extends AbstractComponent {
}
@Override
public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
final MapperService mapperService = indexShard.mapperService();
final Map<String, MappedFieldType> 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) {
public TerminationHandle warmReader(final IndexShard indexShard, final Engine.Searcher searcher) {
final MapperService mapperService = indexShard.mapperService();
final Map<String, MappedFieldType> 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) {
final MappedFieldType fieldType = fieldMapper.fieldType();
final String indexName = fieldType.name();
if (fieldType.eagerGlobalOrdinals() == false) {
continue;
}
if (fieldDataType.getLoading() != MappedFieldType.Loading.EAGER_GLOBAL_ORDINALS) {
continue;
}
if (warmUpGlobalOrdinals.containsKey(indexName)) {
continue;
}
warmUpGlobalOrdinals.put(indexName, fieldMapper.fieldType());
warmUpGlobalOrdinals.put(indexName, fieldType);
}
}
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
@ -210,7 +136,12 @@ public final class IndexWarmer extends AbstractComponent {
try {
final long start = System.nanoTime();
IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType);
ifd.loadGlobal(searcher.getDirectoryReader());
DirectoryReader reader = searcher.getDirectoryReader();
IndexFieldData<?> global = ifd.loadGlobal(reader);
if (reader.leaves().isEmpty() == false) {
global.load(reader.leaves().get(0));
}
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.name(),
TimeValue.timeValueNanos(System.nanoTime() - start));

View File

@ -216,7 +216,7 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L
}
@Override
public IndexWarmer.TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
public IndexWarmer.TerminationHandle warmReader(final IndexShard indexShard, final Engine.Searcher searcher) {
if (indexSettings.getIndex().equals(indexShard.indexSettings().getIndex()) == false) {
// this is from a different index
return TerminationHandle.NO_WAIT;
@ -268,11 +268,6 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L
return () -> latch.await();
}
@Override
public TerminationHandle warmTopReader(IndexShard indexShard, Engine.Searcher searcher) {
return TerminationHandle.NO_WAIT;
}
}
Cache<Object, Cache<Query, Value>> getLoadedFilters() {

View File

@ -1219,12 +1219,9 @@ public abstract class Engine implements Closeable {
*/
public interface Warmer {
/**
* Called once a new Searcher is opened.
* @param searcher the searcer to warm
* @param isTopLevelReader <code>true</code> iff the searcher is build from a top-level reader.
* Otherwise the searcher might be build from a leaf reader to warm in isolation
* Called once a new Searcher is opened on the top-level searcher.
*/
void warm(Engine.Searcher searcher, boolean isTopLevelReader);
void warm(Engine.Searcher searcher);
}
/**

View File

@ -108,7 +108,7 @@ public final class EngineConfig {
final Settings settings = indexSettings.getSettings();
this.indexSettings = indexSettings;
this.threadPool = threadPool;
this.warmer = warmer == null ? (a,b) -> {} : warmer;
this.warmer = warmer == null ? (a) -> {} : warmer;
this.store = store;
this.deletionPolicy = deletionPolicy;
this.mergePolicy = mergePolicy;

View File

@ -24,13 +24,10 @@ import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriter.IndexReaderWarmer;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.LiveIndexWriterConfig;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.SegmentCommitInfo;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.Term;
@ -51,7 +48,6 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.lucene.LoggerInfoStream;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.math.MathUtils;
import org.elasticsearch.common.unit.ByteSizeValue;
@ -70,7 +66,6 @@ import org.elasticsearch.index.translog.TranslogCorruptedException;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
@ -91,7 +86,6 @@ public class InternalEngine extends Engine {
*/
private volatile long lastDeleteVersionPruneTimeMSec;
private final Engine.Warmer warmer;
private final Translog translog;
private final ElasticsearchConcurrentMergeScheduler mergeScheduler;
@ -131,7 +125,6 @@ public class InternalEngine extends Engine {
boolean success = false;
try {
this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis();
this.warmer = engineConfig.getWarmer();
mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings());
this.dirtyLocks = new Object[Runtime.getRuntime().availableProcessors() * 10]; // we multiply it to have enough...
for (int i = 0; i < dirtyLocks.length; i++) {
@ -931,30 +924,6 @@ public class InternalEngine extends Engine {
iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().mbFrac());
iwc.setCodec(engineConfig.getCodec());
iwc.setUseCompoundFile(true); // always use compound on flush - reduces # of file-handles on refresh
// 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
iwc.setMergedSegmentWarmer(new IndexReaderWarmer() {
@Override
public void warm(LeafReader reader) throws IOException {
try {
LeafReader esLeafReader = new ElasticsearchLeafReader(reader, shardId);
assert isMergedSegment(esLeafReader);
if (warmer != null) {
final Engine.Searcher searcher = new Searcher("warmer", searcherFactory.newSearcher(esLeafReader, null));
warmer.warm(searcher, false);
}
} catch (Throwable t) {
// Don't fail a merge if the warm-up failed
if (isClosed.get() == false) {
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(), iwc);
} catch (LockObtainFailedException ex) {
logger.warn("could not lock IndexWriter", ex);
@ -965,14 +934,12 @@ public class InternalEngine extends Engine {
/** Extended SearcherFactory that warms the segments if needed when acquiring a new searcher */
final static class SearchFactory extends EngineSearcherFactory {
private final Engine.Warmer warmer;
private final ShardId shardId;
private final ESLogger logger;
private final AtomicBoolean isEngineClosed;
SearchFactory(ESLogger logger, AtomicBoolean isEngineClosed, EngineConfig engineConfig) {
super(engineConfig);
warmer = engineConfig.getWarmer();
shardId = engineConfig.getShardId();
this.logger = logger;
this.isEngineClosed = isEngineClosed;
}
@ -987,55 +954,13 @@ public class InternalEngine extends Engine {
return searcher;
}
if (warmer != null) {
// we need to pass a custom searcher that does not release anything on Engine.Search Release,
// we will release explicitly
IndexSearcher newSearcher = null;
boolean closeNewSearcher = false;
try {
if (previousReader == null) {
// we are starting up - no writer active so we can't acquire a searcher.
newSearcher = searcher;
} else {
// figure out the newSearcher, with only the new readers that are relevant for us
List<IndexReader> readers = new ArrayList<>();
for (LeafReaderContext newReaderContext : reader.leaves()) {
if (isMergedSegment(newReaderContext.reader())) {
// merged segments are already handled by IndexWriterConfig.setMergedSegmentWarmer
continue;
}
boolean found = false;
for (LeafReaderContext currentReaderContext : previousReader.leaves()) {
if (currentReaderContext.reader().getCoreCacheKey().equals(newReaderContext.reader().getCoreCacheKey())) {
found = true;
break;
}
}
if (!found) {
readers.add(newReaderContext.reader());
}
}
if (!readers.isEmpty()) {
// we don't want to close the inner readers, just increase ref on them
IndexReader newReader = new MultiReader(readers.toArray(new IndexReader[readers.size()]), false);
newSearcher = super.newSearcher(newReader, null);
closeNewSearcher = true;
}
}
if (newSearcher != null) {
warmer.warm(new Searcher("new_reader_warming", newSearcher), false);
}
assert searcher.getIndexReader() instanceof ElasticsearchDirectoryReader : "this class needs an ElasticsearchDirectoryReader but got: " + searcher.getIndexReader().getClass();
warmer.warm(new Searcher("top_reader_warming", searcher), true);
warmer.warm(new Searcher("top_reader_warming", searcher));
} catch (Throwable e) {
if (isEngineClosed.get() == false) {
logger.warn("failed to prepare/warm", e);
}
} finally {
// no need to release the fullSearcher, nothing really is done...
if (newSearcher != null && closeNewSearcher) {
IOUtils.closeWhileHandlingException(newSearcher.getIndexReader()); // ignore
}
}
}
return searcher;

View File

@ -1,92 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.MappedFieldType.Loading;
/**
*/
public class FieldDataType {
public static final String FORMAT_KEY = "format";
public static final String DOC_VALUES_FORMAT_VALUE = "doc_values";
private final String type;
private final String typeFormat;
private final Loading loading;
private final Settings settings;
public FieldDataType(String type) {
this(type, Settings.Builder.EMPTY_SETTINGS);
}
public FieldDataType(String type, Settings.Builder builder) {
this(type, builder.build());
}
public FieldDataType(String type, Settings settings) {
this.type = type;
this.typeFormat = "index.fielddata.type." + type + "." + FORMAT_KEY;
this.settings = settings;
final String loading = settings.get(Loading.KEY);
this.loading = Loading.parse(loading, Loading.LAZY);
}
public String getType() {
return this.type;
}
public Settings getSettings() {
return this.settings;
}
public Loading getLoading() {
return loading;
}
public String getFormat(Settings indexSettings) {
String format = settings.get(FORMAT_KEY);
if (format == null && indexSettings != null) {
format = indexSettings.get(typeFormat);
}
return format;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
FieldDataType that = (FieldDataType) o;
if (!settings.equals(that.settings)) return false;
if (!type.equals(that.type)) return false;
return true;
}
@Override
public int hashCode() {
int result = type.hashCode();
result = 31 * result + settings.hashCode();
return result;
}
}

View File

@ -67,18 +67,6 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
return null;
}
}
/**
* Gets a memory storage hint that should be honored if possible but is not mandatory
*/
public static MemoryStorageFormat getMemoryStorageHint(FieldDataType fieldDataType) {
// backwards compatibility
String s = fieldDataType.getSettings().get("ordinals");
if (s != null) {
return "always".equals(s) ? MemoryStorageFormat.ORDINALS : null;
}
return MemoryStorageFormat.fromString(fieldDataType.getSettings().get(SETTING_MEMORY_STORAGE_HINT));
}
}
/**
@ -86,11 +74,6 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
*/
String getFieldName();
/**
* The field data type.
*/
FieldDataType getFieldDataType();
/**
* Loads the atomic field data for the reader, possibly cached.
*/

View File

@ -48,12 +48,12 @@ public interface IndexFieldDataCache {
/**
* Called after the fielddata is loaded during the cache phase
*/
default void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage){}
default void onCache(ShardId shardId, String fieldName, Accountable ramUsage){}
/**
* Called after the fielddata is unloaded
*/
default void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes){}
default void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes){}
}
class None implements IndexFieldDataCache {

View File

@ -70,116 +70,19 @@ public class IndexFieldDataService extends AbstractIndexComponent implements Clo
}
}, Property.IndexScope);
private static final IndexFieldData.Builder MISSING_DOC_VALUES_BUILDER = (indexProperties, fieldType, cache, breakerService, mapperService1) -> {
throw new IllegalStateException("Can't load fielddata on [" + fieldType.name()
+ "] of index [" + indexProperties.getIndex().getName() + "] because fielddata is unsupported on fields of type ["
+ fieldType.fieldDataType().getType() + "]. Use doc values instead.");
};
private static final String ARRAY_FORMAT = "array";
private static final String DISABLED_FORMAT = "disabled";
private static final String DOC_VALUES_FORMAT = "doc_values";
private static final String PAGED_BYTES_FORMAT = "paged_bytes";
private static final IndexFieldData.Builder DISABLED_BUILDER = new IndexFieldData.Builder() {
@Override
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
throw new IllegalStateException("Field data loading is forbidden on [" + fieldType.name() + "]");
}
};
private final static Map<String, IndexFieldData.Builder> buildersByType;
private final static Map<String, IndexFieldData.Builder> docValuesBuildersByType;
private final static Map<Tuple<String, String>, IndexFieldData.Builder> buildersByTypeAndFormat;
private final CircuitBreakerService circuitBreakerService;
static {
Map<String, IndexFieldData.Builder> buildersByTypeBuilder = new HashMap<>();
buildersByTypeBuilder.put("string", new PagedBytesIndexFieldData.Builder());
buildersByTypeBuilder.put(TextFieldMapper.CONTENT_TYPE, new PagedBytesIndexFieldData.Builder());
buildersByTypeBuilder.put(KeywordFieldMapper.CONTENT_TYPE, MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("float", MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("double", MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("byte", MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("short", MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("int", MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("long", MISSING_DOC_VALUES_BUILDER);
buildersByTypeBuilder.put("geo_point", new GeoPointArrayIndexFieldData.Builder());
buildersByTypeBuilder.put(ParentFieldMapper.NAME, new ParentChildIndexFieldData.Builder());
buildersByTypeBuilder.put(IndexFieldMapper.NAME, new IndexIndexFieldData.Builder());
buildersByTypeBuilder.put("binary", DISABLED_BUILDER);
buildersByTypeBuilder.put(BooleanFieldMapper.CONTENT_TYPE, MISSING_DOC_VALUES_BUILDER);
buildersByType = unmodifiableMap(buildersByTypeBuilder);
docValuesBuildersByType = MapBuilder.<String, IndexFieldData.Builder>newMapBuilder()
.put("string", new DocValuesIndexFieldData.Builder())
.put(KeywordFieldMapper.CONTENT_TYPE, new DocValuesIndexFieldData.Builder())
.put("float", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.FLOAT))
.put("double", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.DOUBLE))
.put("byte", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.BYTE))
.put("short", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.SHORT))
.put("int", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.INT))
.put("long", new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.LONG))
.put("geo_point", new AbstractGeoPointDVIndexFieldData.Builder())
.put("binary", new BytesBinaryDVIndexFieldData.Builder())
.put(BooleanFieldMapper.CONTENT_TYPE, new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.BOOLEAN))
.immutableMap();
buildersByTypeAndFormat = MapBuilder.<Tuple<String, String>, IndexFieldData.Builder>newMapBuilder()
.put(Tuple.tuple("string", PAGED_BYTES_FORMAT), new PagedBytesIndexFieldData.Builder())
.put(Tuple.tuple("string", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder())
.put(Tuple.tuple("string", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple(TextFieldMapper.CONTENT_TYPE, PAGED_BYTES_FORMAT), new PagedBytesIndexFieldData.Builder())
.put(Tuple.tuple(TextFieldMapper.CONTENT_TYPE, DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple(KeywordFieldMapper.CONTENT_TYPE, DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder())
.put(Tuple.tuple(KeywordFieldMapper.CONTENT_TYPE, DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("float", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.FLOAT))
.put(Tuple.tuple("float", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("double", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.DOUBLE))
.put(Tuple.tuple("double", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("byte", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.BYTE))
.put(Tuple.tuple("byte", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("short", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.SHORT))
.put(Tuple.tuple("short", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("int", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.INT))
.put(Tuple.tuple("int", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("long", DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.LONG))
.put(Tuple.tuple("long", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("geo_point", ARRAY_FORMAT), new GeoPointArrayIndexFieldData.Builder())
.put(Tuple.tuple("geo_point", DOC_VALUES_FORMAT), new AbstractGeoPointDVIndexFieldData.Builder())
.put(Tuple.tuple("geo_point", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple("binary", DOC_VALUES_FORMAT), new BytesBinaryDVIndexFieldData.Builder())
.put(Tuple.tuple("binary", DISABLED_FORMAT), DISABLED_BUILDER)
.put(Tuple.tuple(BooleanFieldMapper.CONTENT_TYPE, DOC_VALUES_FORMAT), new DocValuesIndexFieldData.Builder().numericType(IndexNumericFieldData.NumericType.BOOLEAN))
.put(Tuple.tuple(BooleanFieldMapper.CONTENT_TYPE, DISABLED_FORMAT), DISABLED_BUILDER)
.immutableMap();
}
private final IndicesFieldDataCache indicesFieldDataCache;
// the below map needs to be modified under a lock
private final Map<String, IndexFieldDataCache> fieldDataCaches = new HashMap<>();
private final MapperService mapperService;
private static final IndexFieldDataCache.Listener DEFAULT_NOOP_LISTENER = new IndexFieldDataCache.Listener() {
@Override
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, Accountable ramUsage) {
}
@Override
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes) {
}
};
private volatile IndexFieldDataCache.Listener listener = DEFAULT_NOOP_LISTENER;
@ -223,42 +126,15 @@ public class IndexFieldDataService extends AbstractIndexComponent implements Clo
@SuppressWarnings("unchecked")
public <IFD extends IndexFieldData<?>> IFD getForField(MappedFieldType fieldType) {
final String fieldName = fieldType.name();
final FieldDataType type = fieldType.fieldDataType();
if (type == null) {
throw new IllegalArgumentException("found no fielddata type for field [" + fieldName + "]");
}
final boolean docValues = fieldType.hasDocValues();
IndexFieldData.Builder builder = null;
String format = type.getFormat(indexSettings.getSettings());
if (format != null && FieldDataType.DOC_VALUES_FORMAT_VALUE.equals(format) && !docValues) {
logger.warn("field [{}] has no doc values, will use default field data format", fieldName);
format = null;
}
if (format != null) {
builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), format));
if (builder == null) {
logger.warn("failed to find format [{}] for field [{}], will use default", format, fieldName);
}
}
if (builder == null && docValues) {
builder = docValuesBuildersByType.get(type.getType());
}
if (builder == null) {
builder = buildersByType.get(type.getType());
}
if (builder == null) {
throw new IllegalArgumentException("failed to find field data builder for field " + fieldName + ", and type " + type.getType());
}
IndexFieldData.Builder builder = fieldType.fielddataBuilder();
IndexFieldDataCache cache;
synchronized (this) {
cache = fieldDataCaches.get(fieldName);
if (cache == null) {
// we default to node level cache, which in turn defaults to be unbounded
// this means changing the node level settings is simple, just set the bounds there
String cacheType = type.getSettings().get("cache", indexSettings.getValue(INDEX_FIELDDATA_CACHE_KEY));
String cacheType = indexSettings.getValue(INDEX_FIELDDATA_CACHE_KEY);
if (FIELDDATA_CACHE_VALUE_NODE.equals(cacheType)) {
cache = indicesFieldDataCache.buildIndexFieldDataCache(listener, index(), fieldName, type);
cache = indicesFieldDataCache.buildIndexFieldDataCache(listener, index(), fieldName);
} else if ("none".equals(cacheType)){
cache = new IndexFieldDataCache.None();
} else {

View File

@ -51,7 +51,7 @@ public class ShardFieldData implements IndexFieldDataCache.Listener {
}
@Override
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, Accountable ramUsage) {
totalMetric.inc(ramUsage.ramBytesUsed());
CounterMetric total = perFieldTotals.get(fieldName);
if (total != null) {
@ -67,7 +67,7 @@ public class ShardFieldData implements IndexFieldDataCache.Listener {
}
@Override
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes) {
if (wasEvicted) {
evictionsMetric.inc();
}

View File

@ -70,7 +70,7 @@ public enum GlobalOrdinalsBuilder {
);
}
return new InternalGlobalOrdinalsIndexFieldData(indexSettings, indexFieldData.getFieldName(),
indexFieldData.getFieldDataType(), atomicFD, ordinalMap, memorySizeInBytes
atomicFD, ordinalMap, memorySizeInBytes
);
}
@ -104,7 +104,7 @@ public enum GlobalOrdinalsBuilder {
}
final OrdinalMap ordinalMap = OrdinalMap.build(null, subs, PackedInts.DEFAULT);
return new InternalGlobalOrdinalsIndexFieldData(indexSettings, indexFieldData.getFieldName(),
indexFieldData.getFieldDataType(), atomicFD, ordinalMap, 0
atomicFD, ordinalMap, 0
);
}

View File

@ -25,11 +25,9 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.MultiValueMode;
import java.util.Collection;
@ -41,13 +39,11 @@ import java.util.Collections;
public abstract class GlobalOrdinalsIndexFieldData extends AbstractIndexComponent implements IndexOrdinalsFieldData, Accountable {
private final String fieldName;
private final FieldDataType fieldDataType;
private final long memorySizeInBytes;
protected GlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, long memorySizeInBytes) {
protected GlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, long memorySizeInBytes) {
super(indexSettings);
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.memorySizeInBytes = memorySizeInBytes;
}
@ -71,11 +67,6 @@ public abstract class GlobalOrdinalsIndexFieldData extends AbstractIndexComponen
return fieldName;
}
@Override
public FieldDataType getFieldDataType() {
return fieldDataType;
}
@Override
public XFieldComparatorSource comparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) {
throw new UnsupportedOperationException("no global ordinals sorting yet");

View File

@ -24,9 +24,7 @@ import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.plain.AbstractAtomicOrdinalsFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import java.util.Collection;
@ -37,8 +35,8 @@ final class InternalGlobalOrdinalsIndexFieldData extends GlobalOrdinalsIndexFiel
private final Atomic[] atomicReaders;
InternalGlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, AtomicOrdinalsFieldData[] segmentAfd, OrdinalMap ordinalMap, long memorySizeInBytes) {
super(indexSettings, fieldName, fieldDataType, memorySizeInBytes);
InternalGlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, AtomicOrdinalsFieldData[] segmentAfd, OrdinalMap ordinalMap, long memorySizeInBytes) {
super(indexSettings, fieldName, memorySizeInBytes);
this.atomicReaders = new Atomic[segmentAfd.length];
for (int i = 0; i < segmentAfd.length; i++) {
atomicReaders[i] = new Atomic(segmentAfd[i], ordinalMap, i);

View File

@ -35,7 +35,6 @@ import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.packed.GrowableWriter;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PagedGrowableWriter;
import org.elasticsearch.common.settings.Settings;
import java.io.Closeable;
import java.io.IOException;
@ -287,20 +286,13 @@ public final class OrdinalsBuilder implements Closeable {
private OrdinalsStore ordinals;
private final LongsRef spare;
public OrdinalsBuilder(long numTerms, int maxDoc, float acceptableOverheadRatio) throws IOException {
public OrdinalsBuilder(int maxDoc, float acceptableOverheadRatio) throws IOException {
this.maxDoc = maxDoc;
int startBitsPerValue = 8;
if (numTerms >= 0) {
startBitsPerValue = PackedInts.bitsRequired(numTerms);
}
ordinals = new OrdinalsStore(maxDoc, startBitsPerValue, acceptableOverheadRatio);
spare = new LongsRef();
}
public OrdinalsBuilder(int maxDoc, float acceptableOverheadRatio) throws IOException {
this(-1, maxDoc, acceptableOverheadRatio);
}
public OrdinalsBuilder(int maxDoc) throws IOException {
this(maxDoc, DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
}
@ -413,10 +405,9 @@ public final class OrdinalsBuilder implements Closeable {
/**
* Builds an {@link Ordinals} instance from the builders current state.
*/
public Ordinals build(Settings settings) {
final float acceptableOverheadRatio = settings.getAsFloat("acceptable_overhead_ratio", PackedInts.FASTEST);
final boolean forceMultiOrdinals = settings.getAsBoolean(FORCE_MULTI_ORDINALS, false);
if (forceMultiOrdinals || numMultiValuedDocs > 0 || MultiOrdinals.significantlySmallerThanSinglePackedOrdinals(maxDoc, numDocsWithValue, getValueCount(), acceptableOverheadRatio)) {
public Ordinals build() {
final float acceptableOverheadRatio = PackedInts.DEFAULT;
if (numMultiValuedDocs > 0 || MultiOrdinals.significantlySmallerThanSinglePackedOrdinals(maxDoc, numDocsWithValue, getValueCount(), acceptableOverheadRatio)) {
// MultiOrdinals can be smaller than SinglePackedOrdinals for sparse fields
return new MultiOrdinals(this, acceptableOverheadRatio);
} else {

View File

@ -26,7 +26,6 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicGeoPointFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
@ -40,8 +39,8 @@ import java.io.IOException;
public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFieldData implements IndexGeoPointFieldData {
AbstractGeoPointDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super(index, fieldName, fieldDataType);
AbstractGeoPointDVIndexFieldData(Index index, String fieldName) {
super(index, fieldName);
}
@Override
@ -55,8 +54,8 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
public static class GeoPointDVIndexFieldData extends AbstractGeoPointDVIndexFieldData {
final boolean indexCreatedBefore2x;
public GeoPointDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType, final boolean indexCreatedBefore2x) {
super(index, fieldName, fieldDataType);
public GeoPointDVIndexFieldData(Index index, String fieldName, final boolean indexCreatedBefore2x) {
super(index, fieldName);
this.indexCreatedBefore2x = indexCreatedBefore2x;
}
@ -82,8 +81,12 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
@Override
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
if (indexSettings.getIndexVersionCreated().before(Version.V_2_2_0)
&& fieldType.hasDocValues() == false) {
return new GeoPointArrayIndexFieldData(indexSettings, fieldType.name(), cache, breakerService);
}
// Ignore breaker
return new GeoPointDVIndexFieldData(indexSettings.getIndex(), fieldType.name(), fieldType.fieldDataType(),
return new GeoPointDVIndexFieldData(indexSettings.getIndex(), fieldType.name(),
indexSettings.getIndexVersionCreated().before(Version.V_2_2_0));
}
}

View File

@ -27,7 +27,6 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.RamAccountingTermsEnum;
@ -39,13 +38,11 @@ import java.io.IOException;
public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends AbstractIndexComponent implements IndexFieldData<FD> {
private final String fieldName;
protected final FieldDataType fieldDataType;
protected final IndexFieldDataCache cache;
public AbstractIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, IndexFieldDataCache cache) {
public AbstractIndexFieldData(IndexSettings indexSettings, String fieldName, IndexFieldDataCache cache) {
super(indexSettings);
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.cache = cache;
}
@ -54,11 +51,6 @@ public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends
return this.fieldName;
}
@Override
public FieldDataType getFieldDataType() {
return fieldDataType;
}
@Override
public void clear() {
cache.clear(fieldName);

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicGeoPointFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
@ -102,8 +101,8 @@ abstract class AbstractIndexGeoPointFieldData extends AbstractIndexFieldData<Ato
}
}
public AbstractIndexGeoPointFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(indexSettings, fieldName, fieldDataType, cache);
public AbstractIndexGeoPointFieldData(IndexSettings indexSettings, String fieldName, IndexFieldDataCache cache) {
super(indexSettings, fieldName, cache);
}
@Override

View File

@ -25,13 +25,10 @@ import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CharsRefBuilder;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
@ -41,23 +38,21 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldData<AtomicOrdinalsFieldData> implements IndexOrdinalsFieldData {
protected Settings frequency;
protected Settings regex;
private final double minFrequency, maxFrequency;
private final int minSegmentSize;
protected final CircuitBreakerService breakerService;
protected AbstractIndexOrdinalsFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType,
IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldName, fieldDataType, cache);
final Map<String, Settings> groups = fieldDataType.getSettings().getGroups("filter");
frequency = groups.get("frequency");
regex = groups.get("regex");
protected AbstractIndexOrdinalsFieldData(IndexSettings indexSettings, String fieldName,
IndexFieldDataCache cache, CircuitBreakerService breakerService,
double minFrequency, double maxFrequency, int minSegmentSize) {
super(indexSettings, fieldName, cache);
this.breakerService = breakerService;
this.minFrequency = minFrequency;
this.maxFrequency = maxFrequency;
this.minSegmentSize = minSegmentSize;
}
@Override
@ -110,17 +105,24 @@ public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldD
return AbstractAtomicOrdinalsFieldData.empty();
}
protected TermsEnum filter(Terms terms, LeafReader reader) throws IOException {
TermsEnum iterator = terms.iterator();
protected TermsEnum filter(Terms terms, TermsEnum iterator, LeafReader reader) throws IOException {
if (iterator == null) {
return null;
}
if (iterator != null && frequency != null) {
iterator = FrequencyFilter.filter(iterator, terms, reader, frequency);
int docCount = terms.getDocCount();
if (docCount == -1) {
docCount = reader.maxDoc();
}
if (iterator != null && regex != null) {
iterator = RegexFilter.filter(iterator, terms, reader, regex);
if (docCount >= minSegmentSize) {
final int minFreq = minFrequency > 1.0
? (int) minFrequency
: (int)(docCount * minFrequency);
final int maxFreq = maxFrequency > 1.0
? (int) maxFrequency
: (int)(docCount * maxFrequency);
if (minFreq > 1 || maxFreq < docCount) {
iterator = new FrequencyFilter(iterator, minFreq, maxFreq);
}
}
return iterator;
}
@ -135,25 +137,6 @@ public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldD
this.maxFreq = maxFreq;
}
public static TermsEnum filter(TermsEnum toFilter, Terms terms, LeafReader reader, Settings settings) throws IOException {
int docCount = terms.getDocCount();
if (docCount == -1) {
docCount = reader.maxDoc();
}
final double minFrequency = settings.getAsDouble("min", 0d);
final double maxFrequency = settings.getAsDouble("max", docCount+1d);
final double minSegmentSize = settings.getAsInt("min_segment_size", 0);
if (minSegmentSize < docCount) {
final int minFreq = minFrequency > 1.0? (int) minFrequency : (int)(docCount * minFrequency);
final int maxFreq = maxFrequency > 1.0? (int) maxFrequency : (int)(docCount * maxFrequency);
assert minFreq < maxFreq;
return new FrequencyFilter(toFilter, minFreq, maxFreq);
}
return toFilter;
}
@Override
protected AcceptStatus accept(BytesRef arg0) throws IOException {
int docFreq = docFreq();
@ -164,33 +147,4 @@ public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldD
}
}
private static final class RegexFilter extends FilteredTermsEnum {
private final Matcher matcher;
private final CharsRefBuilder spare = new CharsRefBuilder();
public RegexFilter(TermsEnum delegate, Matcher matcher) {
super(delegate, false);
this.matcher = matcher;
}
public static TermsEnum filter(TermsEnum iterator, Terms terms, LeafReader reader, Settings regex) {
String pattern = regex.get("pattern");
if (pattern == null) {
return iterator;
}
Pattern p = Pattern.compile(pattern);
return new RegexFilter(iterator, p.matcher(""));
}
@Override
protected AcceptStatus accept(BytesRef arg0) throws IOException {
spare.copyUTF8Bytes(arg0);
matcher.reset(spare.get());
if (matcher.matches()) {
return AcceptStatus.YES;
}
return AcceptStatus.NO;
}
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
@ -29,8 +28,8 @@ import org.elasticsearch.search.MultiValueMode;
public class BinaryDVIndexFieldData extends DocValuesIndexFieldData implements IndexFieldData<BinaryDVAtomicFieldData> {
public BinaryDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super(index, fieldName, fieldDataType);
public BinaryDVIndexFieldData(Index index, String fieldName) {
super(index, fieldName);
}
@Override

View File

@ -24,7 +24,6 @@ import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
@ -37,8 +36,8 @@ import java.io.IOException;
public class BytesBinaryDVIndexFieldData extends DocValuesIndexFieldData implements IndexFieldData<BytesBinaryDVAtomicFieldData> {
public BytesBinaryDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super(index, fieldName, fieldDataType);
public BytesBinaryDVIndexFieldData(Index index, String fieldName) {
super(index, fieldName);
}
@Override
@ -67,7 +66,7 @@ public class BytesBinaryDVIndexFieldData extends DocValuesIndexFieldData impleme
CircuitBreakerService breakerService, MapperService mapperService) {
// Ignore breaker
final String fieldName = fieldType.name();
return new BytesBinaryDVIndexFieldData(indexSettings.getIndex(), fieldName, fieldType.fieldDataType());
return new BytesBinaryDVIndexFieldData(indexSettings.getIndex(), fieldName);
}
}

View File

@ -25,7 +25,6 @@ import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
@ -46,14 +45,12 @@ public abstract class DocValuesIndexFieldData {
protected final Index index;
protected final String fieldName;
protected final FieldDataType fieldDataType;
protected final ESLogger logger;
public DocValuesIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
public DocValuesIndexFieldData(Index index, String fieldName) {
super();
this.index = index;
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.logger = Loggers.getLogger(getClass());
}
@ -61,10 +58,6 @@ public abstract class DocValuesIndexFieldData {
return fieldName;
}
public final FieldDataType getFieldDataType() {
return fieldDataType;
}
public final void clear() {
// can't do
}
@ -92,19 +85,13 @@ public abstract class DocValuesIndexFieldData {
CircuitBreakerService breakerService, MapperService mapperService) {
// Ignore Circuit Breaker
final String fieldName = fieldType.name();
final Settings fdSettings = fieldType.fieldDataType().getSettings();
final Map<String, Settings> filter = fdSettings.getGroups("filter");
if (filter != null && !filter.isEmpty()) {
throw new IllegalArgumentException("Doc values field data doesn't support filters [" + fieldName + "]");
}
if (BINARY_INDEX_FIELD_NAMES.contains(fieldName)) {
assert numericType == null;
return new BinaryDVIndexFieldData(indexSettings.getIndex(), fieldName, fieldType.fieldDataType());
return new BinaryDVIndexFieldData(indexSettings.getIndex(), fieldName);
} else if (numericType != null) {
return new SortedNumericDVIndexFieldData(indexSettings.getIndex(), fieldName, numericType, fieldType.fieldDataType());
return new SortedNumericDVIndexFieldData(indexSettings.getIndex(), fieldName, numericType);
} else {
return new SortedSetDVOrdinalsIndexFieldData(indexSettings, cache, fieldName, breakerService, fieldType.fieldDataType());
return new SortedSetDVOrdinalsIndexFieldData(indexSettings, cache, fieldName, breakerService);
}
}

View File

@ -35,7 +35,6 @@ import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicGeoPointFieldData;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
@ -50,18 +49,9 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService;
public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData {
private final CircuitBreakerService breakerService;
public static class Builder implements IndexFieldData.Builder {
@Override
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
return new GeoPointArrayIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache,
breakerService);
}
}
public GeoPointArrayIndexFieldData(IndexSettings indexSettings, String fieldName,
FieldDataType fieldDataType, IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldName, fieldDataType, cache);
IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldName, cache);
this.breakerService = breakerService;
}
@ -88,8 +78,7 @@ public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData
private AtomicGeoPointFieldData loadFieldData22(LeafReader reader, NonEstimatingEstimator estimator, Terms terms,
AtomicGeoPointFieldData data) throws Exception {
LongArray indexedPoints = BigArrays.NON_RECYCLING_INSTANCE.newLongArray(128);
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat("acceptable_transient_overhead_ratio",
OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
final float acceptableTransientOverheadRatio = OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO;
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
final TermsEnum termsEnum;
@ -112,10 +101,9 @@ public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData
}
indexedPoints = BigArrays.NON_RECYCLING_INSTANCE.resize(indexedPoints, numTerms);
Ordinals build = builder.build(fieldDataType.getSettings());
Ordinals build = builder.build();
RandomAccessOrds ordinals = build.ordinals();
if (!(FieldData.isMultiValued(ordinals) || CommonSettings.getMemoryStorageHint(fieldDataType) == CommonSettings
.MemoryStorageFormat.ORDINALS)) {
if (FieldData.isMultiValued(ordinals) == false) {
int maxDoc = reader.maxDoc();
LongArray sIndexedPoint = BigArrays.NON_RECYCLING_INSTANCE.newLongArray(reader.maxDoc());
for (int i=0; i<maxDoc; ++i) {
@ -146,9 +134,9 @@ public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData
AtomicGeoPointFieldData data) throws Exception {
DoubleArray lat = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
DoubleArray lon = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat("acceptable_transient_overhead_ratio", OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
final float acceptableTransientOverheadRatio = OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO;
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(terms.size(), reader.maxDoc(), acceptableTransientOverheadRatio)) {
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
final GeoPointTermsEnumLegacy iter = new GeoPointTermsEnumLegacy(builder.buildFromTerms(terms.iterator()));
GeoPoint point;
long numTerms = 0;
@ -162,9 +150,9 @@ public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData
lat = BigArrays.NON_RECYCLING_INSTANCE.resize(lat, numTerms);
lon = BigArrays.NON_RECYCLING_INSTANCE.resize(lon, numTerms);
Ordinals build = builder.build(fieldDataType.getSettings());
Ordinals build = builder.build();
RandomAccessOrds ordinals = build.ordinals();
if (!(FieldData.isMultiValued(ordinals) || CommonSettings.getMemoryStorageHint(fieldDataType) == CommonSettings.MemoryStorageFormat.ORDINALS)) {
if (FieldData.isMultiValued(ordinals) == false) {
int maxDoc = reader.maxDoc();
DoubleArray sLat = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(reader.maxDoc());
DoubleArray sLon = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(reader.maxDoc());

View File

@ -28,12 +28,12 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import java.util.Collection;
@ -101,7 +101,10 @@ public class IndexIndexFieldData extends AbstractIndexOrdinalsFieldData {
private final AtomicOrdinalsFieldData atomicFieldData;
private IndexIndexFieldData(IndexSettings indexSettings, String name) {
super(indexSettings, name, new FieldDataType("string"), null, null);
super(indexSettings, name, null, null,
TextFieldMapper.Defaults.FIELDDATA_MIN_FREQUENCY,
TextFieldMapper.Defaults.FIELDDATA_MAX_FREQUENCY,
TextFieldMapper.Defaults.FIELDDATA_MIN_SEGMENT_SIZE);
atomicFieldData = new IndexAtomicFieldData(index().getName());
}

View File

@ -34,7 +34,6 @@ import org.apache.lucene.util.packed.PackedLongValues;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
@ -54,16 +53,27 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
public static class Builder implements IndexFieldData.Builder {
private final double minFrequency, maxFrequency;
private final int minSegmentSize;
public Builder(double minFrequency, double maxFrequency, int minSegmentSize) {
this.minFrequency = minFrequency;
this.maxFrequency = maxFrequency;
this.minSegmentSize = minSegmentSize;
}
@Override
public IndexOrdinalsFieldData build(IndexSettings indexSettings, MappedFieldType fieldType,
IndexFieldDataCache cache, CircuitBreakerService breakerService, MapperService mapperService) {
return new PagedBytesIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache, breakerService);
return new PagedBytesIndexFieldData(indexSettings, fieldType.name(), cache, breakerService,
minFrequency, maxFrequency, minSegmentSize);
}
}
public PagedBytesIndexFieldData(IndexSettings indexSettings, String fieldName,
FieldDataType fieldDataType, IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldName, fieldDataType, cache, breakerService);
IndexFieldDataCache cache, CircuitBreakerService breakerService,
double minFrequency, double maxFrequency, int minSegmentSize) {
super(indexSettings, fieldName, cache, breakerService, minFrequency, maxFrequency, minSegmentSize);
}
@Override
@ -82,14 +92,7 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
final PagedBytes bytes = new PagedBytes(15);
final PackedLongValues.Builder termOrdToBytesOffset = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
final long numTerms;
if (regex == null && frequency == null) {
numTerms = terms.size();
} else {
numTerms = -1;
}
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat(
FilterSettingFields.ACCEPTABLE_TRANSIENT_OVERHEAD_RATIO, OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
final float acceptableTransientOverheadRatio = OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO;
// Wrap the context in an estimator and use it to either estimate
// the entire set, or wrap the TermsEnum so it can be calculated
@ -98,7 +101,7 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
TermsEnum termsEnum = estimator.beforeLoad(terms);
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(numTerms, reader.maxDoc(), acceptableTransientOverheadRatio)) {
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
PostingsEnum docsEnum = null;
for (BytesRef term = termsEnum.next(); term != null; term = termsEnum.next()) {
final long termOrd = builder.nextOrdinal();
@ -110,7 +113,7 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
}
}
PagedBytes.Reader bytesReader = bytes.freeze(true);
final Ordinals ordinals = builder.build(fieldDataType.getSettings());
final Ordinals ordinals = builder.build();
data = new PagedBytesAtomicFieldData(bytesReader, termOrdToBytesOffset.build(), ordinals);
success = true;
@ -199,33 +202,28 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
*/
@Override
public TermsEnum beforeLoad(Terms terms) throws IOException {
final float acceptableTransientOverheadRatio = fieldDataType.getSettings().getAsFloat(
FilterSettingFields.ACCEPTABLE_TRANSIENT_OVERHEAD_RATIO,
OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO);
LeafReader reader = context.reader();
// Check if one of the following is present:
// - The OrdinalsBuilder overhead has been tweaked away from the default
// - A field data filter is present
// - A regex filter is present
if (acceptableTransientOverheadRatio != OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO ||
fieldDataType.getSettings().getAsDouble(FilterSettingFields.FREQUENCY_MIN, 0d) != 0d ||
fieldDataType.getSettings().getAsDouble(FilterSettingFields.FREQUENCY_MAX, 0d) != 0d ||
fieldDataType.getSettings().getAsDouble(FilterSettingFields.FREQUENCY_MIN_SEGMENT_SIZE, 0d) != 0d ||
fieldDataType.getSettings().get(FilterSettingFields.REGEX_PATTERN) != null) {
TermsEnum iterator = terms.iterator();
TermsEnum filteredIterator = filter(terms, iterator, reader);
final boolean filtered = iterator != filteredIterator;
iterator = filteredIterator;
if (filtered) {
if (logger.isTraceEnabled()) {
logger.trace("Filter exists, can't circuit break normally, using RamAccountingTermsEnum");
}
return new RamAccountingTermsEnum(filter(terms, reader), breaker, this, this.fieldName);
return new RamAccountingTermsEnum(iterator, breaker, this, this.fieldName);
} else {
estimatedBytes = this.estimateStringFieldData();
// If we weren't able to estimate, wrap in the RamAccountingTermsEnum
if (estimatedBytes == 0) {
return new RamAccountingTermsEnum(filter(terms, reader), breaker, this, this.fieldName);
iterator = new RamAccountingTermsEnum(iterator, breaker, this, this.fieldName);
} else {
breaker.addEstimateBytesAndMaybeBreak(estimatedBytes, fieldName);
}
breaker.addEstimateBytesAndMaybeBreak(estimatedBytes, fieldName);
return filter(terms, reader);
return iterator;
}
}
@ -256,12 +254,4 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
breaker.addWithoutBreaking(actualUsed);
}
}
static final class FilterSettingFields {
static final String ACCEPTABLE_TRANSIENT_OVERHEAD_RATIO = "acceptable_transient_overhead_ratio";
static final String FREQUENCY_MIN = "filter.frequency.min";
static final String FREQUENCY_MAX = "filter.frequency.max";
static final String FREQUENCY_MIN_SEGMENT_SIZE = "filter.frequency.min_segment_size";
static final String REGEX_PATTERN = "filter.regex.pattern";
}
}

View File

@ -40,7 +40,6 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicParentChildFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
@ -75,9 +74,9 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
private final CircuitBreakerService breakerService;
public ParentChildIndexFieldData(IndexSettings indexSettings, String fieldName,
FieldDataType fieldDataType, IndexFieldDataCache cache, MapperService mapperService,
IndexFieldDataCache cache, MapperService mapperService,
CircuitBreakerService breakerService) {
super(indexSettings, fieldName, fieldDataType, cache);
super(indexSettings, fieldName, cache);
this.breakerService = breakerService;
Set<String> parentTypes = new HashSet<>();
for (DocumentMapper mapper : mapperService.docMappers(false)) {
@ -146,7 +145,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
MappedFieldType fieldType,
IndexFieldDataCache cache, CircuitBreakerService breakerService,
MapperService mapperService) {
return new ParentChildIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache,
return new ParentChildIndexFieldData(indexSettings, fieldType.name(), cache,
mapperService, breakerService);
}
}
@ -324,11 +323,6 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
return ParentChildIndexFieldData.this.getFieldName();
}
@Override
public FieldDataType getFieldDataType() {
return ParentChildIndexFieldData.this.getFieldDataType();
}
@Override
public AtomicParentChildFieldData load(LeafReaderContext context) {
assert context.reader().getCoreCacheKey() == leaves.get(context.ord).reader().getCoreCacheKey();

View File

@ -30,7 +30,6 @@ import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
@ -51,8 +50,8 @@ import java.util.Collections;
public class SortedNumericDVIndexFieldData extends DocValuesIndexFieldData implements IndexNumericFieldData {
private final NumericType numericType;
public SortedNumericDVIndexFieldData(Index index, String fieldNames, NumericType numericType, FieldDataType fieldDataType) {
super(index, fieldNames, fieldDataType);
public SortedNumericDVIndexFieldData(Index index, String fieldNames, NumericType numericType) {
super(index, fieldNames);
if (numericType == null) {
throw new IllegalArgumentException("numericType must be non-null");
}

View File

@ -24,7 +24,6 @@ import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicOrdinalsFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
@ -42,8 +41,8 @@ public class SortedSetDVOrdinalsIndexFieldData extends DocValuesIndexFieldData i
private final IndexFieldDataCache cache;
private final CircuitBreakerService breakerService;
public SortedSetDVOrdinalsIndexFieldData(IndexSettings indexSettings, IndexFieldDataCache cache, String fieldName, CircuitBreakerService breakerService, FieldDataType fieldDataType) {
super(indexSettings.getIndex(), fieldName, fieldDataType);
public SortedSetDVOrdinalsIndexFieldData(IndexSettings indexSettings, IndexFieldDataCache cache, String fieldName, CircuitBreakerService breakerService) {
super(indexSettings.getIndex(), fieldName);
this.indexSettings = indexSettings;
this.cache = cache;
this.breakerService = breakerService;

View File

@ -25,8 +25,6 @@ import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
@ -140,19 +138,19 @@ public class DocumentMapperParser {
return docBuilder.build(mapperService);
}
public static void checkNoRemainingFields(String fieldName, Map<String, Object> fieldNodeMap, Version indexVersionCreated) {
public static void checkNoRemainingFields(String fieldName, Map<?, ?> fieldNodeMap, Version indexVersionCreated) {
checkNoRemainingFields(fieldNodeMap, indexVersionCreated, "Mapping definition for [" + fieldName + "] has unsupported parameters: ");
}
public static void checkNoRemainingFields(Map<String, Object> fieldNodeMap, Version indexVersionCreated, String message) {
public static void checkNoRemainingFields(Map<?, ?> fieldNodeMap, Version indexVersionCreated, String message) {
if (!fieldNodeMap.isEmpty()) {
throw new MapperParsingException(message + getRemainingFields(fieldNodeMap));
}
}
private static String getRemainingFields(Map<String, ?> map) {
private static String getRemainingFields(Map<?, ?> map) {
StringBuilder remainingFields = new StringBuilder();
for (String key : map.keySet()) {
for (Object key : map.keySet()) {
remainingFields.append(" [").append(key).append(" : ").append(map.get(key)).append("]");
}
return remainingFields.toString();

View File

@ -25,7 +25,6 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.IndexOptions;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Setting;
@ -33,7 +32,6 @@ import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.core.TypeParsers;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import org.elasticsearch.index.similarity.SimilarityProvider;
@ -64,8 +62,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
protected Boolean includeInAll;
protected boolean indexOptionsSet = false;
protected boolean docValuesSet = false;
@Nullable
protected Settings fieldDataSettings;
protected final MultiFields.Builder multiFieldsBuilder;
protected CopyTo copyTo;
@ -203,11 +199,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
return builder;
}
public T fieldDataSettings(Settings settings) {
this.fieldDataSettings = settings;
return builder;
}
public Builder nullValue(Object nullValue) {
this.fieldType.setNullValue(nullValue);
return this;
@ -245,10 +236,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
fieldType.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
fieldType.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
}
if (fieldDataSettings != null) {
Settings settings = Settings.builder().put(fieldType.fieldDataType().getSettings()).put(fieldDataSettings).build();
fieldType.setFieldDataType(new FieldDataType(fieldType.fieldDataType().getType(), settings));
}
boolean defaultDocValues = defaultDocValues(context.indexCreatedVersion());
defaultFieldType.setHasDocValues(defaultDocValues);
if (docValuesSet == false) {
@ -423,6 +410,9 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
if (indexed && (includeDefaults || fieldType().indexOptions() != defaultFieldType.indexOptions())) {
builder.field("index_options", indexOptionToString(fieldType().indexOptions()));
}
if (includeDefaults || fieldType().eagerGlobalOrdinals() != defaultFieldType.eagerGlobalOrdinals()) {
builder.field("eager_global_ordinals", fieldType().eagerGlobalOrdinals());
}
if (fieldType().similarity() != null) {
builder.field("similarity", fieldType().similarity().name());
@ -430,9 +420,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
builder.field("similarity", SimilarityService.DEFAULT_SIMILARITY);
}
if (includeDefaults || hasCustomFieldDataSettings()) {
builder.field("fielddata", fieldType().fieldDataType().getSettings().getAsMap());
}
multiFields.toXContent(builder, params);
if (copyTo != null) {
@ -512,10 +499,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
return indexed;
}
protected boolean hasCustomFieldDataSettings() {
return fieldType().fieldDataType() != null && fieldType().fieldDataType().equals(defaultFieldType.fieldDataType()) == false;
}
protected abstract String contentType();
public static class MultiFields {

View File

@ -37,11 +37,10 @@ import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.index.similarity.SimilarityProvider;
@ -55,46 +54,6 @@ import java.util.Objects;
*/
public abstract class MappedFieldType extends FieldType {
public enum Loading {
LAZY {
@Override
public String toString() {
return LAZY_VALUE;
}
},
EAGER {
@Override
public String toString() {
return EAGER_VALUE;
}
},
EAGER_GLOBAL_ORDINALS {
@Override
public String toString() {
return EAGER_GLOBAL_ORDINALS_VALUE;
}
};
public static final String KEY = "loading";
public static final String EAGER_GLOBAL_ORDINALS_VALUE = "eager_global_ordinals";
public static final String EAGER_VALUE = "eager";
public static final String LAZY_VALUE = "lazy";
public static Loading parse(String loading, Loading defaultValue) {
if (Strings.isNullOrEmpty(loading)) {
return defaultValue;
} else if (EAGER_GLOBAL_ORDINALS_VALUE.equalsIgnoreCase(loading)) {
return EAGER_GLOBAL_ORDINALS;
} else if (EAGER_VALUE.equalsIgnoreCase(loading)) {
return EAGER;
} else if (LAZY_VALUE.equalsIgnoreCase(loading)) {
return LAZY;
} else {
throw new MapperParsingException("Unknown [" + KEY + "] value: [" + loading + "]");
}
}
}
private String name;
private float boost;
// TODO: remove this docvalues flag and use docValuesType
@ -103,9 +62,9 @@ public abstract class MappedFieldType extends FieldType {
private NamedAnalyzer searchAnalyzer;
private NamedAnalyzer searchQuoteAnalyzer;
private SimilarityProvider similarity;
private FieldDataType fieldDataType;
private Object nullValue;
private String nullValueAsString; // for sending null value to _all field
private boolean eagerGlobalOrdinals;
protected MappedFieldType(MappedFieldType ref) {
super(ref);
@ -116,9 +75,9 @@ public abstract class MappedFieldType extends FieldType {
this.searchAnalyzer = ref.searchAnalyzer();
this.searchQuoteAnalyzer = ref.searchQuoteAnalyzer();
this.similarity = ref.similarity();
this.fieldDataType = ref.fieldDataType();
this.nullValue = ref.nullValue();
this.nullValueAsString = ref.nullValueAsString();
this.eagerGlobalOrdinals = ref.eagerGlobalOrdinals;
}
public MappedFieldType() {
@ -128,12 +87,16 @@ public abstract class MappedFieldType extends FieldType {
setOmitNorms(false);
setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
setBoost(1.0f);
fieldDataType = new FieldDataType(typeName());
}
@Override
public abstract MappedFieldType clone();
/** Return a fielddata builder for this field. */
public IndexFieldData.Builder fielddataBuilder() {
throw new IllegalArgumentException("Fielddata is not supported on fields of type [" + typeName() + "]");
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) return false;
@ -156,7 +119,7 @@ public abstract class MappedFieldType extends FieldType {
Objects.equals(indexAnalyzer, fieldType.indexAnalyzer) &&
Objects.equals(searchAnalyzer, fieldType.searchAnalyzer) &&
Objects.equals(searchQuoteAnalyzer(), fieldType.searchQuoteAnalyzer()) &&
Objects.equals(fieldDataType, fieldType.fieldDataType) &&
Objects.equals(eagerGlobalOrdinals, fieldType.eagerGlobalOrdinals) &&
Objects.equals(nullValue, fieldType.nullValue) &&
Objects.equals(nullValueAsString, fieldType.nullValueAsString);
}
@ -164,7 +127,7 @@ public abstract class MappedFieldType extends FieldType {
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), name, boost, docValues, indexAnalyzer, searchAnalyzer, searchQuoteAnalyzer,
similarity == null ? null : similarity.name(), fieldDataType, nullValue, nullValueAsString);
eagerGlobalOrdinals, similarity == null ? null : similarity.name(), nullValue, nullValueAsString);
}
// norelease: we need to override freeze() and add safety checks that all settings are actually set
@ -245,12 +208,12 @@ public abstract class MappedFieldType extends FieldType {
if (Objects.equals(searchQuoteAnalyzer(), other.searchQuoteAnalyzer()) == false) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [search_quote_analyzer] across all types.");
}
if (Objects.equals(fieldDataType(), other.fieldDataType()) == false) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [fielddata] across all types.");
}
if (Objects.equals(nullValue(), other.nullValue()) == false) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [null_value] across all types.");
}
if (eagerGlobalOrdinals() != other.eagerGlobalOrdinals()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [eager_global_ordinals] across all types.");
}
}
}
@ -280,15 +243,6 @@ public abstract class MappedFieldType extends FieldType {
this.boost = boost;
}
public FieldDataType fieldDataType() {
return fieldDataType;
}
public void setFieldDataType(FieldDataType fieldDataType) {
checkIfFrozen();
this.fieldDataType = fieldDataType;
}
public boolean hasDocValues() {
return docValues;
}
@ -453,4 +407,21 @@ public abstract class MappedFieldType extends FieldType {
public Query queryStringTermQuery(Term term) {
return null;
}
protected final void failIfNoDocValues() {
if (hasDocValues() == false) {
throw new IllegalStateException("Can't load fielddata on [" + name()
+ "] because fielddata is unsupported on fields of type ["
+ typeName() + "]. Use doc values instead.");
}
}
public boolean eagerGlobalOrdinals() {
return eagerGlobalOrdinals;
}
public void setEagerGlobalOrdinals(boolean eagerGlobalOrdinals) {
checkIfFrozen();
this.eagerGlobalOrdinals = eagerGlobalOrdinals;
}
}

View File

@ -32,6 +32,8 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.BytesBinaryDVIndexFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -131,6 +133,12 @@ public class BinaryFieldMapper extends FieldMapper {
public Object valueForSearch(Object value) {
return value(value);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new BytesBinaryDVIndexFieldData.Builder();
}
}
protected BinaryFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -29,6 +29,9 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -191,6 +194,12 @@ public class BooleanFieldMapper extends FieldMapper {
public boolean useTermQueryWithQueryString() {
return true;
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.BOOLEAN);
}
}
protected BooleanFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -38,6 +38,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericIntegerAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -185,6 +188,12 @@ public class ByteFieldMapper extends NumberFieldMapper {
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue
);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.BYTE);
}
}
protected ByteFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -183,7 +183,6 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
private ContextMappings contextMappings = null;
public CompletionFieldType() {
setFieldDataType(null);
}
private CompletionFieldType(CompletionFieldType ref) {

View File

@ -44,7 +44,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericDateAnalyzer;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -252,7 +254,6 @@ public class DateFieldMapper extends NumberFieldMapper {
public DateFieldType() {
super(LegacyNumericType.LONG);
setFieldDataType(new FieldDataType("long"));
}
protected DateFieldType(DateFieldType ref) {
@ -434,6 +435,12 @@ public class DateFieldMapper extends NumberFieldMapper {
}
return dateParser.parse(strValue, now(), inclusive, zone);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.LONG);
}
}
protected DateFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit<Boolean> ignoreMalformed,Explicit<Boolean> coerce,

View File

@ -40,6 +40,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericDoubleAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -188,6 +191,12 @@ public class DoubleFieldMapper extends NumberFieldMapper {
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue
);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.DOUBLE);
}
}
protected DoubleFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit<Boolean> ignoreMalformed,

View File

@ -41,6 +41,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericFloatAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -189,6 +192,12 @@ public class FloatFieldMapper extends NumberFieldMapper {
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue
);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.FLOAT);
}
}
protected FloatFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -40,6 +40,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericIntegerAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -194,6 +197,12 @@ public class IntegerFieldMapper extends NumberFieldMapper {
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue
);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.INT);
}
}
protected IntegerFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -29,6 +29,9 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -92,6 +95,11 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap
return super.indexOptions(indexOptions);
}
public Builder eagerGlobalOrdinals(boolean eagerGlobalOrdinals) {
fieldType().setEagerGlobalOrdinals(eagerGlobalOrdinals);
return builder;
}
@Override
public KeywordFieldMapper build(BuilderContext context) {
setupFieldType(context);
@ -123,6 +131,9 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap
} else if (propName.equals("norms")) {
builder.omitNorms(XContentMapValues.nodeBooleanValue(propNode) == false);
iterator.remove();
} else if (propName.equals("eager_global_ordinals")) {
builder.eagerGlobalOrdinals(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (parseMultiField(builder, name, parserContext, propName, propNode)) {
iterator.remove();
}
@ -163,6 +174,12 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap
}
return termQuery(nullValue(), null);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder();
}
}
private Boolean includeInAll;

View File

@ -40,6 +40,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericLongAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -192,6 +195,12 @@ public class LongFieldMapper extends NumberFieldMapper {
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue
);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.LONG);
}
}
protected LongFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -39,6 +39,9 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;

View File

@ -40,6 +40,9 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericIntegerAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -190,6 +193,12 @@ public class ShortFieldMapper extends NumberFieldMapper {
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue
);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.SHORT);
}
}
protected ShortFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -34,6 +34,10 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -47,6 +51,7 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import static org.apache.lucene.index.IndexOptions.NONE;
@ -61,9 +66,13 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
private static final Set<String> SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE = new HashSet<>(Arrays.asList(
"type",
// most common parameters, for which the upgrade is straightforward
"index", "store", "doc_values", "omit_norms", "norms", "fields", "copy_to"));
"index", "store", "doc_values", "omit_norms", "norms", "fields", "copy_to", "fielddata"));
public static class Defaults {
public static double FIELDDATA_MIN_FREQUENCY = 0;
public static double FIELDDATA_MAX_FREQUENCY = Integer.MAX_VALUE;
public static int FIELDDATA_MIN_SEGMENT_SIZE = 0;
public static final MappedFieldType FIELD_TYPE = new StringFieldType();
static {
@ -94,6 +103,11 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
builder = this;
}
@Override
public StringFieldType fieldType() {
return (StringFieldType) super.fieldType();
}
@Override
public Builder searchAnalyzer(NamedAnalyzer searchAnalyzer) {
super.searchAnalyzer(searchAnalyzer);
@ -110,6 +124,31 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
return this;
}
public Builder fielddata(boolean fielddata) {
fieldType().setFielddata(fielddata);
return builder;
}
public Builder eagerGlobalOrdinals(boolean eagerGlobalOrdinals) {
fieldType().setEagerGlobalOrdinals(eagerGlobalOrdinals);
return builder;
}
public Builder fielddataFrequencyFilter(double minFreq, double maxFreq, int minSegmentSize) {
fieldType().setFielddataMinFrequency(minFreq);
fieldType().setFielddataMaxFrequency(maxFreq);
fieldType().setFielddataMinSegmentSize(minSegmentSize);
return builder;
}
@Override
protected void setupFieldType(BuilderContext context) {
super.setupFieldType(context);
if (fieldType().hasDocValues() && ((StringFieldType) fieldType()).fielddata()) {
((StringFieldType) fieldType()).setFielddata(false);
}
}
@Override
public StringFieldMapper build(BuilderContext context) {
if (positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
@ -134,7 +173,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
setupFieldType(context);
StringFieldMapper fieldMapper = new StringFieldMapper(
name, fieldType, defaultFieldType, positionIncrementGap, ignoreAbove,
name, fieldType(), defaultFieldType, positionIncrementGap, ignoreAbove,
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
return fieldMapper.includeInAll(includeInAll);
}
@ -175,6 +214,28 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
node.put("norms", TypeParsers.nodeBooleanValue("omit_norms", omitNorms, parserContext) == false);
}
}
{
// upgrade fielddata settings
Object fielddataO = node.get("fielddata");
if (fielddataO instanceof Map) {
Map<?,?> fielddata = (Map<?, ?>) fielddataO;
if (keyword == false) {
node.put("fielddata", "disabled".equals(fielddata.get("format")) == false);
Map<?,?> fielddataFilter = (Map<?, ?>) fielddata.get("filter");
if (fielddataFilter != null) {
Map<?,?> frequencyFilter = (Map<?, ?>) fielddataFilter.get("frequency");
frequencyFilter.keySet().retainAll(Arrays.asList("min", "max", "min_segment_size"));
node.put("fielddata_frequency_filter", frequencyFilter);
}
} else {
node.remove("fielddata");
}
final Object loading = fielddata.get("loading");
if (loading != null) {
node.put("eager_global_ordinals", "eager_global_ordinals".equals(loading));
}
}
}
if (keyword) {
return new KeywordFieldMapper.TypeParser().parse(fieldName, node, parserContext);
} else {
@ -185,6 +246,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
throw new IllegalArgumentException("The [string] type is removed in 5.0. You should now use either a [text] "
+ "or [keyword] field instead for field [" + fieldName + "]");
}
StringFieldMapper.Builder builder = new StringFieldMapper.Builder(fieldName);
// hack for the fact that string can't just accept true/false for
// the index property and still accepts no/not_analyzed/analyzed
@ -207,6 +269,21 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true], [false], [no], [not_analyzed] or [analyzed]");
}
}
final Object fielddataObject = node.get("fielddata");
if (fielddataObject instanceof Map) {
Map<?,?> fielddata = (Map<?, ?>) fielddataObject;
final Object loading = fielddata.get("loading");
if (loading != null) {
node.put("eager_global_ordinals", "eager_global_ordinals".equals(loading));
}
Map<?,?> fielddataFilter = (Map<?, ?>) fielddata.get("filter");
if (fielddataFilter != null) {
Map<?,?> frequencyFilter = (Map<?, ?>) fielddataFilter.get("frequency");
frequencyFilter.keySet().retainAll(Arrays.asList("min", "max", "min_segment_size"));
node.put("fielddata_frequency_filter", frequencyFilter);
}
node.put("fielddata", "disabled".equals(fielddata.get("format")) == false);
}
parseTextField(builder, fieldName, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
@ -239,6 +316,20 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
} else if (propName.equals("ignore_above")) {
builder.ignoreAbove(XContentMapValues.nodeIntegerValue(propNode, -1));
iterator.remove();
} else if (propName.equals("fielddata")) {
builder.fielddata(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("eager_global_ordinals")) {
builder.eagerGlobalOrdinals(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("fielddata_frequency_filter")) {
Map<?,?> frequencyFilter = (Map<?, ?>) propNode;
double minFrequency = XContentMapValues.nodeDoubleValue(frequencyFilter.remove("min"), 0);
double maxFrequency = XContentMapValues.nodeDoubleValue(frequencyFilter.remove("max"), Integer.MAX_VALUE);
int minSegmentSize = XContentMapValues.nodeIntegerValue(frequencyFilter.remove("min_segment_size"), 0);
builder.fielddataFrequencyFilter(minFrequency, maxFrequency, minSegmentSize);
DocumentMapperParser.checkNoRemainingFields(propName, frequencyFilter, parserContext.indexVersionCreated());
iterator.remove();
} else if (parseMultiField(builder, fieldName, parserContext, propName, propNode)) {
iterator.remove();
}
@ -249,10 +340,42 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
public static final class StringFieldType extends MappedFieldType {
public StringFieldType() {}
private boolean fielddata;
private double fielddataMinFrequency;
private double fielddataMaxFrequency;
private int fielddataMinSegmentSize;
public StringFieldType() {
fielddata = true;
fielddataMinFrequency = Defaults.FIELDDATA_MIN_FREQUENCY;
fielddataMaxFrequency = Defaults.FIELDDATA_MAX_FREQUENCY;
fielddataMinSegmentSize = Defaults.FIELDDATA_MIN_SEGMENT_SIZE;
}
protected StringFieldType(StringFieldType ref) {
super(ref);
this.fielddata = ref.fielddata;
this.fielddataMinFrequency = ref.fielddataMinFrequency;
this.fielddataMaxFrequency = ref.fielddataMaxFrequency;
this.fielddataMinSegmentSize = ref.fielddataMinSegmentSize;
}
@Override
public boolean equals(Object o) {
if (super.equals(o) == false) {
return false;
}
StringFieldType that = (StringFieldType) o;
return fielddata == that.fielddata
&& fielddataMinFrequency == that.fielddataMinFrequency
&& fielddataMaxFrequency == that.fielddataMaxFrequency
&& fielddataMinSegmentSize == that.fielddataMinSegmentSize;
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), fielddata,
fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
}
public StringFieldType clone() {
@ -264,6 +387,67 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
return CONTENT_TYPE;
}
@Override
public void checkCompatibility(MappedFieldType other,
List<String> conflicts, boolean strict) {
super.checkCompatibility(other, conflicts, strict);
StringFieldType otherType = (StringFieldType) other;
if (strict) {
if (fielddata() != otherType.fielddata()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [fielddata] "
+ "across all types.");
}
if (fielddataMinFrequency() != otherType.fielddataMinFrequency()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.min] across all types.");
}
if (fielddataMaxFrequency() != otherType.fielddataMaxFrequency()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.max] across all types.");
}
if (fielddataMinSegmentSize() != otherType.fielddataMinSegmentSize()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.min_segment_size] across all types.");
}
}
}
public boolean fielddata() {
return fielddata;
}
public void setFielddata(boolean fielddata) {
checkIfFrozen();
this.fielddata = fielddata;
}
public double fielddataMinFrequency() {
return fielddataMinFrequency;
}
public void setFielddataMinFrequency(double fielddataMinFrequency) {
checkIfFrozen();
this.fielddataMinFrequency = fielddataMinFrequency;
}
public double fielddataMaxFrequency() {
return fielddataMaxFrequency;
}
public void setFielddataMaxFrequency(double fielddataMaxFrequency) {
checkIfFrozen();
this.fielddataMaxFrequency = fielddataMaxFrequency;
}
public int fielddataMinSegmentSize() {
return fielddataMinSegmentSize;
}
public void setFielddataMinSegmentSize(int fielddataMinSegmentSize) {
checkIfFrozen();
this.fielddataMinSegmentSize = fielddataMinSegmentSize;
}
@Override
public String value(Object value) {
if (value == null) {
@ -279,13 +463,26 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
return termQuery(nullValue(), null);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
if (hasDocValues()) {
return new DocValuesIndexFieldData.Builder();
} else if (fielddata) {
return new PagedBytesIndexFieldData.Builder(fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
} else {
throw new IllegalStateException("Fielddata is disabled on analyzed string fields by default. Set fielddata=true on ["
+ name() + "] in order to load fielddata in memory by uninverting the inverted index. Note that this can however "
+ "use significant memory.");
}
}
}
private Boolean includeInAll;
private int positionIncrementGap;
private int ignoreAbove;
protected StringFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
protected StringFieldMapper(String simpleName, StringFieldType fieldType, MappedFieldType defaultFieldType,
int positionIncrementGap, int ignoreAbove,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
@ -296,6 +493,12 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) {
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot be analyzed and have doc values");
}
if (fieldType.hasDocValues() && (
fieldType.fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY
|| fieldType.fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY
|| fieldType.fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE)) {
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot have doc values and use fielddata filtering");
}
this.positionIncrementGap = positionIncrementGap;
this.ignoreAbove = ignoreAbove;
}
@ -439,6 +642,11 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
}
}
@Override
public StringFieldType fieldType() {
return (StringFieldType) super.fieldType();
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
@ -460,6 +668,27 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
if (includeDefaults || ignoreAbove != Defaults.IGNORE_ABOVE) {
builder.field("ignore_above", ignoreAbove);
}
if (includeDefaults || fieldType().fielddata() != ((StringFieldType) defaultFieldType).fielddata()) {
builder.field("fielddata", fieldType().fielddata());
}
if (fieldType().fielddata()) {
if (includeDefaults
|| fieldType().fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY
|| fieldType().fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY
|| fieldType().fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE) {
builder.startObject("fielddata_frequency_filter");
if (includeDefaults || fieldType().fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY) {
builder.field("min", fieldType().fielddataMinFrequency());
}
if (includeDefaults || fieldType().fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY) {
builder.field("max", fieldType().fielddataMaxFrequency());
}
if (includeDefaults || fieldType().fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE) {
builder.field("min_segment_size", fieldType().fielddataMinSegmentSize());
}
builder.endObject();
}
}
}
/**

View File

@ -27,6 +27,9 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -38,6 +41,7 @@ import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.index.mapper.core.TypeParsers.parseMultiField;
import static org.elasticsearch.index.mapper.core.TypeParsers.parseTextField;
@ -49,6 +53,10 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1;
public static class Defaults {
public static double FIELDDATA_MIN_FREQUENCY = 0;
public static double FIELDDATA_MAX_FREQUENCY = Integer.MAX_VALUE;
public static int FIELDDATA_MIN_SEGMENT_SIZE = 0;
public static final MappedFieldType FIELD_TYPE = new TextFieldType();
static {
@ -72,6 +80,11 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
builder = this;
}
@Override
public TextFieldType fieldType() {
return (TextFieldType) super.fieldType();
}
public Builder positionIncrementGap(int positionIncrementGap) {
if (positionIncrementGap < 0) {
throw new MapperParsingException("[positions_increment_gap] must be positive, got " + positionIncrementGap);
@ -80,6 +93,11 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
return this;
}
public Builder fielddata(boolean fielddata) {
fieldType().setFielddata(fielddata);
return builder;
}
@Override
public Builder docValues(boolean docValues) {
if (docValues) {
@ -88,6 +106,18 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
return super.docValues(docValues);
}
public Builder eagerGlobalOrdinals(boolean eagerGlobalOrdinals) {
fieldType().setEagerGlobalOrdinals(eagerGlobalOrdinals);
return builder;
}
public Builder fielddataFrequencyFilter(double minFreq, double maxFreq, int minSegmentSize) {
fieldType().setFielddataMinFrequency(minFreq);
fieldType().setFielddataMaxFrequency(maxFreq);
fieldType().setFielddataMinSegmentSize(minSegmentSize);
return builder;
}
@Override
public TextFieldMapper build(BuilderContext context) {
if (positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
@ -119,6 +149,20 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
int newPositionIncrementGap = XContentMapValues.nodeIntegerValue(propNode, -1);
builder.positionIncrementGap(newPositionIncrementGap);
iterator.remove();
} else if (propName.equals("fielddata")) {
builder.fielddata(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("eager_global_ordinals")) {
builder.eagerGlobalOrdinals(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("fielddata_frequency_filter")) {
Map<?,?> frequencyFilter = (Map<?, ?>) propNode;
double minFrequency = XContentMapValues.nodeDoubleValue(frequencyFilter.remove("min"), 0);
double maxFrequency = XContentMapValues.nodeDoubleValue(frequencyFilter.remove("max"), Integer.MAX_VALUE);
int minSegmentSize = XContentMapValues.nodeIntegerValue(frequencyFilter.remove("min_segment_size"), 0);
builder.fielddataFrequencyFilter(minFrequency, maxFrequency, minSegmentSize);
DocumentMapperParser.checkNoRemainingFields(propName, frequencyFilter, parserContext.indexVersionCreated());
iterator.remove();
} else if (parseMultiField(builder, fieldName, parserContext, propName, propNode)) {
iterator.remove();
}
@ -129,16 +173,110 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
public static final class TextFieldType extends MappedFieldType {
public TextFieldType() {}
private boolean fielddata;
private double fielddataMinFrequency;
private double fielddataMaxFrequency;
private int fielddataMinSegmentSize;
public TextFieldType() {
// TODO: change the default to false
fielddata = true;
fielddataMinFrequency = Defaults.FIELDDATA_MIN_FREQUENCY;
fielddataMaxFrequency = Defaults.FIELDDATA_MAX_FREQUENCY;
fielddataMinSegmentSize = Defaults.FIELDDATA_MIN_SEGMENT_SIZE;
}
protected TextFieldType(TextFieldType ref) {
super(ref);
this.fielddata = ref.fielddata;
this.fielddataMinFrequency = ref.fielddataMinFrequency;
this.fielddataMaxFrequency = ref.fielddataMaxFrequency;
this.fielddataMinSegmentSize = ref.fielddataMinSegmentSize;
}
public TextFieldType clone() {
return new TextFieldType(this);
}
@Override
public boolean equals(Object o) {
if (super.equals(o) == false) {
return false;
}
TextFieldType that = (TextFieldType) o;
return fielddata == that.fielddata
&& fielddataMinFrequency == that.fielddataMinFrequency
&& fielddataMaxFrequency == that.fielddataMaxFrequency
&& fielddataMinSegmentSize == that.fielddataMinSegmentSize;
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), fielddata,
fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
}
@Override
public void checkCompatibility(MappedFieldType other,
List<String> conflicts, boolean strict) {
super.checkCompatibility(other, conflicts, strict);
TextFieldType otherType = (TextFieldType) other;
if (strict) {
if (fielddata() != otherType.fielddata()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [fielddata] "
+ "across all types.");
}
if (fielddataMinFrequency() != otherType.fielddataMinFrequency()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.min] across all types.");
}
if (fielddataMaxFrequency() != otherType.fielddataMaxFrequency()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.max] across all types.");
}
if (fielddataMinSegmentSize() != otherType.fielddataMinSegmentSize()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.min_segment_size] across all types.");
}
}
}
public boolean fielddata() {
return fielddata;
}
public void setFielddata(boolean fielddata) {
checkIfFrozen();
this.fielddata = fielddata;
}
public double fielddataMinFrequency() {
return fielddataMinFrequency;
}
public void setFielddataMinFrequency(double fielddataMinFrequency) {
checkIfFrozen();
this.fielddataMinFrequency = fielddataMinFrequency;
}
public double fielddataMaxFrequency() {
return fielddataMaxFrequency;
}
public void setFielddataMaxFrequency(double fielddataMaxFrequency) {
checkIfFrozen();
this.fielddataMaxFrequency = fielddataMaxFrequency;
}
public int fielddataMinSegmentSize() {
return fielddataMinSegmentSize;
}
public void setFielddataMinSegmentSize(int fielddataMinSegmentSize) {
checkIfFrozen();
this.fielddataMinSegmentSize = fielddataMinSegmentSize;
}
@Override
public String typeName() {
return CONTENT_TYPE;
@ -159,6 +297,16 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
}
return termQuery(nullValue(), null);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
if (fielddata == false) {
throw new IllegalStateException("Fielddata is disabled on text fields by default. Set fielddata=true on [" + name()
+ "] in order to load fielddata in memory by uninverting the inverted index. Note that this can however "
+ "use significant memory.");
}
return new PagedBytesIndexFieldData.Builder(fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
}
}
private Boolean includeInAll;
@ -249,6 +397,11 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
this.includeInAll = ((TextFieldMapper) mergeWith).includeInAll;
}
@Override
public TextFieldType fieldType() {
return (TextFieldType) super.fieldType();
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
@ -263,5 +416,27 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
if (includeDefaults || positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
builder.field("position_increment_gap", positionIncrementGap);
}
if (includeDefaults || fieldType().fielddata() != ((TextFieldType) defaultFieldType).fielddata()) {
builder.field("fielddata", fieldType().fielddata());
}
if (fieldType().fielddata()) {
if (includeDefaults
|| fieldType().fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY
|| fieldType().fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY
|| fieldType().fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE) {
builder.startObject("fielddata_frequency_filter");
if (includeDefaults || fieldType().fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY) {
builder.field("min", fieldType().fielddataMinFrequency());
}
if (includeDefaults || fieldType().fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY) {
builder.field("max", fieldType().fielddataMaxFrequency());
}
if (includeDefaults || fieldType().fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE) {
builder.field("min_segment_size", fieldType().fielddataMinSegmentSize());
}
builder.endObject();
}
}
}
}

View File

@ -28,13 +28,10 @@ import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType.Loading;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.object.ObjectMapper;
@ -195,7 +192,7 @@ public class TypeParsers {
if (propName2.equals("enabled")) {
builder.omitNorms(!lenientNodeBooleanValue(propNode2));
propsIterator.remove();
} else if (propName2.equals(Loading.KEY)) {
} else if (propName2.equals("loading")) {
// ignore for bw compat
propsIterator.remove();
}
@ -266,9 +263,10 @@ public class TypeParsers {
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
builder.similarity(similarityProvider);
iterator.remove();
} else if (propName.equals("fielddata")) {
final Settings settings = Settings.builder().put(SettingsLoader.Helper.loadNestedFromMap(nodeMapValue(propNode, "fielddata"))).build();
builder.fieldDataSettings(settings);
} else if (propName.equals("fielddata")
&& propNode instanceof Map
&& parserContext.indexVersionCreated().before(Version.V_5_0_0)) {
// ignore for bw compat
iterator.remove();
} else if (propName.equals("copy_to")) {
if (parserContext.isWithinMultiField()) {

View File

@ -34,6 +34,8 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.AbstractGeoPointDVIndexFieldData;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -73,7 +75,7 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
public static final boolean ENABLE_GEOHASH = false;
public static final boolean ENABLE_GEOHASH_PREFIX = false;
public static final int GEO_HASH_PRECISION = GeoHashUtils.PRECISION;
public static final Explicit<Boolean> IGNORE_MALFORMED = new Explicit(false, false);
public static final Explicit<Boolean> IGNORE_MALFORMED = new Explicit<>(false, false);
}
public abstract static class Builder<T extends Builder, Y extends BaseGeoPointFieldMapper> extends FieldMapper.Builder<T, Y> {
@ -99,12 +101,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
return (GeoPointFieldType)fieldType;
}
@Override
public T fieldDataSettings(Settings settings) {
this.fieldDataSettings = settings;
return builder;
}
public T enableLatLon(boolean enableLatLon) {
this.enableLatLon = enableLatLon;
return builder;
@ -345,6 +341,11 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
this.latFieldType = latFieldType;
this.lonFieldType = lonFieldType;
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
return new AbstractGeoPointDVIndexFieldData.Builder();
}
}
protected DoubleFieldMapper latMapper;

View File

@ -24,7 +24,6 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.elasticsearch.Version;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.lucene.Lucene;
@ -32,7 +31,6 @@ import org.elasticsearch.common.lucene.all.AllField;
import org.elasticsearch.common.lucene.all.AllTermQuery;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -168,7 +166,6 @@ public class AllFieldMapper extends MetadataFieldMapper {
static final class AllFieldType extends MappedFieldType {
public AllFieldType() {
setFieldDataType(new FieldDataType("string"));
}
protected AllFieldType(AllFieldType ref) {

View File

@ -27,7 +27,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -132,7 +131,6 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
private boolean enabled = Defaults.ENABLED;
public FieldNamesFieldType() {
setFieldDataType(new FieldDataType("string"));
}
protected FieldNamesFieldType(FieldNamesFieldType ref) {

View File

@ -38,7 +38,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -112,7 +111,6 @@ public class IdFieldMapper extends MetadataFieldMapper {
static final class IdFieldType extends MappedFieldType {
public IdFieldType() {
setFieldDataType(new FieldDataType("string"));
}
protected IdFieldType(IdFieldType ref) {

View File

@ -29,6 +29,8 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.IndexIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -184,6 +186,11 @@ public class IndexFieldMapper extends MetadataFieldMapper {
}
return value.toString();
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
return new IndexIndexFieldData.Builder();
}
}
private EnabledAttributeMapper enabledState;

View File

@ -30,11 +30,14 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
@ -53,7 +56,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeMapValue;
/**
@ -94,6 +96,11 @@ public class ParentFieldMapper extends MetadataFieldMapper {
return builder;
}
public Builder eagerGlobalOrdinals(boolean eagerGlobalOrdinals) {
((ParentFieldType) fieldType()).setEagerGlobalOrdinals(eagerGlobalOrdinals);
return builder;
}
@Override
public ParentFieldMapper build(BuilderContext context) {
if (parentType == null) {
@ -106,6 +113,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
public static class TypeParser implements MetadataFieldMapper.TypeParser {
private static final ParseField FIELDDATA = new ParseField("fielddata").withAllDeprecated("eager_global_ordinals");
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
Builder builder = new Builder(parserContext.type());
@ -116,14 +124,16 @@ public class ParentFieldMapper extends MetadataFieldMapper {
if (fieldName.equals("type")) {
builder.type(fieldNode.toString());
iterator.remove();
} else if (fieldName.equals("fielddata")) {
// Only take over `loading`, since that is the only option now that is configurable:
} else if (parserContext.parseFieldMatcher().match(fieldName, FIELDDATA)) {
// for bw compat only
Map<String, String> fieldDataSettings = SettingsLoader.Helper.loadNestedFromMap(nodeMapValue(fieldNode, "fielddata"));
if (fieldDataSettings.containsKey(MappedFieldType.Loading.KEY)) {
Settings settings = settingsBuilder().put(MappedFieldType.Loading.KEY, fieldDataSettings.get(MappedFieldType.Loading.KEY)).build();
builder.fieldDataSettings(settings);
if (fieldDataSettings.containsKey("loading")) {
builder.eagerGlobalOrdinals("eager_global_ordinals".equals(fieldDataSettings.get("loading")));
}
iterator.remove();
} else if (fieldName.equals("eager_global_ordinals")) {
builder.eagerGlobalOrdinals(XContentMapValues.nodeBooleanValue(fieldNode));
iterator.remove();
}
}
return builder;
@ -143,7 +153,6 @@ public class ParentFieldMapper extends MetadataFieldMapper {
parentJoinField.indexOptions(IndexOptions.NONE);
parentJoinField.docValues(true);
parentJoinField.fieldType().setDocValuesType(DocValuesType.SORTED);
parentJoinField.fieldType().setFieldDataType(null);
return parentJoinField.build(context);
}
@ -152,8 +161,8 @@ public class ParentFieldMapper extends MetadataFieldMapper {
final String documentType;
public ParentFieldType() {
setFieldDataType(new FieldDataType(NAME, settingsBuilder().put(MappedFieldType.Loading.KEY, Loading.EAGER_VALUE)));
documentType = null;
setEagerGlobalOrdinals(true);
}
ParentFieldType(ParentFieldType ref, String documentType) {
@ -200,6 +209,11 @@ public class ParentFieldMapper extends MetadataFieldMapper {
query.add(new TermQuery(new Term(TypeFieldMapper.NAME, documentType)), BooleanClause.Occur.FILTER);
return query.build();
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
return new ParentChildIndexFieldData.Builder();
}
}
private final String parentType;
@ -288,17 +302,13 @@ public class ParentFieldMapper extends MetadataFieldMapper {
builder.startObject(CONTENT_TYPE);
builder.field("type", parentType);
if (includeDefaults || joinFieldHasCustomFieldDataSettings()) {
builder.field("fielddata", (Map) fieldType().fieldDataType().getSettings().getAsMap());
if (includeDefaults || fieldType().eagerGlobalOrdinals() != defaultFieldType.eagerGlobalOrdinals()) {
builder.field("eager_global_ordinals", fieldType().eagerGlobalOrdinals());
}
builder.endObject();
return builder;
}
private boolean joinFieldHasCustomFieldDataSettings() {
return fieldType != null && fieldType.fieldDataType() != null && fieldType.fieldDataType().equals(Defaults.FIELD_TYPE.fieldDataType()) == false;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);

View File

@ -26,7 +26,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -111,7 +110,6 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
static final class RoutingFieldType extends MappedFieldType {
public RoutingFieldType() {
setFieldDataType(new FieldDataType("string"));
}
protected RoutingFieldType(RoutingFieldType ref) {

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.mapper.internal;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
@ -95,7 +94,7 @@ public class TTLFieldMapper extends MetadataFieldMapper {
public TTLFieldMapper build(BuilderContext context) {
setupFieldType(context);
fieldType.setHasDocValues(false);
return new TTLFieldMapper(fieldType, enabledState, defaultTTL, fieldDataSettings, context.indexSettings());
return new TTLFieldMapper(fieldType, enabledState, defaultTTL, context.indexSettings());
}
}
@ -161,11 +160,11 @@ public class TTLFieldMapper extends MetadataFieldMapper {
private long defaultTTL;
private TTLFieldMapper(Settings indexSettings) {
this(Defaults.TTL_FIELD_TYPE.clone(), Defaults.ENABLED_STATE, Defaults.DEFAULT, null, indexSettings);
this(Defaults.TTL_FIELD_TYPE.clone(), Defaults.ENABLED_STATE, Defaults.DEFAULT, indexSettings);
}
private TTLFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabled, long defaultTTL,
@Nullable Settings fieldDataSettings, Settings indexSettings) {
Settings indexSettings) {
super(NAME, fieldType, Defaults.TTL_FIELD_TYPE, indexSettings);
this.enabledState = enabled;
this.defaultTTL = defaultTTL;

View File

@ -34,7 +34,6 @@ import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -102,7 +101,6 @@ public class TypeFieldMapper extends MetadataFieldMapper {
static final class TypeFieldType extends MappedFieldType {
public TypeFieldType() {
setFieldDataType(new FieldDataType("string"));
}
protected TypeFieldType(TypeFieldType ref) {

View File

@ -25,25 +25,24 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.index.mapper.core.TypeParsers.parseField;
/**
*
*/
@ -104,7 +103,6 @@ public class UidFieldMapper extends MetadataFieldMapper {
static final class UidFieldType extends MappedFieldType {
public UidFieldType() {
setFieldDataType(new FieldDataType("string"));
}
protected UidFieldType(UidFieldType ref) {
@ -128,6 +126,15 @@ public class UidFieldMapper extends MetadataFieldMapper {
}
return Uid.createUid(value.toString());
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
// TODO: add doc values support?
return new PagedBytesIndexFieldData.Builder(
TextFieldMapper.Defaults.FIELDDATA_MIN_FREQUENCY,
TextFieldMapper.Defaults.FIELDDATA_MAX_FREQUENCY,
TextFieldMapper.Defaults.FIELDDATA_MIN_SEGMENT_SIZE);
}
}
private UidFieldMapper(Settings indexSettings, MappedFieldType existing) {

View File

@ -24,7 +24,6 @@ import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.index.DocValuesType;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -82,7 +81,6 @@ public class VersionFieldMapper extends MetadataFieldMapper {
static final class VersionFieldType extends MappedFieldType {
public VersionFieldType() {
setFieldDataType(new FieldDataType("long"));
}
protected VersionFieldType(VersionFieldType ref) {

View File

@ -43,7 +43,9 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.analysis.NumericAnalyzer;
import org.elasticsearch.index.analysis.NumericTokenizer;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
@ -162,7 +164,6 @@ public class IpFieldMapper extends NumberFieldMapper {
public static final class IpFieldType extends LongFieldMapper.LongFieldType {
public IpFieldType() {
setFieldDataType(new FieldDataType("long"));
}
protected IpFieldType(IpFieldType ref) {
@ -271,6 +272,12 @@ public class IpFieldMapper extends NumberFieldMapper {
long maxValue = LegacyNumericUtils.getMaxLong(terms);
return new FieldStats.Ip(maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(), minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.LONG);
}
}
protected IpFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,

View File

@ -48,7 +48,9 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexWarmer;
import org.elasticsearch.index.IndexWarmer.TerminationHandle;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.query.PercolatorQuery;
@ -98,9 +100,13 @@ public final class PercolatorQueryCache extends AbstractIndexComponent
final Executor executor = threadPool.executor(ThreadPool.Names.WARMER);
@Override
public IndexWarmer.TerminationHandle warmNewReaders(IndexShard indexShard, Engine.Searcher searcher) {
public TerminationHandle warmReader(IndexShard indexShard, Searcher searcher) {
final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size());
for (final LeafReaderContext ctx : searcher.reader().leaves()) {
if (cache.get(ctx.reader().getCoreCacheKey()) != null) {
latch.countDown();
continue;
}
executor.execute(() -> {
try {
final long start = System.nanoTime();
@ -121,11 +127,6 @@ public final class PercolatorQueryCache extends AbstractIndexComponent
}
return () -> latch.await();
}
@Override
public IndexWarmer.TerminationHandle warmTopReader(IndexShard indexShard, Engine.Searcher searcher) {
return IndexWarmer.TerminationHandle.NO_WAIT;
}
};
}

View File

@ -64,7 +64,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.cache.request.ShardRequestCache;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats;
@ -177,7 +176,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
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) {
public void onRemoval(ShardId shardId, String fieldName, 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);
}

View File

@ -39,7 +39,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.shard.ShardId;
@ -75,8 +74,8 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
cache.invalidateAll();
}
public IndexFieldDataCache buildIndexFieldDataCache(IndexFieldDataCache.Listener listener, Index index, String fieldName, FieldDataType fieldDataType) {
return new IndexFieldCache(logger, cache, index, fieldName, fieldDataType, indicesFieldDataCacheListener, listener);
public IndexFieldDataCache buildIndexFieldDataCache(IndexFieldDataCache.Listener listener, Index index, String fieldName) {
return new IndexFieldCache(logger, cache, index, fieldName, indicesFieldDataCacheListener, listener);
}
public Cache<Key, Accountable> getCache() {
@ -91,7 +90,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final Accountable value = notification.getValue();
for (IndexFieldDataCache.Listener listener : key.listeners) {
try {
listener.onRemoval(key.shardId, indexCache.fieldName, indexCache.fieldDataType, notification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED, value.ramBytesUsed());
listener.onRemoval(key.shardId, indexCache.fieldName, notification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED, value.ramBytesUsed());
} catch (Throwable e) {
// load anyway since listeners should not throw exceptions
logger.error("Failed to call listener on field data cache unloading", e);
@ -114,16 +113,14 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
private final ESLogger logger;
final Index index;
final String fieldName;
final FieldDataType fieldDataType;
private final Cache<Key, Accountable> cache;
private final Listener[] listeners;
IndexFieldCache(ESLogger logger,final Cache<Key, Accountable> cache, Index index, String fieldName, FieldDataType fieldDataType, Listener... listeners) {
IndexFieldCache(ESLogger logger,final Cache<Key, Accountable> cache, Index index, String fieldName, Listener... listeners) {
this.logger = logger;
this.listeners = listeners;
this.index = index;
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.cache = cache;
}
@ -140,7 +137,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final AtomicFieldData fieldData = indexFieldData.loadDirect(context);
for (Listener listener : k.listeners) {
try {
listener.onCache(shardId, fieldName, fieldDataType, fieldData);
listener.onCache(shardId, fieldName, fieldData);
} catch (Throwable e) {
// load anyway since listeners should not throw exceptions
logger.error("Failed to call listener on atomic field data loading", e);
@ -164,7 +161,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final Accountable ifd = (Accountable) indexFieldData.localGlobalDirect(indexReader);
for (Listener listener : k.listeners) {
try {
listener.onCache(shardId, fieldName, fieldDataType, ifd);
listener.onCache(shardId, fieldName, ifd);
} catch (Throwable e) {
// load anyway since listeners should not throw exceptions
logger.error("Failed to call listener on global ordinals loading", e);

View File

@ -22,7 +22,6 @@ package org.elasticsearch.indices.fielddata.cache;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
@ -43,11 +42,11 @@ public class IndicesFieldDataCacheListener implements IndexFieldDataCache.Listen
}
@Override
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable fieldData) {
public void onCache(ShardId shardId, String fieldName, Accountable fieldData) {
}
@Override
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, 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);
}

View File

@ -81,7 +81,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
protected LeafReaderContext readerContext;
protected DirectoryReader topLevelReader;
protected IndicesFieldDataCache indicesFieldDataCache;
protected abstract FieldDataType getFieldDataType();
protected abstract String getFieldDataType();
protected boolean hasDocValues() {
return false;
@ -91,7 +91,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
return getForField(getFieldDataType(), fieldName, hasDocValues());
}
public <IFD extends IndexFieldData<?>> IFD getForField(FieldDataType type, String fieldName) {
public <IFD extends IndexFieldData<?>> IFD getForField(String type, String fieldName) {
return getForField(type, fieldName, hasDocValues());
}
@ -100,35 +100,35 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
return pluginList(InternalSettingsPlugin.class);
}
public <IFD extends IndexFieldData<?>> IFD getForField(FieldDataType type, String fieldName, boolean docValues) {
public <IFD extends IndexFieldData<?>> IFD getForField(String type, String fieldName, boolean docValues) {
final MappedFieldType fieldType;
final BuilderContext context = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1));
if (type.getType().equals("string")) {
fieldType = new StringFieldMapper.Builder(fieldName).tokenized(false).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("float")) {
fieldType = new FloatFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("double")) {
fieldType = new DoubleFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("long")) {
fieldType = new LongFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("int")) {
fieldType = new IntegerFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("short")) {
fieldType = new ShortFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("byte")) {
fieldType = new ByteFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.getType().equals("geo_point")) {
if (type.equals("string")) {
fieldType = new StringFieldMapper.Builder(fieldName).tokenized(false).fielddata(docValues == false).docValues(docValues).build(context).fieldType();
} else if (type.equals("float")) {
fieldType = new FloatFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else if (type.equals("double")) {
fieldType = new DoubleFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else if (type.equals("long")) {
fieldType = new LongFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else if (type.equals("int")) {
fieldType = new IntegerFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else if (type.equals("short")) {
fieldType = new ShortFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else if (type.equals("byte")) {
fieldType = new ByteFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else if (type.equals("geo_point")) {
if (indexService.getIndexSettings().getIndexVersionCreated().before(Version.V_2_2_0)) {
fieldType = new GeoPointFieldMapperLegacy.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
fieldType = new GeoPointFieldMapperLegacy.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else {
fieldType = new GeoPointFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
fieldType = new GeoPointFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
}
} else if (type.getType().equals("_parent")) {
} else if (type.equals("_parent")) {
fieldType = new ParentFieldMapper.Builder("_type").type(fieldName).build(context).fieldType();
} else if (type.getType().equals("binary")) {
fieldType = new BinaryFieldMapper.Builder(fieldName).docValues(docValues).fieldDataSettings(type.getSettings()).build(context).fieldType();
} else if (type.equals("binary")) {
fieldType = new BinaryFieldMapper.Builder(fieldName).docValues(docValues).build(context).fieldType();
} else {
throw new UnsupportedOperationException(type.getType());
throw new UnsupportedOperationException(type);
}
return ifdService.getForField(fieldType);
}
@ -136,8 +136,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
@Before
public void setup() throws Exception {
Version version = VersionUtils.randomVersionBetween(random(), Version.V_2_0_0, Version.V_2_3_0); // we need 2.x so that fielddata is allowed on string fields
Settings settings = Settings.builder().put("index.fielddata.cache", "none")
.put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
indexService = createIndex("test", settings);
mapperService = indexService.mapperService();
indicesFieldDataCache = getInstanceFromNode(IndicesService.class).getIndicesFieldDataCache();

View File

@ -38,7 +38,7 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
*/
public abstract class AbstractGeoFieldDataTestCase extends AbstractFieldDataImplTestCase {
@Override
protected abstract FieldDataType getFieldDataType();
protected abstract String getFieldDataType();
protected Field randomGeoPointField(String fieldName, Field.Store store) {
GeoPoint point = randomPoint(random());

View File

@ -471,8 +471,7 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
public void testGlobalOrdinals() throws Exception {
fillExtendedMvSet();
refreshReader();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("global_values", "fixed"));
IndexOrdinalsFieldData ifd = getForField(fieldDataType, "value", hasDocValues());
IndexOrdinalsFieldData ifd = getForField("string", "value", hasDocValues());
IndexOrdinalsFieldData globalOrdinals = ifd.loadGlobal(topLevelReader);
assertThat(topLevelReader.leaves().size(), equalTo(3));
@ -596,8 +595,7 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
public void testGlobalOrdinalsGetRemovedOnceIndexReaderCloses() throws Exception {
fillExtendedMvSet();
refreshReader();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("global_values", "fixed").put("cache", "node"));
IndexOrdinalsFieldData ifd = getForField(fieldDataType, "value", hasDocValues());
IndexOrdinalsFieldData ifd = getForField("string", "value", hasDocValues());
IndexOrdinalsFieldData globalOrdinals = ifd.loadGlobal(topLevelReader);
assertThat(ifd.loadGlobal(topLevelReader), sameInstance(globalOrdinals));
// 3 b/c 1 segment level caches and 1 top level cache

View File

@ -112,7 +112,7 @@ public class BinaryDVFieldDataTests extends AbstractFieldDataTestCase {
}
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("binary", Settings.builder().put("format", "doc_values"));
protected String getFieldDataType() {
return "binary";
}
}

View File

@ -1,637 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.CompositeReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.English;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.unit.DistanceUnit.Distance;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.ParsedDocument;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Random;
import java.util.Set;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.lessThan;
public class DuelFieldDataTests extends AbstractFieldDataTestCase {
@Override
protected FieldDataType getFieldDataType() {
return null;
}
public void testDuelAllTypesSingleValue() throws Exception {
final String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("bytes").field("type", "string").field("index", "not_analyzed").endObject()
.startObject("byte").field("type", "byte").endObject()
.startObject("short").field("type", "short").endObject()
.startObject("integer").field("type", "integer").endObject()
.startObject("long").field("type", "long").endObject()
.startObject("float").field("type", "float").endObject()
.startObject("double").field("type", "double").endObject()
.endObject().endObject().endObject().string();
final DocumentMapper mapper = mapperService.documentMapperParser().parse("type", new CompressedXContent(mapping));
Random random = getRandom();
int atLeast = scaledRandomIntBetween(200, 1500);
for (int i = 0; i < atLeast; i++) {
String s = Integer.toString(randomByte());
XContentBuilder doc = XContentFactory.jsonBuilder().startObject();
for (String fieldName : Arrays.asList("bytes", "byte", "short", "integer", "long", "float", "double")) {
doc = doc.field(fieldName, s);
}
doc = doc.endObject();
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {
refreshReader();
}
}
LeafReaderContext context = refreshReader();
Map<FieldDataType, Type> typeMap = new HashMap<>();
typeMap.put(new FieldDataType("string", Settings.builder().put("format", "paged_bytes")), Type.Bytes);
typeMap.put(new FieldDataType("byte", Settings.builder().put("format", "doc_values")), Type.Integer);
typeMap.put(new FieldDataType("short", Settings.builder().put("format", "doc_values")), Type.Integer);
typeMap.put(new FieldDataType("int", Settings.builder().put("format", "doc_values")), Type.Integer);
typeMap.put(new FieldDataType("long", Settings.builder().put("format", "doc_values")), Type.Long);
typeMap.put(new FieldDataType("double", Settings.builder().put("format", "doc_values")), Type.Double);
typeMap.put(new FieldDataType("float", Settings.builder().put("format", "doc_values")), Type.Float);
typeMap.put(new FieldDataType("string", Settings.builder().put("format", "doc_values")), Type.Bytes);
ArrayList<Entry<FieldDataType, Type>> list = new ArrayList<>(typeMap.entrySet());
Preprocessor pre = new ToDoublePreprocessor();
while (!list.isEmpty()) {
Entry<FieldDataType, Type> left;
Entry<FieldDataType, Type> right;
if (list.size() > 1) {
left = list.remove(random.nextInt(list.size()));
right = list.remove(random.nextInt(list.size()));
} else {
right = left = list.remove(0);
}
ifdService.clear();
IndexFieldData<?> leftFieldData = getForField(left.getKey(), left.getValue().name().toLowerCase(Locale.ROOT), true);
ifdService.clear();
IndexFieldData<?> rightFieldData = getForField(right.getKey(), right.getValue().name().toLowerCase(Locale.ROOT), true);
duelFieldDataBytes(random, context, leftFieldData, rightFieldData, pre);
duelFieldDataBytes(random, context, rightFieldData, leftFieldData, pre);
DirectoryReader perSegment = DirectoryReader.open(writer);
CompositeReaderContext composite = perSegment.getContext();
List<LeafReaderContext> leaves = composite.leaves();
for (LeafReaderContext atomicReaderContext : leaves) {
duelFieldDataBytes(random, atomicReaderContext, leftFieldData, rightFieldData, pre);
}
}
}
public void testDuelIntegers() throws Exception {
final String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("byte").field("type", "byte").endObject()
.startObject("short").field("type", "short").endObject()
.startObject("integer").field("type", "integer").endObject()
.startObject("long").field("type", "long").endObject()
.endObject().endObject().endObject().string();
final DocumentMapper mapper = mapperService.documentMapperParser().parse("type", new CompressedXContent(mapping));
Random random = getRandom();
int atLeast = scaledRandomIntBetween(200, 1500);
final int maxNumValues = randomBoolean() ? 1 : randomIntBetween(2, 10);
byte[] values = new byte[maxNumValues];
for (int i = 0; i < atLeast; i++) {
int numValues = randomInt(maxNumValues);
// FD loses values if they are duplicated, so we must deduplicate for this test
Set<Byte> vals = new HashSet<Byte>();
for (int j = 0; j < numValues; ++j) {
vals.add(randomByte());
}
numValues = vals.size();
int upto = 0;
for (Byte bb : vals) {
values[upto++] = bb.byteValue();
}
XContentBuilder doc = XContentFactory.jsonBuilder().startObject();
for (String fieldName : Arrays.asList("byte", "short", "integer", "long")) {
doc = doc.startArray(fieldName);
for (int j = 0; j < numValues; ++j) {
doc = doc.value(values[j]);
}
doc = doc.endArray();
}
doc = doc.endObject();
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {
refreshReader();
}
}
LeafReaderContext context = refreshReader();
Map<FieldDataType, Type> typeMap = new HashMap<>();
typeMap.put(new FieldDataType("byte", Settings.builder().put("format", "doc_values")), Type.Integer);
typeMap.put(new FieldDataType("short", Settings.builder().put("format", "doc_values")), Type.Integer);
typeMap.put(new FieldDataType("int", Settings.builder().put("format", "doc_values")), Type.Integer);
typeMap.put(new FieldDataType("long", Settings.builder().put("format", "doc_values")), Type.Long);
ArrayList<Entry<FieldDataType, Type>> list = new ArrayList<>(typeMap.entrySet());
while (!list.isEmpty()) {
Entry<FieldDataType, Type> left;
Entry<FieldDataType, Type> right;
if (list.size() > 1) {
left = list.remove(random.nextInt(list.size()));
right = list.remove(random.nextInt(list.size()));
} else {
right = left = list.remove(0);
}
ifdService.clear();
IndexNumericFieldData leftFieldData = getForField(left.getKey(), left.getValue().name().toLowerCase(Locale.ROOT), true);
ifdService.clear();
IndexNumericFieldData rightFieldData = getForField(right.getKey(), right.getValue().name().toLowerCase(Locale.ROOT), true);
duelFieldDataLong(random, context, leftFieldData, rightFieldData);
duelFieldDataLong(random, context, rightFieldData, leftFieldData);
DirectoryReader perSegment = DirectoryReader.open(writer);
CompositeReaderContext composite = perSegment.getContext();
List<LeafReaderContext> leaves = composite.leaves();
for (LeafReaderContext atomicReaderContext : leaves) {
duelFieldDataLong(random, atomicReaderContext, leftFieldData, rightFieldData);
}
}
}
public void testDuelDoubles() throws Exception {
final String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("float").field("type", "float").endObject()
.startObject("double").field("type", "double").endObject()
.endObject().endObject().endObject().string();
final DocumentMapper mapper = mapperService.documentMapperParser().parse("type", new CompressedXContent(mapping));
Random random = getRandom();
int atLeast = scaledRandomIntBetween(200, 1500);
final int maxNumValues = randomBoolean() ? 1 : randomIntBetween(2, 10);
float[] values = new float[maxNumValues];
for (int i = 0; i < atLeast; i++) {
int numValues = randomInt(maxNumValues);
float def = randomBoolean() ? randomFloat() : Float.NaN;
// FD loses values if they are duplicated, so we must deduplicate for this test
Set<Float> vals = new HashSet<Float>();
for (int j = 0; j < numValues; ++j) {
if (randomBoolean()) {
vals.add(def);
} else {
vals.add(randomFloat());
}
}
numValues = vals.size();
int upto = 0;
for (Float f : vals) {
values[upto++] = f.floatValue();
}
XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startArray("float");
for (int j = 0; j < numValues; ++j) {
doc = doc.value(values[j]);
}
doc = doc.endArray().startArray("double");
for (int j = 0; j < numValues; ++j) {
doc = doc.value(values[j]);
}
doc = doc.endArray().endObject();
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {
refreshReader();
}
}
LeafReaderContext context = refreshReader();
Map<FieldDataType, Type> typeMap = new HashMap<>();
typeMap.put(new FieldDataType("double", Settings.builder().put("format", "doc_values")), Type.Double);
typeMap.put(new FieldDataType("float", Settings.builder().put("format", "doc_values")), Type.Float);
ArrayList<Entry<FieldDataType, Type>> list = new ArrayList<>(typeMap.entrySet());
while (!list.isEmpty()) {
Entry<FieldDataType, Type> left;
Entry<FieldDataType, Type> right;
if (list.size() > 1) {
left = list.remove(random.nextInt(list.size()));
right = list.remove(random.nextInt(list.size()));
} else {
right = left = list.remove(0);
}
ifdService.clear();
IndexNumericFieldData leftFieldData = getForField(left.getKey(), left.getValue().name().toLowerCase(Locale.ROOT), true);
ifdService.clear();
IndexNumericFieldData rightFieldData = getForField(right.getKey(), right.getValue().name().toLowerCase(Locale.ROOT), true);
duelFieldDataDouble(random, context, leftFieldData, rightFieldData);
duelFieldDataDouble(random, context, rightFieldData, leftFieldData);
DirectoryReader perSegment = DirectoryReader.open(writer);
CompositeReaderContext composite = perSegment.getContext();
List<LeafReaderContext> leaves = composite.leaves();
for (LeafReaderContext atomicReaderContext : leaves) {
duelFieldDataDouble(random, atomicReaderContext, leftFieldData, rightFieldData);
}
}
}
public void testDuelStrings() throws Exception {
Random random = getRandom();
int atLeast = scaledRandomIntBetween(200, 1500);
for (int i = 0; i < atLeast; i++) {
Document d = new Document();
d.add(new StringField("_id", "" + i, Field.Store.NO));
if (random.nextInt(15) != 0) {
int[] numbers = getNumbers(random, Integer.MAX_VALUE);
for (int j : numbers) {
final String s = English.longToEnglish(j);
d.add(new StringField("bytes", s, Field.Store.NO));
d.add(new SortedSetDocValuesField("bytes", new BytesRef(s)));
}
if (random.nextInt(10) == 0) {
d.add(new StringField("bytes", "", Field.Store.NO));
d.add(new SortedSetDocValuesField("bytes", new BytesRef()));
}
}
writer.addDocument(d);
if (random.nextInt(10) == 0) {
refreshReader();
}
}
LeafReaderContext context = refreshReader();
Map<FieldDataType, Type> typeMap = new HashMap<>();
typeMap.put(new FieldDataType("string", Settings.builder().put("format", "paged_bytes")), Type.Bytes);
typeMap.put(new FieldDataType("string", Settings.builder().put("format", "doc_values")), Type.Bytes);
// TODO add filters
ArrayList<Entry<FieldDataType, Type>> list = new ArrayList<>(typeMap.entrySet());
Preprocessor pre = new Preprocessor();
while (!list.isEmpty()) {
Entry<FieldDataType, Type> left;
Entry<FieldDataType, Type> right;
if (list.size() > 1) {
left = list.remove(random.nextInt(list.size()));
right = list.remove(random.nextInt(list.size()));
} else {
right = left = list.remove(0);
}
ifdService.clear();
IndexFieldData<?> leftFieldData = getForField(left.getKey(), left.getValue().name().toLowerCase(Locale.ROOT));
ifdService.clear();
IndexFieldData<?> rightFieldData = getForField(right.getKey(), right.getValue().name().toLowerCase(Locale.ROOT));
duelFieldDataBytes(random, context, leftFieldData, rightFieldData, pre);
duelFieldDataBytes(random, context, rightFieldData, leftFieldData, pre);
DirectoryReader perSegment = DirectoryReader.open(writer);
CompositeReaderContext composite = perSegment.getContext();
List<LeafReaderContext> leaves = composite.leaves();
for (LeafReaderContext atomicReaderContext : leaves) {
duelFieldDataBytes(random, atomicReaderContext, leftFieldData, rightFieldData, pre);
}
perSegment.close();
}
}
public void testDuelGlobalOrdinals() throws Exception {
Random random = getRandom();
final int numDocs = scaledRandomIntBetween(10, 1000);
final int numValues = scaledRandomIntBetween(10, 500);
final String[] values = new String[numValues];
for (int i = 0; i < numValues; ++i) {
values[i] = new String(RandomStrings.randomAsciiOfLength(random, 10));
}
for (int i = 0; i < numDocs; i++) {
Document d = new Document();
final int numVals = randomInt(3);
for (int j = 0; j < numVals; ++j) {
final String value = RandomPicks.randomFrom(random, Arrays.asList(values));
d.add(new StringField("string", value, Field.Store.NO));
d.add(new SortedSetDocValuesField("bytes", new BytesRef(value)));
}
writer.addDocument(d);
if (randomInt(10) == 0) {
refreshReader();
}
}
refreshReader();
Map<FieldDataType, Type> typeMap = new HashMap<FieldDataType, DuelFieldDataTests.Type>();
typeMap.put(new FieldDataType("string", Settings.builder().put("format", "paged_bytes")), Type.Bytes);
typeMap.put(new FieldDataType("string", Settings.builder().put("format", "doc_values")), Type.Bytes);
for (Map.Entry<FieldDataType, Type> entry : typeMap.entrySet()) {
ifdService.clear();
IndexOrdinalsFieldData fieldData = getForField(entry.getKey(), entry.getValue().name().toLowerCase(Locale.ROOT));
RandomAccessOrds left = fieldData.load(readerContext).getOrdinalsValues();
fieldData.clear();
RandomAccessOrds right = fieldData.loadGlobal(topLevelReader).load(topLevelReader.leaves().get(0)).getOrdinalsValues();
assertEquals(left.getValueCount(), right.getValueCount());
for (long ord = 0; ord < left.getValueCount(); ++ord) {
assertEquals(left.lookupOrd(ord), right.lookupOrd(ord));
}
}
}
public void testDuelGeoPoints() throws Exception {
final String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("geopoint").field("type", "geo_point").startObject("fielddata").field("format", "doc_values").endObject().endObject()
.endObject().endObject().endObject().string();
final DocumentMapper mapper = mapperService.documentMapperParser().parse("type", new CompressedXContent(mapping));
Random random = getRandom();
int atLeast = scaledRandomIntBetween(200, 1500);
int maxValuesPerDoc = randomBoolean() ? 1 : randomIntBetween(2, 10);
// to test deduplication
double defaultLat = randomDouble() * 180 - 90;
double defaultLon = randomDouble() * 360 - 180;
for (int i = 0; i < atLeast; i++) {
final int numValues = randomInt(maxValuesPerDoc);
XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startArray("geopoint");
for (int j = 0; j < numValues; ++j) {
if (randomBoolean()) {
doc.startObject().field("lat", defaultLat).field("lon", defaultLon).endObject();
} else {
doc.startObject().field("lat", randomDouble() * 180 - 90).field("lon", randomDouble() * 360 - 180).endObject();
}
}
doc = doc.endArray().endObject();
final ParsedDocument d = mapper.parse("test", "type", Integer.toString(i), doc.bytes());
writer.addDocument(d.rootDoc());
if (random.nextInt(10) == 0) {
refreshReader();
}
}
LeafReaderContext context = refreshReader();
Map<FieldDataType, Type> typeMap = new HashMap<>();
final Distance precision = new Distance(1, randomFrom(DistanceUnit.values()));
typeMap.put(new FieldDataType("geo_point", Settings.builder().put("format", "array")), Type.GeoPoint);
typeMap.put(new FieldDataType("geo_point", Settings.builder().put("format", "doc_values")), Type.GeoPoint);
ArrayList<Entry<FieldDataType, Type>> list = new ArrayList<>(typeMap.entrySet());
while (!list.isEmpty()) {
Entry<FieldDataType, Type> left;
Entry<FieldDataType, Type> right;
if (list.size() > 1) {
left = list.remove(random.nextInt(list.size()));
right = list.remove(random.nextInt(list.size()));
} else {
right = left = list.remove(0);
}
ifdService.clear();
IndexGeoPointFieldData leftFieldData = getForField(left.getKey(), left.getValue().name().toLowerCase(Locale.ROOT));
ifdService.clear();
IndexGeoPointFieldData rightFieldData = getForField(right.getKey(), right.getValue().name().toLowerCase(Locale.ROOT));
duelFieldDataGeoPoint(random, context, leftFieldData, rightFieldData, precision);
duelFieldDataGeoPoint(random, context, rightFieldData, leftFieldData, precision);
DirectoryReader perSegment = DirectoryReader.open(writer);
CompositeReaderContext composite = perSegment.getContext();
List<LeafReaderContext> leaves = composite.leaves();
for (LeafReaderContext atomicReaderContext : leaves) {
duelFieldDataGeoPoint(random, atomicReaderContext, leftFieldData, rightFieldData, precision);
}
perSegment.close();
}
}
@Override
public void testEmpty() throws Exception {
// No need to test empty usage here
}
private int[] getNumbers(Random random, int margin) {
if (random.nextInt(20) == 0) {
int[] num = new int[1 + random.nextInt(10)];
for (int i = 0; i < num.length; i++) {
int v = (random.nextBoolean() ? -1 * random.nextInt(margin) : random.nextInt(margin));
num[i] = v;
}
return num;
}
return new int[]{(random.nextBoolean() ? -1 * random.nextInt(margin) : random.nextInt(margin))};
}
private static void duelFieldDataBytes(Random random, LeafReaderContext context, IndexFieldData<?> left, IndexFieldData<?> right, Preprocessor pre) throws Exception {
AtomicFieldData leftData = random.nextBoolean() ? left.load(context) : left.loadDirect(context);
AtomicFieldData rightData = random.nextBoolean() ? right.load(context) : right.loadDirect(context);
int numDocs = context.reader().maxDoc();
SortedBinaryDocValues leftBytesValues = leftData.getBytesValues();
SortedBinaryDocValues rightBytesValues = rightData.getBytesValues();
BytesRefBuilder leftSpare = new BytesRefBuilder();
BytesRefBuilder rightSpare = new BytesRefBuilder();
for (int i = 0; i < numDocs; i++) {
leftBytesValues.setDocument(i);
rightBytesValues.setDocument(i);
int numValues = leftBytesValues.count();
assertThat(numValues, equalTo(rightBytesValues.count()));
BytesRef previous = null;
for (int j = 0; j < numValues; j++) {
rightSpare.copyBytes(rightBytesValues.valueAt(j));
leftSpare.copyBytes(leftBytesValues.valueAt(j));
if (previous != null) {
assertThat(pre.compare(previous, rightSpare.get()), lessThan(0));
}
previous = BytesRef.deepCopyOf(rightSpare.get());
pre.toString(rightSpare.get());
pre.toString(leftSpare.get());
assertThat(pre.toString(leftSpare.get()), equalTo(pre.toString(rightSpare.get())));
}
}
}
private static void duelFieldDataDouble(Random random, LeafReaderContext context, IndexNumericFieldData left, IndexNumericFieldData right) throws Exception {
AtomicNumericFieldData leftData = random.nextBoolean() ? left.load(context) : left.loadDirect(context);
AtomicNumericFieldData rightData = random.nextBoolean() ? right.load(context) : right.loadDirect(context);
int numDocs = context.reader().maxDoc();
SortedNumericDoubleValues leftDoubleValues = leftData.getDoubleValues();
SortedNumericDoubleValues rightDoubleValues = rightData.getDoubleValues();
for (int i = 0; i < numDocs; i++) {
leftDoubleValues.setDocument(i);
rightDoubleValues.setDocument(i);
int numValues = leftDoubleValues.count();
assertThat(numValues, equalTo(rightDoubleValues.count()));
double previous = 0;
for (int j = 0; j < numValues; j++) {
double current = rightDoubleValues.valueAt(j);
if (Double.isNaN(current)) {
assertTrue(Double.isNaN(leftDoubleValues.valueAt(j)));
} else {
assertThat(leftDoubleValues.valueAt(j), closeTo(current, 0.0001));
}
if (j > 0) {
assertThat(Double.compare(previous,current), lessThan(0));
}
previous = current;
}
}
}
private static void duelFieldDataLong(Random random, LeafReaderContext context, IndexNumericFieldData left, IndexNumericFieldData right) throws Exception {
AtomicNumericFieldData leftData = random.nextBoolean() ? left.load(context) : left.loadDirect(context);
AtomicNumericFieldData rightData = random.nextBoolean() ? right.load(context) : right.loadDirect(context);
int numDocs = context.reader().maxDoc();
SortedNumericDocValues leftLongValues = leftData.getLongValues();
SortedNumericDocValues rightLongValues = rightData.getLongValues();
for (int i = 0; i < numDocs; i++) {
leftLongValues.setDocument(i);
rightLongValues.setDocument(i);
int numValues = leftLongValues.count();
long previous = 0;
assertThat(numValues, equalTo(rightLongValues.count()));
for (int j = 0; j < numValues; j++) {
long current;
assertThat(leftLongValues.valueAt(j), equalTo(current = rightLongValues.valueAt(j)));
if (j > 0) {
assertThat(previous, lessThan(current));
}
previous = current;
}
}
}
private static void duelFieldDataGeoPoint(Random random, LeafReaderContext context, IndexGeoPointFieldData left, IndexGeoPointFieldData right, Distance precision) throws Exception {
AtomicGeoPointFieldData leftData = random.nextBoolean() ? left.load(context) : left.loadDirect(context);
AtomicGeoPointFieldData rightData = random.nextBoolean() ? right.load(context) : right.loadDirect(context);
int numDocs = context.reader().maxDoc();
MultiGeoPointValues leftValues = leftData.getGeoPointValues();
MultiGeoPointValues rightValues = rightData.getGeoPointValues();
for (int i = 0; i < numDocs; ++i) {
leftValues.setDocument(i);
final int numValues = leftValues.count();
rightValues.setDocument(i);
assertEquals(numValues, rightValues.count());
List<GeoPoint> leftPoints = new ArrayList<>();
List<GeoPoint> rightPoints = new ArrayList<>();
for (int j = 0; j < numValues; ++j) {
GeoPoint l = leftValues.valueAt(j);
leftPoints.add(new GeoPoint(l.getLat(), l.getLon()));
GeoPoint r = rightValues.valueAt(j);
rightPoints.add(new GeoPoint(r.getLat(), r.getLon()));
}
// missing values were treated as 0,0 which are valid geopoints, this now correctly tests for missing values
if (leftPoints.isEmpty() == false) {
for (GeoPoint l : leftPoints) {
assertTrue("Couldn't find " + l + " among " + rightPoints, contains(l, rightPoints, precision));
}
for (GeoPoint r : rightPoints) {
assertTrue("Couldn't find " + r + " among " + leftPoints, contains(r, leftPoints, precision));
}
}
}
}
private static boolean contains(GeoPoint point, List<GeoPoint> set, Distance precision) {
for (GeoPoint r : set) {
final double distance = GeoDistance.PLANE.calculate(point.getLat(), point.getLon(), r.getLat(), r.getLon(), DistanceUnit.METERS);
if (new Distance(distance, DistanceUnit.METERS).compareTo(precision) <= 0) {
return true;
}
}
return false;
}
private static class Preprocessor {
public String toString(BytesRef ref) {
return ref.utf8ToString();
}
public int compare(BytesRef a, BytesRef b) {
return a.compareTo(b);
}
}
private static class ToDoublePreprocessor extends Preprocessor {
@Override
public String toString(BytesRef ref) {
assertTrue(ref.length > 0);
return Double.toString(Double.parseDouble(super.toString(ref)));
}
@Override
public int compare(BytesRef a, BytesRef b) {
Double _a = Double.parseDouble(super.toString(a));
return _a.compareTo(Double.parseDouble(super.toString(b)));
}
}
private static enum Type {
Float, Double, Integer, Long, Bytes, GeoPoint;
}
}

View File

@ -37,15 +37,12 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.FieldMaskingReader;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
public class FieldDataCacheTests extends ESTestCase {
@ -89,13 +86,14 @@ public class FieldDataCacheTests extends ESTestCase {
}
private SortedSetDVOrdinalsIndexFieldData createSortedDV(String fieldName, IndexFieldDataCache indexFieldDataCache) {
FieldDataType fieldDataType = new StringFieldMapper.StringFieldType().fieldDataType();
return new SortedSetDVOrdinalsIndexFieldData(createIndexSettings(), indexFieldDataCache, fieldName, new NoneCircuitBreakerService(), fieldDataType);
return new SortedSetDVOrdinalsIndexFieldData(createIndexSettings(), indexFieldDataCache, fieldName, new NoneCircuitBreakerService());
}
private PagedBytesIndexFieldData createPagedBytes(String fieldName, IndexFieldDataCache indexFieldDataCache) {
FieldDataType fieldDataType = new StringFieldMapper.StringFieldType().fieldDataType();
return new PagedBytesIndexFieldData(createIndexSettings(), fieldName, fieldDataType, indexFieldDataCache, new NoneCircuitBreakerService());
return new PagedBytesIndexFieldData(createIndexSettings(), fieldName, indexFieldDataCache, new NoneCircuitBreakerService(),
TextFieldMapper.Defaults.FIELDDATA_MIN_FREQUENCY,
TextFieldMapper.Defaults.FIELDDATA_MAX_FREQUENCY,
TextFieldMapper.Defaults.FIELDDATA_MIN_SEGMENT_SIZE);
}
private IndexSettings createIndexSettings() {

View File

@ -1,86 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.test.ESIntegTestCase;
import org.hamcrest.Matchers;
import java.io.IOException;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.search.aggregations.AggregationBuilders.terms;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
public class FieldDataFilterIntegrationIT extends ESIntegTestCase {
@Override
protected int numberOfReplicas() {
return 0;
}
public void testRegexpFilter() throws IOException {
CreateIndexRequestBuilder builder = prepareCreate("test");
XContentBuilder mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("name")
.field("type", "text")
.startObject("fielddata")
.startObject("filter")
.startObject("regex")
.field("pattern", "^bac.*")
.endObject()
.endObject()
.endObject()
.endObject()
.startObject("not_filtered")
.field("type", "text")
.endObject()
.endObject()
.endObject().endObject();
assertAcked(builder.addMapping("type", mapping));
ensureGreen();
int numDocs = scaledRandomIntBetween(5, 50);
for (int i = 0; i < numDocs; i++) {
client().prepareIndex("test", "type", "" + 0).setSource("name", "bacon bastards", "not_filtered", "bacon bastards").get();
}
refresh();
SearchResponse searchResponse = client().prepareSearch()
.setSize(0)
.setQuery(matchAllQuery())
.addAggregation(terms("name").field("name"))
.addAggregation(terms("not_filtered").field("not_filtered")).get();
Aggregations aggs = searchResponse.getAggregations();
Terms nameAgg = aggs.get("name");
assertThat(nameAgg.getBuckets().size(), Matchers.equalTo(1));
assertThat(nameAgg.getBuckets().iterator().next().getKeyAsString(), Matchers.equalTo("bacon"));
Terms notFilteredAgg = aggs.get("not_filtered");
assertThat(notFilteredAgg.getBuckets().size(), Matchers.equalTo(2));
assertThat(notFilteredAgg.getBuckets().get(0).getKeyAsString(), Matchers.isOneOf("bacon", "bastards"));
assertThat(notFilteredAgg.getBuckets().get(1).getKeyAsString(), Matchers.isOneOf("bacon", "bastards"));
}
}

View File

@ -29,29 +29,13 @@ import static org.hamcrest.Matchers.greaterThan;
/**
*/
public class FieldDataLoadingIT extends ESIntegTestCase {
public void testEagerFieldDataLoading() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("type", jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("name")
.field("type", "text")
.startObject("fielddata").field("loading", "eager").endObject()
.endObject()
.endObject().endObject().endObject()));
ensureGreen();
client().prepareIndex("test", "type", "1").setSource("name", "name").get();
client().admin().indices().prepareRefresh("test").get();
ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get();
assertThat(response.getIndicesStats().getFieldData().getMemorySizeInBytes(), greaterThan(0L));
}
public void testEagerGlobalOrdinalsFieldDataLoading() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("type", jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("name")
.field("type", "text")
.startObject("fielddata").field("loading", "eager_global_ordinals").endObject()
.field("eager_global_ordinals", true)
.endObject()
.endObject().endObject().endObject()));
ensureGreen();

View File

@ -23,7 +23,10 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import java.util.Random;
@ -31,8 +34,7 @@ import static org.hamcrest.Matchers.equalTo;
public class FilterFieldDataTests extends AbstractFieldDataTestCase {
@Override
protected FieldDataType getFieldDataType() {
// TODO Auto-generated method stub
protected String getFieldDataType() {
return null;
}
@ -57,120 +59,56 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
}
writer.forceMerge(1, true);
LeafReaderContext context = refreshReader();
String[] formats = new String[] { "paged_bytes"};
final BuilderContext builderCtx = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1));
for (String format : formats) {
{
ifdService.clear();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.frequency.min_segment_size", 100).put("filter.frequency.min", 0.0d).put("filter.frequency.max", random.nextBoolean() ? 100 : 0.5d));
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "high_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
{
ifdService.clear();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.frequency.min_segment_size", 100).put("filter.frequency.min", random.nextBoolean() ? 101 : 101d/200.0d).put("filter.frequency.max", 201));
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "high_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(1L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("5"));
}
{
ifdService.clear(); // test # docs with value
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.frequency.min_segment_size", 101).put("filter.frequency.min", random.nextBoolean() ? 101 : 101d/200.0d));
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "med_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
{
ifdService.clear();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.frequency.min_segment_size", 101).put("filter.frequency.min", random.nextBoolean() ? 101 : 101d/200.0d));
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "med_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
{
ifdService.clear();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.regex.pattern", "\\d{2,3}") // allows 10 & 100
.put("filter.frequency.min_segment_size", 0)
.put("filter.frequency.min", random.nextBoolean() ? 2 : 1d/200.0d) // 100, 10, 5
.put("filter.frequency.max", random.nextBoolean() ? 99 : 99d/200.0d)); // 100
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "high_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(1L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("100"));
}
{
ifdService.clear();
MappedFieldType ft = new TextFieldMapper.Builder("high_freq")
.fielddataFrequencyFilter(0, random.nextBoolean() ? 100 : 0.5d, 0)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
{
ifdService.clear();
MappedFieldType ft = new TextFieldMapper.Builder("high_freq")
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, 201, 100)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(1L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("5"));
}
}
public void testFilterByRegExp() throws Exception {
int hundred = 0;
int ten = 0;
int five = 0;
for (int i = 0; i < 1000; i++) {
Document d = new Document();
d.add(new StringField("id", "" + i, Field.Store.NO));
if (i % 100 == 0) {
hundred++;
d.add(new StringField("high_freq", "100", Field.Store.NO));
}
if (i % 10 == 0) {
ten++;
d.add(new StringField("high_freq", "10", Field.Store.NO));
}
if (i % 5 == 0) {
five++;
d.add(new StringField("high_freq", "5", Field.Store.NO));
}
writer.addDocument(d);
{
ifdService.clear(); // test # docs with value
MappedFieldType ft = new TextFieldMapper.Builder("med_freq")
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, Integer.MAX_VALUE, 101)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
logger.debug("{} {} {}", hundred, ten, five);
writer.forceMerge(1, true);
LeafReaderContext context = refreshReader();
String[] formats = new String[] { "paged_bytes"};
for (String format : formats) {
{
ifdService.clear();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.regex.pattern", "\\d"));
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "high_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(1L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("5"));
}
{
ifdService.clear();
FieldDataType fieldDataType = new FieldDataType("string", Settings.builder().put("format", format)
.put("filter.regex.pattern", "\\d{1,2}"));
IndexOrdinalsFieldData fieldData = getForField(fieldDataType, "high_freq");
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("5"));
}
{
ifdService.clear();
MappedFieldType ft = new TextFieldMapper.Builder("med_freq")
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, Integer.MAX_VALUE, 101)
.build(builderCtx).fieldType();
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
RandomAccessOrds bytesValues = loadDirect.getOrdinalsValues();
assertThat(2L, equalTo(bytesValues.getValueCount()));
assertThat(bytesValues.lookupOrd(0).utf8ToString(), equalTo("10"));
assertThat(bytesValues.lookupOrd(1).utf8ToString(), equalTo("100"));
}
}

View File

@ -35,8 +35,8 @@ public class GeoFieldDataTests extends AbstractGeoFieldDataTestCase {
private static String FIELD_NAME = "value";
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("geo_point");
protected String getFieldDataType() {
return "geo_point";
}
@Override

View File

@ -46,7 +46,6 @@ import org.elasticsearch.index.mapper.core.IntegerFieldMapper;
import org.elasticsearch.index.mapper.core.KeywordFieldMapper;
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.mapper.core.TextFieldMapper;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService;
@ -109,7 +108,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
indicesService.getIndicesFieldDataCache(), indicesService.getCircuitBreakerService(), indexService.mapperService());
final BuilderContext ctx = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1));
final MappedFieldType mapper1 = new TextFieldMapper.Builder("s").build(ctx).fieldType();
final MappedFieldType mapper1 = new TextFieldMapper.Builder("s").fielddata(true).build(ctx).fieldType();
final IndexWriter writer = new IndexWriter(new RAMDirectory(), new IndexWriterConfig(new KeywordAnalyzer()));
Document doc = new Document();
doc.add(new StringField("s", "thisisastring", Store.NO));
@ -121,7 +120,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
final AtomicInteger onRemovalCalled = new AtomicInteger();
ifdService.setListener(new IndexFieldDataCache.Listener() {
@Override
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, Accountable ramUsage) {
if (wrap) {
assertEquals(new ShardId("test", "_na_", 1), shardId);
} else {
@ -131,7 +130,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
}
@Override
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes) {
if (wrap) {
assertEquals(new ShardId("test", "_na_", 1), shardId);
} else {
@ -159,12 +158,12 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
try {
shardPrivateService.setListener(new IndexFieldDataCache.Listener() {
@Override
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, Accountable ramUsage) {
}
@Override
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes) {
}
});
@ -206,22 +205,4 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
doTestRequireDocValues(new BooleanFieldMapper.BooleanFieldType());
}
public void testDisabled() {
ThreadPool threadPool = new ThreadPool("random_threadpool_name");
StringFieldMapper.StringFieldType ft = new StringFieldMapper.StringFieldType();
try {
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()));
try {
ifds.getForField(ft);
fail();
} catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("Field data loading is forbidden on [some_str]"));
}
} finally {
threadPool.shutdown();
}
}
}

View File

@ -42,11 +42,6 @@ public class NoOrdinalsStringFieldDataTests extends PagedBytesStringFieldDataTes
return in.getFieldName();
}
@Override
public FieldDataType getFieldDataType() {
return in.getFieldDataType();
}
@Override
public AtomicFieldData load(LeafReaderContext context) {
return in.load(context);

View File

@ -19,15 +19,12 @@
package org.elasticsearch.index.fielddata;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
/**
*/
public class PagedBytesStringFieldDataTests extends AbstractStringFieldDataTestCase {
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("string", Settings.builder().put("format", "paged_bytes").put(OrdinalsBuilder.FORCE_MULTI_ORDINALS, randomBoolean()));
protected String getFieldDataType() {
return "string";
}
}

View File

@ -35,11 +35,14 @@ import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.search.MultiValueMode;
import org.junit.Before;
@ -211,7 +214,8 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTestCase {
public void testThreads() throws Exception {
final ParentChildIndexFieldData indexFieldData = getForField(childType);
final DirectoryReader reader = DirectoryReader.open(writer);
final DirectoryReader reader = ElasticsearchDirectoryReader.wrap(
DirectoryReader.open(writer), new ShardId(new Index("test", ""), 0));
final IndexParentChildFieldData global = indexFieldData.loadGlobal(reader);
final AtomicReference<Exception> error = new AtomicReference<>();
final int numThreads = scaledRandomIntBetween(3, 8);
@ -266,7 +270,7 @@ public class ParentChildFieldDataTests extends AbstractFieldDataTestCase {
}
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("_parent");
protected String getFieldDataType() {
return "_parent";
}
}

View File

@ -19,14 +19,11 @@
package org.elasticsearch.index.fielddata;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
public class SortedSetDVStringFieldDataTests extends AbstractStringFieldDataTestCase {
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("string", Settings.builder().put("format", "doc_values").put(OrdinalsBuilder.FORCE_MULTI_ORDINALS, randomBoolean()));
protected String getFieldDataType() {
return "string";
}
@Override

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.fielddata.ordinals;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.packed.PackedInts;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.test.ESTestCase;
@ -41,13 +40,8 @@ import static org.hamcrest.Matchers.equalTo;
*/
public class MultiOrdinalsTests extends ESTestCase {
protected final Ordinals creationMultiOrdinals(OrdinalsBuilder builder) {
return this.creationMultiOrdinals(builder, Settings.builder());
}
protected Ordinals creationMultiOrdinals(OrdinalsBuilder builder, Settings.Builder settings) {
return builder.build(settings.build());
protected Ordinals creationMultiOrdinals(OrdinalsBuilder builder) {
return builder.build();
}
public void testRandomValues() throws IOException {

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.fielddata.ordinals;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
@ -49,7 +48,7 @@ public class SingleOrdinalsTests extends ESTestCase {
builder.addDoc(doc);
}
Ordinals ords = builder.build(Settings.EMPTY);
Ordinals ords = builder.build();
assertThat(ords, instanceOf(SinglePackedOrdinals.class));
RandomAccessOrds docs = ords.ordinals();
final SortedDocValues singleOrds = DocValues.unwrapSingleton(docs);
@ -68,12 +67,12 @@ public class SingleOrdinalsTests extends ESTestCase {
builder.addDoc(doc);
}
Ordinals ords = builder.build(Settings.EMPTY);
Ordinals ords = builder.build();
assertThat(ords, instanceOf(SinglePackedOrdinals.class));
builder.nextOrdinal();
builder.addDoc(0);
ords = builder.build(Settings.EMPTY);
ords = builder.build();
assertThat(ords, not(instanceOf(SinglePackedOrdinals.class)));
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.similarity.BM25SimilarityProvider;
import org.elasticsearch.test.ESTestCase;
@ -130,10 +129,10 @@ public abstract class FieldTypeTestCase extends ESTestCase {
other.setSimilarity(new BM25SimilarityProvider("bar", Settings.EMPTY));
}
},
new Modifier("fielddata", true) {
new Modifier("eager_global_ordinals", true) {
@Override
public void modify(MappedFieldType ft) {
ft.setFieldDataType(new FieldDataType("foo", Settings.builder().put("loading", "eager").build()));
ft.setEagerGlobalOrdinals(ft.eagerGlobalOrdinals() == false);
}
},
new Modifier("null_value", true) {
@ -211,7 +210,7 @@ public abstract class FieldTypeTestCase extends ESTestCase {
", searchAnalyzer=" + ft.searchAnalyzer() +
", searchQuoteAnalyzer=" + ft.searchQuoteAnalyzer() +
", similarity=" + ft.similarity() +
", fieldDataType=" + ft.fieldDataType() +
", eagerGlobalOrdinals=" + ft.eagerGlobalOrdinals() +
", nullValue=" + ft.nullValue() +
", nullValueAsString='" + ft.nullValueAsString() + "'" +
"} " + super.toString();

View File

@ -20,10 +20,43 @@ package org.elasticsearch.index.mapper.core;
import org.elasticsearch.index.mapper.FieldTypeTestCase;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.FieldTypeTestCase.Modifier;
import org.junit.Before;
public class StringFieldTypeTests extends FieldTypeTestCase {
@Override
protected MappedFieldType createDefaultFieldType() {
return new StringFieldMapper.StringFieldType();
}
@Before
public void setupProperties() {
addModifier(new Modifier("fielddata", true) {
@Override
public void modify(MappedFieldType ft) {
StringFieldMapper.StringFieldType tft = (StringFieldMapper.StringFieldType)ft;
tft.setFielddata(tft.fielddata() == false);
}
});
addModifier(new Modifier("fielddata_frequency_filter.min", true) {
@Override
public void modify(MappedFieldType ft) {
StringFieldMapper.StringFieldType tft = (StringFieldMapper.StringFieldType)ft;
tft.setFielddataMinFrequency(3);
}
});
addModifier(new Modifier("fielddata_frequency_filter.max", true) {
@Override
public void modify(MappedFieldType ft) {
StringFieldMapper.StringFieldType tft = (StringFieldMapper.StringFieldType)ft;
tft.setFielddataMaxFrequency(0.2);
}
});
addModifier(new Modifier("fielddata_frequency_filter.min_segment_size", true) {
@Override
public void modify(MappedFieldType ft) {
StringFieldMapper.StringFieldType tft = (StringFieldMapper.StringFieldType)ft;
tft.setFielddataMinSegmentSize(1000);
}
});
}
}

View File

@ -32,6 +32,7 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.core.TextFieldMapper.TextFieldType;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.InternalSettingsPlugin;
@ -40,6 +41,8 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.instanceOf;
@ -108,6 +111,43 @@ public class StringMappingUpgradeTests extends ESSingleNodeTestCase {
assertThat(e.getMessage(), containsString("The [string] type is removed in 5.0"));
}
public void testUpgradeFielddataSettings() throws IOException {
IndexService indexService = createIndex("test");
DocumentMapperParser parser = indexService.mapperService().documentMapperParser();
String format = randomFrom("paged_bytes", "disabled");
String loading = randomFrom("lazy", "eager", "eager_global_ordinals");
boolean keyword = random().nextBoolean();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("field")
.field("type", "string")
.field("index", keyword ? "not_analyzed" : "analyzed")
.startObject("fielddata")
.field("forwat", format)
.field("loading", loading)
.startObject("filter")
.startObject("frequency")
.field("min", 3)
.endObject()
.endObject()
.endObject()
.endObject()
.endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
FieldMapper field = mapper.mappers().getMapper("field");
if (keyword) {
assertThat(field, instanceOf(KeywordFieldMapper.class));
} else {
assertThat(field, instanceOf(TextFieldMapper.class));
TextFieldType fieldType = (TextFieldType) field.fieldType();
assertEquals("disabled".equals(format) == false, fieldType.fielddata());
assertEquals(3, fieldType.fielddataMinFrequency(), 0d);
assertEquals(Integer.MAX_VALUE, fieldType.fielddataMaxFrequency(), 0d);
}
assertEquals("eager_global_ordinals".equals(loading), field.fieldType().eagerGlobalOrdinals());
}
public void testUpgradeRandomMapping() throws IOException {
final int iters = 20;
for (int i = 0; i < iters; ++i) {
@ -152,6 +192,21 @@ public class StringMappingUpgradeTests extends ESSingleNodeTestCase {
mapping.field("norms", Collections.singletonMap("enabled", hasNorms));
}
}
if (randomBoolean()) {
Map<String, Object> fielddata = new HashMap<>();
if (randomBoolean()) {
fielddata.put("format", randomFrom("paged_bytes", "disabled"));
}
if (randomBoolean()) {
fielddata.put("loading", randomFrom("lazy", "eager", "eager_global_ordinals"));
}
if (randomBoolean()) {
Map<String, Object> frequencyFilter = new HashMap<>();
frequencyFilter.put("min", 10);
frequencyFilter.put("max", 1000);
frequencyFilter.put("min_segment_size", 10000);
}
}
if (randomBoolean()) {
mapping.startObject("fields").startObject("raw").field("type", "keyword").endObject().endObject();
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MapperService.MergeReason;
import org.elasticsearch.index.mapper.core.TextFieldMapper.TextFieldType;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.test.ESSingleNodeTestCase;
@ -45,6 +46,7 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class TextFieldMapperTests extends ESSingleNodeTestCase {
@ -385,4 +387,53 @@ public class TextFieldMapperTests extends ESSingleNodeTestCase {
assertThat(doc.rootDoc().getField("field6").fieldType().storeTermVectorPayloads(), equalTo(true));
}
public void testEagerGlobalOrdinals() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "text")
.field("eager_global_ordinals", true)
.endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
assertEquals(mapping, mapper.mappingSource().toString());
assertTrue(mapper.mappers().getMapper("field").fieldType().eagerGlobalOrdinals());
}
public void testFielddata() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "text")
.field("fielddata", false)
.endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
assertEquals(mapping, mapper.mappingSource().toString());
IllegalStateException e = expectThrows(IllegalStateException.class,
() -> mapper.mappers().getMapper("field").fieldType().fielddataBuilder());
assertThat(e.getMessage(), containsString("Fielddata is disabled"));
}
public void testFrequencyFilter() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "text")
.startObject("fielddata_frequency_filter")
.field("min", 2d)
.field("min_segment_size", 1000)
.endObject()
.endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
assertEquals(mapping, mapper.mappingSource().toString());
TextFieldType fieldType = (TextFieldType) mapper.mappers().getMapper("field").fieldType();
assertThat(fieldType.fielddataMinFrequency(), equalTo(2d));
assertThat(fieldType.fielddataMaxFrequency(), equalTo((double) Integer.MAX_VALUE));
assertThat(fieldType.fielddataMinSegmentSize(), equalTo(1000));
}
}

View File

@ -20,10 +20,43 @@ package org.elasticsearch.index.mapper.core;
import org.elasticsearch.index.mapper.FieldTypeTestCase;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.junit.Before;
public class TextFieldTypeTests extends FieldTypeTestCase {
@Override
protected MappedFieldType createDefaultFieldType() {
return new TextFieldMapper.TextFieldType();
}
@Before
public void setupProperties() {
addModifier(new Modifier("fielddata", true) {
@Override
public void modify(MappedFieldType ft) {
TextFieldMapper.TextFieldType tft = (TextFieldMapper.TextFieldType)ft;
tft.setFielddata(tft.fielddata() == false);
}
});
addModifier(new Modifier("fielddata_frequency_filter.min", true) {
@Override
public void modify(MappedFieldType ft) {
TextFieldMapper.TextFieldType tft = (TextFieldMapper.TextFieldType)ft;
tft.setFielddataMinFrequency(3);
}
});
addModifier(new Modifier("fielddata_frequency_filter.max", true) {
@Override
public void modify(MappedFieldType ft) {
TextFieldMapper.TextFieldType tft = (TextFieldMapper.TextFieldType)ft;
tft.setFielddataMaxFrequency(0.2);
}
});
addModifier(new Modifier("fielddata_frequency_filter.min_segment_size", true) {
@Override
public void modify(MappedFieldType ft) {
TextFieldMapper.TextFieldType tft = (TextFieldMapper.TextFieldType)ft;
tft.setFielddataMinSegmentSize(1000);
}
});
}
}

View File

@ -128,9 +128,7 @@ public class TokenCountFieldMapperIntegrationIT extends ESIntegTestCase {
.startObject("token_count_with_doc_values")
.field("type", "token_count")
.field("analyzer", "standard")
.startObject("fielddata")
.field("format", "doc_values")
.endObject()
.field("doc_values", true)
.endObject()
.endObject()
.endObject()

View File

@ -22,33 +22,28 @@ import org.apache.lucene.index.DocValuesType;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType.Loading;
import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.test.ESTestCase;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.nullValue;
public class ParentFieldMapperTests extends ESTestCase {
public void testPost2Dot0LazyLoading() {
ParentFieldMapper.Builder builder = new ParentFieldMapper.Builder("child");
builder.type("parent");
builder.fieldDataSettings(createFDSettings(Loading.LAZY));
builder.eagerGlobalOrdinals(false);
ParentFieldMapper parentFieldMapper = builder.build(new Mapper.BuilderContext(post2Dot0IndexSettings(), new ContentPath(0)));
assertThat(parentFieldMapper.getParentJoinFieldType().name(), equalTo("_parent#child"));
assertThat(parentFieldMapper.getParentJoinFieldType().fieldDataType(), nullValue());
assertThat(parentFieldMapper.getParentJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getParentJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.fieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.fieldType().fieldDataType().getLoading(), equalTo(Loading.LAZY));
assertThat(parentFieldMapper.fieldType().eagerGlobalOrdinals(), equalTo(false));
assertThat(parentFieldMapper.fieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.fieldType().docValuesType(), equalTo(DocValuesType.SORTED));
}
@ -56,35 +51,16 @@ public class ParentFieldMapperTests extends ESTestCase {
public void testPost2Dot0EagerLoading() {
ParentFieldMapper.Builder builder = new ParentFieldMapper.Builder("child");
builder.type("parent");
builder.fieldDataSettings(createFDSettings(Loading.EAGER));
builder.eagerGlobalOrdinals(true);
ParentFieldMapper parentFieldMapper = builder.build(new Mapper.BuilderContext(post2Dot0IndexSettings(), new ContentPath(0)));
assertThat(parentFieldMapper.getParentJoinFieldType().name(), equalTo("_parent#child"));
assertThat(parentFieldMapper.getParentJoinFieldType().fieldDataType(), nullValue());
assertThat(parentFieldMapper.getParentJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getParentJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.fieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.fieldType().fieldDataType().getLoading(), equalTo(Loading.EAGER));
assertThat(parentFieldMapper.fieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.fieldType().docValuesType(), equalTo(DocValuesType.SORTED));
}
public void testPost2Dot0EagerGlobalOrdinalsLoading() {
ParentFieldMapper.Builder builder = new ParentFieldMapper.Builder("child");
builder.type("parent");
builder.fieldDataSettings(createFDSettings(Loading.EAGER_GLOBAL_ORDINALS));
ParentFieldMapper parentFieldMapper = builder.build(new Mapper.BuilderContext(post2Dot0IndexSettings(), new ContentPath(0)));
assertThat(parentFieldMapper.getParentJoinFieldType().name(), equalTo("_parent#child"));
assertThat(parentFieldMapper.getParentJoinFieldType().fieldDataType(), nullValue());
assertThat(parentFieldMapper.getParentJoinFieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.getParentJoinFieldType().docValuesType(), equalTo(DocValuesType.SORTED));
assertThat(parentFieldMapper.fieldType().name(), equalTo("_parent#parent"));
assertThat(parentFieldMapper.fieldType().fieldDataType().getLoading(), equalTo(Loading.EAGER_GLOBAL_ORDINALS));
assertThat(parentFieldMapper.fieldType().eagerGlobalOrdinals(), equalTo(true));
assertThat(parentFieldMapper.fieldType().hasDocValues(), is(true));
assertThat(parentFieldMapper.fieldType().docValuesType(), equalTo(DocValuesType.SORTED));
}
@ -93,8 +69,4 @@ public class ParentFieldMapperTests extends ESTestCase {
return Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_2_1_0).build();
}
private static Settings createFDSettings(Loading loading) {
return new FieldDataType("child", settingsBuilder().put(Loading.KEY, loading)).getSettings();
}
}

View File

@ -30,13 +30,11 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.core.DateFieldMapper;
import org.elasticsearch.index.mapper.core.KeywordFieldMapper;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import org.elasticsearch.index.mapper.core.TokenCountFieldMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
@ -118,7 +116,7 @@ public class MultiFieldTests extends ESSingleNodeTestCase {
assertNotSame(IndexOptions.NONE, docMapper.mappers().getMapper("name.test1").fieldType().indexOptions());
assertThat(docMapper.mappers().getMapper("name.test1").fieldType().stored(), equalTo(true));
assertThat(docMapper.mappers().getMapper("name.test1").fieldType().tokenized(), equalTo(true));
assertThat(docMapper.mappers().getMapper("name.test1").fieldType().fieldDataType().getLoading(), equalTo(MappedFieldType.Loading.EAGER));
assertThat(docMapper.mappers().getMapper("name.test1").fieldType().eagerGlobalOrdinals(), equalTo(true));
assertThat(docMapper.mappers().getMapper("name.test2"), notNullValue());
assertThat(docMapper.mappers().getMapper("name.test2"), instanceOf(TokenCountFieldMapper.class));
@ -204,37 +202,6 @@ public class MultiFieldTests extends ESSingleNodeTestCase {
}
}
// The fielddata settings need to be the same after deserializing/re-serialsing, else unnecessary mapping sync's can be triggered
public void testMultiFieldsFieldDataSettingsInConsistentOrder() throws Exception {
final String MY_MULTI_FIELD = "multi_field";
// Possible fielddata settings
Map<String, Object> possibleSettings = new TreeMap<String, Object>();
possibleSettings.put("filter.frequency.min", 1);
possibleSettings.put("filter.frequency.max", 2);
possibleSettings.put("filter.regex.pattern", ".*");
possibleSettings.put("loading", "eager");
possibleSettings.put("foo", "bar");
possibleSettings.put("zetting", "zValue");
possibleSettings.put("aSetting", "aValue");
// Generate a mapping with the a random subset of possible fielddata settings
XContentBuilder builder = jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("my_field").field("type", "text").startObject("fields").startObject(MY_MULTI_FIELD)
.field("type", "text").startObject("fielddata");
String[] keys = possibleSettings.keySet().toArray(new String[]{});
Collections.shuffle(Arrays.asList(keys), random());
for(int i = randomIntBetween(0, possibleSettings.size()-1); i >= 0; --i)
builder.field(keys[i], possibleSettings.get(keys[i]));
builder.endObject().endObject().endObject().endObject().endObject().endObject().endObject();
// Check the mapping remains identical when deserialed/re-serialsed
final DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper docMapper = parser.parse("type", new CompressedXContent(builder.string()));
DocumentMapper docMapper2 = parser.parse("type", docMapper.mappingSource());
assertThat(docMapper.mappingSource(), equalTo(docMapper2.mappingSource()));
}
public void testObjectFieldNotAllowed() throws Exception {
String mapping = jsonBuilder().startObject().startObject("type").startObject("properties").startObject("my_field")
.field("type", "text").startObject("fields").startObject("multi").field("type", "object").endObject().endObject()

View File

@ -418,15 +418,11 @@ public class SimpleNumericTests extends ESSingleNodeTestCase {
.startObject("properties")
.startObject("int")
.field("type", "integer")
.startObject("fielddata")
.field("format", "doc_values")
.endObject()
.field("doc_values", true)
.endObject()
.startObject("double")
.field("type", "double")
.startObject("fielddata")
.field("format", "doc_values")
.endObject()
.field("doc_values", true)
.endObject()
.endObject()
.endObject()
@ -702,4 +698,30 @@ public class SimpleNumericTests extends ESSingleNodeTestCase {
assertThat(e.getMessage(), containsString("Mapping definition for [foo] has unsupported parameters: [norms"));
}
}
public void testIgnoreFielddata() throws IOException {
for (String type : Arrays.asList("byte", "short", "integer", "long", "float", "double")) {
Settings oldIndexSettings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_2_1_0)
.build();
DocumentMapperParser parser = createIndex("index-" + type, oldIndexSettings).mapperService().documentMapperParser();
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("foo")
.field("type", type)
.startObject("fielddata")
.field("loading", "eager")
.endObject()
.endObject()
.endObject().endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
String expectedMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties")
.startObject("foo")
.field("type", type)
.endObject()
.endObject().endObject().endObject().string();
assertEquals(expectedMapping, mapper.mappingSource().string());
}
}
}

View File

@ -44,6 +44,7 @@ import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.core.StringFieldMapper;
import org.elasticsearch.index.mapper.core.StringFieldMapper.Builder;
import org.elasticsearch.index.mapper.core.StringFieldMapper.StringFieldType;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.InternalSettingsPlugin;
@ -592,4 +593,88 @@ public class SimpleStringMappingTests extends ESSingleNodeTestCase {
}
}
public void testFielddataLoading() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "string")
.startObject("fielddata")
.field("loading", "eager_global_ordinals")
.endObject()
.endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
String expectedMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "string")
.field("eager_global_ordinals", true)
.endObject().endObject()
.endObject().endObject().string();
assertEquals(expectedMapping, mapper.mappingSource().toString());
assertTrue(mapper.mappers().getMapper("field").fieldType().eagerGlobalOrdinals());
}
public void testFielddataFilter() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "string")
.startObject("fielddata")
.startObject("filter")
.startObject("frequency")
.field("min", 2d)
.field("min_segment_size", 1000)
.endObject()
.startObject("regex")
.field("pattern", "^#.*")
.endObject()
.endObject()
.endObject()
.endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
String expectedMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "string")
.startObject("fielddata_frequency_filter")
.field("min", 2d)
.field("min_segment_size", 1000)
.endObject()
.endObject().endObject()
.endObject().endObject().string();
assertEquals(expectedMapping, mapper.mappingSource().toString());
StringFieldType fieldType = (StringFieldType) mapper.mappers().getMapper("field").fieldType();
assertThat(fieldType.fielddataMinFrequency(), equalTo(2d));
assertThat(fieldType.fielddataMaxFrequency(), equalTo((double) Integer.MAX_VALUE));
assertThat(fieldType.fielddataMinSegmentSize(), equalTo(1000));
}
public void testDisabledFielddata() throws IOException {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "string")
.startObject("fielddata")
.field("format", "disabled")
.endObject()
.endObject().endObject()
.endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
String expectedMapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "string")
.field("fielddata", false)
.endObject().endObject()
.endObject().endObject().string();
assertEquals(expectedMapping, mapper.mappingSource().toString());
IllegalStateException e = expectThrows(IllegalStateException.class,
() -> mapper.mappers().getMapper("field").fieldType().fielddataBuilder());
assertThat(e.getMessage(), containsString("Fielddata is disabled"));
}
}

View File

@ -210,7 +210,7 @@ public class PercolatorQueryCacheTests extends ESTestCase {
IndexShard indexShard = mockIndexShard();
ThreadPool threadPool = mockThreadPool();
IndexWarmer.Listener listener = cache.createListener(threadPool);
listener.warmNewReaders(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
listener.warmReader(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
PercolatorQueryCacheStats stats = cache.getStats(shardId);
assertThat(stats.getNumQueries(), equalTo(9L));
@ -257,7 +257,7 @@ public class PercolatorQueryCacheTests extends ESTestCase {
IndexShard indexShard = mockIndexShard();
ThreadPool threadPool = mockThreadPool();
IndexWarmer.Listener listener = cache.createListener(threadPool);
listener.warmNewReaders(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
listener.warmReader(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
assertThat(cache.getStats(shardId).getNumQueries(), equalTo(3L));
PercolatorQuery.QueryRegistry.Leaf leaf = cache.getQueries(indexReader.leaves().get(0));
@ -277,7 +277,7 @@ public class PercolatorQueryCacheTests extends ESTestCase {
indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(indexWriter), shardId);
assertThat(indexReader.leaves().size(), equalTo(2));
assertThat(indexReader.maxDoc(), equalTo(2));
listener.warmNewReaders(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
listener.warmReader(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
assertThat(cache.getStats(shardId).getNumQueries(), equalTo(2L));
leaf = cache.getQueries(indexReader.leaves().get(0));
@ -291,7 +291,7 @@ public class PercolatorQueryCacheTests extends ESTestCase {
indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(indexWriter), shardId);
assertThat(indexReader.leaves().size(), equalTo(1));
assertThat(indexReader.maxDoc(), equalTo(2));
listener.warmNewReaders(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
listener.warmReader(indexShard, new Engine.Searcher("test", new IndexSearcher(indexReader)));
assertThat(cache.getStats(shardId).getNumQueries(), equalTo(2L));
leaf = cache.getQueries(indexReader.leaves().get(0));

View File

@ -299,7 +299,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
String type = randomAsciiOfLengthBetween(1, 10);
mapperService.merge(type, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(type,
STRING_FIELD_NAME, "type=text",
STRING_FIELD_NAME_2, "type=text",
STRING_FIELD_NAME_2, "type=keyword",
INT_FIELD_NAME, "type=integer",
DOUBLE_FIELD_NAME, "type=double",
BOOLEAN_FIELD_NAME, "type=boolean",

View File

@ -72,6 +72,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
MapperService mapperService = queryShardContext().getMapperService();
mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE,
STRING_FIELD_NAME, "type=text",
STRING_FIELD_NAME_2, "type=keyword",
INT_FIELD_NAME, "type=integer",
DOUBLE_FIELD_NAME, "type=double",
BOOLEAN_FIELD_NAME, "type=boolean",
@ -117,7 +118,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
protected HasChildQueryBuilder doCreateTestQueryBuilder() {
int min = randomIntBetween(0, Integer.MAX_VALUE / 2);
int max = randomIntBetween(min, Integer.MAX_VALUE);
InnerHitsBuilder.InnerHit innerHit = new InnerHitsBuilder.InnerHit().setSize(100).addSort(STRING_FIELD_NAME, SortOrder.ASC);
InnerHitsBuilder.InnerHit innerHit = new InnerHitsBuilder.InnerHit().setSize(100).addSort(STRING_FIELD_NAME_2, SortOrder.ASC);
return new HasChildQueryBuilder(CHILD_TYPE,
RandomQueryBuilder.createQuery(random()), max, min,
RandomPicks.randomFrom(random(), ScoreMode.values()),
@ -145,7 +146,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
InnerHitsContext.BaseInnerHits innerHits = SearchContext.current().innerHits().getInnerHits().get("inner_hits_name");
assertEquals(innerHits.size(), 100);
assertEquals(innerHits.sort().getSort().length, 1);
assertEquals(innerHits.sort().getSort()[0].getField(), STRING_FIELD_NAME);
assertEquals(innerHits.sort().getSort()[0].getField(), STRING_FIELD_NAME_2);
} else {
assertThat(SearchContext.current().innerHits().getInnerHits().size(), equalTo(0));
}

View File

@ -58,6 +58,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
MapperService mapperService = queryShardContext().getMapperService();
mapperService.merge(PARENT_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(PARENT_TYPE,
STRING_FIELD_NAME, "type=text",
STRING_FIELD_NAME_2, "type=keyword",
INT_FIELD_NAME, "type=integer",
DOUBLE_FIELD_NAME, "type=double",
BOOLEAN_FIELD_NAME, "type=boolean",
@ -67,6 +68,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
mapperService.merge(CHILD_TYPE, new CompressedXContent(PutMappingRequest.buildFromSimplifiedDef(CHILD_TYPE,
"_parent", "type=" + PARENT_TYPE,
STRING_FIELD_NAME, "type=text",
STRING_FIELD_NAME_2, "type=keyword",
INT_FIELD_NAME, "type=integer",
DOUBLE_FIELD_NAME, "type=double",
BOOLEAN_FIELD_NAME, "type=boolean",
@ -102,7 +104,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
*/
@Override
protected HasParentQueryBuilder doCreateTestQueryBuilder() {
InnerHitsBuilder.InnerHit innerHit = new InnerHitsBuilder.InnerHit().setSize(100).addSort(STRING_FIELD_NAME, SortOrder.ASC);
InnerHitsBuilder.InnerHit innerHit = new InnerHitsBuilder.InnerHit().setSize(100).addSort(STRING_FIELD_NAME_2, SortOrder.ASC);
return new HasParentQueryBuilder(PARENT_TYPE,
RandomQueryBuilder.createQuery(random()),randomBoolean(),
randomBoolean() ? null : new QueryInnerHits("inner_hits_name", innerHit));
@ -127,7 +129,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
InnerHitsContext.BaseInnerHits innerHits = SearchContext.current().innerHits().getInnerHits().get("inner_hits_name");
assertEquals(innerHits.size(), 100);
assertEquals(innerHits.sort().getSort().length, 1);
assertEquals(innerHits.sort().getSort()[0].getField(), STRING_FIELD_NAME);
assertEquals(innerHits.sort().getSort()[0].getField(), STRING_FIELD_NAME_2);
} else {
assertThat(SearchContext.current().innerHits().getInnerHits().size(), equalTo(0));
}

View File

@ -54,7 +54,6 @@ import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.ScriptDocValues;
@ -89,11 +88,6 @@ public class FunctionScoreTests extends ESTestCase {
return "test";
}
@Override
public FieldDataType getFieldDataType() {
throw new UnsupportedOperationException(UNSUPPORTED);
}
@Override
public AtomicFieldData load(LeafReaderContext context) {
return new AtomicFieldData() {
@ -174,11 +168,6 @@ public class FunctionScoreTests extends ESTestCase {
return "test";
}
@Override
public FieldDataType getFieldDataType() {
throw new UnsupportedOperationException(UNSUPPORTED);
}
@Override
public AtomicNumericFieldData load(LeafReaderContext context) {
return new AtomicNumericFieldData() {

View File

@ -32,7 +32,6 @@ import org.apache.lucene.search.join.ScoreMode;
import org.apache.lucene.search.join.ToParentBlockJoinQuery;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
@ -49,8 +48,8 @@ import static org.hamcrest.Matchers.equalTo;
public class DoubleNestedSortingTests extends AbstractNumberNestedSortingTestCase {
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("double");
protected String getFieldDataType() {
return "double";
}
@Override

View File

@ -32,7 +32,6 @@ import org.apache.lucene.search.join.ScoreMode;
import org.apache.lucene.search.join.ToParentBlockJoinQuery;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
@ -49,8 +48,8 @@ import static org.hamcrest.Matchers.equalTo;
public class FloatNestedSortingTests extends DoubleNestedSortingTests {
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("float");
protected String getFieldDataType() {
return "float";
}
@Override

View File

@ -20,7 +20,6 @@ package org.elasticsearch.index.search.nested;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.IndexableField;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
@ -32,8 +31,8 @@ import org.elasticsearch.search.MultiValueMode;
public class LongNestedSortingTests extends AbstractNumberNestedSortingTestCase {
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("long");
protected String getFieldDataType() {
return "long";
}
@Override

View File

@ -42,9 +42,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TestUtil;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.AbstractFieldDataTestCase;
import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource;
import org.elasticsearch.index.fielddata.NoOrdinalsStringFieldDataTests;
@ -64,8 +62,8 @@ import static org.hamcrest.Matchers.equalTo;
*/
public class NestedSortingTests extends AbstractFieldDataTestCase {
@Override
protected FieldDataType getFieldDataType() {
return new FieldDataType("string", Settings.builder().put("format", "paged_bytes"));
protected String getFieldDataType() {
return "string";
}
public void testDuel() throws Exception {

View File

@ -1083,6 +1083,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService indexService = indicesService.indexService(resolveIndex("test"));
IndexShard shard = indexService.getShardOrNull(0);
client().admin().indices().preparePutMapping("test").setType("test").setSource("foo", "type=text,fielddata=true").get();
client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get();
client().prepareIndex("test", "test", "1").setSource("{\"foobar\" : \"bar\"}").setRefresh(true).get();

Some files were not shown because too many files have changed in this diff Show More