Remove `index_name` back compat.

Since 2.0 we enforce that fields have the same full and index names. So in 3.x
we can remove the ability to have different names on the same field.
This commit is contained in:
Adrien Grand 2015-12-16 19:41:34 +01:00
parent b771daa3c7
commit d8d8666877
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