Merge pull request #15488 from jpountz/remove/index_name_back_compat

Remove `index_name` back compat.
This commit is contained in:
Adrien Grand 2015-12-23 14:55:41 +01:00
commit 76fa9023b6
150 changed files with 533 additions and 823 deletions

View File

@ -226,7 +226,7 @@ public class MapperQueryParser extends QueryParser {
}
}
if (query == null) {
query = super.getFieldQuery(currentFieldType.names().indexName(), queryText, quoted);
query = super.getFieldQuery(currentFieldType.name(), queryText, quoted);
}
return query;
}
@ -466,7 +466,7 @@ public class MapperQueryParser extends QueryParser {
query = currentFieldType.prefixQuery(termStr, multiTermRewriteMethod, context);
}
if (query == null) {
query = getPossiblyAnalyzedPrefixQuery(currentFieldType.names().indexName(), termStr);
query = getPossiblyAnalyzedPrefixQuery(currentFieldType.name(), termStr);
}
return query;
}
@ -592,7 +592,7 @@ public class MapperQueryParser extends QueryParser {
if (!settings.forceAnalyzer()) {
setAnalyzer(context.getSearchAnalyzer(currentFieldType));
}
indexedNameField = currentFieldType.names().indexName();
indexedNameField = currentFieldType.name();
return getPossiblyAnalyzedWildcardQuery(indexedNameField, termStr);
}
return getPossiblyAnalyzedWildcardQuery(indexedNameField, termStr);

View File

@ -126,13 +126,13 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
if (indexService == null) {
throw new IllegalArgumentException("No index provided, and trying to analyzer based on a specific field which requires the index parameter");
}
MappedFieldType fieldType = indexService.mapperService().smartNameFieldType(request.field());
MappedFieldType fieldType = indexService.mapperService().fullName(request.field());
if (fieldType != null) {
if (fieldType.isNumeric()) {
throw new IllegalArgumentException("Can't process field [" + request.field() + "], Analysis requests are not supported on numeric fields");
}
analyzer = fieldType.indexAnalyzer();
field = fieldType.names().indexName();
field = fieldType.name();
}
}
if (field == null) {

View File

@ -171,7 +171,7 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleShardAc
for (String field : request.fields()) {
if (Regex.isMatchAllPattern(field)) {
for (FieldMapper fieldMapper : allFieldMappers) {
addFieldMapper(fieldMapper.fieldType().names().fullName(), fieldMapper, fieldMappings, request.includeDefaults());
addFieldMapper(fieldMapper.fieldType().name(), fieldMapper, fieldMappings, request.includeDefaults());
}
} else if (Regex.isSimpleMatchPattern(field)) {
// go through the field mappers 3 times, to make sure we give preference to the resolve order: full name, index name, name.
@ -179,15 +179,15 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleShardAc
Collection<FieldMapper> remainingFieldMappers = newLinkedList(allFieldMappers);
for (Iterator<FieldMapper> it = remainingFieldMappers.iterator(); it.hasNext(); ) {
final FieldMapper fieldMapper = it.next();
if (Regex.simpleMatch(field, fieldMapper.fieldType().names().fullName())) {
addFieldMapper(fieldMapper.fieldType().names().fullName(), fieldMapper, fieldMappings, request.includeDefaults());
if (Regex.simpleMatch(field, fieldMapper.fieldType().name())) {
addFieldMapper(fieldMapper.fieldType().name(), fieldMapper, fieldMappings, request.includeDefaults());
it.remove();
}
}
for (Iterator<FieldMapper> it = remainingFieldMappers.iterator(); it.hasNext(); ) {
final FieldMapper fieldMapper = it.next();
if (Regex.simpleMatch(field, fieldMapper.fieldType().names().indexName())) {
addFieldMapper(fieldMapper.fieldType().names().indexName(), fieldMapper, fieldMappings, request.includeDefaults());
if (Regex.simpleMatch(field, fieldMapper.fieldType().name())) {
addFieldMapper(fieldMapper.fieldType().name(), fieldMapper, fieldMappings, request.includeDefaults());
it.remove();
}
}
@ -214,7 +214,7 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleShardAc
builder.startObject();
fieldMapper.toXContent(builder, includeDefaults ? includeDefaultsParams : ToXContent.EMPTY_PARAMS);
builder.endObject();
fieldMappings.put(field, new FieldMappingMetaData(fieldMapper.fieldType().names().fullName(), builder.bytes()));
fieldMappings.put(field, new FieldMappingMetaData(fieldMapper.fieldType().name(), builder.bytes()));
} catch (IOException e) {
throw new ElasticsearchException("failed to serialize XContent of field [" + field + "]", e);
}

View File

@ -459,21 +459,21 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
}
@Override
public void onCache(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
if (shardId != null) {
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
shard.fieldData().onCache(shardId, fieldNames, fieldDataType, ramUsage);
shard.fieldData().onCache(shardId, fieldName, fieldDataType, ramUsage);
}
}
}
@Override
public void onRemoval(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
if (shardId != null) {
final IndexShard shard = indexService.getShardOrNull(shardId.id());
if (shard != null) {
shard.fieldData().onRemoval(shardId, fieldNames, fieldDataType, wasEvicted, sizeInBytes);
shard.fieldData().onRemoval(shardId, fieldName, fieldDataType, wasEvicted, sizeInBytes);
}
}
}

View File

@ -54,7 +54,7 @@ public class PerFieldMappingPostingFormatCodec extends Lucene54Codec {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final MappedFieldType indexName = mapperService.indexName(field);
final MappedFieldType indexName = mapperService.fullName(field);
if (indexName == null) {
logger.warn("no index mapper found for field: [{}] returning default postings format", field);
} else if (indexName instanceof CompletionFieldMapper.CompletionFieldType) {

View File

@ -80,7 +80,7 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
/**
* The field name.
*/
MappedFieldType.Names getFieldNames();
String getFieldName();
/**
* The field data type.

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index.fielddata;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.shard.ShardId;
/**
@ -49,12 +48,12 @@ public interface IndexFieldDataCache {
/**
* Called after the fielddata is loaded during the cache phase
*/
void onCache(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, Accountable ramUsage);
void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage);
/**
* Called after the fielddata is unloaded
*/
void onRemoval(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes);
void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes);
}
class None implements IndexFieldDataCache {

View File

@ -34,7 +34,6 @@ import org.elasticsearch.index.fielddata.plain.IndexIndexFieldData;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import org.elasticsearch.index.fielddata.plain.ParentChildIndexFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
import org.elasticsearch.index.mapper.internal.IndexFieldMapper;
@ -61,7 +60,7 @@ public class IndexFieldDataService extends AbstractIndexComponent implements Clo
public static final String FIELDDATA_CACHE_VALUE_NODE = "node";
private static final IndexFieldData.Builder MISSING_DOC_VALUES_BUILDER = (indexProperties, fieldType, cache, breakerService, mapperService1) -> {
throw new IllegalStateException("Can't load fielddata on [" + fieldType.names().fullName()
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.");
};
@ -148,11 +147,11 @@ public class IndexFieldDataService extends AbstractIndexComponent implements Clo
private final MapperService mapperService;
private static final IndexFieldDataCache.Listener DEFAULT_NOOP_LISTENER = new IndexFieldDataCache.Listener() {
@Override
public void onCache(ShardId shardId, Names fieldNames, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
}
@Override
public void onRemoval(ShardId shardId, Names fieldNames, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
}
};
private volatile IndexFieldDataCache.Listener listener = DEFAULT_NOOP_LISTENER;
@ -195,22 +194,22 @@ public class IndexFieldDataService extends AbstractIndexComponent implements Clo
@SuppressWarnings("unchecked")
public <IFD extends IndexFieldData<?>> IFD getForField(MappedFieldType fieldType) {
final Names fieldNames = fieldType.names();
final String fieldName = fieldType.name();
final FieldDataType type = fieldType.fieldDataType();
if (type == null) {
throw new IllegalArgumentException("found no fielddata type for field [" + fieldNames.fullName() + "]");
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 [" + fieldNames.fullName() + "] has no doc values, will use default field data format");
logger.warn("field [" + fieldName + "] has no doc values, will use default field data format");
format = null;
}
if (format != null) {
builder = buildersByTypeAndFormat.get(Tuple.tuple(type.getType(), format));
if (builder == null) {
logger.warn("failed to find format [" + format + "] for field [" + fieldNames.fullName() + "], will use default");
logger.warn("failed to find format [" + format + "] for field [" + fieldName + "], will use default");
}
}
if (builder == null && docValues) {
@ -220,24 +219,24 @@ public class IndexFieldDataService extends AbstractIndexComponent implements Clo
builder = buildersByType.get(type.getType());
}
if (builder == null) {
throw new IllegalArgumentException("failed to find field data builder for field " + fieldNames.fullName() + ", and type " + type.getType());
throw new IllegalArgumentException("failed to find field data builder for field " + fieldName + ", and type " + type.getType());
}
IndexFieldDataCache cache;
synchronized (this) {
cache = fieldDataCaches.get(fieldNames.indexName());
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.getSettings().get(FIELDDATA_CACHE_KEY, FIELDDATA_CACHE_VALUE_NODE));
if (FIELDDATA_CACHE_VALUE_NODE.equals(cacheType)) {
cache = indicesFieldDataCache.buildIndexFieldDataCache(listener, index(), fieldNames, type);
cache = indicesFieldDataCache.buildIndexFieldDataCache(listener, index(), fieldName, type);
} else if ("none".equals(cacheType)){
cache = new IndexFieldDataCache.None();
} else {
throw new IllegalArgumentException("cache type not supported [" + cacheType + "] for field [" + fieldNames.fullName() + "]");
throw new IllegalArgumentException("cache type not supported [" + cacheType + "] for field [" + fieldName + "]");
}
fieldDataCaches.put(fieldNames.indexName(), cache);
fieldDataCaches.put(fieldName, cache);
}
}

View File

@ -24,7 +24,6 @@ import org.apache.lucene.util.Accountable;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.shard.ShardId;
import java.util.Map;
@ -52,16 +51,15 @@ public class ShardFieldData implements IndexFieldDataCache.Listener {
}
@Override
public void onCache(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, Accountable ramUsage) {
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable ramUsage) {
totalMetric.inc(ramUsage.ramBytesUsed());
String keyFieldName = fieldNames.indexName();
CounterMetric total = perFieldTotals.get(keyFieldName);
CounterMetric total = perFieldTotals.get(fieldName);
if (total != null) {
total.inc(ramUsage.ramBytesUsed());
} else {
total = new CounterMetric();
total.inc(ramUsage.ramBytesUsed());
CounterMetric prev = perFieldTotals.putIfAbsent(keyFieldName, total);
CounterMetric prev = perFieldTotals.putIfAbsent(fieldName, total);
if (prev != null) {
prev.inc(ramUsage.ramBytesUsed());
}
@ -69,15 +67,14 @@ public class ShardFieldData implements IndexFieldDataCache.Listener {
}
@Override
public void onRemoval(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
if (wasEvicted) {
evictionsMetric.inc();
}
if (sizeInBytes != -1) {
totalMetric.dec(sizeInBytes);
String keyFieldName = fieldNames.indexName();
CounterMetric total = perFieldTotals.get(keyFieldName);
CounterMetric total = perFieldTotals.get(fieldName);
if (total != null) {
total.dec(sizeInBytes);
}

View File

@ -80,7 +80,7 @@ public class BytesRefFieldComparatorSource extends IndexFieldData.XFieldComparat
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldNames().indexName());
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldName());
final boolean sortMissingLast = sortMissingLast(missingValue) ^ reversed;
final BytesRef missingBytes = (BytesRef) missingObject(missingValue, reversed);

View File

@ -65,7 +65,7 @@ public class DoubleValuesComparatorSource extends IndexFieldData.XFieldComparato
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldNames().indexName());
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldName());
final double dMissingValue = (Double) missingObject(missingValue, reversed);
// NOTE: it's important to pass null as a missing value in the constructor so that

View File

@ -57,7 +57,7 @@ public class FloatValuesComparatorSource extends IndexFieldData.XFieldComparator
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldNames().indexName());
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldName());
final float dMissingValue = (Float) missingObject(missingValue, reversed);
// NOTE: it's important to pass null as a missing value in the constructor so that

View File

@ -56,7 +56,7 @@ public class LongValuesComparatorSource extends IndexFieldData.XFieldComparatorS
@Override
public FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldNames().indexName());
assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldName());
final Long dMissingValue = (Long) missingObject(missingValue, reversed);
// NOTE: it's important to pass null as a missing value in the constructor so that

View File

