mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-02-17 10:25:15 +00:00
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:
parent
435558a5c0
commit
e50eeeaffb
@ -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();
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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));
|
||||
|
@ -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() {
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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.
|
||||
*/
|
||||
|
@ -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 {
|
||||
|
@ -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 {
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -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");
|
||||
|
@ -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);
|
||||
|
@ -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 {
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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());
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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");
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
|
@ -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 {
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -183,7 +183,6 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
|
||||
private ContextMappings contextMappings = null;
|
||||
|
||||
public CompletionFieldType() {
|
||||
setFieldDataType(null);
|
||||
}
|
||||
|
||||
private CompletionFieldType(CompletionFieldType ref) {
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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()) {
|
||||
|
@ -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;
|
||||
|
@ -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) {
|
||||
|
@ -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) {
|
||||
|
@ -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) {
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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) {
|
||||
|
@ -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;
|
||||
|
@ -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) {
|
||||
|
@ -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) {
|
||||
|
@ -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) {
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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());
|
||||
|
@ -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
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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() {
|
||||
|
@ -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"));
|
||||
}
|
||||
|
||||
}
|
@ -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();
|
||||
|
@ -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"));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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)));
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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()
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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"));
|
||||
}
|
||||
}
|
||||
|
@ -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));
|
||||
|
@ -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",
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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));
|
||||
}
|
||||
|
@ -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() {
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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
Loading…
x
Reference in New Issue
Block a user