@ -64,12 +64,12 @@ public enum GlobalOrdinalsBuilder {
if (logger.isDebugEnabled()) {
logger.debug(
"Global-ordinals[{}][{}] took {} ms",
indexFieldData.getFieldNames().fullName(),
indexFieldData.getFieldName(),
ordinalMap.getValueCount(),
TimeValue.nsecToMSec(System.nanoTime() - startTimeNS)
);
}
return new InternalGlobalOrdinalsIndexFieldData(indexSettings, indexFieldData.getFieldNames(),
return new InternalGlobalOrdinalsIndexFieldData(indexSettings, indexFieldData.getFieldName(),
indexFieldData.getFieldDataType(), atomicFD, ordinalMap, memorySizeInBytes
);
}
@ -103,7 +103,7 @@ public enum GlobalOrdinalsBuilder {
subs[i] = atomicFD[i].getOrdinalsValues();
}
final OrdinalMap ordinalMap = OrdinalMap.build(null, subs, PackedInts.DEFAULT);
return new InternalGlobalOrdinalsIndexFieldData(indexSettings, indexFieldData.getFieldNames(),
return new InternalGlobalOrdinalsIndexFieldData(indexSettings, indexFieldData.getFieldName(),
indexFieldData.getFieldDataType(), atomicFD, ordinalMap, 0
);
}

View File

@ -40,13 +40,13 @@ import java.util.Collections;
*/
public abstract class GlobalOrdinalsIndexFieldData extends AbstractIndexComponent implements IndexOrdinalsFieldData, Accountable {
private final MappedFieldType.Names fieldNames;
private final String fieldName;
private final FieldDataType fieldDataType;
private final long memorySizeInBytes;
protected GlobalOrdinalsIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, long memorySizeInBytes) {
protected GlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, long memorySizeInBytes) {
super(indexSettings);
this.fieldNames = fieldNames;
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.memorySizeInBytes = memorySizeInBytes;
}
@ -67,8 +67,8 @@ public abstract class GlobalOrdinalsIndexFieldData extends AbstractIndexComponen
}
@Override
public MappedFieldType.Names getFieldNames() {
return fieldNames;
public String getFieldName() {
return fieldName;
}
@Override

View File

@ -37,8 +37,8 @@ final class InternalGlobalOrdinalsIndexFieldData extends GlobalOrdinalsIndexFiel
private final Atomic[] atomicReaders;
InternalGlobalOrdinalsIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, AtomicOrdinalsFieldData[] segmentAfd, OrdinalMap ordinalMap, long memorySizeInBytes) {
super(indexSettings, fieldNames, fieldDataType, memorySizeInBytes);
InternalGlobalOrdinalsIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, AtomicOrdinalsFieldData[] segmentAfd, OrdinalMap ordinalMap, long memorySizeInBytes) {
super(indexSettings, fieldName, fieldDataType, memorySizeInBytes);
this.atomicReaders = new Atomic[segmentAfd.length];
for (int i = 0; i < segmentAfd.length; i++) {
atomicReaders[i] = new Atomic(segmentAfd[i], ordinalMap, i);

View File

@ -32,7 +32,6 @@ import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.N
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
@ -41,8 +40,8 @@ import java.io.IOException;
public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFieldData implements IndexGeoPointFieldData {
AbstractGeoPointDVIndexFieldData(Index index, Names fieldNames, FieldDataType fieldDataType) {
super(index, fieldNames, fieldDataType);
AbstractGeoPointDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super(index, fieldName, fieldDataType);
}
@Override
@ -56,8 +55,8 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
public static class GeoPointDVIndexFieldData extends AbstractGeoPointDVIndexFieldData {
final boolean indexCreatedBefore2x;
public GeoPointDVIndexFieldData(Index index, Names fieldNames, FieldDataType fieldDataType, final boolean indexCreatedBefore2x) {
super(index, fieldNames, fieldDataType);
public GeoPointDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType, final boolean indexCreatedBefore2x) {
super(index, fieldName, fieldDataType);
this.indexCreatedBefore2x = indexCreatedBefore2x;
}
@ -65,9 +64,9 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
public AtomicGeoPointFieldData load(LeafReaderContext context) {
try {
if (indexCreatedBefore2x) {
return new GeoPointLegacyDVAtomicFieldData(DocValues.getBinary(context.reader(), fieldNames.indexName()));
return new GeoPointLegacyDVAtomicFieldData(DocValues.getBinary(context.reader(), fieldName));
}
return new GeoPointDVAtomicFieldData(DocValues.getSortedNumeric(context.reader(), fieldNames.indexName()));
return new GeoPointDVAtomicFieldData(DocValues.getSortedNumeric(context.reader(), fieldName));
} catch (IOException e) {
throw new IllegalStateException("Cannot load doc values", e);
}
@ -84,7 +83,7 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
// Ignore breaker
return new GeoPointDVIndexFieldData(indexSettings.getIndex(), fieldType.names(), fieldType.fieldDataType(),
return new GeoPointDVIndexFieldData(indexSettings.getIndex(), fieldType.name(), fieldType.fieldDataType(),
indexSettings.getIndexVersionCreated().before(Version.V_2_2_0));
}
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.RamAccountingTermsEnum;
import org.elasticsearch.index.mapper.MappedFieldType;
import java.io.IOException;
@ -39,20 +38,20 @@ import java.io.IOException;
*/
public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends AbstractIndexComponent implements IndexFieldData<FD> {
private final MappedFieldType.Names fieldNames;
private final String fieldName;
protected final FieldDataType fieldDataType;
protected final IndexFieldDataCache cache;
public AbstractIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, IndexFieldDataCache cache) {
public AbstractIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(indexSettings);
this.fieldNames = fieldNames;
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.cache = cache;
}
@Override
public MappedFieldType.Names getFieldNames() {
return this.fieldNames;
public String getFieldName() {
return this.fieldName;
}
@Override
@ -62,12 +61,12 @@ public abstract class AbstractIndexFieldData<FD extends AtomicFieldData> extends
@Override
public void clear() {
cache.clear(fieldNames.indexName());
cache.clear(fieldName);
}
@Override
public FD load(LeafReaderContext context) {
if (context.reader().getFieldInfos().fieldInfo(fieldNames.indexName()) == null) {
if (context.reader().getFieldInfos().fieldInfo(fieldName) == null) {
// Some leaf readers may be wrapped and report different set of fields and use the same cache key.
// If a field can't be found then it doesn't mean it isn't there,
// so if a field doesn't exist then we don't cache it and just return an empty field data instance.

View File

@ -31,7 +31,6 @@ 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;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
@ -92,8 +91,8 @@ abstract class AbstractIndexGeoPointFieldData extends AbstractIndexFieldData<Ato
}
}
public AbstractIndexGeoPointFieldData(IndexSettings indexSettings, Names fieldNames, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(indexSettings, fieldNames, fieldDataType, cache);
public AbstractIndexGeoPointFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(indexSettings, fieldName, fieldDataType, cache);
}
@Override

View File

@ -37,7 +37,6 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
@ -52,9 +51,9 @@ public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldD
protected Settings regex;
protected final CircuitBreakerService breakerService;
protected AbstractIndexOrdinalsFieldData(IndexSettings indexSettings, Names fieldNames, FieldDataType fieldDataType,
protected AbstractIndexOrdinalsFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType,
IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldNames, fieldDataType, cache);
super(indexSettings, fieldName, fieldDataType, cache);
final Map<String, Settings> groups = fieldDataType.getSettings().getGroups("filter");
frequency = groups.get("frequency");
regex = groups.get("regex");
@ -74,7 +73,7 @@ public abstract class AbstractIndexOrdinalsFieldData extends AbstractIndexFieldD
}
boolean fieldFound = false;
for (LeafReaderContext context : indexReader.leaves()) {
if (context.reader().getFieldInfos().fieldInfo(getFieldNames().indexName()) != null) {
if (context.reader().getFieldInfos().fieldInfo(getFieldName()) != null) {
fieldFound = true;
break;
}

View File

@ -25,18 +25,17 @@ 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;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.search.MultiValueMode;
public class BinaryDVIndexFieldData extends DocValuesIndexFieldData implements IndexFieldData<BinaryDVAtomicFieldData> {
public BinaryDVIndexFieldData(Index index, Names fieldNames, FieldDataType fieldDataType) {
super(index, fieldNames, fieldDataType);
public BinaryDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super(index, fieldName, fieldDataType);
}
@Override
public BinaryDVAtomicFieldData load(LeafReaderContext context) {
return new BinaryDVAtomicFieldData(context.reader(), fieldNames.indexName());
return new BinaryDVAtomicFieldData(context.reader(), fieldName);
}
@Override

View File

@ -29,7 +29,6 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
@ -38,8 +37,8 @@ import java.io.IOException;
public class BytesBinaryDVIndexFieldData extends DocValuesIndexFieldData implements IndexFieldData<BytesBinaryDVAtomicFieldData> {
public BytesBinaryDVIndexFieldData(Index index, Names fieldNames, FieldDataType fieldDataType) {
super(index, fieldNames, fieldDataType);
public BytesBinaryDVIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super(index, fieldName, fieldDataType);
}
@Override
@ -50,7 +49,7 @@ public class BytesBinaryDVIndexFieldData extends DocValuesIndexFieldData impleme
@Override
public BytesBinaryDVAtomicFieldData load(LeafReaderContext context) {
try {
return new BytesBinaryDVAtomicFieldData(DocValues.getBinary(context.reader(), fieldNames.indexName()));
return new BytesBinaryDVAtomicFieldData(DocValues.getBinary(context.reader(), fieldName));
} catch (IOException e) {
throw new IllegalStateException("Cannot load doc values", e);
}
@ -67,8 +66,8 @@ public class BytesBinaryDVIndexFieldData extends DocValuesIndexFieldData impleme
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
// Ignore breaker
final Names fieldNames = fieldType.names();
return new BytesBinaryDVIndexFieldData(indexSettings.getIndex(), fieldNames, fieldType.fieldDataType());
final String fieldName = fieldType.name();
return new BytesBinaryDVIndexFieldData(indexSettings.getIndex(), fieldName, fieldType.fieldDataType());
}
}

View File

@ -27,7 +27,6 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
@ -43,12 +42,12 @@ public final class DisabledIndexFieldData extends AbstractIndexFieldData<AtomicF
public IndexFieldData<AtomicFieldData> build(IndexSettings indexSettings, MappedFieldType fieldType,
IndexFieldDataCache cache, CircuitBreakerService breakerService, MapperService mapperService) {
// Ignore Circuit Breaker
return new DisabledIndexFieldData(indexSettings, fieldType.names(), fieldType.fieldDataType(), cache);
return new DisabledIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache);
}
}
public DisabledIndexFieldData(IndexSettings indexSettings, Names fieldNames, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(indexSettings, fieldNames, fieldDataType, cache);
public DisabledIndexFieldData(IndexSettings indexSettings, String fieldName, FieldDataType fieldDataType, IndexFieldDataCache cache) {
super(indexSettings, fieldName, fieldDataType, cache);
}
@Override
@ -67,7 +66,7 @@ public final class DisabledIndexFieldData extends AbstractIndexFieldData<AtomicF
}
private IllegalStateException fail() {
return new IllegalStateException("Field data loading is forbidden on " + getFieldNames().fullName());
return new IllegalStateException("Field data loading is forbidden on " + getFieldName());
}
}

View File

@ -30,7 +30,6 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.internal.IdFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
@ -46,20 +45,20 @@ import static org.elasticsearch.common.util.set.Sets.newHashSet;
public abstract class DocValuesIndexFieldData {
protected final Index index;
protected final Names fieldNames;
protected final String fieldName;
protected final FieldDataType fieldDataType;
protected final ESLogger logger;
public DocValuesIndexFieldData(Index index, Names fieldNames, FieldDataType fieldDataType) {
public DocValuesIndexFieldData(Index index, String fieldName, FieldDataType fieldDataType) {
super();
this.index = index;
this.fieldNames = fieldNames;
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.logger = Loggers.getLogger(getClass());
}
public final Names getFieldNames() {
return fieldNames;
public final String getFieldName() {
return fieldName;
}
public final FieldDataType getFieldDataType() {
@ -92,20 +91,20 @@ public abstract class DocValuesIndexFieldData {
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
// Ignore Circuit Breaker
final Names fieldNames = fieldType.names();
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 [" + fieldNames.fullName() + "]");
throw new IllegalArgumentException("Doc values field data doesn't support filters [" + fieldName + "]");
}
if (BINARY_INDEX_FIELD_NAMES.contains(fieldNames.indexName())) {
if (BINARY_INDEX_FIELD_NAMES.contains(fieldName)) {
assert numericType == null;
return new BinaryDVIndexFieldData(indexSettings.getIndex(), fieldNames, fieldType.fieldDataType());
return new BinaryDVIndexFieldData(indexSettings.getIndex(), fieldName, fieldType.fieldDataType());
} else if (numericType != null) {
return new SortedNumericDVIndexFieldData(indexSettings.getIndex(), fieldNames, numericType, fieldType.fieldDataType());
return new SortedNumericDVIndexFieldData(indexSettings.getIndex(), fieldName, numericType, fieldType.fieldDataType());
} else {
return new SortedSetDVOrdinalsIndexFieldData(indexSettings, cache, fieldNames, breakerService, fieldType.fieldDataType());
return new SortedSetDVOrdinalsIndexFieldData(indexSettings, cache, fieldName, breakerService, fieldType.fieldDataType());
}
}

View File

@ -54,17 +54,17 @@ public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData
@Override
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
return new GeoPointArrayIndexFieldData(indexSettings, fieldType.names(), fieldType.fieldDataType(), cache,
return new GeoPointArrayIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache,
breakerService, fieldType.fieldDataType().getSettings()
.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_2_2_0) ||
indexSettings.getIndexVersionCreated().before(Version.V_2_2_0));
}
}
public GeoPointArrayIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names fieldNames,
public GeoPointArrayIndexFieldData(IndexSettings indexSettings, String fieldName,
FieldDataType fieldDataType, IndexFieldDataCache cache, CircuitBreakerService breakerService,
final boolean indexCreatedBefore22) {
super(indexSettings, fieldNames, fieldDataType, cache);
super(indexSettings, fieldName, fieldDataType, cache);
this.breakerService = breakerService;
this.indexCreatedBefore22 = indexCreatedBefore22;
}
@ -73,7 +73,7 @@ public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData
public AtomicGeoPointFieldData loadDirect(LeafReaderContext context) throws Exception {
LeafReader reader = context.reader();
Terms terms = reader.terms(getFieldNames().indexName());
Terms terms = reader.terms(getFieldName());
AtomicGeoPointFieldData data = null;
// TODO: Use an actual estimator to estimate before loading.
NonEstimatingEstimator estimator = new NonEstimatingEstimator(breakerService.getBreaker(CircuitBreaker.FIELDDATA));

View File

@ -46,7 +46,7 @@ public class IndexIndexFieldData extends AbstractIndexOrdinalsFieldData {
@Override
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
return new IndexIndexFieldData(indexSettings, fieldType.names());
return new IndexIndexFieldData(indexSettings, fieldType.name());
}
}
@ -100,8 +100,8 @@ public class IndexIndexFieldData extends AbstractIndexOrdinalsFieldData {
private final AtomicOrdinalsFieldData atomicFieldData;
private IndexIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names names) {
super(indexSettings, names, new FieldDataType("string"), null, null);
private IndexIndexFieldData(IndexSettings indexSettings, String name) {
super(indexSettings, name, new FieldDataType("string"), null, null);
atomicFieldData = new IndexAtomicFieldData(index().name());
}

View File

@ -57,13 +57,13 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
@Override
public IndexOrdinalsFieldData build(IndexSettings indexSettings, MappedFieldType fieldType,
IndexFieldDataCache cache, CircuitBreakerService breakerService, MapperService mapperService) {
return new PagedBytesIndexFieldData(indexSettings, fieldType.names(), fieldType.fieldDataType(), cache, breakerService);
return new PagedBytesIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache, breakerService);
}
}
public PagedBytesIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names fieldNames,
public PagedBytesIndexFieldData(IndexSettings indexSettings, String fieldName,
FieldDataType fieldDataType, IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldNames, fieldDataType, cache, breakerService);
super(indexSettings, fieldName, fieldDataType, cache, breakerService);
}
@Override
@ -71,8 +71,8 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
LeafReader reader = context.reader();
AtomicOrdinalsFieldData data = null;
PagedBytesEstimator estimator = new PagedBytesEstimator(context, breakerService.getBreaker(CircuitBreaker.FIELDDATA), getFieldNames().fullName());
Terms terms = reader.terms(getFieldNames().indexName());
PagedBytesEstimator estimator = new PagedBytesEstimator(context, breakerService.getBreaker(CircuitBreaker.FIELDDATA), getFieldName());
Terms terms = reader.terms(getFieldName());
if (terms == null) {
data = AbstractAtomicOrdinalsFieldData.empty();
estimator.afterLoad(null, data.ramBytesUsed());
@ -167,10 +167,10 @@ public class PagedBytesIndexFieldData extends AbstractIndexOrdinalsFieldData {
public long estimateStringFieldData() {
try {
LeafReader reader = context.reader();
Terms terms = reader.terms(getFieldNames().indexName());
Terms terms = reader.terms(getFieldName());
Fields fields = reader.fields();
final Terms fieldTerms = fields.terms(getFieldNames().indexName());
final Terms fieldTerms = fields.terms(getFieldName());
if (fieldTerms instanceof FieldReader) {
final Stats stats = ((FieldReader) fieldTerms).getStats();

View File

@ -48,7 +48,6 @@ import org.elasticsearch.index.fielddata.IndexParentChildFieldData;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
@ -75,10 +74,10 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
private final Set<String> parentTypes;
private final CircuitBreakerService breakerService;
public ParentChildIndexFieldData(IndexSettings indexSettings, MappedFieldType.Names fieldNames,
public ParentChildIndexFieldData(IndexSettings indexSettings, String fieldName,
FieldDataType fieldDataType, IndexFieldDataCache cache, MapperService mapperService,
CircuitBreakerService breakerService) {
super(indexSettings, fieldNames, fieldDataType, cache);
super(indexSettings, fieldName, fieldDataType, cache);
this.breakerService = breakerService;
Set<String> parentTypes = new HashSet<>();
for (DocumentMapper mapper : mapperService.docMappers(false)) {
@ -147,7 +146,7 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
MappedFieldType fieldType,
IndexFieldDataCache cache, CircuitBreakerService breakerService,
MapperService mapperService) {
return new ParentChildIndexFieldData(indexSettings, fieldType.names(), fieldType.fieldDataType(), cache,
return new ParentChildIndexFieldData(indexSettings, fieldType.name(), fieldType.fieldDataType(), cache,
mapperService, breakerService);
}
}
@ -319,8 +318,8 @@ public class ParentChildIndexFieldData extends AbstractIndexFieldData<AtomicPare
}
@Override
public Names getFieldNames() {
return ParentChildIndexFieldData.this.getFieldNames();
public String getFieldName() {
return ParentChildIndexFieldData.this.getFieldName();
}
@Override

View File

@ -38,7 +38,6 @@ import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.index.fielddata.fieldcomparator.FloatValuesComparatorSource;
import org.elasticsearch.index.fielddata.fieldcomparator.LongValuesComparatorSource;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
@ -52,7 +51,7 @@ import java.util.Collections;
public class SortedNumericDVIndexFieldData extends DocValuesIndexFieldData implements IndexNumericFieldData {
private final NumericType numericType;
public SortedNumericDVIndexFieldData(Index index, Names fieldNames, NumericType numericType, FieldDataType fieldDataType) {
public SortedNumericDVIndexFieldData(Index index, String fieldNames, NumericType numericType, FieldDataType fieldDataType) {
super(index, fieldNames, fieldDataType);
if (numericType == null) {
throw new IllegalArgumentException("numericType must be non-null");
@ -86,7 +85,7 @@ public class SortedNumericDVIndexFieldData extends DocValuesIndexFieldData imple
@Override
public AtomicNumericFieldData load(LeafReaderContext context) {
final LeafReader reader = context.reader();
final String field = fieldNames.indexName();
final String field = fieldName;
switch (numericType) {
case FLOAT:

View File

@ -31,9 +31,8 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexOrdinalsFieldData;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsBuilder;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import java.io.IOException;
@ -43,8 +42,8 @@ public class SortedSetDVOrdinalsIndexFieldData extends DocValuesIndexFieldData i
private final IndexFieldDataCache cache;
private final CircuitBreakerService breakerService;
public SortedSetDVOrdinalsIndexFieldData(IndexSettings indexSettings, IndexFieldDataCache cache, Names fieldNames, CircuitBreakerService breakerService, FieldDataType fieldDataType) {
super(indexSettings.getIndex(), fieldNames, fieldDataType);
public SortedSetDVOrdinalsIndexFieldData(IndexSettings indexSettings, IndexFieldDataCache cache, String fieldName, CircuitBreakerService breakerService, FieldDataType fieldDataType) {
super(indexSettings.getIndex(), fieldName, fieldDataType);
this.indexSettings = indexSettings;
this.cache = cache;
this.breakerService = breakerService;
@ -57,7 +56,7 @@ public class SortedSetDVOrdinalsIndexFieldData extends DocValuesIndexFieldData i
@Override
public AtomicOrdinalsFieldData load(LeafReaderContext context) {
return new SortedSetDVBytesAtomicFieldData(context.reader(), fieldNames.indexName());
return new SortedSetDVBytesAtomicFieldData(context.reader(), fieldName);
}
@Override
@ -73,7 +72,7 @@ public class SortedSetDVOrdinalsIndexFieldData extends DocValuesIndexFieldData i
}
boolean fieldFound = false;
for (LeafReaderContext context : indexReader.leaves()) {
if (context.reader().getFieldInfos().fieldInfo(getFieldNames().indexName()) != null) {
if (context.reader().getFieldInfos().fieldInfo(getFieldName()) != null) {
fieldFound = true;
break;
}

View File

@ -94,7 +94,7 @@ public class FieldsVisitor extends StoredFieldVisitor {
}
// can't derive exact mapping type
for (Map.Entry<String, List<Object>> entry : fields().entrySet()) {
MappedFieldType fieldType = mapperService.indexName(entry.getKey());
MappedFieldType fieldType = mapperService.fullName(entry.getKey());
if (fieldType == null) {
continue;
}
@ -112,7 +112,7 @@ public class FieldsVisitor extends StoredFieldVisitor {
if (fieldMapper == null) {
// it's possible index name doesn't match field name (legacy feature)
for (FieldMapper mapper : documentMapper.mappers()) {
if (mapper.fieldType().names().indexName().equals(indexName)) {
if (mapper.fieldType().name().equals(indexName)) {
fieldMapper = mapper;
break;
}

View File

@ -69,7 +69,7 @@ public class SingleFieldsVisitor extends FieldsVisitor {
if (fieldsValues == null) {
return;
}
List<Object> fieldValues = fieldsValues.get(fieldType.names().indexName());
List<Object> fieldValues = fieldsValues.get(fieldType.name());
if (fieldValues == null) {
return;
}

View File

@ -58,9 +58,9 @@ public final class DocumentFieldMappers implements Iterable<FieldMapper> {
for (FieldMapper mapper : mappers) {
fieldMappers.put(mapper.name(), mapper);
MappedFieldType fieldType = mapper.fieldType();
put(indexAnalyzers, fieldType.names().indexName(), fieldType.indexAnalyzer(), defaultIndex);
put(searchAnalyzers, fieldType.names().indexName(), fieldType.searchAnalyzer(), defaultSearch);
put(searchQuoteAnalyzers, fieldType.names().indexName(), fieldType.searchQuoteAnalyzer(), defaultSearchQuote);
put(indexAnalyzers, fieldType.name(), fieldType.indexAnalyzer(), defaultIndex);
put(searchAnalyzers, fieldType.name(), fieldType.searchAnalyzer(), defaultSearch);
put(searchQuoteAnalyzers, fieldType.name(), fieldType.searchQuoteAnalyzer(), defaultSearchQuote);
}
this.fieldMappers = Collections.unmodifiableMap(fieldMappers);
this.indexAnalyzer = new FieldNameAnalyzer(indexAnalyzers);
@ -76,10 +76,10 @@ public final class DocumentFieldMappers implements Iterable<FieldMapper> {
public Collection<String> simpleMatchToFullName(String pattern) {
Set<String> fields = new HashSet<>();
for (FieldMapper fieldMapper : this) {
if (Regex.simpleMatch(pattern, fieldMapper.fieldType().names().fullName())) {
fields.add(fieldMapper.fieldType().names().fullName());
} else if (Regex.simpleMatch(pattern, fieldMapper.fieldType().names().indexName())) {
fields.add(fieldMapper.fieldType().names().fullName());
if (Regex.simpleMatch(pattern, fieldMapper.fieldType().name())) {
fields.add(fieldMapper.fieldType().name());
} else if (Regex.simpleMatch(pattern, fieldMapper.fieldType().name())) {
fields.add(fieldMapper.fieldType().name());
}
}
return fields;
@ -91,7 +91,7 @@ public final class DocumentFieldMappers implements Iterable<FieldMapper> {
return fieldMapper;
}
for (FieldMapper otherFieldMapper : this) {
if (otherFieldMapper.fieldType().names().indexName().equals(name)) {
if (otherFieldMapper.fieldType().name().equals(name)) {
return otherFieldMapper;
}
}

View File

@ -217,31 +217,12 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
return builder;
}
protected MappedFieldType.Names buildNames(BuilderContext context) {
return new MappedFieldType.Names(buildIndexName(context), buildIndexNameClean(context), buildFullName(context));
}
protected String buildIndexName(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0_beta1)) {
return buildFullName(context);
}
String actualIndexName = indexName == null ? name : indexName;
return context.path().pathAsText(actualIndexName);
}
protected String buildIndexNameClean(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_2_0_0_beta1)) {
return buildFullName(context);
}
return indexName == null ? name : indexName;
}
protected String buildFullName(BuilderContext context) {
return context.path().pathAsText(name);
}
protected void setupFieldType(BuilderContext context) {
fieldType.setNames(buildNames(context));
fieldType.setName(buildFullName(context));
if (fieldType.indexAnalyzer() == null && fieldType.tokenized() == false && fieldType.indexOptions() != IndexOptions.NONE) {
fieldType.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
fieldType.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
@ -288,7 +269,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
@Override
public String name() {
return fieldType().names().fullName();
return fieldType().name();
}
public MappedFieldType fieldType() {
@ -318,7 +299,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
context.doc().add(field);
}
} catch (Exception e) {
throw new MapperParsingException("failed to parse [" + fieldType().names().fullName() + "]", e);
throw new MapperParsingException("failed to parse [" + fieldType().name() + "]", e);
}
multiFields.parse(this, context);
return null;
@ -367,7 +348,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
if (mergeWith instanceof FieldMapper) {
mergedType = ((FieldMapper) mergeWith).contentType();
}
throw new IllegalArgumentException("mapper [" + fieldType().names().fullName() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]");
throw new IllegalArgumentException("mapper [" + fieldType().name() + "] of different type, current_type [" + contentType() + "], merged_type [" + mergedType + "]");
}
FieldMapper fieldMergeWith = (FieldMapper) mergeWith;
multiFields = multiFields.merge(fieldMergeWith.multiFields);
@ -379,7 +360,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
@Override
public FieldMapper updateFieldType(Map<String, MappedFieldType> fullNameToFieldType) {
final MappedFieldType newFieldType = fullNameToFieldType.get(fieldType.names().fullName());
final MappedFieldType newFieldType = fullNameToFieldType.get(fieldType.name());
if (newFieldType == null) {
throw new IllegalStateException();
}
@ -404,9 +385,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
builder.field("type", contentType());
if (indexCreatedBefore2x && (includeDefaults || !simpleName().equals(fieldType().names().originalIndexName()))) {
builder.field("index_name", fieldType().names().originalIndexName());
}
if (includeDefaults || fieldType().boost() != 1.0f) {
builder.field("boost", fieldType().boost());

View File

@ -42,29 +42,17 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
/** Full field name to types containing a mapping for this full name. */
final CopyOnWriteHashMap<String, Set<String>> fullNameToTypes;
/** Index field name to field type */
final CopyOnWriteHashMap<String, MappedFieldType> indexNameToFieldType;
/** Index field name to types containing a mapping for this index name. */
final CopyOnWriteHashMap<String, Set<String>> indexNameToTypes;
/** Create a new empty instance. */
public FieldTypeLookup() {
fullNameToFieldType = new CopyOnWriteHashMap<>();
fullNameToTypes = new CopyOnWriteHashMap<>();
indexNameToFieldType = new CopyOnWriteHashMap<>();
indexNameToTypes = new CopyOnWriteHashMap<>();
}
private FieldTypeLookup(
CopyOnWriteHashMap<String, MappedFieldType> fullName,
CopyOnWriteHashMap<String, Set<String>> fullNameToTypes,
CopyOnWriteHashMap<String, MappedFieldType> indexName,
CopyOnWriteHashMap<String, Set<String>> indexNameToTypes) {
CopyOnWriteHashMap<String, Set<String>> fullNameToTypes) {
this.fullNameToFieldType = fullName;
this.fullNameToTypes = fullNameToTypes;
this.indexNameToFieldType = indexName;
this.indexNameToTypes = indexNameToTypes;
}
private static CopyOnWriteHashMap<String, Set<String>> addType(CopyOnWriteHashMap<String, Set<String>> map, String key, String type) {
@ -97,31 +85,21 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
CopyOnWriteHashMap<String, MappedFieldType> fullName = this.fullNameToFieldType;
CopyOnWriteHashMap<String, Set<String>> fullNameToTypes = this.fullNameToTypes;
CopyOnWriteHashMap<String, MappedFieldType> indexName = this.indexNameToFieldType;
CopyOnWriteHashMap<String, Set<String>> indexNameToTypes = this.indexNameToTypes;
for (FieldMapper fieldMapper : fieldMappers) {
MappedFieldType fieldType = fieldMapper.fieldType();
MappedFieldType fullNameFieldType = fullName.get(fieldType.names().fullName());
MappedFieldType indexNameFieldType = indexName.get(fieldType.names().indexName());
if (fullNameFieldType != null && indexNameFieldType != null && fullNameFieldType != indexNameFieldType) {
// this new field bridges between two existing field names (a full and index name), which we cannot support
throw new IllegalStateException("insane mappings found. field " + fieldType.names().fullName() + " maps across types to field " + fieldType.names().indexName());
}
MappedFieldType fullNameFieldType = fullName.get(fieldType.name());
// is the update even legal?
checkCompatibility(type, fieldMapper, updateAllTypes);
if (fieldType != fullNameFieldType || fieldType != indexNameFieldType) {
fullName = fullName.copyAndPut(fieldType.names().fullName(), fieldMapper.fieldType());
indexName = indexName.copyAndPut(fieldType.names().indexName(), fieldMapper.fieldType());
if (fieldType != fullNameFieldType) {
fullName = fullName.copyAndPut(fieldType.name(), fieldMapper.fieldType());
}
fullNameToTypes = addType(fullNameToTypes, fieldType.names().fullName(), type);
indexNameToTypes = addType(indexNameToTypes, fieldType.names().indexName(), type);
fullNameToTypes = addType(fullNameToTypes, fieldType.name(), type);
}
return new FieldTypeLookup(fullName, fullNameToTypes, indexName, indexNameToTypes);
return new FieldTypeLookup(fullName, fullNameToTypes);
}
private static boolean beStrict(String type, Set<String> types, boolean updateAllTypes) {
@ -142,26 +120,14 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
* If updateAllTypes is true, only basic compatibility is checked.
*/
private void checkCompatibility(String type, FieldMapper fieldMapper, boolean updateAllTypes) {
MappedFieldType fieldType = fullNameToFieldType.get(fieldMapper.fieldType().names().fullName());
MappedFieldType fieldType = fullNameToFieldType.get(fieldMapper.fieldType().name());
if (fieldType != null) {
List<String> conflicts = new ArrayList<>();
final Set<String> types = fullNameToTypes.get(fieldMapper.fieldType().names().fullName());
final Set<String> types = fullNameToTypes.get(fieldMapper.fieldType().name());
boolean strict = beStrict(type, types, updateAllTypes);
fieldType.checkCompatibility(fieldMapper.fieldType(), conflicts, strict);
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Mapper for [" + fieldMapper.fieldType().names().fullName() + "] conflicts with existing mapping in other types:\n" + conflicts.toString());
}
}
// field type for the index name must be compatible too
fieldType = indexNameToFieldType.get(fieldMapper.fieldType().names().indexName());
if (fieldType != null) {
List<String> conflicts = new ArrayList<>();
final Set<String> types = indexNameToTypes.get(fieldMapper.fieldType().names().indexName());
boolean strict = beStrict(type, types, updateAllTypes);
fieldType.checkCompatibility(fieldMapper.fieldType(), conflicts, strict);
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Mapper for [" + fieldMapper.fieldType().names().fullName() + "] conflicts with mapping with the same index name in other types" + conflicts.toString());
throw new IllegalArgumentException("Mapper for [" + fieldMapper.fieldType().name() + "] conflicts with existing mapping in other types:\n" + conflicts.toString());
}
}
}
@ -180,45 +146,16 @@ class FieldTypeLookup implements Iterable<MappedFieldType> {
return types;
}
/** Returns the field type for the given index name */
public MappedFieldType getByIndexName(String field) {
return indexNameToFieldType.get(field);
}
/** Get the set of types that have a mapping for the given field. */
public Set<String> getTypesByIndexName(String field) {
Set<String> types = indexNameToTypes.get(field);
if (types == null) {
types = Collections.emptySet();
}
return types;
}
/**
* Returns a list of the index names of a simple match regex like pattern against full name and index name.
*/
public Collection<String> simpleMatchToIndexNames(String pattern) {
Set<String> fields = new HashSet<>();
for (MappedFieldType fieldType : this) {
if (Regex.simpleMatch(pattern, fieldType.names().fullName())) {
fields.add(fieldType.names().indexName());
} else if (Regex.simpleMatch(pattern, fieldType.names().indexName())) {
fields.add(fieldType.names().indexName());
}
}
return fields;
}
/**
* Returns a list of the full names of a simple match regex like pattern against full name and index name.
*/
public Collection<String> simpleMatchToFullName(String pattern) {
Set<String> fields = new HashSet<>();
for (MappedFieldType fieldType : this) {
if (Regex.simpleMatch(pattern, fieldType.names().fullName())) {
fields.add(fieldType.names().fullName());
} else if (Regex.simpleMatch(pattern, fieldType.names().indexName())) {
fields.add(fieldType.names().fullName());
if (Regex.simpleMatch(pattern, fieldType.name())) {
fields.add(fieldType.name());
} else if (Regex.simpleMatch(pattern, fieldType.name())) {
fields.add(fieldType.name());
}
}
return fields;

View File

@ -53,68 +53,6 @@ import java.util.Objects;
*/
public abstract class MappedFieldType extends FieldType {
public static class Names {
private final String indexName;
private final String originalIndexName;
private final String fullName;
public Names(String name) {
this(name, name, name);
}
public Names(String indexName, String originalIndexName, String fullName) {
this.indexName = indexName;
this.originalIndexName = originalIndexName;
this.fullName = fullName;
}
/**
* The indexed name of the field. This is the name under which we will
* store it in the index.
*/
public String indexName() {
return indexName;
}
/**
* The original index name, before any "path" modifications performed on it.
*/
public String originalIndexName() {
return originalIndexName;
}
/**
* The full name, including dot path.
*/
public String fullName() {
return fullName;
}
@Override
public boolean equals(Object o) {
if (o == null || getClass() != o.getClass()) return false;
Names names = (Names) o;
if (!fullName.equals(names.fullName)) return false;
if (!indexName.equals(names.indexName)) return false;
if (!originalIndexName.equals(names.originalIndexName)) return false;
return true;
}
@Override
public int hashCode() {
int result = indexName.hashCode();
result = 31 * result + originalIndexName.hashCode();
result = 31 * result + fullName.hashCode();
return result;
}
}
public enum Loading {
LAZY {
@Override
@ -155,7 +93,7 @@ public abstract class MappedFieldType extends FieldType {
}
}
private Names names;
private String name;
private float boost;
// TODO: remove this docvalues flag and use docValuesType
private boolean docValues;
@ -170,7 +108,7 @@ public abstract class MappedFieldType extends FieldType {
protected MappedFieldType(MappedFieldType ref) {
super(ref);
this.names = ref.names();
this.name = ref.name();
this.boost = ref.boost();
this.docValues = ref.hasDocValues();
this.indexAnalyzer = ref.indexAnalyzer();
@ -214,7 +152,7 @@ public abstract class MappedFieldType extends FieldType {
return boost == fieldType.boost &&
docValues == fieldType.docValues &&
Objects.equals(names, fieldType.names) &&
Objects.equals(name, fieldType.name) &&
Objects.equals(indexAnalyzer, fieldType.indexAnalyzer) &&
Objects.equals(searchAnalyzer, fieldType.searchAnalyzer) &&
Objects.equals(searchQuoteAnalyzer(), fieldType.searchQuoteAnalyzer()) &&
@ -226,7 +164,7 @@ public abstract class MappedFieldType extends FieldType {
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), names, boost, docValues, indexAnalyzer, searchAnalyzer, searchQuoteAnalyzer,
return Objects.hash(super.hashCode(), name, boost, docValues, indexAnalyzer, searchAnalyzer, searchQuoteAnalyzer,
similarity == null ? null : similarity.name(), normsLoading, fieldDataType, nullValue, nullValueAsString);
}
@ -238,7 +176,7 @@ public abstract class MappedFieldType extends FieldType {
/** Checks this type is the same type as other. Adds a conflict if they are different. */
private final void checkTypeName(MappedFieldType other) {
if (typeName().equals(other.typeName()) == false) {
throw new IllegalArgumentException("mapper [" + names().fullName() + "] cannot be changed from type [" + typeName() + "] to [" + other.typeName() + "]");
throw new IllegalArgumentException("mapper [" + name + "] cannot be changed from type [" + typeName() + "] to [" + other.typeName() + "]");
} else if (getClass() != other.getClass()) {
throw new IllegalStateException("Type names equal for class " + getClass().getSimpleName() + " and " + other.getClass().getSimpleName());
}
@ -256,71 +194,68 @@ public abstract class MappedFieldType extends FieldType {
boolean mergeWithIndexed = other.indexOptions() != IndexOptions.NONE;
// TODO: should be validating if index options go "up" (but "down" is ok)
if (indexed != mergeWithIndexed || tokenized() != other.tokenized()) {
conflicts.add("mapper [" + names().fullName() + "] has different [index] values");
conflicts.add("mapper [" + name() + "] has different [index] values");
}
if (stored() != other.stored()) {
conflicts.add("mapper [" + names().fullName() + "] has different [store] values");
conflicts.add("mapper [" + name() + "] has different [store] values");
}
if (hasDocValues() == false && other.hasDocValues()) {
// don't add conflict if this mapper has doc values while the mapper to merge doesn't since doc values are implicitly set
// when the doc_values field data format is configured
conflicts.add("mapper [" + names().fullName() + "] has different [doc_values] values, cannot change from disabled to enabled");
conflicts.add("mapper [" + name() + "] has different [doc_values] values, cannot change from disabled to enabled");
}
if (omitNorms() && !other.omitNorms()) {
conflicts.add("mapper [" + names().fullName() + "] has different [omit_norms] values, cannot change from disable to enabled");
conflicts.add("mapper [" + name() + "] has different [omit_norms] values, cannot change from disable to enabled");
}
if (storeTermVectors() != other.storeTermVectors()) {
conflicts.add("mapper [" + names().fullName() + "] has different [store_term_vector] values");
conflicts.add("mapper [" + name() + "] has different [store_term_vector] values");
}
if (storeTermVectorOffsets() != other.storeTermVectorOffsets()) {
conflicts.add("mapper [" + names().fullName() + "] has different [store_term_vector_offsets] values");
conflicts.add("mapper [" + name() + "] has different [store_term_vector_offsets] values");
}
if (storeTermVectorPositions() != other.storeTermVectorPositions()) {
conflicts.add("mapper [" + names().fullName() + "] has different [store_term_vector_positions] values");
conflicts.add("mapper [" + name() + "] has different [store_term_vector_positions] values");
}
if (storeTermVectorPayloads() != other.storeTermVectorPayloads()) {
conflicts.add("mapper [" + names().fullName() + "] has different [store_term_vector_payloads] values");
conflicts.add("mapper [" + name() + "] has different [store_term_vector_payloads] values");
}
// null and "default"-named index analyzers both mean the default is used
if (indexAnalyzer() == null || "default".equals(indexAnalyzer().name())) {
if (other.indexAnalyzer() != null && "default".equals(other.indexAnalyzer().name()) == false) {
conflicts.add("mapper [" + names().fullName() + "] has different [analyzer]");
conflicts.add("mapper [" + name() + "] has different [analyzer]");
}
} else if (other.indexAnalyzer() == null || "default".equals(other.indexAnalyzer().name())) {
conflicts.add("mapper [" + names().fullName() + "] has different [analyzer]");
conflicts.add("mapper [" + name() + "] has different [analyzer]");
} else if (indexAnalyzer().name().equals(other.indexAnalyzer().name()) == false) {
conflicts.add("mapper [" + names().fullName() + "] has different [analyzer]");
conflicts.add("mapper [" + name() + "] has different [analyzer]");
}
if (!names().indexName().equals(other.names().indexName())) {
conflicts.add("mapper [" + names().fullName() + "] has different [index_name]");
}
if (Objects.equals(similarity(), other.similarity()) == false) {
conflicts.add("mapper [" + names().fullName() + "] has different [similarity]");
conflicts.add("mapper [" + name() + "] has different [similarity]");
}
if (strict) {
if (omitNorms() != other.omitNorms()) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [omit_norms] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [omit_norms] across all types.");
}
if (boost() != other.boost()) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [boost] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [boost] across all types.");
}
if (normsLoading() != other.normsLoading()) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [norms.loading] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [norms.loading] across all types.");
}
if (Objects.equals(searchAnalyzer(), other.searchAnalyzer()) == false) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [search_analyzer] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [search_analyzer] across all types.");
}
if (Objects.equals(searchQuoteAnalyzer(), other.searchQuoteAnalyzer()) == false) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [search_quote_analyzer] across all types.");
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 [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [fielddata] across all types.");
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 [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [null_value] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [null_value] across all types.");
}
}
}
@ -333,13 +268,13 @@ public abstract class MappedFieldType extends FieldType {
return true;
}
public Names names() {
return names;
public String name() {
return name;
}
public void setNames(Names names) {
public void setName(String name) {
checkIfFrozen();
this.names = names;
this.name = name;
}
public float boost() {
@ -456,7 +391,7 @@ public abstract class MappedFieldType extends FieldType {
/** Creates a term associated with the field of this mapper for the given value */
protected Term createTerm(Object value) {
return new Term(names().indexName(), indexedValueForSearch(value));
return new Term(name(), indexedValueForSearch(value));
}
public Query termQuery(Object value, @Nullable QueryShardContext context) {
@ -468,11 +403,11 @@ public abstract class MappedFieldType extends FieldType {
for (int i = 0; i < bytesRefs.length; i++) {
bytesRefs[i] = indexedValueForSearch(values.get(i));
}
return new TermsQuery(names.indexName(), bytesRefs);
return new TermsQuery(name(), bytesRefs);
}
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return new TermRangeQuery(names().indexName(),
return new TermRangeQuery(name(),
lowerTerm == null ? null : indexedValueForSearch(lowerTerm),
upperTerm == null ? null : indexedValueForSearch(upperTerm),
includeLower, includeUpper);
@ -492,7 +427,7 @@ public abstract class MappedFieldType extends FieldType {
public Query regexpQuery(String value, int flags, int maxDeterminizedStates, @Nullable MultiTermQuery.RewriteMethod method, @Nullable QueryShardContext context) {
if (numericType() != null) {
throw new QueryShardException(context, "Cannot use regular expression to filter numeric field [" + names.fullName + "]");
throw new QueryShardException(context, "Cannot use regular expression to filter numeric field [" + name + "]");
}
RegexpQuery query = new RegexpQuery(createTerm(value), flags, maxDeterminizedStates);

View File

@ -551,15 +551,6 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
}
}
/**
* Returns an {@link MappedFieldType} which has the given index name.
*
* If multiple types have fields with the same index name, the first is returned.
*/
public MappedFieldType indexName(String indexName) {
return fieldTypes.getByIndexName(indexName);
}
/**
* Returns the {@link MappedFieldType} for the give fullName.
*
@ -578,32 +569,13 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
// no wildcards
return Collections.singletonList(pattern);
}
return fieldTypes.simpleMatchToIndexNames(pattern);
return fieldTypes.simpleMatchToFullName(pattern);
}
// TODO: remove this since the underlying index names are now the same across all types
public Collection<String> simpleMatchToIndexNames(String pattern, @Nullable String[] types) {
return simpleMatchToIndexNames(pattern);
}
// TODO: remove types param, since the object mapper must be the same across all types
public ObjectMapper getObjectMapper(String name, @Nullable String[] types) {
public ObjectMapper getObjectMapper(String name) {
return fullPathObjectMappers.get(name);
}
public MappedFieldType smartNameFieldType(String smartName) {
MappedFieldType fieldType = fullName(smartName);
if (fieldType != null) {
return fieldType;
}
return indexName(smartName);
}
// TODO: remove this since the underlying index names are now the same across all types
public MappedFieldType smartNameFieldType(String smartName, @Nullable String[] types) {
return smartNameFieldType(smartName);
}
/**
* Given a type (eg. long, string, ...), return an anonymous field mapper that can be used for search operations.
*/
@ -697,7 +669,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
@Override
protected Analyzer getWrappedAnalyzer(String fieldName) {
MappedFieldType fieldType = smartNameFieldType(fieldName);
MappedFieldType fieldType = fullName(fieldName);
if (fieldType != null) {
Analyzer analyzer = extractAnalyzer.apply(fieldType);
if (analyzer != null) {

View File

@ -170,14 +170,14 @@ public class BinaryFieldMapper extends FieldMapper {
return;
}
if (fieldType().stored()) {
fields.add(new Field(fieldType().names().indexName(), value, fieldType()));
fields.add(new Field(fieldType().name(), value, fieldType()));
}
if (fieldType().hasDocValues()) {
CustomBinaryDocValuesField field = (CustomBinaryDocValuesField) context.doc().getByKey(fieldType().names().indexName());
CustomBinaryDocValuesField field = (CustomBinaryDocValuesField) context.doc().getByKey(fieldType().name());
if (field == null) {
field = new CustomBinaryDocValuesField(fieldType().names().indexName(), value);
context.doc().addWithKey(fieldType().names().indexName(), field);
field = new CustomBinaryDocValuesField(fieldType().name(), value);
context.doc().addWithKey(fieldType().name(), field);
} else {
field.add(value);
}

View File

@ -222,9 +222,9 @@ public class BooleanFieldMapper extends FieldMapper {
if (value == null) {
return;
}
fields.add(new Field(fieldType().names().indexName(), value ? "T" : "F", fieldType()));
fields.add(new Field(fieldType().name(), value ? "T" : "F", fieldType()));
if (fieldType().hasDocValues()) {
fields.add(new SortedNumericDocValuesField(fieldType().names().indexName(), value ? 1 : 0));
fields.add(new SortedNumericDocValuesField(fieldType().name(), value ? 1 : 0));
}
}

View File

@ -161,7 +161,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm),
includeLower, includeUpper);
@ -171,7 +171,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
byte iValue = parseValue(value);
byte iSim = fuzziness.asByte();
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -238,7 +238,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
value = ((Number) externalValue).byteValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), Byte.toString(value), boost);
context.allEntries().addText(fieldType().name(), Byte.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
@ -249,7 +249,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().names().fullName(), fieldType().nullValueAsString(), boost);
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
XContentParser.Token token;
@ -278,7 +278,7 @@ public class ByteFieldMapper extends NumberFieldMapper {
} else {
value = (byte) parser.shortValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), parser.text(), boost);
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}

View File

@ -326,15 +326,15 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
CompletionFieldType other = (CompletionFieldType)fieldType;
if (preservePositionIncrements != other.preservePositionIncrements) {
conflicts.add("mapper [" + names().fullName() + "] has different [preserve_position_increments] values");
conflicts.add("mapper [" + name() + "] has different [preserve_position_increments] values");
}
if (preserveSep != other.preserveSep) {
conflicts.add("mapper [" + names().fullName() + "] has different [preserve_separators] values");
conflicts.add("mapper [" + name() + "] has different [preserve_separators] values");
}
if (hasContextMappings() != other.hasContextMappings()) {
conflicts.add("mapper [" + names().fullName() + "] has different [context_mappings] values");
conflicts.add("mapper [" + name() + "] has different [context_mappings] values");
} else if (hasContextMappings() && contextMappings.equals(other.contextMappings) == false) {
conflicts.add("mapper [" + names().fullName() + "] has different [context_mappings] values");
conflicts.add("mapper [" + name() + "] has different [context_mappings] values");
}
}
@ -446,7 +446,7 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
Token token = parser.currentToken();
Map<String, CompletionInputMetaData> inputMap = new HashMap<>(1);
if (token == Token.VALUE_NULL) {
throw new MapperParsingException("completion field [" + fieldType().names().fullName() + "] does not support null values");
throw new MapperParsingException("completion field [" + fieldType().name() + "] does not support null values");
} else if (token == Token.START_ARRAY) {
while ((token = parser.nextToken()) != Token.END_ARRAY) {
parse(context, token, parser, inputMap);
@ -469,10 +469,10 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
}
CompletionInputMetaData metaData = completionInput.getValue();
if (fieldType().hasContextMappings()) {
fieldType().getContextMappings().addField(context.doc(), fieldType().names().indexName(),
fieldType().getContextMappings().addField(context.doc(), fieldType().name(),
input, metaData.weight, metaData.contexts);
} else {
context.doc().add(new SuggestField(fieldType().names().indexName(), input, metaData.weight));
context.doc().add(new SuggestField(fieldType().name(), input, metaData.weight));
}
}
multiFields.parse(this, context);
@ -536,7 +536,7 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
weight = weightValue.intValue();
} else if (Fields.CONTENT_FIELD_NAME_CONTEXTS.equals(currentFieldName)) {
if (fieldType().hasContextMappings() == false) {
throw new IllegalArgumentException("contexts field is not supported for field: [" + fieldType().names().fullName() + "]");
throw new IllegalArgumentException("contexts field is not supported for field: [" + fieldType().name() + "]");
}
ContextMappings contextMappings = fieldType().getContextMappings();
XContentParser.Token currentToken = parser.currentToken();

View File

@ -249,7 +249,7 @@ public class DateFieldMapper extends NumberFieldMapper {
@Override
public String toString(String s) {
final StringBuilder sb = new StringBuilder();
return sb.append(names().indexName()).append(':')
return sb.append(name()).append(':')
.append(includeLower ? '[' : '{')
.append((lowerTerm == null) ? "*" : lowerTerm.toString())
.append(" TO ")
@ -306,13 +306,13 @@ public class DateFieldMapper extends NumberFieldMapper {
if (strict) {
DateFieldType other = (DateFieldType)fieldType;
if (Objects.equals(dateTimeFormatter().format(), other.dateTimeFormatter().format()) == false) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [format] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [format] across all types.");
}
if (Objects.equals(dateTimeFormatter().locale(), other.dateTimeFormatter().locale()) == false) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [locale] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [locale] across all types.");
}
if (Objects.equals(timeUnit(), other.timeUnit()) == false) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [numeric_resolution] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [numeric_resolution] across all types.");
}
}
}
@ -404,7 +404,7 @@ public class DateFieldMapper extends NumberFieldMapper {
// not a time format
iSim = fuzziness.asLong();
}
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -424,7 +424,7 @@ public class DateFieldMapper extends NumberFieldMapper {
}
private Query innerRangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, @Nullable DateTimeZone timeZone, @Nullable DateMathParser forcedDateParser) {
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseToMilliseconds(lowerTerm, !includeLower, timeZone, forcedDateParser == null ? dateMathParser : forcedDateParser),
upperTerm == null ? null : parseToMilliseconds(upperTerm, includeUpper, timeZone, forcedDateParser == null ? dateMathParser : forcedDateParser),
includeLower, includeUpper);
@ -516,7 +516,7 @@ public class DateFieldMapper extends NumberFieldMapper {
Long value = null;
if (dateAsString != null) {
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), dateAsString, boost);
context.allEntries().addText(fieldType().name(), dateAsString, boost);
}
value = fieldType().parseStringValue(dateAsString);
}

View File

@ -164,7 +164,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newDoubleRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newDoubleRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseDoubleValue(lowerTerm),
upperTerm == null ? null : parseDoubleValue(upperTerm),
includeLower, includeUpper);
@ -174,7 +174,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
double iValue = parseDoubleValue(value);
double iSim = fuzziness.asDouble();
return NumericRangeQuery.newDoubleRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newDoubleRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -230,7 +230,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
value = ((Number) externalValue).doubleValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), Double.toString(value), boost);
context.allEntries().addText(fieldType().name(), Double.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
@ -241,7 +241,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().names().fullName(), fieldType().nullValueAsString(), boost);
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
XContentParser.Token token;
@ -270,7 +270,7 @@ public class DoubleFieldMapper extends NumberFieldMapper {
} else {
value = parser.doubleValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), parser.text(), boost);
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}

View File

@ -165,7 +165,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newFloatRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newFloatRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper);
@ -175,7 +175,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
float iValue = parseValue(value);
final float iSim = fuzziness.asFloat();
return NumericRangeQuery.newFloatRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newFloatRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -242,7 +242,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
value = ((Number) externalValue).floatValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), Float.toString(value), boost);
context.allEntries().addText(fieldType().name(), Float.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
@ -253,7 +253,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().names().fullName(), fieldType().nullValueAsString(), boost);
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
XContentParser.Token token;
@ -282,7 +282,7 @@ public class FloatFieldMapper extends NumberFieldMapper {
} else {
value = parser.floatValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), parser.text(), boost);
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}

View File

@ -170,7 +170,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper);
@ -180,7 +180,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
int iValue = parseValue(value);
int iSim = fuzziness.asInt();
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -247,7 +247,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
value = ((Number) externalValue).intValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), Integer.toString(value), boost);
context.allEntries().addText(fieldType().name(), Integer.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
@ -258,7 +258,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().names().fullName(), fieldType().nullValueAsString(), boost);
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
XContentParser.Token token;
@ -287,7 +287,7 @@ public class IntegerFieldMapper extends NumberFieldMapper {
} else {
value = parser.intValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), parser.text(), boost);
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}

View File

@ -168,7 +168,7 @@ public class LongFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseLongValue(lowerTerm),
upperTerm == null ? null : parseLongValue(upperTerm),
includeLower, includeUpper);
@ -178,7 +178,7 @@ public class LongFieldMapper extends NumberFieldMapper {
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
long iValue = parseLongValue(value);
final long iSim = fuzziness.asLong();
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -235,7 +235,7 @@ public class LongFieldMapper extends NumberFieldMapper {
value = ((Number) externalValue).longValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), Long.toString(value), boost);
context.allEntries().addText(fieldType().name(), Long.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
@ -246,7 +246,7 @@ public class LongFieldMapper extends NumberFieldMapper {
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().names().fullName(), fieldType().nullValueAsString(), boost);
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
XContentParser.Token token;
@ -275,7 +275,7 @@ public class LongFieldMapper extends NumberFieldMapper {
} else {
value = parser.longValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), parser.text(), boost);
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}

View File

@ -144,7 +144,7 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
List<String> conflicts, boolean strict) {
super.checkCompatibility(other, conflicts, strict);
if (numericPrecisionStep() != other.numericPrecisionStep()) {
conflicts.add("mapper [" + names().fullName() + "] has different [precision_step] values");
conflicts.add("mapper [" + name() + "] has different [precision_step] values");
}
}
@ -243,7 +243,7 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
protected abstract void innerParseCreateField(ParseContext context, List<Field> fields) throws IOException;
protected final void addDocValue(ParseContext context, List<Field> fields, long value) {
fields.add(new SortedNumericDocValuesField(fieldType().names().indexName(), value));
fields.add(new SortedNumericDocValuesField(fieldType().name(), value));
}
/**
@ -329,7 +329,7 @@ public abstract class NumberFieldMapper extends FieldMapper implements AllFieldM
};
public CustomNumericField(Number value, MappedFieldType fieldType) {
super(fieldType.names().indexName(), fieldType);
super(fieldType.name(), fieldType);
if (value != null) {
this.fieldsData = value;
}

View File

@ -166,7 +166,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm),
includeLower, includeUpper);
@ -176,7 +176,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
short iValue = parseValue(value);
short iSim = fuzziness.asShort();
return NumericRangeQuery.newIntRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -243,7 +243,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
value = ((Number) externalValue).shortValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), Short.toString(value), boost);
context.allEntries().addText(fieldType().name(), Short.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
@ -254,7 +254,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().names().fullName(), fieldType().nullValueAsString(), boost);
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT) {
XContentParser.Token token;
@ -283,7 +283,7 @@ public class ShortFieldMapper extends NumberFieldMapper {
} else {
value = parser.shortValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), parser.text(), boost);
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}

View File

@ -248,7 +248,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) {
throw new MapperParsingException("Field [" + fieldType.names().fullName() + "] cannot be analyzed and have doc values");
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot be analyzed and have doc values");
}
this.positionIncrementGap = positionIncrementGap;
this.ignoreAbove = ignoreAbove;
@ -315,19 +315,19 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
return;
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), valueAndBoost.value(), valueAndBoost.boost());
context.allEntries().addText(fieldType().name(), valueAndBoost.value(), valueAndBoost.boost());
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
Field field = new Field(fieldType().names().indexName(), valueAndBoost.value(), fieldType());
Field field = new Field(fieldType().name(), valueAndBoost.value(), fieldType());
field.setBoost(valueAndBoost.boost());
fields.add(field);
}
if (fieldType().hasDocValues()) {
fields.add(new SortedSetDocValuesField(fieldType().names().indexName(), new BytesRef(valueAndBoost.value())));
fields.add(new SortedSetDocValuesField(fieldType().name(), new BytesRef(valueAndBoost.value())));
}
if (fields.isEmpty()) {
context.ignoredValue(fieldType().names().indexName(), valueAndBoost.value());
context.ignoredValue(fieldType().name(), valueAndBoost.value());
}
}

View File

@ -148,7 +148,7 @@ public class TokenCountFieldMapper extends IntegerFieldMapper {
addIntegerFields(context, fields, count, valueAndBoost.boost());
}
if (fields.isEmpty()) {
context.ignoredValue(fieldType().names().indexName(), valueAndBoost.value());
context.ignoredValue(fieldType().name(), valueAndBoost.value());
}
}

View File

@ -241,10 +241,7 @@ public class TypeParsers {
Map.Entry<String, Object> entry = iterator.next();
final String propName = Strings.toUnderscoreCase(entry.getKey());
final Object propNode = entry.getValue();
if (propName.equals("index_name") && indexVersionCreated.before(Version.V_2_0_0_beta1)) {
builder.indexName(propNode.toString());
iterator.remove();
} else if (propName.equals("store")) {
if (propName.equals("store")) {
builder.store(parseStore(name, propNode.toString()));
iterator.remove();
} else if (propName.equals("index")) {

View File

@ -287,20 +287,20 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
super.checkCompatibility(fieldType, conflicts, strict);
GeoPointFieldType other = (GeoPointFieldType)fieldType;
if (isLatLonEnabled() != other.isLatLonEnabled()) {
conflicts.add("mapper [" + names().fullName() + "] has different [lat_lon]");
conflicts.add("mapper [" + name() + "] has different [lat_lon]");
}
if (isLatLonEnabled() && other.isLatLonEnabled() &&
latFieldType().numericPrecisionStep() != other.latFieldType().numericPrecisionStep()) {
conflicts.add("mapper [" + names().fullName() + "] has different [precision_step]");
conflicts.add("mapper [" + name() + "] has different [precision_step]");
}
if (isGeoHashEnabled() != other.isGeoHashEnabled()) {
conflicts.add("mapper [" + names().fullName() + "] has different [geohash]");
conflicts.add("mapper [" + name() + "] has different [geohash]");
}
if (geoHashPrecision() != other.geoHashPrecision()) {
conflicts.add("mapper [" + names().fullName() + "] has different [geohash_precision]");
conflicts.add("mapper [" + name() + "] has different [geohash_precision]");
}
if (isGeoHashPrefixEnabled() != other.isGeoHashPrefixEnabled()) {
conflicts.add("mapper [" + names().fullName() + "] has different [geohash_prefix]");
conflicts.add("mapper [" + name() + "] has different [geohash_prefix]");
}
}

View File

@ -123,7 +123,7 @@ public class GeoPointFieldMapper extends BaseGeoPointFieldMapper {
GeoUtils.normalizePoint(point);
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
context.doc().add(new GeoPointField(fieldType().names().indexName(), point.lon(), point.lat(), fieldType() ));
context.doc().add(new GeoPointField(fieldType().name(), point.lon(), point.lat(), fieldType() ));
}
super.parse(context, point, geoHash);
}

View File

@ -301,7 +301,7 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement
GeoPointFieldMapperLegacy gpfmMergeWith = (GeoPointFieldMapperLegacy) mergeWith;
if (gpfmMergeWith.coerce.explicit()) {
if (coerce.explicit() && coerce.value() != gpfmMergeWith.coerce.value()) {
throw new IllegalArgumentException("mapper [" + fieldType().names().fullName() + "] has different [coerce]");
throw new IllegalArgumentException("mapper [" + fieldType().name() + "] has different [coerce]");
}
}
@ -330,17 +330,17 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
Field field = new Field(fieldType().names().indexName(), Double.toString(point.lat()) + ',' + Double.toString(point.lon()), fieldType());
Field field = new Field(fieldType().name(), Double.toString(point.lat()) + ',' + Double.toString(point.lon()), fieldType());
context.doc().add(field);
}
super.parse(context, point, geoHash);
if (fieldType().hasDocValues()) {
CustomGeoPointDocValuesField field = (CustomGeoPointDocValuesField) context.doc().getByKey(fieldType().names().indexName());
CustomGeoPointDocValuesField field = (CustomGeoPointDocValuesField) context.doc().getByKey(fieldType().name());
if (field == null) {
field = new CustomGeoPointDocValuesField(fieldType().names().indexName(), point.lat(), point.lon());
context.doc().addWithKey(fieldType().names().indexName(), field);
field = new CustomGeoPointDocValuesField(fieldType().name(), point.lat(), point.lon());
context.doc().addWithKey(fieldType().name(), field);
} else {
field.add(point.lat(), point.lon());
}

View File

@ -105,7 +105,7 @@ public class GeoShapeFieldMapper extends FieldMapper {
static {
// setting name here is a hack so freeze can be called...instead all these options should be
// moved to the default ctor for GeoShapeFieldType, and defaultFieldType() should be removed from mappers...
FIELD_TYPE.setNames(new MappedFieldType.Names("DoesNotExist"));
FIELD_TYPE.setName("DoesNotExist");
FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
FIELD_TYPE.setTokenized(false);
FIELD_TYPE.setStored(false);
@ -278,10 +278,10 @@ public class GeoShapeFieldMapper extends FieldMapper {
throw new IllegalArgumentException("Unknown prefix tree type [" + tree + "]");
}
recursiveStrategy = new RecursivePrefixTreeStrategy(prefixTree, names().indexName());
recursiveStrategy = new RecursivePrefixTreeStrategy(prefixTree, name());
recursiveStrategy.setDistErrPct(distanceErrorPct());
recursiveStrategy.setPruneLeafyBranches(false);
termStrategy = new TermQueryPrefixTreeStrategy(prefixTree, names().indexName());
termStrategy = new TermQueryPrefixTreeStrategy(prefixTree, name());
termStrategy.setDistErrPct(distanceErrorPct());
defaultStrategy = resolveStrategy(strategyName);
defaultStrategy.setPointsOnly(pointsOnly);
@ -293,33 +293,33 @@ public class GeoShapeFieldMapper extends FieldMapper {
GeoShapeFieldType other = (GeoShapeFieldType)fieldType;
// prevent user from changing strategies
if (strategyName().equals(other.strategyName()) == false) {
conflicts.add("mapper [" + names().fullName() + "] has different [strategy]");
conflicts.add("mapper [" + name() + "] has different [strategy]");
}
// prevent user from changing trees (changes encoding)
if (tree().equals(other.tree()) == false) {
conflicts.add("mapper [" + names().fullName() + "] has different [tree]");
conflicts.add("mapper [" + name() + "] has different [tree]");
}
if ((pointsOnly() != other.pointsOnly())) {
conflicts.add("mapper [" + names().fullName() + "] has different points_only");
conflicts.add("mapper [" + name() + "] has different points_only");
}
// TODO we should allow this, but at the moment levels is used to build bookkeeping variables
// in lucene's SpatialPrefixTree implementations, need a patch to correct that first
if (treeLevels() != other.treeLevels()) {
conflicts.add("mapper [" + names().fullName() + "] has different [tree_levels]");
conflicts.add("mapper [" + name() + "] has different [tree_levels]");
}
if (precisionInMeters() != other.precisionInMeters()) {
conflicts.add("mapper [" + names().fullName() + "] has different [precision]");
conflicts.add("mapper [" + name() + "] has different [precision]");
}
if (strict) {
if (orientation() != other.orientation()) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [orientation] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [orientation] across all types.");
}
if (distanceErrorPct() != other.distanceErrorPct()) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [distance_error_pct] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [distance_error_pct] across all types.");
}
}
}
@ -450,7 +450,7 @@ public class GeoShapeFieldMapper extends FieldMapper {
shape = shapeBuilder.build();
}
if (fieldType().pointsOnly() && !(shape instanceof Point)) {
throw new MapperParsingException("[{" + fieldType().names().fullName() + "}] is configured for points only but a " +
throw new MapperParsingException("[{" + fieldType().name() + "}] is configured for points only but a " +
((shape instanceof JtsGeometry) ? ((JtsGeometry)shape).getGeom().getGeometryType() : shape.getClass()) + " was found");
}
Field[] fields = fieldType().defaultStrategy().createIndexableFields(shape);
@ -464,7 +464,7 @@ public class GeoShapeFieldMapper extends FieldMapper {
context.doc().add(field);
}
} catch (Exception e) {
throw new MapperParsingException("failed to parse [" + fieldType().names().fullName() + "]", e);
throw new MapperParsingException("failed to parse [" + fieldType().name() + "]", e);
}
return null;
}

View File

@ -91,7 +91,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
static {
FIELD_TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
FIELD_TYPE.setTokenized(true);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
}
@ -246,7 +246,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
// reset the entries
context.allEntries().reset();
Analyzer analyzer = findAnalyzer(context);
fields.add(new AllField(fieldType().names().indexName(), context.allEntries(), analyzer, fieldType()));
fields.add(new AllField(fieldType().name(), context.allEntries(), analyzer, fieldType()));
}
private Analyzer findAnalyzer(ParseContext context) {
@ -323,7 +323,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
if (((AllFieldMapper)mergeWith).enabled() != this.enabled() && ((AllFieldMapper)mergeWith).enabledState != Defaults.ENABLED) {
throw new IllegalArgumentException("mapper [" + fieldType().names().fullName() + "] enabled is " + this.enabled() + " now encountering "+ ((AllFieldMapper)mergeWith).enabled());
throw new IllegalArgumentException("mapper [" + fieldType().name() + "] enabled is " + this.enabled() + " now encountering "+ ((AllFieldMapper)mergeWith).enabled());
}
super.doMerge(mergeWith, updateAllTypes);
}

View File

@ -69,7 +69,7 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
}
@ -175,7 +175,7 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
if (strict) {
FieldNamesFieldType other = (FieldNamesFieldType)fieldType;
if (isEnabled() != other.isEnabled()) {
conflicts.add("mapper [" + names().fullName() + "] is used by multiple types. Set update_all_types to true to update [enabled] across all types.");
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [enabled] across all types.");
}
}
}
@ -290,7 +290,7 @@ public class FieldNamesFieldMapper extends MetadataFieldMapper {
for (String path : paths) {
for (String fieldName : extractFieldNames(path)) {
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
document.add(new Field(fieldType().names().indexName(), fieldName, fieldType()));
document.add(new Field(fieldType().name(), fieldName, fieldType()));
}
}
}

View File

@ -77,7 +77,7 @@ public class IdFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
@ -285,10 +285,10 @@ public class IdFieldMapper extends MetadataFieldMapper {
} // else we are in the pre/post parse phase
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
fields.add(new Field(fieldType().names().indexName(), context.id(), fieldType()));
fields.add(new Field(fieldType().name(), context.id(), fieldType()));
}
if (fieldType().hasDocValues()) {
fields.add(new BinaryDocValuesField(fieldType().names().indexName(), new BytesRef(context.id())));
fields.add(new BinaryDocValuesField(fieldType().name(), new BytesRef(context.id())));
}
}

View File

@ -67,7 +67,7 @@ public class IndexFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
@ -223,7 +223,7 @@ public class IndexFieldMapper extends MetadataFieldMapper {
}
public String value(Document document) {
Field field = (Field) document.getField(fieldType().names().indexName());
Field field = (Field) document.getField(fieldType().name());
return field == null ? null : (String)fieldType().value(field);
}
@ -247,7 +247,7 @@ public class IndexFieldMapper extends MetadataFieldMapper {
if (!enabledState.enabled) {
return;
}
fields.add(new Field(fieldType().names().indexName(), context.index(), fieldType()));
fields.add(new Field(fieldType().name(), context.index(), fieldType()));
}
@Override

View File

@ -75,7 +75,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
JOIN_FIELD_TYPE.setHasDocValues(true);
@ -120,9 +120,9 @@ public class ParentFieldMapper extends MetadataFieldMapper {
if (parentType == null) {
throw new MapperParsingException("[_parent] field mapping must contain the [type] option");
}
parentJoinFieldType.setNames(new MappedFieldType.Names(joinField(documentType)));
parentJoinFieldType.setName(joinField(documentType));
parentJoinFieldType.setFieldDataType(null);
childJoinFieldType.setNames(new MappedFieldType.Names(joinField(parentType)));
childJoinFieldType.setName(joinField(parentType));
return new ParentFieldMapper(fieldType, parentJoinFieldType, childJoinFieldType, parentType, context.indexSettings());
}
}
@ -242,7 +242,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
}
}
}
return new TermsQuery(names().indexName(), bValues);
return new TermsQuery(name(), bValues);
}
}
@ -269,7 +269,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
private static MappedFieldType joinFieldTypeForParentType(String parentType, Settings indexSettings) {
MappedFieldType parentJoinFieldType = Defaults.JOIN_FIELD_TYPE.clone();
parentJoinFieldType.setNames(new MappedFieldType.Names(joinField(parentType)));
parentJoinFieldType.setName(joinField(parentType));
parentJoinFieldType.freeze();
return parentJoinFieldType;
}
@ -312,7 +312,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
// we are in the parsing of _parent phase
String parentId = context.parser().text();
context.sourceToParse().parent(parentId);
fields.add(new Field(fieldType().names().indexName(), Uid.createUid(context.stringBuilder(), parentType, parentId), fieldType()));
fields.add(new Field(fieldType().name(), Uid.createUid(context.stringBuilder(), parentType, parentId), fieldType()));
addJoinFieldIfNeeded(fields, childJoinFieldType, parentId);
} else {
// otherwise, we are running it post processing of the xcontent
@ -324,7 +324,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
throw new MapperParsingException("No parent id provided, not within the document, and not externally");
}
// we did not add it in the parsing phase, add it now
fields.add(new Field(fieldType().names().indexName(), Uid.createUid(context.stringBuilder(), parentType, parentId), fieldType()));
fields.add(new Field(fieldType().name(), Uid.createUid(context.stringBuilder(), parentType, parentId), fieldType()));
addJoinFieldIfNeeded(fields, childJoinFieldType, parentId);
} else if (parentId != null && !parsedParentId.equals(Uid.createUid(context.stringBuilder(), parentType, parentId))) {
throw new MapperParsingException("Parent id mismatch, document value is [" + Uid.createUid(parsedParentId).id() + "], while external value is [" + parentId + "]");
@ -336,7 +336,7 @@ public class ParentFieldMapper extends MetadataFieldMapper {
private void addJoinFieldIfNeeded(List<Field> fields, MappedFieldType fieldType, String id) {
if (fieldType.hasDocValues()) {
fields.add(new SortedDocValuesField(fieldType.names().indexName(), new BytesRef(id)));
fields.add(new SortedDocValuesField(fieldType.name(), new BytesRef(id)));
}
}

View File

@ -62,7 +62,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
@ -179,7 +179,7 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
}
public String value(Document document) {
Field field = (Field) document.getField(fieldType().names().indexName());
Field field = (Field) document.getField(fieldType().name());
return field == null ? null : (String)fieldType().value(field);
}
@ -206,10 +206,10 @@ public class RoutingFieldMapper extends MetadataFieldMapper {
String routing = context.sourceToParse().routing();
if (routing != null) {
if (fieldType().indexOptions() == IndexOptions.NONE && !fieldType().stored()) {
context.ignoredValue(fieldType().names().indexName(), routing);
context.ignoredValue(fieldType().name(), routing);
return;
}
fields.add(new Field(fieldType().names().indexName(), routing, fieldType()));
fields.add(new Field(fieldType().name(), routing, fieldType()));
}
}
}

View File

@ -74,7 +74,7 @@ public class SourceFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
@ -272,7 +272,7 @@ public class SourceFieldMapper extends MetadataFieldMapper {
if (!source.hasArray()) {
source = source.toBytesArray();
}
fields.add(new StoredField(fieldType().names().indexName(), source.array(), source.arrayOffset(), source.length()));
fields.add(new StoredField(fieldType().name(), source.array(), source.arrayOffset(), source.length()));
}
@Override

View File

@ -64,7 +64,7 @@ public class TTLFieldMapper extends MetadataFieldMapper {
TTL_FIELD_TYPE.setNumericPrecisionStep(Defaults.PRECISION_STEP_64_BIT);
TTL_FIELD_TYPE.setIndexAnalyzer(NumericLongAnalyzer.buildNamedAnalyzer(Defaults.PRECISION_STEP_64_BIT));
TTL_FIELD_TYPE.setSearchAnalyzer(NumericLongAnalyzer.buildNamedAnalyzer(Integer.MAX_VALUE));
TTL_FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
TTL_FIELD_TYPE.setName(NAME);
TTL_FIELD_TYPE.freeze();
}

View File

@ -67,7 +67,7 @@ public class TimestampFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setStored(true);
FIELD_TYPE.setTokenized(false);
FIELD_TYPE.setNumericPrecisionStep(Defaults.PRECISION_STEP_64_BIT);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setDateTimeFormatter(DATE_TIME_FORMATTER);
FIELD_TYPE.setIndexAnalyzer(NumericDateAnalyzer.buildNamedAnalyzer(DATE_TIME_FORMATTER, Defaults.PRECISION_STEP_64_BIT));
FIELD_TYPE.setSearchAnalyzer(NumericDateAnalyzer.buildNamedAnalyzer(DATE_TIME_FORMATTER, Integer.MAX_VALUE));
@ -313,13 +313,13 @@ public class TimestampFieldMapper extends MetadataFieldMapper {
if (enabledState.enabled) {
long timestamp = context.sourceToParse().timestamp();
if (fieldType().indexOptions() == IndexOptions.NONE && !fieldType().stored() && !fieldType().hasDocValues()) {
context.ignoredValue(fieldType().names().indexName(), String.valueOf(timestamp));
context.ignoredValue(fieldType().name(), String.valueOf(timestamp));
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
fields.add(new LongFieldMapper.CustomLongNumericField(timestamp, fieldType()));
}
if (fieldType().hasDocValues()) {
fields.add(new NumericDocValuesField(fieldType().names().indexName(), timestamp));
fields.add(new NumericDocValuesField(fieldType().name(), timestamp));
}
}
}

View File

@ -70,7 +70,7 @@ public class TypeFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
}
}
@ -84,7 +84,7 @@ public class TypeFieldMapper extends MetadataFieldMapper {
@Override
public TypeFieldMapper build(BuilderContext context) {
fieldType.setNames(buildNames(context));
fieldType.setName(buildFullName(context));
return new TypeFieldMapper(fieldType, context.indexSettings());
}
}
@ -186,9 +186,9 @@ public class TypeFieldMapper extends MetadataFieldMapper {
if (fieldType().indexOptions() == IndexOptions.NONE && !fieldType().stored()) {
return;
}
fields.add(new Field(fieldType().names().indexName(), context.type(), fieldType()));
fields.add(new Field(fieldType().name(), context.type(), fieldType()));
if (fieldType().hasDocValues()) {
fields.add(new SortedSetDocValuesField(fieldType().names().indexName(), new BytesRef(context.type())));
fields.add(new SortedSetDocValuesField(fieldType().name(), new BytesRef(context.type())));
}
}

View File

@ -66,7 +66,7 @@ public class UidFieldMapper extends MetadataFieldMapper {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.freeze();
NESTED_FIELD_TYPE = FIELD_TYPE.clone();
@ -193,7 +193,7 @@ public class UidFieldMapper extends MetadataFieldMapper {
}
public Term term(String uid) {
return new Term(fieldType().names().indexName(), fieldType().indexedValueForSearch(uid));
return new Term(fieldType().name(), fieldType().indexedValueForSearch(uid));
}
@Override

View File

@ -51,7 +51,7 @@ public class VersionFieldMapper extends MetadataFieldMapper {
public static final MappedFieldType FIELD_TYPE = new VersionFieldType();
static {
FIELD_TYPE.setNames(new MappedFieldType.Names(NAME));
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setDocValuesType(DocValuesType.NUMERIC);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.freeze();

View File

@ -229,7 +229,7 @@ public class IpFieldMapper extends NumberFieldMapper {
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper);
@ -244,7 +244,7 @@ public class IpFieldMapper extends NumberFieldMapper {
} catch (IllegalArgumentException e) {
iSim = fuzziness.asLong();
}
return NumericRangeQuery.newLongRange(names().indexName(), numericPrecisionStep(),
return NumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
iValue - iSim,
iValue + iSim,
true, true);
@ -287,7 +287,7 @@ public class IpFieldMapper extends NumberFieldMapper {
return;
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().names().fullName(), ipAsString, fieldType().boost());
context.allEntries().addText(fieldType().name(), ipAsString, fieldType().boost());
}
final long value = ipToLong(ipAsString);

View File

@ -54,7 +54,7 @@ final class QueriesLoaderCollector extends SimpleCollector {
QueriesLoaderCollector(PercolatorQueriesRegistry percolator, ESLogger logger, MapperService mapperService, IndexFieldDataService indexFieldDataService) {
this.percolator = percolator;
this.logger = logger;
final MappedFieldType uidMapper = mapperService.smartNameFieldType(UidFieldMapper.NAME);
final MappedFieldType uidMapper = mapperService.fullName(UidFieldMapper.NAME);
this.uidFieldData = indexFieldDataService.getForField(uidMapper);
}

View File

@ -235,7 +235,7 @@ public class CommonTermsQueryBuilder extends AbstractQueryBuilder<CommonTermsQue
String field;
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType != null) {
field = fieldType.names().indexName();
field = fieldType.name();
} else {
field = fieldName;
}

View File

@ -101,7 +101,7 @@ public class ExistsQueryBuilder extends AbstractQueryBuilder<ExistsQueryBuilder>
if (fieldNamesFieldType.isEnabled()) {
final String f;
if (fieldType != null) {
f = fieldType.names().indexName();
f = fieldType.name();
} else {
f = field;
}

View File

@ -87,7 +87,7 @@ public class FieldMaskingSpanQueryBuilder extends AbstractQueryBuilder<FieldMask
String fieldInQuery = fieldName;
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType != null) {
fieldInQuery = fieldType.names().indexName();
fieldInQuery = fieldType.name();
}
Query innerQuery = queryBuilder.toQuery(context);
assert innerQuery instanceof SpanQuery;

View File

@ -265,7 +265,7 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
}
if (context.indexVersionCreated().onOrAfter(Version.V_2_2_0)) {
return new GeoPointInBBoxQuery(fieldType.names().fullName(), luceneTopLeft.lon(), luceneBottomRight.lat(),
return new GeoPointInBBoxQuery(fieldType.name(), luceneTopLeft.lon(), luceneBottomRight.lat(),
luceneBottomRight.lon(), luceneTopLeft.lat());
}

View File

@ -236,7 +236,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
}
normDistance = GeoUtils.maxRadialDistance(center, normDistance);
return new GeoPointDistanceQuery(fieldType.names().fullName(), center.lon(), center.lat(), normDistance);
return new GeoPointDistanceQuery(fieldType.name(), center.lon(), center.lat(), normDistance);
}
@Override

View File

@ -273,7 +273,7 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
indexFieldData, optimizeBbox);
}
return new GeoPointDistanceRangeQuery(fieldType.names().fullName(), point.lon(), point.lat(),
return new GeoPointDistanceRangeQuery(fieldType.name(), point.lon(), point.lat(),
(includeLower) ? fromValue : fromValue + TOLERANCE,
(includeUpper) ? toValue : toValue - TOLERANCE);
}

View File

@ -149,7 +149,7 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
lats[i] = p.lat();
lons[i] = p.lon();
}
return new GeoPointInPolygonQuery(fieldType.names().fullName(), lons, lats);
return new GeoPointInPolygonQuery(fieldType.name(), lons, lats);
}
@Override

View File

@ -810,7 +810,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
} else {
for (String field : fields) {
MappedFieldType fieldType = context.fieldMapper(field);
moreLikeFields.add(fieldType == null ? field : fieldType.names().indexName());
moreLikeFields.add(fieldType == null ? field : fieldType.name());
}
}

View File

@ -251,11 +251,11 @@ public class QueryShardContext {
}
public MappedFieldType fieldMapper(String name) {
return failIfFieldMappingNotFound(name, mapperService.smartNameFieldType(name, getTypes()));
return failIfFieldMappingNotFound(name, mapperService.fullName(name));
}
public ObjectMapper getObjectMapper(String name) {
return mapperService.getObjectMapper(name, getTypes());
return mapperService.getObjectMapper(name);
}
/**

View File

@ -294,7 +294,7 @@ public class SimpleQueryStringBuilder extends AbstractQueryBuilder<SimpleQuerySt
private static String resolveIndexName(String fieldName, QueryShardContext context) {
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType != null) {
return fieldType.names().indexName();
return fieldType.name();
}
return fieldName;
}

View File

@ -73,7 +73,7 @@ public class SpanTermQueryBuilder extends BaseTermQueryBuilder<SpanTermQueryBuil
String fieldName = this.fieldName;
MappedFieldType mapper = context.fieldMapper(fieldName);
if (mapper != null) {
fieldName = mapper.names().indexName();
fieldName = mapper.name();
valueBytes = mapper.indexedValueForSearch(value);
}
if (valueBytes == null) {

View File

@ -262,7 +262,7 @@ public class TermsQueryBuilder extends AbstractQueryBuilder<TermsQueryBuilder> {
MappedFieldType fieldType = context.fieldMapper(fieldName);
String indexFieldName;
if (fieldType != null) {
indexFieldName = fieldType.names().indexName();
indexFieldName = fieldType.name();
} else {
indexFieldName = fieldName;
}

View File

@ -118,7 +118,7 @@ public class WildcardQueryBuilder extends AbstractQueryBuilder<WildcardQueryBuil
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType != null) {
indexFieldName = fieldType.names().indexName();
indexFieldName = fieldType.name();
valueBytes = fieldType.indexedValueForSearch(value);
} else {
indexFieldName = fieldName;

View File

@ -374,7 +374,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder> ext
@Override
protected String getFieldName() {
return fieldData.getFieldNames().fullName();
return fieldData.getFieldName();
}
@Override
@ -450,7 +450,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder> ext
@Override
protected String getFieldName() {
return fieldData.getFieldNames().fullName();
return fieldData.getFieldName();
}
@Override

View File

@ -148,7 +148,7 @@ public class FieldValueFactorFunctionBuilder extends ScoreFunctionBuilder<FieldV
@Override
protected ScoreFunction doToFunction(QueryShardContext context) {
MappedFieldType fieldType = context.getMapperService().smartNameFieldType(field);
MappedFieldType fieldType = context.getMapperService().fullName(field);
IndexNumericFieldData fieldData = null;
if (fieldType == null) {
if(missing == null) {

View File

@ -117,7 +117,7 @@ public class RandomScoreFunctionBuilder extends ScoreFunctionBuilder<RandomScore
@Override
protected ScoreFunction doToFunction(QueryShardContext context) {
final MappedFieldType fieldType = context.getMapperService().smartNameFieldType("_uid");
final MappedFieldType fieldType = context.getMapperService().fullName("_uid");
if (fieldType == null) {
// mapper could be null if we are on a shard with no docs yet, so this won't actually be used
return new RandomScoreFunction();

View File

@ -235,7 +235,7 @@ public class MatchQuery {
final String field;
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType != null) {
field = fieldType.names().indexName();
field = fieldType.name();
} else {
field = fieldName;
}

View File

@ -167,7 +167,7 @@ public class MultiMatchQuery extends MatchQuery {
MappedFieldType fieldType = context.fieldMapper(name);
if (fieldType != null) {
Analyzer actualAnalyzer = getAnalyzer(fieldType);
name = fieldType.names().indexName();
name = fieldType.name();
if (!groups.containsKey(actualAnalyzer)) {
groups.put(actualAnalyzer, new ArrayList<>());
}

View File

@ -123,7 +123,7 @@ public class GeoDistanceRangeQuery extends Query {
}
public String fieldName() {
return indexFieldData.getFieldNames().indexName();
return indexFieldData.getFieldName();
}
@Override
@ -198,7 +198,7 @@ public class GeoDistanceRangeQuery extends Query {
if (Double.compare(filter.inclusiveUpperPoint, inclusiveUpperPoint) != 0) return false;
if (Double.compare(filter.lat, lat) != 0) return false;
if (Double.compare(filter.lon, lon) != 0) return false;
if (!indexFieldData.getFieldNames().indexName().equals(filter.indexFieldData.getFieldNames().indexName()))
if (!indexFieldData.getFieldName().equals(filter.indexFieldData.getFieldName()))
return false;
if (geoDistance != filter.geoDistance) return false;
@ -207,7 +207,7 @@ public class GeoDistanceRangeQuery extends Query {
@Override
public String toString(String field) {
return "GeoDistanceRangeQuery(" + indexFieldData.getFieldNames().indexName() + ", " + geoDistance + ", [" + inclusiveLowerPoint + " - " + inclusiveUpperPoint + "], " + lat + ", " + lon + ")";
return "GeoDistanceRangeQuery(" + indexFieldData.getFieldName() + ", " + geoDistance + ", [" + inclusiveLowerPoint + " - " + inclusiveUpperPoint + "], " + lat + ", " + lon + ")";
}
@Override
@ -223,7 +223,7 @@ public class GeoDistanceRangeQuery extends Query {
temp = inclusiveUpperPoint != +0.0d ? Double.doubleToLongBits(inclusiveUpperPoint) : 0L;
result = 31 * result + Long.hashCode(temp);
result = 31 * result + (geoDistance != null ? geoDistance.hashCode() : 0);
result = 31 * result + indexFieldData.getFieldNames().indexName().hashCode();
result = 31 * result + indexFieldData.getFieldName().hashCode();
return result;
}

View File

@ -51,7 +51,7 @@ public class GeoPolygonQuery extends Query {
}
public String fieldName() {
return indexFieldData.getFieldNames().indexName();
return indexFieldData.getFieldName();
}
@Override
@ -104,7 +104,7 @@ public class GeoPolygonQuery extends Query {
@Override
public String toString(String field) {
StringBuilder sb = new StringBuilder("GeoPolygonQuery(");
sb.append(indexFieldData.getFieldNames().indexName());
sb.append(indexFieldData.getFieldName());
sb.append(", ").append(Arrays.toString(points)).append(')');
return sb.toString();
}
@ -115,14 +115,14 @@ public class GeoPolygonQuery extends Query {
return false;
}
GeoPolygonQuery that = (GeoPolygonQuery) obj;
return indexFieldData.getFieldNames().indexName().equals(that.indexFieldData.getFieldNames().indexName())
return indexFieldData.getFieldName().equals(that.indexFieldData.getFieldName())
&& Arrays.equals(points, that.points);
}
@Override
public int hashCode() {
int h = super.hashCode();
h = 31 * h + indexFieldData.getFieldNames().indexName().hashCode();
h = 31 * h + indexFieldData.getFieldName().hashCode();
h = 31 * h + Arrays.hashCode(points);
return h;
}

View File

@ -57,7 +57,7 @@ public class InMemoryGeoBoundingBoxQuery extends Query {
}
public String fieldName() {
return indexFieldData.getFieldNames().indexName();
return indexFieldData.getFieldName();
}
@Override
@ -79,7 +79,7 @@ public class InMemoryGeoBoundingBoxQuery extends Query {
@Override
public String toString(String field) {
return "GeoBoundingBoxFilter(" + indexFieldData.getFieldNames().indexName() + ", " + topLeft + ", " + bottomRight + ")";
return "GeoBoundingBoxFilter(" + indexFieldData.getFieldName() + ", " + topLeft + ", " + bottomRight + ")";
}
@Override

View File

@ -32,7 +32,7 @@ public class IndexedGeoBoundingBoxQuery {
public static Query create(GeoPoint topLeft, GeoPoint bottomRight, GeoPointFieldMapperLegacy.GeoPointFieldType fieldType) {
if (!fieldType.isLatLonEnabled()) {
throw new IllegalArgumentException("lat/lon is not enabled (indexed) for field [" + fieldType.names().fullName() + "], can't use indexed filter on it");
throw new IllegalArgumentException("lat/lon is not enabled (indexed) for field [" + fieldType.name() + "], can't use indexed filter on it");
}
//checks to see if bounding box crosses 180 degrees
if (topLeft.lon() > bottomRight.lon()) {

View File

@ -129,7 +129,7 @@ public final class SimilarityService extends AbstractIndexComponent {
@Override
public Similarity get(String name) {
MappedFieldType fieldType = mapperService.smartNameFieldType(name);
MappedFieldType fieldType = mapperService.fullName(name);
return (fieldType != null && fieldType.similarity() != null) ? fieldType.similarity().get() : defaultSimilarity;
}
}

View File

@ -188,7 +188,7 @@ public class TermVectorsService {
/* only keep valid fields */
Set<String> validFields = new HashSet<>();
for (String field : selectedFields) {
MappedFieldType fieldType = indexShard.mapperService().smartNameFieldType(field);
MappedFieldType fieldType = indexShard.mapperService().fullName(field);
if (!isValidField(fieldType)) {
continue;
}
@ -223,7 +223,7 @@ public class TermVectorsService {
if (perFieldAnalyzer != null && perFieldAnalyzer.containsKey(field)) {
analyzer = mapperService.analysisService().analyzer(perFieldAnalyzer.get(field).toString());
} else {
analyzer = mapperService.smartNameFieldType(field).indexAnalyzer();
analyzer = mapperService.fullName(field).indexAnalyzer();
}
if (analyzer == null) {
analyzer = mapperService.analysisService().defaultIndexAnalyzer();
@ -269,7 +269,7 @@ public class TermVectorsService {
Set<String> seenFields = new HashSet<>();
Collection<GetField> getFields = new HashSet<>();
for (IndexableField field : doc.getFields()) {
MappedFieldType fieldType = indexShard.mapperService().smartNameFieldType(field.name());
MappedFieldType fieldType = indexShard.mapperService().fullName(field.name());
if (!isValidField(fieldType)) {
continue;
}

View File

@ -41,7 +41,6 @@ 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.mapper.MappedFieldType;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils;
import org.elasticsearch.threadpool.ThreadPool;
@ -91,8 +90,8 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
this.closed = true;
}
public IndexFieldDataCache buildIndexFieldDataCache(IndexFieldDataCache.Listener listener, Index index, MappedFieldType.Names fieldNames, FieldDataType fieldDataType) {
return new IndexFieldCache(logger, cache, index, fieldNames, fieldDataType, indicesFieldDataCacheListener, listener);
public IndexFieldDataCache buildIndexFieldDataCache(IndexFieldDataCache.Listener listener, Index index, String fieldName, FieldDataType fieldDataType) {
return new IndexFieldCache(logger, cache, index, fieldName, fieldDataType, indicesFieldDataCacheListener, listener);
}
public Cache<Key, Accountable> getCache() {
@ -107,7 +106,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.fieldNames, indexCache.fieldDataType, notification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED, value.ramBytesUsed());
listener.onRemoval(key.shardId, indexCache.fieldName, indexCache.fieldDataType, 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);
@ -129,16 +128,16 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
static class IndexFieldCache implements IndexFieldDataCache, SegmentReader.CoreClosedListener, IndexReader.ReaderClosedListener {
private final ESLogger logger;
final Index index;
final MappedFieldType.Names fieldNames;
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, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, Listener... listeners) {
IndexFieldCache(ESLogger logger,final Cache<Key, Accountable> cache, Index index, String fieldName, FieldDataType fieldDataType, Listener... listeners) {
this.logger = logger;
this.listeners = listeners;
this.index = index;
this.fieldNames = fieldNames;
this.fieldName = fieldName;
this.fieldDataType = fieldDataType;
this.cache = cache;
}
@ -156,7 +155,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final AtomicFieldData fieldData = indexFieldData.loadDirect(context);
for (Listener listener : k.listeners) {
try {
listener.onCache(shardId, fieldNames, fieldDataType, fieldData);
listener.onCache(shardId, fieldName, fieldDataType, fieldData);
} catch (Throwable e) {
// load anyway since listeners should not throw exceptions
logger.error("Failed to call listener on atomic field data loading", e);
@ -180,7 +179,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final Accountable ifd = (Accountable) indexFieldData.localGlobalDirect(indexReader);
for (Listener listener : k.listeners) {
try {
listener.onCache(shardId, fieldNames, fieldDataType, ifd);
listener.onCache(shardId, fieldName, fieldDataType, ifd);
} catch (Throwable e) {
// load anyway since listeners should not throw exceptions
logger.error("Failed to call listener on global ordinals loading", e);
@ -218,7 +217,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
public void clear(String fieldName) {
for (Key key : cache.keys()) {
if (key.indexCache.index.equals(index)) {
if (key.indexCache.fieldNames.fullName().equals(fieldName)) {
if (key.indexCache.fieldName.equals(fieldName)) {
cache.invalidate(key);
}
}

View File

@ -24,7 +24,6 @@ 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.mapper.MappedFieldType;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
@ -44,11 +43,11 @@ public class IndicesFieldDataCacheListener implements IndexFieldDataCache.Listen
}
@Override
public void onCache(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, Accountable fieldData) {
public void onCache(ShardId shardId, String fieldName, FieldDataType fieldDataType, Accountable fieldData) {
}
@Override
public void onRemoval(ShardId shardId, MappedFieldType.Names fieldNames, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
public void onRemoval(ShardId shardId, String fieldName, FieldDataType fieldDataType, boolean wasEvicted, long sizeInBytes) {
assert sizeInBytes >= 0 : "When reducing circuit breaker, it should be adjusted with a number higher or equal to 0 and not [" + sizeInBytes + "]";
circuitBreakerService.getBreaker(CircuitBreaker.FIELDDATA).addWithoutBreaking(-sizeInBytes);
}

View File

@ -196,7 +196,7 @@ public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLServ
private void purgeShards(List<IndexShard> shardsToPurge) {
for (IndexShard shardToPurge : shardsToPurge) {
Query query = shardToPurge.mapperService().smartNameFieldType(TTLFieldMapper.NAME).rangeQuery(null, System.currentTimeMillis(), false, true);
Query query = shardToPurge.mapperService().fullName(TTLFieldMapper.NAME).rangeQuery(null, System.currentTimeMillis(), false, true);
Engine.Searcher searcher = shardToPurge.acquireSearcher("indices_ttl");
try {
logger.debug("[{}][{}] purging shard", shardToPurge.routingEntry().index(), shardToPurge.routingEntry().id());

View File

@ -637,12 +637,7 @@ public class PercolateContext extends SearchContext {
@Override
public MappedFieldType smartNameFieldType(String name) {
return mapperService().smartNameFieldType(name, types);
}
@Override
public MappedFieldType smartNameFieldTypeFromAnyType(String name) {
return mapperService().smartNameFieldType(name);
return mapperService().fullName(name);
}
@Override

View File

@ -760,7 +760,7 @@ public class PercolatorService extends AbstractComponent {
hls = new ArrayList<>(topDocs.scoreDocs.length);
}
final MappedFieldType uidMapper = context.mapperService().smartNameFieldType(UidFieldMapper.NAME);
final MappedFieldType uidMapper = context.mapperService().fullName(UidFieldMapper.NAME);
final IndexFieldData<?> uidFieldData = context.fieldData().getForField(uidMapper);
int i = 0;
for (ScoreDoc scoreDoc : topDocs.scoreDocs) {

View File

@ -73,7 +73,7 @@ abstract class QueryCollector extends SimpleCollector {
this.logger = logger;
this.queries = context.percolateQueries();
this.searcher = context.docSearcher();
final MappedFieldType uidMapper = context.mapperService().smartNameFieldType(UidFieldMapper.NAME);
final MappedFieldType uidMapper = context.mapperService().fullName(UidFieldMapper.NAME);
this.uidFieldData = context.fieldData().getForField(uidMapper);
this.isNestedDoc = isNestedDoc;

View File

@ -971,7 +971,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
final ObjectSet<String> warmUp = new ObjectHashSet<>();
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
for (FieldMapper fieldMapper : docMapper.mappers()) {
final String indexName = fieldMapper.fieldType().names().indexName();
final String indexName = fieldMapper.fieldType().name();
Loading normsLoading = fieldMapper.fieldType().normsLoading();
if (normsLoading == null) {
normsLoading = defaultLoading;
@ -1047,10 +1047,10 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
fieldDataType = joinFieldType.fieldDataType();
// TODO: this can be removed in 3.0 when the old parent/child impl is removed:
// related to: https://github.com/elastic/elasticsearch/pull/12418
indexName = fieldMapper.fieldType().names().indexName();
indexName = fieldMapper.fieldType().name();
} else {
fieldDataType = fieldMapper.fieldType().fieldDataType();
indexName = fieldMapper.fieldType().names().indexName();
indexName = fieldMapper.fieldType().name();
}
if (fieldDataType == null) {
@ -1079,10 +1079,10 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
final long start = System.nanoTime();
indexFieldDataService.getForField(fieldType).load(ctx);
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed fielddata for [{}], took [{}]", fieldType.names().fullName(), TimeValue.timeValueNanos(System.nanoTime() - start));
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.names().fullName());
indexShard.warmerService().logger().warn("failed to warm-up fielddata for [{}]", t, fieldType.name());
} finally {
latch.countDown();
}
@ -1115,10 +1115,10 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
fieldDataType = joinFieldType.fieldDataType();
// TODO: this can be removed in 3.0 when the old parent/child impl is removed:
// related to: https://github.com/elastic/elasticsearch/pull/12418
indexName = fieldMapper.fieldType().names().indexName();
indexName = fieldMapper.fieldType().name();
} else {
fieldDataType = fieldMapper.fieldType().fieldDataType();
indexName = fieldMapper.fieldType().names().indexName();
indexName = fieldMapper.fieldType().name();
}
if (fieldDataType == null) {
continue;
@ -1144,10 +1144,10 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType);
ifd.loadGlobal(searcher.getDirectoryReader());
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.names().fullName(), TimeValue.timeValueNanos(System.nanoTime() - start));
indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.name(), TimeValue.timeValueNanos(System.nanoTime() - start));
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("failed to warm-up global ordinals for [{}]", t, fieldType.names().fullName());
indexShard.warmerService().logger().warn("failed to warm-up global ordinals for [{}]", t, fieldType.name());
} finally {
latch.countDown();
}

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