Remove 2.x backward compatibility of mappings. (#21670)

For the record, I also had to remove the geo-hash cell and geo-distance range
queries to make the code compile. These queries already throw an exception in
all cases with 5.x indices, so that does not hurt any more.

I also had to rename all 2.x bwc indices from `index-${version}` to
`unsupported-${version}` to make `OldIndexBackwardCompatibilityIT`
happy.
This commit is contained in:
Adrien Grand 2016-11-30 13:34:46 +01:00 committed by GitHub
parent 235e6acd73
commit 6231009a8f
279 changed files with 473 additions and 24545 deletions

View File

@ -41,7 +41,6 @@ import org.apache.lucene.util.automaton.RegExp;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.LegacyDateFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.StringFieldType;
@ -336,11 +335,7 @@ public class MapperQueryParser extends AnalyzingQueryParser {
BytesRef part1Binary = part1 == null ? null : getAnalyzer().normalize(field, part1);
BytesRef part2Binary = part2 == null ? null : getAnalyzer().normalize(field, part2);
Query rangeQuery;
if (currentFieldType instanceof LegacyDateFieldMapper.DateFieldType && settings.timeZone() != null) {
LegacyDateFieldMapper.DateFieldType dateFieldType = (LegacyDateFieldMapper.DateFieldType) this.currentFieldType;
rangeQuery = dateFieldType.rangeQuery(part1Binary, part2Binary,
startInclusive, endInclusive, settings.timeZone(), null, context);
} else if (currentFieldType instanceof DateFieldMapper.DateFieldType && settings.timeZone() != null) {
if (currentFieldType instanceof DateFieldMapper.DateFieldType && settings.timeZone() != null) {
DateFieldMapper.DateFieldType dateFieldType = (DateFieldMapper.DateFieldType) this.currentFieldType;
rangeQuery = dateFieldType.rangeQuery(part1Binary, part2Binary,
startInclusive, endInclusive, settings.timeZone(), null, context);

View File

@ -305,8 +305,6 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
String parent = null;
FetchSourceContext fetchSourceContext = defaultFetchSourceContext;
String[] fields = defaultFields;
String timestamp = null;
TimeValue ttl = null;
String opType = null;
long version = Versions.MATCH_ANY;
VersionType versionType = VersionType.INTERNAL;
@ -336,14 +334,6 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
routing = parser.text();
} else if ("_parent".equals(currentFieldName) || "parent".equals(currentFieldName)) {
parent = parser.text();
} else if ("_timestamp".equals(currentFieldName) || "timestamp".equals(currentFieldName)) {
timestamp = parser.text();
} else if ("_ttl".equals(currentFieldName) || "ttl".equals(currentFieldName)) {
if (parser.currentToken() == XContentParser.Token.VALUE_STRING) {
ttl = TimeValue.parseTimeValue(parser.text(), null, currentFieldName);
} else {
ttl = new TimeValue(parser.longValue());
}
} else if ("op_type".equals(currentFieldName) || "opType".equals(currentFieldName)) {
opType = parser.text();
} else if ("_version".equals(currentFieldName) || "version".equals(currentFieldName)) {
@ -394,15 +384,15 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
// of index request.
if ("index".equals(action)) {
if (opType == null) {
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType)
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).version(version).versionType(versionType)
.setPipeline(pipeline).source(data.slice(from, nextMarker - from)), payload);
} else {
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType)
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).version(version).versionType(versionType)
.create("create".equals(opType)).setPipeline(pipeline)
.source(data.slice(from, nextMarker - from)), payload);
}
} else if ("create".equals(action)) {
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType)
internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).version(version).versionType(versionType)
.create(true).setPipeline(pipeline)
.source(data.slice(from, nextMarker - from)), payload);
} else if ("update".equals(action)) {
@ -420,15 +410,11 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques
IndexRequest upsertRequest = updateRequest.upsertRequest();
if (upsertRequest != null) {
upsertRequest.timestamp(timestamp);
upsertRequest.ttl(ttl);
upsertRequest.version(version);
upsertRequest.versionType(versionType);
}
IndexRequest doc = updateRequest.doc();
if (doc != null) {
doc.timestamp(timestamp);
doc.ttl(ttl);
doc.version(version);
doc.versionType(versionType);
}

View File

@ -20,10 +20,10 @@
package org.elasticsearch.action.index;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.action.TimestampParsingException;
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.metadata.MappingMetaData;
@ -41,7 +41,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
@ -75,10 +74,6 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
private String routing;
@Nullable
private String parent;
@Nullable
private String timestamp;
@Nullable
private TimeValue ttl;
private BytesReference source;
@ -164,12 +159,6 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
validationException = addValidationError("version type [force] may no longer be used", validationException);
}
if (ttl != null) {
if (ttl.millis() < 0) {
validationException = addValidationError("ttl must not be negative", validationException);
}
}
if (id != null && id.getBytes(StandardCharsets.UTF_8).length > 512) {
validationException = addValidationError("id is too long, must be no longer than 512 bytes but was: " +
id.getBytes(StandardCharsets.UTF_8).length, validationException);
@ -265,49 +254,6 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
return this.parent;
}
/**
* Sets the timestamp either as millis since the epoch, or, in the configured date format.
*/
public IndexRequest timestamp(String timestamp) {
this.timestamp = timestamp;
return this;
}
public String timestamp() {
return this.timestamp;
}
/**
* Sets the ttl value as a time value expression.
*/
public IndexRequest ttl(String ttl) {
this.ttl = TimeValue.parseTimeValue(ttl, null, "ttl");
return this;
}
/**
* Sets the ttl as a {@link TimeValue} instance.
*/
public IndexRequest ttl(TimeValue ttl) {
this.ttl = ttl;
return this;
}
/**
* Sets the relative ttl value in milliseconds. It musts be greater than 0 as it makes little sense otherwise.
*/
public IndexRequest ttl(long ttl) {
this.ttl = new TimeValue(ttl);
return this;
}
/**
* Returns the ttl as a {@link TimeValue}
*/
public TimeValue ttl() {
return this.ttl;
}
/**
* Sets the ingest pipeline to be executed before indexing the document
*/
@ -537,11 +483,6 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
public void process(@Nullable MappingMetaData mappingMd, boolean allowIdGeneration, String concreteIndex) {
// resolve timestamp if provided externally
if (timestamp != null) {
timestamp = MappingMetaData.Timestamp.parseStringTimestamp(timestamp,
mappingMd != null ? mappingMd.timestamp().dateTimeFormatter() : TimestampFieldMapper.Defaults.DATE_TIME_FORMATTER);
}
if (mappingMd != null) {
// might as well check for routing here
if (mappingMd.routing().required() && routing == null) {
@ -563,30 +504,6 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
autoGeneratedTimestamp = Math.max(0, System.currentTimeMillis()); // extra paranoia
id(UUIDs.base64UUID());
}
// generate timestamp if not provided, we always have one post this stage...
if (timestamp == null) {
String defaultTimestamp = TimestampFieldMapper.Defaults.DEFAULT_TIMESTAMP;
if (mappingMd != null && mappingMd.timestamp() != null) {
// If we explicitly ask to reject null timestamp
if (mappingMd.timestamp().ignoreMissing() != null && mappingMd.timestamp().ignoreMissing() == false) {
throw new TimestampParsingException("timestamp is required by mapping");
}
defaultTimestamp = mappingMd.timestamp().defaultTimestamp();
}
if (defaultTimestamp.equals(TimestampFieldMapper.Defaults.DEFAULT_TIMESTAMP)) {
timestamp = Long.toString(System.currentTimeMillis());
} else {
// if we are here, the defaultTimestamp is not
// TimestampFieldMapper.Defaults.DEFAULT_TIMESTAMP but
// this can only happen if defaultTimestamp was
// assigned again because mappingMd and
// mappingMd#timestamp() are not null
assert mappingMd != null;
timestamp = MappingMetaData.Timestamp.parseStringTimestamp(defaultTimestamp, mappingMd.timestamp().dateTimeFormatter());
}
}
}
/* resolve the routing if needed */
@ -601,8 +518,10 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
id = in.readOptionalString();
routing = in.readOptionalString();
parent = in.readOptionalString();
timestamp = in.readOptionalString();
ttl = in.readOptionalWriteable(TimeValue::new);
if (in.getVersion().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
in.readOptionalString(); // timestamp
in.readOptionalWriteable(TimeValue::new); // ttl
}
source = in.readBytesReference();
opType = OpType.fromId(in.readByte());
version = in.readLong();
@ -619,8 +538,10 @@ public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implement
out.writeOptionalString(id);
out.writeOptionalString(routing);
out.writeOptionalString(parent);
out.writeOptionalString(timestamp);
out.writeOptionalWriteable(ttl);
if (out.getVersion().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
out.writeOptionalString(null);
out.writeOptionalWriteable(null);
}
out.writeBytesReference(source);
out.writeByte(opType.getId());
out.writeLong(version);

View File

@ -25,7 +25,6 @@ import org.elasticsearch.action.support.replication.ReplicationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
@ -231,38 +230,6 @@ public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest,
return this;
}
/**
* Sets the timestamp either as millis since the epoch, or, in the configured date format.
*/
public IndexRequestBuilder setTimestamp(String timestamp) {
request.timestamp(timestamp);
return this;
}
/**
* Sets the ttl value as a time value expression.
*/
public IndexRequestBuilder setTTL(String ttl) {
request.ttl(ttl);
return this;
}
/**
* Sets the relative ttl value in milliseconds. It musts be greater than 0 as it makes little sense otherwise.
*/
public IndexRequestBuilder setTTL(long ttl) {
request.ttl(ttl);
return this;
}
/**
* Sets the ttl as a {@link TimeValue} instance.
*/
public IndexRequestBuilder setTTL(TimeValue ttl) {
request.ttl(ttl);
return this;
}
/**
* Sets the ingest pipeline to be executed before indexing the document
*/

View File

@ -171,7 +171,7 @@ public class TransportIndexAction extends TransportWriteAction<IndexRequest, Ind
public static Engine.IndexResult executeIndexRequestOnReplica(IndexRequest request, IndexShard replica) {
final ShardId shardId = replica.shardId();
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, shardId.getIndexName(), request.type(), request.id(), request.source())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());
.routing(request.routing()).parent(request.parent());
final Engine.Index operation;
try {
@ -189,7 +189,7 @@ public class TransportIndexAction extends TransportWriteAction<IndexRequest, Ind
/** Utility method to prepare an index operation on primary shards */
static Engine.Index prepareIndexOperationOnPrimary(IndexRequest request, IndexShard primary) {
SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.index(), request.type(), request.id(), request.source())
.routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl());
.routing(request.routing()).parent(request.parent());
return primary.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType(), request.getAutoGeneratedTimestamp(), request.isRetry());
}

View File

@ -156,8 +156,6 @@ public class SimulatePipelineRequest extends ActionRequest {
ConfigurationUtils.readStringProperty(null, null, dataMap, MetaData.ID.getFieldName(), "_id"),
ConfigurationUtils.readOptionalStringProperty(null, null, dataMap, MetaData.ROUTING.getFieldName()),
ConfigurationUtils.readOptionalStringProperty(null, null, dataMap, MetaData.PARENT.getFieldName()),
ConfigurationUtils.readOptionalStringProperty(null, null, dataMap, MetaData.TIMESTAMP.getFieldName()),
ConfigurationUtils.readOptionalStringProperty(null, null, dataMap, MetaData.TTL.getFieldName()),
document);
ingestDocumentList.add(ingestDocument);
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
@ -42,8 +41,6 @@ import org.elasticsearch.index.get.GetField;
import org.elasticsearch.index.get.GetResult;
import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.mapper.RoutingFieldMapper;
import org.elasticsearch.index.mapper.TTLFieldMapper;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.script.ExecutableScript;
@ -55,7 +52,6 @@ import org.elasticsearch.search.lookup.SourceLookup;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.function.LongSupplier;
@ -76,7 +72,7 @@ public class UpdateHelper extends AbstractComponent {
*/
public Result prepare(UpdateRequest request, IndexShard indexShard, LongSupplier nowInMillis) {
final GetResult getResult = indexShard.getService().get(request.type(), request.id(),
new String[]{RoutingFieldMapper.NAME, ParentFieldMapper.NAME, TTLFieldMapper.NAME, TimestampFieldMapper.NAME},
new String[]{RoutingFieldMapper.NAME, ParentFieldMapper.NAME},
true, request.version(), request.versionType(), FetchSourceContext.FETCH_SOURCE);
return prepare(indexShard.shardId(), request, getResult, nowInMillis);
}
@ -86,13 +82,11 @@ public class UpdateHelper extends AbstractComponent {
*/
@SuppressWarnings("unchecked")
protected Result prepare(ShardId shardId, UpdateRequest request, final GetResult getResult, LongSupplier nowInMillis) {
long getDateNS = System.nanoTime();
if (!getResult.isExists()) {
if (request.upsertRequest() == null && !request.docAsUpsert()) {
throw new DocumentMissingException(shardId, request.type(), request.id());
}
IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest();
TimeValue ttl = indexRequest.ttl();
if (request.scriptedUpsert() && request.script() != null) {
// Run the script to perform the create logic
IndexRequest upsert = request.upsertRequest();
@ -103,10 +97,6 @@ public class UpdateHelper extends AbstractComponent {
ctx.put("_source", upsertDoc);
ctx.put("_now", nowInMillis.getAsLong());
ctx = executeScript(request.script, ctx);
//Allow the script to set TTL using ctx._ttl
if (ttl == null) {
ttl = getTTLFromScriptContext(ctx);
}
//Allow the script to abort the create by setting "op" to "none"
String scriptOpChoice = (String) ctx.get("op");
@ -129,7 +119,6 @@ public class UpdateHelper extends AbstractComponent {
indexRequest.index(request.index()).type(request.type()).id(request.id())
// it has to be a "create!"
.create(true)
.ttl(ttl)
.setRefreshPolicy(request.getRefreshPolicy())
.routing(request.routing())
.parent(request.parent())
@ -155,8 +144,6 @@ public class UpdateHelper extends AbstractComponent {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true);
String operation = null;
String timestamp = null;
TimeValue ttl = null;
final Map<String, Object> updatedSourceAsMap;
final XContentType updateSourceContentType = sourceAndContent.v1();
String routing = getResult.getFields().containsKey(RoutingFieldMapper.NAME) ? getResult.field(RoutingFieldMapper.NAME).getValue().toString() : null;
@ -165,10 +152,6 @@ public class UpdateHelper extends AbstractComponent {
if (request.script() == null && request.doc() != null) {
IndexRequest indexRequest = request.doc();
updatedSourceAsMap = sourceAndContent.v2();
if (indexRequest.ttl() != null) {
ttl = indexRequest.ttl();
}
timestamp = indexRequest.timestamp();
if (indexRequest.routing() != null) {
routing = indexRequest.routing();
}
@ -184,16 +167,12 @@ public class UpdateHelper extends AbstractComponent {
}
} else {
Map<String, Object> ctx = new HashMap<>(16);
Long originalTtl = getResult.getFields().containsKey(TTLFieldMapper.NAME) ? (Long) getResult.field(TTLFieldMapper.NAME).getValue() : null;
Long originalTimestamp = getResult.getFields().containsKey(TimestampFieldMapper.NAME) ? (Long) getResult.field(TimestampFieldMapper.NAME).getValue() : null;
ctx.put("_index", getResult.getIndex());
ctx.put("_type", getResult.getType());
ctx.put("_id", getResult.getId());
ctx.put("_version", getResult.getVersion());
ctx.put("_routing", routing);
ctx.put("_parent", parent);
ctx.put("_timestamp", originalTimestamp);
ctx.put("_ttl", originalTtl);
ctx.put("_source", sourceAndContent.v2());
ctx.put("_now", nowInMillis.getAsLong());
@ -201,34 +180,14 @@ public class UpdateHelper extends AbstractComponent {
operation = (String) ctx.get("op");
Object fetchedTimestamp = ctx.get("_timestamp");
if (fetchedTimestamp != null) {
timestamp = fetchedTimestamp.toString();
} else if (originalTimestamp != null) {
// No timestamp has been given in the update script, so we keep the previous timestamp if there is one
timestamp = originalTimestamp.toString();
}
ttl = getTTLFromScriptContext(ctx);
updatedSourceAsMap = (Map<String, Object>) ctx.get("_source");
}
// apply script to update the source
// No TTL has been given in the update script so we keep previous TTL value if there is one
if (ttl == null) {
Long ttlAsLong = getResult.getFields().containsKey(TTLFieldMapper.NAME) ? (Long) getResult.field(TTLFieldMapper.NAME).getValue() : null;
if (ttlAsLong != null) {
ttl = new TimeValue(ttlAsLong - TimeValue.nsecToMSec(System.nanoTime() - getDateNS));// It is an approximation of exact TTL value, could be improved
}
}
if (operation == null || "index".equals(operation)) {
final IndexRequest indexRequest = Requests.indexRequest(request.index()).type(request.type()).id(request.id()).routing(routing).parent(parent)
.source(updatedSourceAsMap, updateSourceContentType)
.version(updateVersion).versionType(request.versionType())
.waitForActiveShards(request.waitForActiveShards())
.timestamp(timestamp).ttl(ttl)
.setRefreshPolicy(request.getRefreshPolicy());
return new Result(indexRequest, DocWriteResponse.Result.UPDATED, updatedSourceAsMap, updateSourceContentType);
} else if ("delete".equals(operation)) {
@ -263,17 +222,6 @@ public class UpdateHelper extends AbstractComponent {
return ctx;
}
private TimeValue getTTLFromScriptContext(Map<String, Object> ctx) {
Object fetchedTTL = ctx.get("_ttl");
if (fetchedTTL != null) {
if (fetchedTTL instanceof Number) {
return new TimeValue(((Number) fetchedTTL).longValue());
}
return TimeValue.parseTimeValue((String) fetchedTTL, null, "_ttl");
}
return null;
}
/**
* Applies {@link UpdateRequest#fetchSource()} to the _source of the updated document to be returned in a update response.
* For BWC this function also extracts the {@link UpdateRequest#fields()} from the updated document to be returned in a update response

View File

@ -28,7 +28,6 @@ import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
@ -355,33 +354,4 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
return this;
}
/**
* Set the new ttl of the document as a long. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
public UpdateRequestBuilder setTtl(Long ttl) {
request.doc().ttl(ttl);
return this;
}
/**
* Set the new ttl of the document as a time value expression. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
public UpdateRequestBuilder setTtl(String ttl) {
request.doc().ttl(ttl);
return this;
}
/**
* Set the new ttl of the document as a {@link TimeValue} instance. Note that if detectNoop is true (the default)
* and the source of the document isn't changed then the ttl update won't take
* effect.
*/
public UpdateRequestBuilder setTtl(TimeValue ttl) {
request.doc().ttl(ttl);
return this;
}
}

View File

@ -19,19 +19,17 @@
package org.elasticsearch.cluster.metadata;
import org.elasticsearch.action.TimestampParsingException;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.AbstractDiffable;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import java.io.IOException;
import java.util.Map;
@ -75,103 +73,17 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
}
}
public static class Timestamp {
public static String parseStringTimestamp(String timestampAsString, FormatDateTimeFormatter dateTimeFormatter) throws TimestampParsingException {
try {
return Long.toString(dateTimeFormatter.parser().parseMillis(timestampAsString));
} catch (RuntimeException e) {
throw new TimestampParsingException(timestampAsString, e);
}
}
public static final Timestamp EMPTY = new Timestamp(false, TimestampFieldMapper.DEFAULT_DATE_TIME_FORMAT,
TimestampFieldMapper.Defaults.DEFAULT_TIMESTAMP, null);
private final boolean enabled;
private final String format;
private final FormatDateTimeFormatter dateTimeFormatter;
private final String defaultTimestamp;
private final Boolean ignoreMissing;
public Timestamp(boolean enabled, String format, String defaultTimestamp, Boolean ignoreMissing) {
this.enabled = enabled;
this.format = format;
this.dateTimeFormatter = Joda.forPattern(format);
this.defaultTimestamp = defaultTimestamp;
this.ignoreMissing = ignoreMissing;
}
public boolean enabled() {
return enabled;
}
public String format() {
return this.format;
}
public String defaultTimestamp() {
return this.defaultTimestamp;
}
public boolean hasDefaultTimestamp() {
return this.defaultTimestamp != null;
}
public Boolean ignoreMissing() {
return ignoreMissing;
}
public FormatDateTimeFormatter dateTimeFormatter() {
return this.dateTimeFormatter;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Timestamp timestamp = (Timestamp) o;
if (enabled != timestamp.enabled) return false;
if (format != null ? !format.equals(timestamp.format) : timestamp.format != null) return false;
if (defaultTimestamp != null ? !defaultTimestamp.equals(timestamp.defaultTimestamp) : timestamp.defaultTimestamp != null) return false;
if (ignoreMissing != null ? !ignoreMissing.equals(timestamp.ignoreMissing) : timestamp.ignoreMissing != null) return false;
return true;
}
@Override
public int hashCode() {
int result = (enabled ? 1 : 0);
result = 31 * result + (format != null ? format.hashCode() : 0);
result = 31 * result + (dateTimeFormatter != null ? dateTimeFormatter.hashCode() : 0);
result = 31 * result + (defaultTimestamp != null ? defaultTimestamp.hashCode() : 0);
result = 31 * result + (ignoreMissing != null ? ignoreMissing.hashCode() : 0);
return result;
}
}
private final String type;
private final CompressedXContent source;
private Routing routing;
private Timestamp timestamp;
private boolean hasParentField;
public MappingMetaData(DocumentMapper docMapper) {
this.type = docMapper.type();
this.source = docMapper.mappingSource();
this.routing = new Routing(docMapper.routingFieldMapper().required());
this.timestamp = new Timestamp(docMapper.timestampFieldMapper().enabled(),
docMapper.timestampFieldMapper().fieldType().dateTimeFormatter().format(), docMapper.timestampFieldMapper().defaultTimestamp(),
docMapper.timestampFieldMapper().ignoreMissing());
this.hasParentField = docMapper.parentFieldMapper().active();
}
@ -227,29 +139,6 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
} else {
this.routing = Routing.EMPTY;
}
if (withoutType.containsKey("_timestamp")) {
boolean enabled = false;
String format = TimestampFieldMapper.DEFAULT_DATE_TIME_FORMAT;
String defaultTimestamp = TimestampFieldMapper.Defaults.DEFAULT_TIMESTAMP;
Boolean ignoreMissing = null;
Map<String, Object> timestampNode = (Map<String, Object>) withoutType.get("_timestamp");
for (Map.Entry<String, Object> entry : timestampNode.entrySet()) {
String fieldName = entry.getKey();
Object fieldNode = entry.getValue();
if (fieldName.equals("enabled")) {
enabled = lenientNodeBooleanValue(fieldNode);
} else if (fieldName.equals("format")) {
format = fieldNode.toString();
} else if (fieldName.equals("default") && fieldNode != null) {
defaultTimestamp = fieldNode.toString();
} else if (fieldName.equals("ignore_missing")) {
ignoreMissing = lenientNodeBooleanValue(fieldNode);
}
}
this.timestamp = new Timestamp(enabled, format, defaultTimestamp, ignoreMissing);
} else {
this.timestamp = Timestamp.EMPTY;
}
if (withoutType.containsKey("_parent")) {
this.hasParentField = true;
} else {
@ -257,11 +146,10 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
}
}
public MappingMetaData(String type, CompressedXContent source, Routing routing, Timestamp timestamp, boolean hasParentField) {
public MappingMetaData(String type, CompressedXContent source, Routing routing, boolean hasParentField) {
this.type = type;
this.source = source;
this.routing = routing;
this.timestamp = timestamp;
this.hasParentField = hasParentField;
}
@ -269,9 +157,6 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
if (routing == Routing.EMPTY) {
routing = defaultMapping.routing();
}
if (timestamp == Timestamp.EMPTY) {
timestamp = defaultMapping.timestamp();
}
}
public String type() {
@ -309,21 +194,19 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
return this.routing;
}
public Timestamp timestamp() {
return this.timestamp;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(type());
source().writeTo(out);
// routing
out.writeBoolean(routing().required());
// timestamp
out.writeBoolean(timestamp().enabled());
out.writeString(timestamp().format());
out.writeOptionalString(timestamp().defaultTimestamp());
out.writeOptionalBoolean(timestamp().ignoreMissing());
if (out.getVersion().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
// timestamp
out.writeBoolean(false); // enabled
out.writeString(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.format());
out.writeOptionalString(null);
out.writeOptionalBoolean(null);
}
out.writeBoolean(hasParentField());
}
@ -336,7 +219,6 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
if (!routing.equals(that.routing)) return false;
if (!source.equals(that.source)) return false;
if (!timestamp.equals(that.timestamp)) return false;
if (!type.equals(that.type)) return false;
return true;
@ -347,7 +229,6 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
int result = type.hashCode();
result = 31 * result + source.hashCode();
result = 31 * result + routing.hashCode();
result = 31 * result + timestamp.hashCode();
return result;
}
@ -356,18 +237,19 @@ public class MappingMetaData extends AbstractDiffable<MappingMetaData> {
CompressedXContent source = CompressedXContent.readCompressedString(in);
// routing
Routing routing = new Routing(in.readBoolean());
// timestamp
if (in.getVersion().before(Version.V_6_0_0_alpha1_UNRELEASED)) {
// timestamp
boolean enabled = in.readBoolean();
if (enabled) {
throw new IllegalArgumentException("_timestamp may not be enabled");
}
in.readString(); // format
in.readOptionalString(); // defaultTimestamp
in.readOptionalBoolean(); // ignoreMissing
}
boolean enabled = in.readBoolean();
String format = in.readString();
String defaultTimestamp = in.readOptionalString();
Boolean ignoreMissing = null;
ignoreMissing = in.readOptionalBoolean();
final Timestamp timestamp = new Timestamp(enabled, format, defaultTimestamp, ignoreMissing);
final boolean hasParentField = in.readBoolean();
return new MappingMetaData(type, source, routing, timestamp, hasParentField);
return new MappingMetaData(type, source, routing, hasParentField);
}
}

View File

@ -55,7 +55,6 @@ import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.ingest.IngestMetadata;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.script.ScriptMetaData;
@ -761,7 +760,6 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
/** All known time cluster settings. */
public static final Set<String> CLUSTER_TIME_SETTINGS = unmodifiableSet(newHashSet(
IndicesTTLService.INDICES_TTL_INTERVAL_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING.getKey(),

View File

@ -106,7 +106,7 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
* Returns true if this index can be supported by the current version of elasticsearch
*/
private static boolean isSupportedVersion(IndexMetaData indexMetaData) {
return indexMetaData.getCreationVersion().onOrAfter(Version.V_2_0_0_beta1);
return indexMetaData.getCreationVersion().onOrAfter(Version.V_5_0_0_beta1);
}
/**

View File

@ -26,7 +26,6 @@ import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import java.io.IOException;
@ -41,9 +40,9 @@ public class GeoUtils {
/** Minimum valid longitude in degrees. */
public static final double MIN_LON = -180.0;
public static final String LATITUDE = GeoPointFieldMapper.Names.LAT;
public static final String LONGITUDE = GeoPointFieldMapper.Names.LON;
public static final String GEOHASH = GeoPointFieldMapper.Names.GEOHASH;
public static final String LATITUDE = "lat";
public static final String LONGITUDE = "lon";
public static final String GEOHASH = "geohash";
/** Earth ellipsoid major axis defined by WGS 84 in meters */
public static final double EARTH_SEMI_MAJOR_AXIS = 6378137.0; // meters (WGS 84)

View File

@ -72,7 +72,6 @@ import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.monitor.fs.FsService;
import org.elasticsearch.monitor.jvm.JvmGcMonitorService;
import org.elasticsearch.monitor.jvm.JvmService;
@ -184,7 +183,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
IndicesQueryCache.INDICES_CACHE_QUERY_SIZE_SETTING,
IndicesQueryCache.INDICES_CACHE_QUERY_COUNT_SETTING,
IndicesQueryCache.INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING,
IndicesTTLService.INDICES_TTL_INTERVAL_SETTING,
MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING,
MetaData.SETTING_READ_ONLY_SETTING,
RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING,

View File

@ -26,7 +26,6 @@ import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.index.mapper.CompletionFieldMapper;
import org.elasticsearch.index.mapper.CompletionFieldMapper2x;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
@ -60,9 +59,6 @@ public class PerFieldMappingPostingFormatCodec extends Lucene62Codec {
logger.warn("no index mapper found for field: [{}] returning default postings format", field);
} else if (fieldType instanceof CompletionFieldMapper.CompletionFieldType) {
return CompletionFieldMapper.CompletionFieldType.postingsFormat();
} else if (fieldType instanceof CompletionFieldMapper2x.CompletionFieldType) {
return ((CompletionFieldMapper2x.CompletionFieldType) fieldType).postingsFormat(
super.getPostingsFormatForField(field));
}
return super.getPostingsFormatForField(field);
}

View File

@ -69,7 +69,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import javax.net.ssl.SNIServerName;
import java.io.Closeable;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -1031,14 +1030,6 @@ public abstract class Engine implements Closeable {
return this.doc.routing();
}
public long timestamp() {
return this.doc.timestamp();
}
public long ttl() {
return this.doc.ttl();
}
public String parent() {
return this.doc.parent();
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.LeafReaderContext;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
@ -52,19 +51,14 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
* Lucene 5.4 GeoPointFieldType
*/
public static class GeoPointDVIndexFieldData extends AbstractGeoPointDVIndexFieldData {
final boolean indexCreatedBefore2x;
public GeoPointDVIndexFieldData(Index index, String fieldName, final boolean indexCreatedBefore2x) {
public GeoPointDVIndexFieldData(Index index, String fieldName) {
super(index, fieldName);
this.indexCreatedBefore2x = indexCreatedBefore2x;
}
@Override
public AtomicGeoPointFieldData load(LeafReaderContext context) {
try {
if (indexCreatedBefore2x) {
return new GeoPointLegacyDVAtomicFieldData(DocValues.getBinary(context.reader(), fieldName));
}
return new GeoPointDVAtomicFieldData(DocValues.getSortedNumeric(context.reader(), fieldName));
} catch (IOException e) {
throw new IllegalStateException("Cannot load doc values", e);
@ -81,13 +75,8 @@ public abstract class AbstractGeoPointDVIndexFieldData extends DocValuesIndexFie
@Override
public IndexFieldData<?> build(IndexSettings indexSettings, MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
if (indexSettings.getIndexVersionCreated().before(Version.V_2_2_0)
&& fieldType.hasDocValues() == false) {
return new GeoPointArrayIndexFieldData(indexSettings, fieldType.name(), cache, breakerService);
}
// Ignore breaker
return new GeoPointDVIndexFieldData(indexSettings.getIndex(), fieldType.name(),
indexSettings.getIndexVersionCreated().before(Version.V_2_2_0));
return new GeoPointDVIndexFieldData(indexSettings.getIndex(), fieldType.name());
}
}
}

View File

@ -1,145 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.GeoPointValues;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
public abstract class GeoPointArrayAtomicFieldData extends AbstractAtomicGeoPointFieldData {
@Override
public void close() {
}
static class WithOrdinals extends GeoPointArrayAtomicFieldData {
private final LongArray indexedPoints;
private final Ordinals ordinals;
private final int maxDoc;
public WithOrdinals(LongArray indexedPoints, Ordinals ordinals, int maxDoc) {
super();
this.indexedPoints = indexedPoints;
this.ordinals = ordinals;
this.maxDoc = maxDoc;
}
@Override
public long ramBytesUsed() {
return Integer.BYTES + indexedPoints.ramBytesUsed();
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("indexedPoints", indexedPoints));
return Collections.unmodifiableList(resources);
}
@Override
public MultiGeoPointValues getGeoPointValues() {
final RandomAccessOrds ords = ordinals.ordinals();
final SortedDocValues singleOrds = DocValues.unwrapSingleton(ords);
final GeoPoint point = new GeoPoint(Double.NaN, Double.NaN);
if (singleOrds != null) {
final GeoPointValues values = new GeoPointValues() {
@Override
public GeoPoint get(int docID) {
final int ord = singleOrds.getOrd(docID);
if (ord >= 0) {
return point.resetFromIndexHash(indexedPoints.get(ord));
}
return point.reset(Double.NaN, Double.NaN);
}
};
return FieldData.singleton(values, DocValues.docsWithValue(singleOrds, maxDoc));
}
return new MultiGeoPointValues() {
@Override
public GeoPoint valueAt(int index) {
return point.resetFromIndexHash(indexedPoints.get(ords.ordAt(index)));
}
@Override
public void setDocument(int docId) {
ords.setDocument(docId);
}
@Override
public int count() {
return ords.cardinality();
}
};
}
}
public static class Single extends GeoPointArrayAtomicFieldData {
private final LongArray indexedPoint;
private final BitSet set;
public Single(LongArray indexedPoint, BitSet set) {
this.indexedPoint = indexedPoint;
this.set = set;
}
@Override
public long ramBytesUsed() {
return Integer.BYTES + indexedPoint.ramBytesUsed()
+ (set == null ? 0 : set.ramBytesUsed());
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("indexedPoints", indexedPoint));
if (set != null) {
resources.add(Accountables.namedAccountable("missing bitset", set));
}
return Collections.unmodifiableList(resources);
}
@Override
public MultiGeoPointValues getGeoPointValues() {
final GeoPoint point = new GeoPoint();
final GeoPointValues values = new GeoPointValues() {
@Override
public GeoPoint get(int docID) {
if (set == null || set.get(docID)) {
return point.resetFromIndexHash(indexedPoint.get(docID));
}
return point.reset(Double.NaN, Double.NaN);
}
};
return FieldData.singleton(values, set);
}
}
}

View File

@ -1,180 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.Version;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicGeoPointFieldData;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import org.elasticsearch.index.fielddata.ordinals.OrdinalsBuilder;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
/**
* Loads FieldData for an array of GeoPoints supporting both long encoded points and backward compatible double arrays
*/
public class GeoPointArrayIndexFieldData extends AbstractIndexGeoPointFieldData {
private final CircuitBreakerService breakerService;
public GeoPointArrayIndexFieldData(IndexSettings indexSettings, String fieldName,
IndexFieldDataCache cache, CircuitBreakerService breakerService) {
super(indexSettings, fieldName, cache);
this.breakerService = breakerService;
}
@Override
public AtomicGeoPointFieldData loadDirect(LeafReaderContext context) throws Exception {
LeafReader reader = context.reader();
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));
if (terms == null) {
data = AbstractAtomicGeoPointFieldData.empty(reader.maxDoc());
estimator.afterLoad(null, data.ramBytesUsed());
return data;
}
return (indexSettings.getIndexVersionCreated().before(Version.V_2_2_0)) ?
loadLegacyFieldData(reader, estimator, terms, data) : loadFieldData22(reader, estimator, terms, data);
}
/**
* long encoded geopoint field data
*/
private AtomicGeoPointFieldData loadFieldData22(LeafReader reader, NonEstimatingEstimator estimator, Terms terms,
AtomicGeoPointFieldData data) throws Exception {
LongArray indexedPoints = BigArrays.NON_RECYCLING_INSTANCE.newLongArray(128);
final float acceptableTransientOverheadRatio = OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO;
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
final TermsEnum termsEnum;
final GeoPointField.TermEncoding termEncoding;
if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_2_3_0)) {
termEncoding = GeoPointField.TermEncoding.PREFIX;
termsEnum = OrdinalsBuilder.wrapGeoPointTerms(terms.iterator());
} else {
termEncoding = GeoPointField.TermEncoding.NUMERIC;
termsEnum = OrdinalsBuilder.wrapNumeric64Bit(terms.iterator());
}
final GeoPointTermsEnum iter = new GeoPointTermsEnum(builder.buildFromTerms(termsEnum), termEncoding);
Long hashedPoint;
long numTerms = 0;
while ((hashedPoint = iter.next()) != null) {
indexedPoints = BigArrays.NON_RECYCLING_INSTANCE.resize(indexedPoints, numTerms + 1);
indexedPoints.set(numTerms++, hashedPoint);
}
indexedPoints = BigArrays.NON_RECYCLING_INSTANCE.resize(indexedPoints, numTerms);
Ordinals build = builder.build();
RandomAccessOrds ordinals = build.ordinals();
if (FieldData.isMultiValued(ordinals) == false) {
int maxDoc = reader.maxDoc();
LongArray sIndexedPoint = BigArrays.NON_RECYCLING_INSTANCE.newLongArray(reader.maxDoc());
for (int i=0; i<maxDoc; ++i) {
ordinals.setDocument(i);
long nativeOrdinal = ordinals.nextOrd();
if (nativeOrdinal != RandomAccessOrds.NO_MORE_ORDS) {
sIndexedPoint.set(i, indexedPoints.get(nativeOrdinal));
}
}
BitSet set = builder.buildDocsWithValuesSet();
data = new GeoPointArrayAtomicFieldData.Single(sIndexedPoint, set);
} else {
data = new GeoPointArrayAtomicFieldData.WithOrdinals(indexedPoints, build, reader.maxDoc());
}
success = true;
return data;
} finally {
if (success) {
estimator.afterLoad(null, data.ramBytesUsed());
}
}
}
/**
* Backward compatibility support for legacy lat/lon double arrays
*/
private AtomicGeoPointFieldData loadLegacyFieldData(LeafReader reader, NonEstimatingEstimator estimator, Terms terms,
AtomicGeoPointFieldData data) throws Exception {
DoubleArray lat = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
DoubleArray lon = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(128);
final float acceptableTransientOverheadRatio = OrdinalsBuilder.DEFAULT_ACCEPTABLE_OVERHEAD_RATIO;
boolean success = false;
try (OrdinalsBuilder builder = new OrdinalsBuilder(reader.maxDoc(), acceptableTransientOverheadRatio)) {
final GeoPointTermsEnumLegacy iter = new GeoPointTermsEnumLegacy(builder.buildFromTerms(terms.iterator()));
GeoPoint point;
long numTerms = 0;
while ((point = iter.next()) != null) {
lat = BigArrays.NON_RECYCLING_INSTANCE.resize(lat, numTerms + 1);
lon = BigArrays.NON_RECYCLING_INSTANCE.resize(lon, numTerms + 1);
lat.set(numTerms, point.getLat());
lon.set(numTerms, point.getLon());
++numTerms;
}
lat = BigArrays.NON_RECYCLING_INSTANCE.resize(lat, numTerms);
lon = BigArrays.NON_RECYCLING_INSTANCE.resize(lon, numTerms);
Ordinals build = builder.build();
RandomAccessOrds ordinals = build.ordinals();
if (FieldData.isMultiValued(ordinals) == false) {
int maxDoc = reader.maxDoc();
DoubleArray sLat = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(reader.maxDoc());
DoubleArray sLon = BigArrays.NON_RECYCLING_INSTANCE.newDoubleArray(reader.maxDoc());
for (int i = 0; i < maxDoc; i++) {
ordinals.setDocument(i);
long nativeOrdinal = ordinals.nextOrd();
if (nativeOrdinal != RandomAccessOrds.NO_MORE_ORDS) {
sLat.set(i, lat.get(nativeOrdinal));
sLon.set(i, lon.get(nativeOrdinal));
}
}
BitSet set = builder.buildDocsWithValuesSet();
data = new GeoPointArrayLegacyAtomicFieldData.Single(sLon, sLat, set);
} else {
data = new GeoPointArrayLegacyAtomicFieldData.WithOrdinals(lon, lat, build, reader.maxDoc());
}
success = true;
return data;
} finally {
if (success) {
estimator.afterLoad(null, data.ramBytesUsed());
}
}
}
}

View File

@ -1,162 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.RandomAccessOrds;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.util.DoubleArray;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.GeoPointValues;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.ordinals.Ordinals;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
public abstract class GeoPointArrayLegacyAtomicFieldData extends AbstractAtomicGeoPointFieldData {
@Override
public void close() {
}
static class WithOrdinals extends GeoPointArrayLegacyAtomicFieldData {
private final DoubleArray lon, lat;
private final Ordinals ordinals;
private final int maxDoc;
public WithOrdinals(DoubleArray lon, DoubleArray lat, Ordinals ordinals, int maxDoc) {
super();
this.lon = lon;
this.lat = lat;
this.ordinals = ordinals;
this.maxDoc = maxDoc;
}
@Override
public long ramBytesUsed() {
return Integer.BYTES/*size*/ + lon.ramBytesUsed() + lat.ramBytesUsed();
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("latitude", lat));
resources.add(Accountables.namedAccountable("longitude", lon));
return Collections.unmodifiableList(resources);
}
@Override
public MultiGeoPointValues getGeoPointValues() {
final RandomAccessOrds ords = ordinals.ordinals();
final SortedDocValues singleOrds = DocValues.unwrapSingleton(ords);
if (singleOrds != null) {
final GeoPoint point = new GeoPoint();
final GeoPointValues values = new GeoPointValues() {
@Override
public GeoPoint get(int docID) {
final int ord = singleOrds.getOrd(docID);
if (ord >= 0) {
return point.reset(lat.get(ord), lon.get(ord));
}
return point.reset(Double.NaN, Double.NaN);
}
};
return FieldData.singleton(values, DocValues.docsWithValue(singleOrds, maxDoc));
} else {
final GeoPoint point = new GeoPoint();
return new MultiGeoPointValues() {
@Override
public GeoPoint valueAt(int index) {
final long ord = ords.ordAt(index);
if (ord >= 0) {
return point.reset(lat.get(ord), lon.get(ord));
}
return point.reset(Double.NaN, Double.NaN);
}
@Override
public void setDocument(int docId) {
ords.setDocument(docId);
}
@Override
public int count() {
return ords.cardinality();
}
};
}
}
}
/**
* Assumes unset values are marked in bitset, and docId is used as the index to the value array.
*/
public static class Single extends GeoPointArrayLegacyAtomicFieldData {
private final DoubleArray lon, lat;
private final BitSet set;
public Single(DoubleArray lon, DoubleArray lat, BitSet set) {
this.lon = lon;
this.lat = lat;
this.set = set;
}
@Override
public long ramBytesUsed() {
return Integer.BYTES + lon.ramBytesUsed() + lat.ramBytesUsed() + (set == null ? 0 : set.ramBytesUsed());
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
resources.add(Accountables.namedAccountable("latitude", lat));
resources.add(Accountables.namedAccountable("longitude", lon));
if (set != null) {
resources.add(Accountables.namedAccountable("missing bitset", set));
}
return Collections.unmodifiableList(resources);
}
@Override
public MultiGeoPointValues getGeoPointValues() {
final GeoPoint point = new GeoPoint();
final GeoPointValues values = new GeoPointValues() {
@Override
public GeoPoint get(int docID) {
if (set == null || set.get(docID)) {
return point.reset(lat.get(docID), lon.get(docID));
}
return point.reset(Double.NaN, Double.NaN);
}
};
return FieldData.singleton(values, set);
}
}
}

View File

@ -1,100 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.fielddata.plain;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.util.ByteUtils;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
final class GeoPointLegacyDVAtomicFieldData extends AbstractAtomicGeoPointFieldData {
private static final int COORDINATE_SIZE = 8; // number of bytes per coordinate
private static final int GEOPOINT_SIZE = COORDINATE_SIZE * 2; // lat + lon
private final BinaryDocValues values;
GeoPointLegacyDVAtomicFieldData(BinaryDocValues values) {
super();
this.values = values;
}
@Override
public long ramBytesUsed() {
return 0; // not exposed by Lucene
}
@Override
public Collection<Accountable> getChildResources() {
return Collections.emptyList();
}
@Override
public void close() {
// no-op
}
@Override
public MultiGeoPointValues getGeoPointValues() {
return new MultiGeoPointValues() {
int count;
GeoPoint[] points = new GeoPoint[0];
@Override
public void setDocument(int docId) {
final BytesRef bytes = values.get(docId);
assert bytes.length % GEOPOINT_SIZE == 0;
count = (bytes.length >>> 4);
if (count > points.length) {
final int previousLength = points.length;
points = Arrays.copyOf(points, ArrayUtil.oversize(count, RamUsageEstimator.NUM_BYTES_OBJECT_REF));
for (int i = previousLength; i < points.length; ++i) {
points[i] = new GeoPoint(Double.NaN, Double.NaN);
}
}
for (int i = 0; i < count; ++i) {
final double lat = ByteUtils.readDoubleLE(bytes.bytes, bytes.offset + i * GEOPOINT_SIZE);
final double lon = ByteUtils.readDoubleLE(bytes.bytes, bytes.offset + i * GEOPOINT_SIZE + COORDINATE_SIZE);
points[i].reset(lat, lon);
}
}
@Override
public int count() {
return count;
}
@Override
public GeoPoint valueAt(int index) {
return points[index];
}
};
}
}

View File

@ -28,8 +28,6 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.mapper.RoutingFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.TTLFieldMapper;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.UidFieldMapper;
@ -52,8 +50,6 @@ import static org.elasticsearch.common.util.set.Sets.newHashSet;
public class FieldsVisitor extends StoredFieldVisitor {
private static final Set<String> BASE_REQUIRED_FIELDS = unmodifiableSet(newHashSet(
UidFieldMapper.NAME,
TimestampFieldMapper.NAME,
TTLFieldMapper.NAME,
RoutingFieldMapper.NAME,
ParentFieldMapper.NAME));

View File

@ -24,12 +24,9 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.logging.DeprecationLogger;
@ -46,7 +43,6 @@ import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -78,16 +74,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
public abstract static class Builder<T extends Builder, Y extends BaseGeoPointFieldMapper> extends FieldMapper.Builder<T, Y> {
protected boolean enableLatLon = Defaults.ENABLE_LATLON;
protected Integer precisionStep;
protected boolean enableGeoHash = Defaults.ENABLE_GEOHASH;
protected boolean enableGeoHashPrefix = Defaults.ENABLE_GEOHASH_PREFIX;
protected int geoHashPrecision = Defaults.GEO_HASH_PRECISION;
protected Boolean ignoreMalformed;
public Builder(String name, MappedFieldType fieldType) {
@ -99,31 +85,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
return (GeoPointFieldType)fieldType;
}
public T enableLatLon(boolean enableLatLon) {
this.enableLatLon = enableLatLon;
return builder;
}
public T precisionStep(int precisionStep) {
this.precisionStep = precisionStep;
return builder;
}
public T enableGeoHash(boolean enableGeoHash) {
this.enableGeoHash = enableGeoHash;
return builder;
}
public T geoHashPrefix(boolean enableGeoHashPrefix) {
this.enableGeoHashPrefix = enableGeoHashPrefix;
return builder;
}
public T geoHashPrecision(int precision) {
this.geoHashPrecision = precision;
return builder;
}
public T ignoreMalformed(boolean ignoreMalformed) {
this.ignoreMalformed = ignoreMalformed;
return builder;
@ -144,112 +105,21 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
FieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo);
public Y build(Mapper.BuilderContext context) {
// version 5.0 cuts over to LatLonPoint and no longer indexes geohash, or lat/lon separately
if (context.indexCreatedVersion().before(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
return buildLegacy(context);
}
return build(context, name, fieldType, defaultFieldType, context.indexSettings(),
null, null, null, multiFieldsBuilder.build(this, context), ignoreMalformed(context), copyTo);
}
private Y buildLegacy(Mapper.BuilderContext context) {
LegacyGeoPointFieldType geoPointFieldType = (LegacyGeoPointFieldType)fieldType;
FieldMapper latMapper = null;
FieldMapper lonMapper = null;
context.path().add(name);
if (enableLatLon) {
if (context.indexCreatedVersion().before(Version.V_5_0_0_alpha2)) {
LegacyNumberFieldMapper.Builder<?, ?> latMapperBuilder = new LegacyDoubleFieldMapper.Builder(Names.LAT).includeInAll(false);
LegacyNumberFieldMapper.Builder<?, ?> lonMapperBuilder = new LegacyDoubleFieldMapper.Builder(Names.LON).includeInAll(false);
if (precisionStep != null) {
latMapperBuilder.precisionStep(precisionStep);
lonMapperBuilder.precisionStep(precisionStep);
}
latMapper = (LegacyDoubleFieldMapper) latMapperBuilder.includeInAll(false).store(fieldType.stored()).docValues(false).build(context);
lonMapper = (LegacyDoubleFieldMapper) lonMapperBuilder.includeInAll(false).store(fieldType.stored()).docValues(false).build(context);
} else {
latMapper = new NumberFieldMapper.Builder(Names.LAT, NumberFieldMapper.NumberType.DOUBLE)
.includeInAll(false).store(fieldType.stored()).docValues(false).build(context);
lonMapper = new NumberFieldMapper.Builder(Names.LON, NumberFieldMapper.NumberType.DOUBLE)
.includeInAll(false).store(fieldType.stored()).docValues(false).build(context);
}
geoPointFieldType.setLatLonEnabled(latMapper.fieldType(), lonMapper.fieldType());
}
FieldMapper geoHashMapper = null;
if (enableGeoHash || enableGeoHashPrefix) {
// TODO: possible also implicitly enable geohash if geohash precision is set
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha1)) {
geoHashMapper = new KeywordFieldMapper.Builder(Names.GEOHASH)
.index(true).includeInAll(false).store(fieldType.stored()).build(context);
} else {
geoHashMapper = new StringFieldMapper.Builder(Names.GEOHASH)
.tokenized(false).index(true).omitNorms(true).indexOptions(IndexOptions.DOCS)
.includeInAll(false).store(fieldType.stored()).build(context);
}
geoPointFieldType.setGeoHashEnabled(geoHashMapper.fieldType(), geoHashPrecision, enableGeoHashPrefix);
}
context.path().remove();
return build(context, name, fieldType, defaultFieldType, context.indexSettings(),
latMapper, lonMapper, geoHashMapper, multiFieldsBuilder.build(this, context), ignoreMalformed(context), copyTo);
}
}
public abstract static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
Builder builder;
Version indexVersionCreated = parserContext.indexVersionCreated();
if (indexVersionCreated.before(Version.V_2_2_0)) {
builder = new LegacyGeoPointFieldMapper.Builder(name);
} else if (indexVersionCreated.onOrAfter(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
builder = new LatLonPointFieldMapper.Builder(name);
} else {
builder = new GeoPointFieldMapper.Builder(name);
}
Builder<?, ?> builder = new LatLonPointFieldMapper.Builder(name);
parseField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (indexVersionCreated.before(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
if (propName.equals("lat_lon")) {
deprecationLogger.deprecated(CONTENT_TYPE + " lat_lon parameter is deprecated and will be removed "
+ "in the next major release");
builder.enableLatLon(XContentMapValues.lenientNodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("precision_step")) {
deprecationLogger.deprecated(CONTENT_TYPE + " precision_step parameter is deprecated and will be removed "
+ "in the next major release");
builder.precisionStep(XContentMapValues.nodeIntegerValue(propNode));
iterator.remove();
} else if (propName.equals("geohash")) {
deprecationLogger.deprecated(CONTENT_TYPE + " geohash parameter is deprecated and will be removed "
+ "in the next major release");
builder.enableGeoHash(XContentMapValues.lenientNodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("geohash_prefix")) {
deprecationLogger.deprecated(CONTENT_TYPE + " geohash_prefix parameter is deprecated and will be removed "
+ "in the next major release");
builder.geoHashPrefix(XContentMapValues.lenientNodeBooleanValue(propNode));
if (XContentMapValues.lenientNodeBooleanValue(propNode)) {
builder.enableGeoHash(true);
}
iterator.remove();
} else if (propName.equals("geohash_precision")) {
deprecationLogger.deprecated(CONTENT_TYPE + " geohash_precision parameter is deprecated and will be removed "
+ "in the next major release");
if (propNode instanceof Integer) {
builder.geoHashPrecision(XContentMapValues.nodeIntegerValue(propNode));
} else {
builder.geoHashPrecision(GeoUtils.geoHashLevelsForPrecision(propNode.toString()));
}
iterator.remove();
}
}
if (propName.equals(Names.IGNORE_MALFORMED)) {
builder.ignoreMalformed(XContentMapValues.lenientNodeBooleanValue(propNode));
@ -257,19 +127,13 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
}
if (builder instanceof LegacyGeoPointFieldMapper.Builder) {
return LegacyGeoPointFieldMapper.parse((LegacyGeoPointFieldMapper.Builder) builder, node, parserContext);
} else if (builder instanceof LatLonPointFieldMapper.Builder) {
return (LatLonPointFieldMapper.Builder) builder;
}
return (GeoPointFieldMapper.Builder) builder;
return builder;
}
}
public abstract static class GeoPointFieldType extends MappedFieldType {
GeoPointFieldType() {
}
public static class GeoPointFieldType extends MappedFieldType {
GeoPointFieldType() {}
GeoPointFieldType(GeoPointFieldType ref) {
super(ref);
@ -279,110 +143,10 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
public String typeName() {
return CONTENT_TYPE;
}
}
public static class LegacyGeoPointFieldType extends GeoPointFieldType {
protected MappedFieldType geoHashFieldType;
protected int geoHashPrecision;
protected boolean geoHashPrefixEnabled;
protected MappedFieldType latFieldType;
protected MappedFieldType lonFieldType;
LegacyGeoPointFieldType() {}
LegacyGeoPointFieldType(LegacyGeoPointFieldType ref) {
super(ref);
this.geoHashFieldType = ref.geoHashFieldType; // copying ref is ok, this can never be modified
this.geoHashPrecision = ref.geoHashPrecision;
this.geoHashPrefixEnabled = ref.geoHashPrefixEnabled;
this.latFieldType = ref.latFieldType; // copying ref is ok, this can never be modified
this.lonFieldType = ref.lonFieldType; // copying ref is ok, this can never be modified
}
@Override
public MappedFieldType clone() {
return new LegacyGeoPointFieldType(this);
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) return false;
LegacyGeoPointFieldType that = (LegacyGeoPointFieldType) o;
return geoHashPrecision == that.geoHashPrecision &&
geoHashPrefixEnabled == that.geoHashPrefixEnabled &&
java.util.Objects.equals(geoHashFieldType, that.geoHashFieldType) &&
java.util.Objects.equals(latFieldType, that.latFieldType) &&
java.util.Objects.equals(lonFieldType, that.lonFieldType);
}
@Override
public int hashCode() {
return java.util.Objects.hash(super.hashCode(), geoHashFieldType, geoHashPrecision, geoHashPrefixEnabled, latFieldType,
lonFieldType);
}
@Override
public void checkCompatibility(MappedFieldType fieldType, List<String> conflicts, boolean strict) {
super.checkCompatibility(fieldType, conflicts, strict);
LegacyGeoPointFieldType other = (LegacyGeoPointFieldType)fieldType;
if (isLatLonEnabled() != other.isLatLonEnabled()) {
conflicts.add("mapper [" + name() + "] has different [lat_lon]");
}
if (isLatLonEnabled() && other.isLatLonEnabled() &&
latFieldType().numericPrecisionStep() != other.latFieldType().numericPrecisionStep()) {
conflicts.add("mapper [" + name() + "] has different [precision_step]");
}
if (isGeoHashEnabled() != other.isGeoHashEnabled()) {
conflicts.add("mapper [" + name() + "] has different [geohash]");
}
if (geoHashPrecision() != other.geoHashPrecision()) {
conflicts.add("mapper [" + name() + "] has different [geohash_precision]");
}
if (isGeoHashPrefixEnabled() != other.isGeoHashPrefixEnabled()) {
conflicts.add("mapper [" + name() + "] has different [geohash_prefix]");
}
}
public boolean isGeoHashEnabled() {
return geoHashFieldType != null;
}
public MappedFieldType geoHashFieldType() {
return geoHashFieldType;
}
public int geoHashPrecision() {
return geoHashPrecision;
}
public boolean isGeoHashPrefixEnabled() {
return geoHashPrefixEnabled;
}
public void setGeoHashEnabled(MappedFieldType geoHashFieldType, int geoHashPrecision, boolean geoHashPrefixEnabled) {
checkIfFrozen();
this.geoHashFieldType = geoHashFieldType;
this.geoHashPrecision = geoHashPrecision;
this.geoHashPrefixEnabled = geoHashPrefixEnabled;
}
public boolean isLatLonEnabled() {
return latFieldType != null;
}
public MappedFieldType latFieldType() {
return latFieldType;
}
public MappedFieldType lonFieldType() {
return lonFieldType;
}
public void setLatLonEnabled(MappedFieldType latFieldType, MappedFieldType lonFieldType) {
checkIfFrozen();
this.latFieldType = latFieldType;
this.lonFieldType = lonFieldType;
return new GeoPointFieldType(this);
}
@Override
@ -408,30 +172,15 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
}
protected FieldMapper latMapper;
protected FieldMapper lonMapper;
protected FieldMapper geoHashMapper;
protected Explicit<Boolean> ignoreMalformed;
protected BaseGeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings,
FieldMapper latMapper, FieldMapper lonMapper, FieldMapper geoHashMapper,
MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
this.latMapper = latMapper;
this.lonMapper = lonMapper;
this.geoHashMapper = geoHashMapper;
this.ignoreMalformed = ignoreMalformed;
}
public LegacyGeoPointFieldType legacyFieldType() {
return (LegacyGeoPointFieldType) super.fieldType();
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
@ -441,26 +190,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
}
@Override
public Iterator<Mapper> iterator() {
if (this instanceof LatLonPointFieldMapper == false) {
return Iterators.concat(super.iterator(), legacyIterator());
}
return super.iterator();
}
public Iterator<Mapper> legacyIterator() {
List<Mapper> extras = new ArrayList<>();
if (legacyFieldType().isGeoHashEnabled()) {
extras.add(geoHashMapper);
}
if (legacyFieldType().isLatLonEnabled()) {
extras.add(latMapper);
extras.add(lonMapper);
}
return extras.iterator();
}
@Override
protected String contentType() {
return CONTENT_TYPE;
@ -472,16 +201,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
}
protected void parse(ParseContext context, GeoPoint point, String geoHash) throws IOException {
if (legacyFieldType().isGeoHashEnabled()) {
if (geoHash == null) {
geoHash = GeoHashUtils.stringEncode(point.lon(), point.lat());
}
addGeoHashField(context, geoHash);
}
if (legacyFieldType().isLatLonEnabled()) {
latMapper.parse(context.createExternalValueContext(point.lat()));
lonMapper.parse(context.createExternalValueContext(point.lon()));
}
multiFields.parse(this, context.createExternalValueContext(point));
}
@ -552,17 +271,6 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
return null;
}
private void addGeoHashField(ParseContext context, String geoHash) throws IOException {
LegacyGeoPointFieldType ft = (LegacyGeoPointFieldType)fieldType;
int len = Math.min(ft.geoHashPrecision(), geoHash.length());
int min = ft.isGeoHashPrefixEnabled() ? 1 : len;
for (int i = len; i >= min; i--) {
// side effect of this call is adding the field
geoHashMapper.parse(context.createExternalValueContext(geoHash.substring(0, i)));
}
}
private void parsePointFromString(ParseContext context, GeoPoint sparse, String point) throws IOException {
if (point.indexOf(',') < 0) {
parse(context, sparse.resetFromGeoHash(point), point);
@ -574,51 +282,9 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (this instanceof LatLonPointFieldMapper == false) {
legacyDoXContentBody(builder, includeDefaults, params);
}
if (includeDefaults || ignoreMalformed.explicit()) {
builder.field(Names.IGNORE_MALFORMED, ignoreMalformed.value());
}
}
protected void legacyDoXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
LegacyGeoPointFieldType ft = (LegacyGeoPointFieldType) fieldType;
if (includeDefaults || ft.isLatLonEnabled() != GeoPointFieldMapper.Defaults.ENABLE_LATLON) {
builder.field("lat_lon", ft.isLatLonEnabled());
}
if (ft.isLatLonEnabled() && (includeDefaults || ft.latFieldType().numericPrecisionStep() != LegacyNumericUtils.PRECISION_STEP_DEFAULT)) {
builder.field("precision_step", ft.latFieldType().numericPrecisionStep());
}
if (includeDefaults || ft.isGeoHashEnabled() != Defaults.ENABLE_GEOHASH) {
builder.field("geohash", ft.isGeoHashEnabled());
}
if (includeDefaults || ft.isGeoHashPrefixEnabled() != Defaults.ENABLE_GEOHASH_PREFIX) {
builder.field("geohash_prefix", ft.isGeoHashPrefixEnabled());
}
if (ft.isGeoHashEnabled() && (includeDefaults || ft.geoHashPrecision() != Defaults.GEO_HASH_PRECISION)) {
builder.field("geohash_precision", ft.geoHashPrecision());
}
}
@Override
public FieldMapper updateFieldType(Map<String, MappedFieldType> fullNameToFieldType) {
BaseGeoPointFieldMapper updated = (BaseGeoPointFieldMapper) super.updateFieldType(fullNameToFieldType);
FieldMapper geoUpdated = geoHashMapper == null ? null : geoHashMapper.updateFieldType(fullNameToFieldType);
FieldMapper latUpdated = latMapper == null ? null : latMapper.updateFieldType(fullNameToFieldType);
FieldMapper lonUpdated = lonMapper == null ? null : lonMapper.updateFieldType(fullNameToFieldType);
if (updated == this
&& geoUpdated == geoHashMapper
&& latUpdated == latMapper
&& lonUpdated == lonMapper) {
return this;
}
if (updated == this) {
updated = (BaseGeoPointFieldMapper) updated.clone();
}
updated.geoHashMapper = geoUpdated;
updated.latMapper = latUpdated;
updated.lonMapper = lonUpdated;
return updated;
}
}

View File

@ -115,9 +115,6 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
@Override
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha1)) {
return new CompletionFieldMapper2x.TypeParser().parse(name, node, parserContext);
}
CompletionFieldMapper.Builder builder = new CompletionFieldMapper.Builder(name);
NamedAnalyzer indexAnalyzer = null;
NamedAnalyzer searchAnalyzer = null;

View File

@ -1,603 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.suggest.analyzing.XAnalyzingSuggester;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.NumberType;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.search.suggest.completion2x.AnalyzingCompletionLookupProvider;
import org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat;
import org.elasticsearch.search.suggest.completion2x.CompletionTokenStream;
import org.elasticsearch.search.suggest.completion2x.context.ContextBuilder;
import org.elasticsearch.search.suggest.completion2x.context.ContextMapping;
import org.elasticsearch.search.suggest.completion2x.context.ContextMapping.ContextConfig;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import static org.elasticsearch.index.mapper.TypeParsers.parseMultiField;
public class CompletionFieldMapper2x extends FieldMapper {
public static final String CONTENT_TYPE = "completion";
public static class Defaults {
public static final CompletionFieldType FIELD_TYPE = new CompletionFieldType();
static {
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.freeze();
}
public static final boolean DEFAULT_PRESERVE_SEPARATORS = true;
public static final boolean DEFAULT_POSITION_INCREMENTS = true;
public static final boolean DEFAULT_HAS_PAYLOADS = false;
public static final int DEFAULT_MAX_INPUT_LENGTH = 50;
}
public static class Fields {
// Mapping field names
public static final String ANALYZER = "analyzer";
public static final ParseField SEARCH_ANALYZER = new ParseField("search_analyzer");
public static final ParseField PRESERVE_SEPARATORS = new ParseField("preserve_separators");
public static final ParseField PRESERVE_POSITION_INCREMENTS = new ParseField("preserve_position_increments");
public static final String PAYLOADS = "payloads";
public static final String TYPE = "type";
public static final ParseField MAX_INPUT_LENGTH = new ParseField("max_input_length", "max_input_len");
// Content field names
public static final String CONTENT_FIELD_NAME_INPUT = "input";
public static final String CONTENT_FIELD_NAME_OUTPUT = "output";
public static final String CONTENT_FIELD_NAME_PAYLOAD = "payload";
public static final String CONTENT_FIELD_NAME_WEIGHT = "weight";
public static final String CONTEXT = "context";
}
public static final Set<String> ALLOWED_CONTENT_FIELD_NAMES;
static {
ALLOWED_CONTENT_FIELD_NAMES = new HashSet<>();
ALLOWED_CONTENT_FIELD_NAMES.add(Fields.CONTENT_FIELD_NAME_INPUT);
ALLOWED_CONTENT_FIELD_NAMES.add(Fields.CONTENT_FIELD_NAME_OUTPUT);
ALLOWED_CONTENT_FIELD_NAMES.add(Fields.CONTENT_FIELD_NAME_PAYLOAD);
ALLOWED_CONTENT_FIELD_NAMES.add(Fields.CONTENT_FIELD_NAME_WEIGHT);
ALLOWED_CONTENT_FIELD_NAMES.add(Fields.CONTEXT);
}
public static class Builder extends FieldMapper.Builder<Builder, CompletionFieldMapper2x> {
private boolean preserveSeparators = Defaults.DEFAULT_PRESERVE_SEPARATORS;
private boolean payloads = Defaults.DEFAULT_HAS_PAYLOADS;
private boolean preservePositionIncrements = Defaults.DEFAULT_POSITION_INCREMENTS;
private int maxInputLength = Defaults.DEFAULT_MAX_INPUT_LENGTH;
private SortedMap<String, ContextMapping> contextMapping = ContextMapping.EMPTY_MAPPING;
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE);
builder = this;
}
public Builder payloads(boolean payloads) {
this.payloads = payloads;
return this;
}
public Builder preserveSeparators(boolean preserveSeparators) {
this.preserveSeparators = preserveSeparators;
return this;
}
public Builder preservePositionIncrements(boolean preservePositionIncrements) {
this.preservePositionIncrements = preservePositionIncrements;
return this;
}
public Builder maxInputLength(int maxInputLength) {
if (maxInputLength <= 0) {
throw new IllegalArgumentException(
Fields.MAX_INPUT_LENGTH.getPreferredName() + " must be > 0 but was [" + maxInputLength + "]");
}
this.maxInputLength = maxInputLength;
return this;
}
public Builder contextMapping(SortedMap<String, ContextMapping> contextMapping) {
this.contextMapping = contextMapping;
return this;
}
@Override
public CompletionFieldMapper2x build(Mapper.BuilderContext context) {
setupFieldType(context);
CompletionFieldType completionFieldType = (CompletionFieldType) fieldType;
completionFieldType.setProvider(
new AnalyzingCompletionLookupProvider(preserveSeparators, preservePositionIncrements, payloads));
completionFieldType.setContextMapping(contextMapping);
return new CompletionFieldMapper2x(name, fieldType, maxInputLength, context.indexSettings(),
multiFieldsBuilder.build(this, context), copyTo);
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext)
throws MapperParsingException {
CompletionFieldMapper2x.Builder builder = new Builder(name);
NamedAnalyzer indexAnalyzer = null;
NamedAnalyzer searchAnalyzer = null;
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext(); ) {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = entry.getKey();
Object fieldNode = entry.getValue();
if (fieldName.equals("type")) {
continue;
}
if (Fields.ANALYZER.equals(fieldName) || // index_analyzer is for backcompat, remove for v3.0
fieldName.equals("index_analyzer") && parserContext.indexVersionCreated().before(Version.V_2_0_0_beta1)) {
indexAnalyzer = getNamedAnalyzer(parserContext, fieldNode.toString());
iterator.remove();
} else if (parserContext.parseFieldMatcher().match(fieldName, Fields.SEARCH_ANALYZER)) {
searchAnalyzer = getNamedAnalyzer(parserContext, fieldNode.toString());
iterator.remove();
} else if (fieldName.equals(Fields.PAYLOADS)) {
builder.payloads(Boolean.parseBoolean(fieldNode.toString()));
iterator.remove();
} else if (parserContext.parseFieldMatcher().match(fieldName, Fields.PRESERVE_SEPARATORS)) {
builder.preserveSeparators(Boolean.parseBoolean(fieldNode.toString()));
iterator.remove();
} else if (parserContext.parseFieldMatcher().match(fieldName, Fields.PRESERVE_POSITION_INCREMENTS)) {
builder.preservePositionIncrements(Boolean.parseBoolean(fieldNode.toString()));
iterator.remove();
} else if (parserContext.parseFieldMatcher().match(fieldName, Fields.MAX_INPUT_LENGTH)) {
builder.maxInputLength(Integer.parseInt(fieldNode.toString()));
iterator.remove();
} else if (parseMultiField(builder, name, parserContext, fieldName, fieldNode)) {
iterator.remove();
} else if (fieldName.equals(Fields.CONTEXT)) {
builder.contextMapping(ContextBuilder.loadMappings(fieldNode, parserContext.indexVersionCreated()));
iterator.remove();
}
}
if (indexAnalyzer == null) {
if (searchAnalyzer != null) {
throw new MapperParsingException(
"analyzer on completion field [" + name + "] must be set when search_analyzer is set");
}
indexAnalyzer = searchAnalyzer = parserContext.getIndexAnalyzers().get("simple");
} else if (searchAnalyzer == null) {
searchAnalyzer = indexAnalyzer;
}
builder.indexAnalyzer(indexAnalyzer);
builder.searchAnalyzer(searchAnalyzer);
return builder;
}
private NamedAnalyzer getNamedAnalyzer(ParserContext parserContext, String name) {
NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(name);
if (analyzer == null) {
throw new IllegalArgumentException("Can't find default or mapped analyzer with name [" + name + "]");
}
return analyzer;
}
}
public static final class CompletionFieldType extends TermBasedFieldType {
private PostingsFormat postingsFormat;
private AnalyzingCompletionLookupProvider analyzingSuggestLookupProvider;
private SortedMap<String, ContextMapping> contextMapping = ContextMapping.EMPTY_MAPPING;
public CompletionFieldType() {
}
protected CompletionFieldType(CompletionFieldType ref) {
super(ref);
this.postingsFormat = ref.postingsFormat;
this.analyzingSuggestLookupProvider = ref.analyzingSuggestLookupProvider;
this.contextMapping = ref.contextMapping;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (!(o instanceof CompletionFieldType)) return false;
if (!super.equals(o)) return false;
CompletionFieldType fieldType = (CompletionFieldType) o;
return analyzingSuggestLookupProvider.getPreserveSep() == fieldType.analyzingSuggestLookupProvider.getPreserveSep()
&& analyzingSuggestLookupProvider.getPreservePositionsIncrements() ==
fieldType.analyzingSuggestLookupProvider.getPreservePositionsIncrements() &&
analyzingSuggestLookupProvider.hasPayloads() == fieldType.analyzingSuggestLookupProvider.hasPayloads() &&
Objects.equals(getContextMapping(), fieldType.getContextMapping());
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(),
analyzingSuggestLookupProvider.getPreserveSep(),
analyzingSuggestLookupProvider.getPreservePositionsIncrements(),
analyzingSuggestLookupProvider.hasPayloads(),
getContextMapping());
}
@Override
public CompletionFieldType clone() {
return new CompletionFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public void checkCompatibility(MappedFieldType fieldType, List<String> conflicts, boolean strict) {
super.checkCompatibility(fieldType, conflicts, strict);
CompletionFieldType other = (CompletionFieldType) fieldType;
if (analyzingSuggestLookupProvider.hasPayloads() != other.analyzingSuggestLookupProvider.hasPayloads()) {
conflicts.add("mapper [" + name() + "] has different [payload] values");
}
if (analyzingSuggestLookupProvider.getPreservePositionsIncrements() !=
other.analyzingSuggestLookupProvider.getPreservePositionsIncrements()) {
conflicts.add("mapper [" + name() + "] has different [preserve_position_increments] values");
}
if (analyzingSuggestLookupProvider.getPreserveSep() != other.analyzingSuggestLookupProvider.getPreserveSep()) {
conflicts.add("mapper [" + name() + "] has different [preserve_separators] values");
}
if (!ContextMapping.mappingsAreEqual(getContextMapping(), other.getContextMapping())) {
conflicts.add("mapper [" + name() + "] has different [context_mapping] values");
}
}
public void setProvider(AnalyzingCompletionLookupProvider provider) {
checkIfFrozen();
this.analyzingSuggestLookupProvider = provider;
}
public synchronized PostingsFormat postingsFormat(PostingsFormat in) {
if (in instanceof Completion090PostingsFormat) {
throw new IllegalStateException("Double wrapping of " + Completion090PostingsFormat.class);
}
if (postingsFormat == null) {
postingsFormat = new Completion090PostingsFormat(in, analyzingSuggestLookupProvider);
}
return postingsFormat;
}
public void setContextMapping(SortedMap<String, ContextMapping> contextMapping) {
checkIfFrozen();
this.contextMapping = contextMapping;
}
/**
* Get the context mapping associated with this completion field
*/
public SortedMap<String, ContextMapping> getContextMapping() {
return contextMapping;
}
/**
* @return true if a context mapping has been defined
*/
public boolean requiresContext() {
return contextMapping.isEmpty() == false;
}
}
private static final BytesRef EMPTY = new BytesRef();
private int maxInputLength;
public CompletionFieldMapper2x(String simpleName, MappedFieldType fieldType, int maxInputLength,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, Defaults.FIELD_TYPE, indexSettings, multiFields, copyTo);
this.maxInputLength = maxInputLength;
}
@Override
public CompletionFieldType fieldType() {
return (CompletionFieldType) super.fieldType();
}
/**
* Parses and indexes inputs
* Parsing:
* Acceptable format:
* "STRING" - interpreted as field value (input)
* "ARRAY" - each element can be one of "OBJECT" (see below)
* "OBJECT" - { "input": STRING|ARRAY, "weight": STRING|INT, "contexts": ARRAY|OBJECT }
*/
@Override
public Mapper parse(ParseContext context) throws IOException {
XContentParser parser = context.parser();
XContentParser.Token token = parser.currentToken();
if (token == XContentParser.Token.VALUE_NULL) {
throw new MapperParsingException("completion field [" + fieldType().name() + "] does not support null values");
}
String surfaceForm = null;
BytesRef payload = null;
long weight = -1;
List<String> inputs = new ArrayList<>(4);
SortedMap<String, ContextConfig> contextConfig = null;
if (token == XContentParser.Token.VALUE_STRING) {
inputs.add(parser.text());
multiFields.parse(this, context);
} else {
String currentFieldName = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
if (!ALLOWED_CONTENT_FIELD_NAMES.contains(currentFieldName)) {
throw new IllegalArgumentException(
"Unknown field name[" + currentFieldName + "], must be one of " + ALLOWED_CONTENT_FIELD_NAMES);
}
} else if (Fields.CONTEXT.equals(currentFieldName)) {
SortedMap<String, ContextConfig> configs = new TreeMap<>();
if (token == Token.START_OBJECT) {
while ((token = parser.nextToken()) != Token.END_OBJECT) {
String name = parser.currentName();
ContextMapping mapping = fieldType().getContextMapping().get(name);
if (mapping == null) {
throw new ElasticsearchParseException("context [{}] is not defined", name);
} else {
token = parser.nextToken();
configs.put(name, mapping.parseContext(context, parser));
}
}
contextConfig = new TreeMap<>();
for (ContextMapping mapping : fieldType().getContextMapping().values()) {
ContextConfig config = configs.get(mapping.name());
contextConfig.put(mapping.name(), config == null ? mapping.defaultConfig() : config);
}
} else {
throw new ElasticsearchParseException("context must be an object");
}
} else if (Fields.CONTENT_FIELD_NAME_PAYLOAD.equals(currentFieldName)) {
if (!isStoringPayloads()) {
throw new MapperException("Payloads disabled in mapping");
}
if (token == XContentParser.Token.START_OBJECT) {
XContentBuilder payloadBuilder =
XContentFactory.contentBuilder(parser.contentType()).copyCurrentStructure(parser);
payload = payloadBuilder.bytes().toBytesRef();
payloadBuilder.close();
} else if (token.isValue()) {
payload = parser.utf8BytesOrNull();
} else {
throw new MapperException("payload doesn't support type " + token);
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if (Fields.CONTENT_FIELD_NAME_OUTPUT.equals(currentFieldName)) {
surfaceForm = parser.text();
}
if (Fields.CONTENT_FIELD_NAME_INPUT.equals(currentFieldName)) {
inputs.add(parser.text());
}
if (Fields.CONTENT_FIELD_NAME_WEIGHT.equals(currentFieldName)) {
Number weightValue;
try {
weightValue = Long.parseLong(parser.text());
} catch (NumberFormatException e) {
throw new IllegalArgumentException(
"Weight must be a string representing a numeric value, but was [" + parser.text() + "]");
}
weight = weightValue.longValue(); // always parse a long to make sure we don't get overflow
checkWeight(weight);
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (Fields.CONTENT_FIELD_NAME_WEIGHT.equals(currentFieldName)) {
NumberType numberType = parser.numberType();
if (NumberType.LONG != numberType && NumberType.INT != numberType) {
throw new IllegalArgumentException(
"Weight must be an integer, but was [" + parser.numberValue() + "]");
}
weight = parser.longValue(); // always parse a long to make sure we don't get overflow
checkWeight(weight);
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (Fields.CONTENT_FIELD_NAME_INPUT.equals(currentFieldName)) {
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
inputs.add(parser.text());
}
}
}
}
}
if (contextConfig == null) {
contextConfig = new TreeMap<>();
for (ContextMapping mapping : fieldType().getContextMapping().values()) {
contextConfig.put(mapping.name(), mapping.defaultConfig());
}
}
final ContextMapping.Context ctx = new ContextMapping.Context(contextConfig, context.doc());
payload = payload == null ? EMPTY : payload;
if (surfaceForm == null) { // no surface form use the input
for (String input : inputs) {
if (input.length() == 0) {
continue;
}
BytesRef suggestPayload = fieldType().analyzingSuggestLookupProvider.buildPayload(new BytesRef(
input), weight, payload);
context.doc().add(getCompletionField(ctx, input, suggestPayload));
}
} else {
BytesRef suggestPayload = fieldType().analyzingSuggestLookupProvider.buildPayload(new BytesRef(
surfaceForm), weight, payload);
for (String input : inputs) {
if (input.length() == 0) {
continue;
}
context.doc().add(getCompletionField(ctx, input, suggestPayload));
}
}
return null;
}
private void checkWeight(long weight) {
if (weight < 0 || weight > Integer.MAX_VALUE) {
throw new IllegalArgumentException("Weight must be in the interval [0..2147483647], but was [" + weight + "]");
}
}
public Field getCompletionField(ContextMapping.Context ctx, String input, BytesRef payload) {
final String originalInput = input;
if (input.length() > maxInputLength) {
final int len = correctSubStringLen(input, Math.min(maxInputLength, input.length()));
input = input.substring(0, len);
}
for (int i = 0; i < input.length(); i++) {
if (isReservedChar(input.charAt(i))) {
throw new IllegalArgumentException("Illegal input [" + originalInput + "] UTF-16 codepoint [0x"
+ Integer.toHexString(input.charAt(i)).toUpperCase(Locale.ROOT)
+ "] at position " + i + " is a reserved character");
}
}
return new SuggestField(
fieldType().name(), ctx, input, fieldType(), payload, fieldType().analyzingSuggestLookupProvider);
}
public static int correctSubStringLen(String input, int len) {
if (Character.isHighSurrogate(input.charAt(len - 1))) {
assert input.length() >= len + 1 && Character.isLowSurrogate(input.charAt(len));
return len + 1;
}
return len;
}
public BytesRef buildPayload(BytesRef surfaceForm, long weight, BytesRef payload) throws IOException {
return fieldType().analyzingSuggestLookupProvider.buildPayload(surfaceForm, weight, payload);
}
private static final class SuggestField extends Field {
private final BytesRef payload;
private final CompletionTokenStream.ToFiniteStrings toFiniteStrings;
private final ContextMapping.Context ctx;
public SuggestField(String name, ContextMapping.Context ctx,
String value, MappedFieldType type, BytesRef payload,
CompletionTokenStream.ToFiniteStrings toFiniteStrings) {
super(name, value, type);
this.payload = payload;
this.toFiniteStrings = toFiniteStrings;
this.ctx = ctx;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
TokenStream ts = ctx.wrapTokenStream(super.tokenStream(analyzer, previous));
return new CompletionTokenStream(ts, payload, toFiniteStrings);
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(simpleName())
.field(Fields.TYPE, CONTENT_TYPE);
builder.field(Fields.ANALYZER, fieldType().indexAnalyzer().name());
if (fieldType().indexAnalyzer().name().equals(fieldType().searchAnalyzer().name()) == false) {
builder.field(Fields.SEARCH_ANALYZER.getPreferredName(), fieldType().searchAnalyzer().name());
}
builder.field(Fields.PAYLOADS, fieldType().analyzingSuggestLookupProvider.hasPayloads());
builder.field(Fields.PRESERVE_SEPARATORS.getPreferredName(),
fieldType().analyzingSuggestLookupProvider.getPreserveSep());
builder.field(Fields.PRESERVE_POSITION_INCREMENTS.getPreferredName(),
fieldType().analyzingSuggestLookupProvider.getPreservePositionsIncrements());
builder.field(Fields.MAX_INPUT_LENGTH.getPreferredName(), this.maxInputLength);
multiFields.toXContent(builder, params);
if (fieldType().requiresContext()) {
builder.startObject(Fields.CONTEXT);
for (ContextMapping mapping : fieldType().getContextMapping().values()) {
builder.value(mapping);
}
builder.endObject();
}
return builder.endObject();
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
public boolean isStoringPayloads() {
return fieldType().analyzingSuggestLookupProvider.hasPayloads();
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
CompletionFieldMapper2x fieldMergeWith = (CompletionFieldMapper2x) mergeWith;
this.maxInputLength = fieldMergeWith.maxInputLength;
}
// this should be package private but our tests don't allow it.
public static boolean isReservedChar(char character) {
/* we use 0x001F as a SEP_LABEL in the suggester but we can use the UTF-16 representation since they
* are equivalent. We also don't need to convert the input character to UTF-8 here to check for
* the 0x00 end label since all multi-byte UTF-8 chars start with 0x10 binary so if the UTF-16 CP is == 0x00
* it's the single byte UTF-8 CP */
assert XAnalyzingSuggester.PAYLOAD_SEP == XAnalyzingSuggester.SEP_LABEL; // ensure they are the same!
switch (character) {
case XAnalyzingSuggester.END_BYTE:
case XAnalyzingSuggester.SEP_LABEL:
case XAnalyzingSuggester.HOLE_CHARACTER:
case ContextMapping.SEPARATOR:
return true;
default:
return false;
}
}
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
@ -42,7 +41,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.LegacyNumberFieldMapper.Defaults;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
@ -63,6 +61,10 @@ public class DateFieldMapper extends FieldMapper {
public static final FormatDateTimeFormatter DEFAULT_DATE_TIME_FORMATTER = Joda.forPattern(
"strict_date_optional_time||epoch_millis", Locale.ROOT);
public static class Defaults {
public static final Explicit<Boolean> IGNORE_MALFORMED = new Explicit<>(false, false);
}
public static class Builder extends FieldMapper.Builder<Builder, DateFieldMapper> {
private Boolean ignoreMalformed;
@ -128,9 +130,6 @@ public class DateFieldMapper extends FieldMapper {
@Override
public Mapper.Builder<?,?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha2)) {
return new LegacyDateFieldMapper.TypeParser().parse(name, node, parserContext);
}
Builder builder = new Builder(name);
TypeParsers.parseField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {

View File

@ -235,14 +235,6 @@ public class DocumentMapper implements ToXContent {
return metadataMapper(ParentFieldMapper.class);
}
public TimestampFieldMapper timestampFieldMapper() {
return metadataMapper(TimestampFieldMapper.class);
}
public TTLFieldMapper TTLFieldMapper() {
return metadataMapper(TTLFieldMapper.class);
}
public IndexFieldMapper IndexFieldMapper() {
return metadataMapper(IndexFieldMapper.class);
}

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.DynamicTemplate.XContentFieldType;
import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType;
import org.elasticsearch.index.mapper.StringFieldMapper.StringFieldType;
import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType;
import java.io.IOException;
@ -153,8 +152,6 @@ final class DocumentParser {
context.sourceToParse().id(),
context.sourceToParse().type(),
source.routing(),
source.timestamp(),
source.ttl(),
context.docs(),
context.sourceToParse().source(),
update
@ -636,35 +633,19 @@ final class DocumentParser {
}
private static Mapper.Builder<?, ?> newLongBuilder(String name, Version indexCreated) {
if (indexCreated.onOrAfter(Version.V_5_0_0_alpha2)) {
return new NumberFieldMapper.Builder(name, NumberFieldMapper.NumberType.LONG);
} else {
return new LegacyLongFieldMapper.Builder(name);
}
return new NumberFieldMapper.Builder(name, NumberFieldMapper.NumberType.LONG);
}
private static Mapper.Builder<?, ?> newFloatBuilder(String name, Version indexCreated) {
if (indexCreated.onOrAfter(Version.V_5_0_0_alpha2)) {
return new NumberFieldMapper.Builder(name, NumberFieldMapper.NumberType.FLOAT);
} else {
return new LegacyFloatFieldMapper.Builder(name);
}
return new NumberFieldMapper.Builder(name, NumberFieldMapper.NumberType.FLOAT);
}
private static Mapper.Builder<?, ?> newDateBuilder(String name, FormatDateTimeFormatter dateTimeFormatter, Version indexCreated) {
if (indexCreated.onOrAfter(Version.V_5_0_0_alpha2)) {
DateFieldMapper.Builder builder = new DateFieldMapper.Builder(name);
if (dateTimeFormatter != null) {
builder.dateTimeFormatter(dateTimeFormatter);
}
return builder;
} else {
LegacyDateFieldMapper.Builder builder = new LegacyDateFieldMapper.Builder(name);
if (dateTimeFormatter != null) {
builder.dateTimeFormatter(dateTimeFormatter);
}
return builder;
DateFieldMapper.Builder builder = new DateFieldMapper.Builder(name);
if (dateTimeFormatter != null) {
builder.dateTimeFormatter(dateTimeFormatter);
}
return builder;
}
private static Mapper.Builder<?,?> createBuilderFromDynamicValue(final ParseContext context, XContentParser.Token token, String currentFieldName) throws IOException {

View File

@ -1,135 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.elasticsearch.Version;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.settings.Settings;
import java.io.IOException;
import java.util.Map;
/**
* Parsing: We handle:
* <p>
* - "field" : "geo_hash"
* - "field" : "lat,lon"
* - "field" : {
* "lat" : 1.1,
* "lon" : 2.1
* }
*/
public class GeoPointFieldMapper extends BaseGeoPointFieldMapper {
public static final String CONTENT_TYPE = "geo_point";
public static class Defaults extends BaseGeoPointFieldMapper.Defaults {
public static final GeoPointFieldType FIELD_TYPE = new LegacyGeoPointFieldType();
static {
FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
FIELD_TYPE.setTokenized(false);
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.freeze();
}
}
/**
* Concrete builder for indexed GeoPointField type
*/
public static class Builder extends BaseGeoPointFieldMapper.Builder<Builder, GeoPointFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE);
this.builder = this;
}
@Override
public GeoPointFieldMapper build(BuilderContext context, String simpleName, MappedFieldType fieldType,
MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper,
FieldMapper lonMapper, FieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed,
CopyTo copyTo) {
fieldType.setTokenized(false);
if (context.indexCreatedVersion().before(Version.V_2_3_0)) {
fieldType.setNumericPrecisionStep(GeoPointField.PRECISION_STEP);
fieldType.setNumericType(FieldType.LegacyNumericType.LONG);
}
setupFieldType(context);
return new GeoPointFieldMapper(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper,
geoHashMapper, multiFields, ignoreMalformed, copyTo);
}
@Override
public GeoPointFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().before(Version.V_2_3_0)) {
fieldType.setNumericPrecisionStep(GeoPointField.PRECISION_STEP);
fieldType.setNumericType(FieldType.LegacyNumericType.LONG);
}
return super.build(context);
}
}
public static class TypeParser extends BaseGeoPointFieldMapper.TypeParser {
@Override
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
return super.parse(name, node, parserContext);
}
}
public GeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings,
FieldMapper latMapper, FieldMapper lonMapper,
FieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields,
ignoreMalformed, copyTo);
}
@Override
protected void parse(ParseContext context, GeoPoint point, String geoHash) throws IOException {
if (ignoreMalformed.value() == false) {
if (point.lat() > 90.0 || point.lat() < -90.0) {
throw new IllegalArgumentException("illegal latitude value [" + point.lat() + "] for " + name());
}
if (point.lon() > 180.0 || point.lon() < -180) {
throw new IllegalArgumentException("illegal longitude value [" + point.lon() + "] for " + name());
}
} else {
// LUCENE WATCH: This will be folded back into Lucene's GeoPointField
GeoUtils.normalizePoint(point);
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
context.doc().add(new GeoPointField(fieldType().name(), point.lat(), point.lon(), fieldType()));
}
super.parse(context, point, geoHash);
}
@Override
public LegacyGeoPointFieldType fieldType() {
return (LegacyGeoPointFieldType) super.fieldType();
}
}

View File

@ -39,7 +39,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.LegacyNumberFieldMapper.Defaults;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
@ -55,6 +54,10 @@ public class IpFieldMapper extends FieldMapper {
public static final String CONTENT_TYPE = "ip";
public static class Defaults {
public static final Explicit<Boolean> IGNORE_MALFORMED = new Explicit<>(false, false);
}
public static class Builder extends FieldMapper.Builder<Builder, IpFieldMapper> {
private Boolean ignoreMalformed;
@ -94,9 +97,6 @@ public class IpFieldMapper extends FieldMapper {
@Override
public Mapper.Builder<?,?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha2)) {
return new LegacyIpFieldMapper.TypeParser().parse(name, node, parserContext);
}
Builder builder = new Builder(name);
TypeParsers.parseField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {

View File

@ -25,7 +25,6 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -34,14 +33,10 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static java.util.Collections.unmodifiableList;
import static org.elasticsearch.index.mapper.TypeParsers.parseField;
/**
@ -51,12 +46,6 @@ public final class KeywordFieldMapper extends FieldMapper {
public static final String CONTENT_TYPE = "keyword";
private static final List<String> SUPPORTED_PARAMETERS_FOR_AUTO_DOWNGRADE_TO_STRING = unmodifiableList(Arrays.asList(
"type",
// common keyword parameters, for which the upgrade is straightforward
"index", "store", "doc_values", "omit_norms", "norms", "boost", "fields", "copy_to",
"include_in_all", "ignore_above", "index_options", "similarity"));
public static class Defaults {
public static final MappedFieldType FIELD_TYPE = new KeywordFieldType();
@ -115,29 +104,6 @@ public final class KeywordFieldMapper extends FieldMapper {
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha1)) {
// Downgrade "keyword" to "string" in indexes created in 2.x so you can use modern syntax against old indexes
Set<String> unsupportedParameters = new HashSet<>(node.keySet());
unsupportedParameters.removeAll(SUPPORTED_PARAMETERS_FOR_AUTO_DOWNGRADE_TO_STRING);
if (false == SUPPORTED_PARAMETERS_FOR_AUTO_DOWNGRADE_TO_STRING.containsAll(node.keySet())) {
throw new IllegalArgumentException("Automatic downgrade from [keyword] to [string] failed because parameters "
+ unsupportedParameters + " are not supported for automatic downgrades.");
}
{ // Downgrade "index"
Object index = node.get("index");
if (index == null || Boolean.TRUE.equals(index)) {
index = "not_analyzed";
} else if (Boolean.FALSE.equals(index)) {
index = "no";
} else {
throw new IllegalArgumentException(
"Can't parse [index] value [" + index + "] for field [" + name + "], expected [true] or [false]");
}
node.put("index", index);
}
return new StringFieldMapper.TypeParser().parse(name, node, parserContext);
}
KeywordFieldMapper.Builder builder = new KeywordFieldMapper.Builder(name);
parseField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {

View File

@ -23,7 +23,6 @@ import org.apache.lucene.document.LatLonPoint;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.search.Query;
import org.elasticsearch.Version;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
@ -43,7 +42,6 @@ import java.util.Map;
*/
public class LatLonPointFieldMapper extends BaseGeoPointFieldMapper {
public static final String CONTENT_TYPE = "geo_point";
public static final Version LAT_LON_FIELD_VERSION = Version.V_5_0_0_beta1;
public static class Defaults extends BaseGeoPointFieldMapper.Defaults {
public static final LatLonPointFieldType FIELD_TYPE = new LatLonPointFieldType();
@ -59,6 +57,7 @@ public class LatLonPointFieldMapper extends BaseGeoPointFieldMapper {
public static class Builder extends BaseGeoPointFieldMapper.Builder<Builder, LatLonPointFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE);
builder = this;
}
@Override

View File

@ -1,328 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeByteValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyByteFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "byte";
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final MappedFieldType FIELD_TYPE = new ByteFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyByteFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_8_BIT);
builder = this;
}
@Override
public LegacyByteFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyByteFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context),
coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 32;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyByteFieldMapper.Builder builder = new LegacyByteFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(nodeByteValue(propNode));
iterator.remove();
}
}
return builder;
}
}
static final class ByteFieldType extends NumberFieldType {
public ByteFieldType() {
super(LegacyNumericType.INT);
}
protected ByteFieldType(ByteFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new ByteFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public Byte nullValue() {
return (Byte)super.nullValue();
}
@Override
public Byte valueForDisplay(Object value) {
if (value == null) {
return null;
}
return ((Number) value).byteValue();
}
@Override
public BytesRef indexedValueForSearch(Object value) {
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.intToPrefixCoded(parseValue(value), 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats.Long stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
long minValue = LegacyNumericUtils.getMinInt(terms);
long maxValue = LegacyNumericUtils.getMaxInt(terms);
return new FieldStats.Long(maxDoc, terms.getDocCount(),
terms.getSumDocFreq(), terms.getSumTotalTermFreq(), isSearchable(), isAggregatable(),
minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.BYTE);
}
}
protected LegacyByteFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public ByteFieldType fieldType() {
return (ByteFieldType) super.fieldType();
}
private static byte parseValue(Object value) {
if (value instanceof Number) {
return ((Number) value).byteValue();
}
if (value instanceof BytesRef) {
return Byte.parseByte(((BytesRef) value).utf8ToString());
}
return Byte.parseByte(value.toString());
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
byte value;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
if (externalValue == null) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else if (externalValue instanceof String) {
String sExternalValue = (String) externalValue;
if (sExternalValue.length() == 0) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else {
value = Byte.parseByte(sExternalValue);
}
} else {
value = ((Number) externalValue).byteValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), Byte.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL ||
(parser.currentToken() == XContentParser.Token.VALUE_STRING && parser.textLength() == 0)) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
Byte objValue = fieldType().nullValue();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (parser.currentToken() != XContentParser.Token.VALUE_NULL) {
objValue = (byte) parser.shortValue(coerce.value());
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
if (objValue == null) {
// no value
return;
}
value = objValue;
} else {
value = (byte) parser.shortValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomByteNumericField field = new CustomByteNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, value);
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_8_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
public static class CustomByteNumericField extends CustomNumericField {
private final byte number;
public CustomByteNumericField(byte number, MappedFieldType fieldType) {
super(number, fieldType);
this.number = number;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
if (fieldType().indexOptions() != IndexOptions.NONE) {
return getCachedStream().setIntValue(number);
}
return null;
}
@Override
public String numericAsString() {
return Byte.toString(number);
}
}
}

View File

@ -1,529 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.joda.DateMathParser;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.LocaleUtils;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.LegacyLongFieldMapper.CustomLongNumericField;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.index.mapper.TypeParsers.parseDateTimeFormatter;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyDateFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "date";
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern("strict_date_optional_time||epoch_millis", Locale.ROOT);
public static final TimeUnit TIME_UNIT = TimeUnit.MILLISECONDS;
public static final DateFieldType FIELD_TYPE = new DateFieldType();
static {
FIELD_TYPE.freeze();
}
public static final String NULL_VALUE = null;
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyDateFieldMapper> {
protected String nullValue = Defaults.NULL_VALUE;
private Locale locale;
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_64_BIT);
builder = this;
// do *NOT* rely on the default locale
locale = Locale.ROOT;
}
@Override
public DateFieldType fieldType() {
return (DateFieldType)fieldType;
}
public Builder timeUnit(TimeUnit timeUnit) {
fieldType().setTimeUnit(timeUnit);
return this;
}
public Builder nullValue(String nullValue) {
this.nullValue = nullValue;
return this;
}
public Builder dateTimeFormatter(FormatDateTimeFormatter dateTimeFormatter) {
fieldType().setDateTimeFormatter(dateTimeFormatter);
return this;
}
@Override
public LegacyDateFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
fieldType.setNullValue(nullValue);
return new LegacyDateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context),
coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected void setupFieldType(BuilderContext context) {
FormatDateTimeFormatter dateTimeFormatter = fieldType().dateTimeFormatter;
if (!locale.equals(dateTimeFormatter.locale())) {
fieldType().setDateTimeFormatter(new FormatDateTimeFormatter(dateTimeFormatter.format(), dateTimeFormatter.parser(), dateTimeFormatter.printer(), locale));
}
super.setupFieldType(context);
}
public Builder locale(Locale locale) {
this.locale = locale;
return this;
}
@Override
protected int maxPrecisionStep() {
return 64;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyDateFieldMapper.Builder builder = new LegacyDateFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
boolean configuredFormat = false;
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(propNode.toString());
iterator.remove();
} else if (propName.equals("format")) {
builder.dateTimeFormatter(parseDateTimeFormatter(propNode));
configuredFormat = true;
iterator.remove();
} else if (propName.equals("numeric_resolution")) {
builder.timeUnit(TimeUnit.valueOf(propNode.toString().toUpperCase(Locale.ROOT)));
iterator.remove();
} else if (propName.equals("locale")) {
builder.locale(LocaleUtils.parse(propNode.toString()));
iterator.remove();
}
}
if (!configuredFormat) {
builder.dateTimeFormatter(Defaults.DATE_TIME_FORMATTER);
}
return builder;
}
}
public static class DateFieldType extends NumberFieldType {
protected FormatDateTimeFormatter dateTimeFormatter = Defaults.DATE_TIME_FORMATTER;
protected TimeUnit timeUnit = Defaults.TIME_UNIT;
protected DateMathParser dateMathParser = new DateMathParser(dateTimeFormatter);
public DateFieldType() {
super(LegacyNumericType.LONG);
}
protected DateFieldType(DateFieldType ref) {
super(ref);
this.dateTimeFormatter = ref.dateTimeFormatter;
this.timeUnit = ref.timeUnit;
this.dateMathParser = ref.dateMathParser;
}
@Override
public DateFieldType clone() {
return new DateFieldType(this);
}
@Override
public boolean equals(Object o) {
if (!super.equals(o)) return false;
DateFieldType that = (DateFieldType) o;
return Objects.equals(dateTimeFormatter.format(), that.dateTimeFormatter.format()) &&
Objects.equals(dateTimeFormatter.locale(), that.dateTimeFormatter.locale()) &&
Objects.equals(timeUnit, that.timeUnit);
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), dateTimeFormatter.format(), timeUnit);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public void checkCompatibility(MappedFieldType fieldType, List<String> conflicts, boolean strict) {
super.checkCompatibility(fieldType, conflicts, strict);
if (strict) {
DateFieldType other = (DateFieldType)fieldType;
if (Objects.equals(dateTimeFormatter().format(), other.dateTimeFormatter().format()) == false) {
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 [" + 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 [" + name() + "] is used by multiple types. Set update_all_types to true to update [numeric_resolution] across all types.");
}
}
}
public FormatDateTimeFormatter dateTimeFormatter() {
return dateTimeFormatter;
}
public void setDateTimeFormatter(FormatDateTimeFormatter dateTimeFormatter) {
checkIfFrozen();
this.dateTimeFormatter = dateTimeFormatter;
this.dateMathParser = new DateMathParser(dateTimeFormatter);
}
public TimeUnit timeUnit() {
return timeUnit;
}
public void setTimeUnit(TimeUnit timeUnit) {
checkIfFrozen();
this.timeUnit = timeUnit;
this.dateMathParser = new DateMathParser(dateTimeFormatter);
}
protected DateMathParser dateMathParser() {
return dateMathParser;
}
private long parseValue(Object value) {
if (value instanceof Number) {
return ((Number) value).longValue();
}
if (value instanceof BytesRef) {
return dateTimeFormatter().parser().parseMillis(((BytesRef) value).utf8ToString());
}
return dateTimeFormatter().parser().parseMillis(value.toString());
}
protected long parseStringValue(String value) {
return dateTimeFormatter().parser().parseMillis(value);
}
@Override
public BytesRef indexedValueForSearch(Object value) {
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.longToPrefixCoded(parseValue(value), 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Object valueForDisplay(Object value) {
Long val = (Long) value;
if (val == null) {
return null;
}
return dateTimeFormatter().printer().print(val);
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return rangeQuery(lowerTerm, upperTerm, includeLower, includeUpper, null, null, context);
}
@Override
public FieldStats.Date stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
long minValue = LegacyNumericUtils.getMinLong(terms);
long maxValue = LegacyNumericUtils.getMaxLong(terms);
return new FieldStats.Date(maxDoc, terms.getDocCount(),
terms.getSumDocFreq(), terms.getSumTotalTermFreq(), isSearchable(), isAggregatable(),
dateTimeFormatter(), minValue, maxValue);
}
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper,
@Nullable DateTimeZone timeZone, @Nullable DateMathParser forcedDateParser, QueryShardContext context) {
return innerRangeQuery(lowerTerm, upperTerm, includeLower, includeUpper, timeZone, forcedDateParser, context);
}
private Query innerRangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper,
@Nullable DateTimeZone timeZone, @Nullable DateMathParser forcedDateParser, QueryRewriteContext context) {
return LegacyNumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
lowerTerm == null ? null
: parseToMilliseconds(lowerTerm, !includeLower, timeZone,
forcedDateParser == null ? dateMathParser : forcedDateParser, context),
upperTerm == null ? null
: parseToMilliseconds(upperTerm, includeUpper, timeZone,
forcedDateParser == null ? dateMathParser : forcedDateParser, context),
includeLower, includeUpper);
}
@Override
public Relation isFieldWithinQuery(IndexReader reader,
Object from, Object to,
boolean includeLower, boolean includeUpper,
DateTimeZone timeZone, DateMathParser dateParser, QueryRewriteContext context) throws IOException {
if (dateParser == null) {
dateParser = this.dateMathParser;
}
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
// no terms, so nothing matches
return Relation.DISJOINT;
}
long minValue = LegacyNumericUtils.getMinLong(terms);
long maxValue = LegacyNumericUtils.getMaxLong(terms);
long fromInclusive = Long.MIN_VALUE;
if (from != null) {
fromInclusive = parseToMilliseconds(from, !includeLower, timeZone, dateParser, context);
if (includeLower == false) {
if (fromInclusive == Long.MAX_VALUE) {
return Relation.DISJOINT;
}
++fromInclusive;
}
}
long toInclusive = Long.MAX_VALUE;
if (to != null) {
toInclusive = parseToMilliseconds(to, includeUpper, timeZone, dateParser, context);
if (includeUpper == false) {
if (toInclusive == Long.MIN_VALUE) {
return Relation.DISJOINT;
}
--toInclusive;
}
}
if (minValue >= fromInclusive && maxValue <= toInclusive) {
return Relation.WITHIN;
} else if (maxValue < fromInclusive || minValue > toInclusive) {
return Relation.DISJOINT;
} else {
return Relation.INTERSECTS;
}
}
public long parseToMilliseconds(Object value, boolean inclusive, @Nullable DateTimeZone zone,
@Nullable DateMathParser forcedDateParser, QueryRewriteContext context) {
if (value instanceof Long) {
return ((Long) value).longValue();
}
DateMathParser dateParser = dateMathParser();
if (forcedDateParser != null) {
dateParser = forcedDateParser;
}
String strValue;
if (value instanceof BytesRef) {
strValue = ((BytesRef) value).utf8ToString();
} else {
strValue = value.toString();
}
return dateParser.parse(strValue, context::nowInMillis, inclusive, zone);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.LONG);
}
@Override
public DocValueFormat docValueFormat(@Nullable String format, DateTimeZone timeZone) {
FormatDateTimeFormatter dateTimeFormatter = this.dateTimeFormatter;
if (format != null) {
dateTimeFormatter = Joda.forPattern(format);
}
if (timeZone == null) {
timeZone = DateTimeZone.UTC;
}
return new DocValueFormat.DateTime(dateTimeFormatter, timeZone);
}
}
protected LegacyDateFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit<Boolean> ignoreMalformed,Explicit<Boolean> coerce,
Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public DateFieldType fieldType() {
return (DateFieldType) super.fieldType();
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
String dateAsString = null;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
dateAsString = (String) externalValue;
if (dateAsString == null) {
dateAsString = fieldType().nullValueAsString();
}
} else {
XContentParser parser = context.parser();
XContentParser.Token token = parser.currentToken();
if (token == XContentParser.Token.VALUE_NULL) {
dateAsString = fieldType().nullValueAsString();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
dateAsString = parser.text();
} else if (token == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
String currentFieldName = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (token == XContentParser.Token.VALUE_NULL) {
dateAsString = fieldType().nullValueAsString();
} else {
dateAsString = parser.text();
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
} else {
dateAsString = parser.text();
}
}
Long value = null;
if (dateAsString != null) {
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), dateAsString, boost);
}
value = fieldType().parseStringValue(dateAsString);
}
if (value != null) {
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomLongNumericField field = new CustomLongNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, value);
}
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_64_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
builder.field("format", fieldType().dateTimeFormatter().format());
if (includeDefaults || fieldType().nullValueAsString() != null) {
builder.field("null_value", fieldType().nullValueAsString());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
if (includeDefaults || fieldType().timeUnit() != Defaults.TIME_UNIT) {
builder.field("numeric_resolution", fieldType().timeUnit().name().toLowerCase(Locale.ROOT));
}
// only serialize locale if needed, ROOT is the default, so no need to serialize that case as well...
if (fieldType().dateTimeFormatter().locale() != null && fieldType().dateTimeFormatter().locale() != Locale.ROOT) {
builder.field("locale", fieldType().dateTimeFormatter().locale());
} else if (includeDefaults) {
if (fieldType().dateTimeFormatter().locale() == null) {
builder.field("locale", Locale.ROOT);
} else {
builder.field("locale", fieldType().dateTimeFormatter().locale());
}
}
}
}

View File

@ -1,331 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Numbers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeDoubleValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyDoubleFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "double";
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final MappedFieldType FIELD_TYPE = new DoubleFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyDoubleFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_64_BIT);
builder = this;
}
@Override
public LegacyDoubleFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyDoubleFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context),
includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 64;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyDoubleFieldMapper.Builder builder = new LegacyDoubleFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("nullValue") || propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(nodeDoubleValue(propNode));
iterator.remove();
}
}
return builder;
}
}
public static final class DoubleFieldType extends NumberFieldType {
public DoubleFieldType() {
super(LegacyNumericType.DOUBLE);
}
protected DoubleFieldType(DoubleFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new DoubleFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public java.lang.Double nullValue() {
return (java.lang.Double)super.nullValue();
}
@Override
public java.lang.Double valueForDisplay(Object value) {
if (value == null) {
return null;
}
if (value instanceof Number) {
return ((Number) value).doubleValue();
}
if (value instanceof BytesRef) {
return Numbers.bytesToDouble((BytesRef) value);
}
return java.lang.Double.parseDouble(value.toString());
}
@Override
public BytesRef indexedValueForSearch(Object value) {
long longValue = NumericUtils.doubleToSortableLong(parseDoubleValue(value));
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.longToPrefixCoded(longValue, 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newDoubleRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseDoubleValue(lowerTerm),
upperTerm == null ? null : parseDoubleValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats.Double stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
double minValue = NumericUtils.sortableLongToDouble(LegacyNumericUtils.getMinLong(terms));
double maxValue = NumericUtils.sortableLongToDouble(LegacyNumericUtils.getMaxLong(terms));
return new FieldStats.Double(maxDoc, terms.getDocCount(),
terms.getSumDocFreq(), terms.getSumTotalTermFreq(), isSearchable(), isAggregatable(),
minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.DOUBLE);
}
}
protected LegacyDoubleFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit<Boolean> ignoreMalformed,
Explicit<Boolean> coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public DoubleFieldType fieldType() {
return (DoubleFieldType) super.fieldType();
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
double value;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
if (externalValue == null) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else if (externalValue instanceof String) {
String sExternalValue = (String) externalValue;
if (sExternalValue.length() == 0) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else {
value = java.lang.Double.parseDouble(sExternalValue);
}
} else {
value = ((Number) externalValue).doubleValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), java.lang.Double.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL ||
(parser.currentToken() == XContentParser.Token.VALUE_STRING && parser.textLength() == 0)) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
java.lang.Double objValue = fieldType().nullValue();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (parser.currentToken() != XContentParser.Token.VALUE_NULL) {
objValue = parser.doubleValue(coerce.value());
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
if (objValue == null) {
// no value
return;
}
value = objValue;
} else {
value = parser.doubleValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomDoubleNumericField field = new CustomDoubleNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, NumericUtils.doubleToSortableLong(value));
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_64_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
public static class CustomDoubleNumericField extends CustomNumericField {
private final double number;
public CustomDoubleNumericField(double number, NumberFieldType fieldType) {
super(number, fieldType);
this.number = number;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
if (fieldType().indexOptions() != IndexOptions.NONE) {
return getCachedStream().setDoubleValue(number);
}
return null;
}
@Override
public String numericAsString() {
return java.lang.Double.toString(number);
}
}
}

View File

@ -1,325 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeFloatValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyFloatFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "float";
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final MappedFieldType FIELD_TYPE = new FloatFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyFloatFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_32_BIT);
builder = this;
}
@Override
public LegacyFloatFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyFloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context),
includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 32;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyFloatFieldMapper.Builder builder = new LegacyFloatFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(nodeFloatValue(propNode));
iterator.remove();
}
}
return builder;
}
}
static final class FloatFieldType extends NumberFieldType {
public FloatFieldType() {
super(LegacyNumericType.FLOAT);
}
protected FloatFieldType(FloatFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new FloatFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public Float nullValue() {
return (Float)super.nullValue();
}
@Override
public BytesRef indexedValueForSearch(Object value) {
int intValue = NumericUtils.floatToSortableInt(parseValue(value));
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.intToPrefixCoded(intValue, 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newFloatRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats.Double stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
float minValue = NumericUtils.sortableIntToFloat(LegacyNumericUtils.getMinInt(terms));
float maxValue = NumericUtils.sortableIntToFloat(LegacyNumericUtils.getMaxInt(terms));
return new FieldStats.Double(maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(),
isSearchable(), isAggregatable(), minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.FLOAT);
}
}
protected LegacyFloatFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public FloatFieldType fieldType() {
return (FloatFieldType) super.fieldType();
}
private static float parseValue(Object value) {
if (value instanceof Number) {
return ((Number) value).floatValue();
}
if (value instanceof BytesRef) {
return Float.parseFloat(((BytesRef) value).utf8ToString());
}
return Float.parseFloat(value.toString());
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
float value;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
if (externalValue == null) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else if (externalValue instanceof String) {
String sExternalValue = (String) externalValue;
if (sExternalValue.length() == 0) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else {
value = Float.parseFloat(sExternalValue);
}
} else {
value = ((Number) externalValue).floatValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), Float.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL ||
(parser.currentToken() == XContentParser.Token.VALUE_STRING && parser.textLength() == 0)) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
Float objValue = fieldType().nullValue();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (parser.currentToken() != XContentParser.Token.VALUE_NULL) {
objValue = parser.floatValue(coerce.value());
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
if (objValue == null) {
// no value
return;
}
value = objValue;
} else {
value = parser.floatValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomFloatNumericField field = new CustomFloatNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, NumericUtils.floatToSortableInt(value));
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_32_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
public static class CustomFloatNumericField extends CustomNumericField {
private final float number;
public CustomFloatNumericField(float number, NumberFieldType fieldType) {
super(number, fieldType);
this.number = number;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
if (fieldType().indexOptions() != IndexOptions.NONE) {
return getCachedStream().setFloatValue(number);
}
return null;
}
@Override
public String numericAsString() {
return Float.toString(number);
}
}
}

View File

@ -1,367 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import com.carrotsearch.hppc.ObjectHashSet;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.util.ByteUtils;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
/**
* Parsing: We handle:
* <p>
* - "field" : "geo_hash"
* - "field" : "lat,lon"
* - "field" : {
* "lat" : 1.1,
* "lon" : 2.1
* }
*/
public class LegacyGeoPointFieldMapper extends BaseGeoPointFieldMapper implements ArrayValueMapperParser {
public static final String CONTENT_TYPE = "geo_point";
public static class Names extends BaseGeoPointFieldMapper.Names {
public static final String COERCE = "coerce";
}
public static class Defaults extends BaseGeoPointFieldMapper.Defaults{
public static final Explicit<Boolean> COERCE = new Explicit<>(false, false);
public static final GeoPointFieldType FIELD_TYPE = new LegacyGeoPointFieldType();
static {
FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
FIELD_TYPE.setTokenized(false);
FIELD_TYPE.setOmitNorms(true);
FIELD_TYPE.freeze();
}
}
/**
* Concrete builder for legacy GeoPointField
*/
public static class Builder extends BaseGeoPointFieldMapper.Builder<Builder, LegacyGeoPointFieldMapper> {
private Boolean coerce;
public Builder(String name) {
super(name, Defaults.FIELD_TYPE);
this.builder = this;
}
public Builder coerce(boolean coerce) {
this.coerce = coerce;
return builder;
}
protected Explicit<Boolean> coerce(BuilderContext context) {
if (coerce != null) {
return new Explicit<>(coerce, true);
}
if (context.indexSettings() != null) {
return new Explicit<>(COERCE_SETTING.get(context.indexSettings()), false);
}
return Defaults.COERCE;
}
@Override
public LegacyGeoPointFieldMapper build(BuilderContext context, String simpleName, MappedFieldType fieldType,
MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper, FieldMapper lonMapper,
FieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed,
CopyTo copyTo) {
fieldType.setTokenized(false);
setupFieldType(context);
fieldType.setHasDocValues(false);
defaultFieldType.setHasDocValues(false);
return new LegacyGeoPointFieldMapper(simpleName, fieldType, defaultFieldType, indexSettings, latMapper,
lonMapper, geoHashMapper, multiFields, ignoreMalformed, coerce(context), copyTo);
}
@Override
public LegacyGeoPointFieldMapper build(BuilderContext context) {
return super.build(context);
}
}
public static Builder parse(Builder builder, Map<String, Object> node, Mapper.TypeParser.ParserContext parserContext)
throws MapperParsingException {
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals(Names.COERCE)) {
builder.coerce = XContentMapValues.lenientNodeBooleanValue(propNode);
iterator.remove();
}
}
return builder;
}
/**
* A byte-aligned fixed-length encoding for latitudes and longitudes.
*/
public static final class Encoding {
// With 14 bytes we already have better precision than a double since a double has 11 bits of exponent
private static final int MAX_NUM_BYTES = 14;
private static final Encoding[] INSTANCES;
static {
INSTANCES = new Encoding[MAX_NUM_BYTES + 1];
for (int numBytes = 2; numBytes <= MAX_NUM_BYTES; numBytes += 2) {
INSTANCES[numBytes] = new Encoding(numBytes);
}
}
/** Get an instance based on the number of bytes that has been used to encode values. */
public static Encoding of(int numBytesPerValue) {
final Encoding instance = INSTANCES[numBytesPerValue];
if (instance == null) {
throw new IllegalStateException("No encoding for " + numBytesPerValue + " bytes per value");
}
return instance;
}
/** Get an instance based on the expected precision. Here are examples of the number of required bytes per value
* depending on the
* expected precision:<ul>
* <li>1km: 4 bytes</li>
* <li>3m: 6 bytes</li>
* <li>1m: 8 bytes</li>
* <li>1cm: 8 bytes</li>
* <li>1mm: 10 bytes</li></ul> */
public static Encoding of(DistanceUnit.Distance precision) {
for (Encoding encoding : INSTANCES) {
if (encoding != null && encoding.precision().compareTo(precision) <= 0) {
return encoding;
}
}
return INSTANCES[MAX_NUM_BYTES];
}
private final DistanceUnit.Distance precision;
private final int numBytes;
private final int numBytesPerCoordinate;
private final double factor;
private Encoding(int numBytes) {
assert numBytes >= 1 && numBytes <= MAX_NUM_BYTES;
assert (numBytes & 1) == 0; // we don't support odd numBytes for the moment
this.numBytes = numBytes;
this.numBytesPerCoordinate = numBytes / 2;
this.factor = Math.pow(2, - numBytesPerCoordinate * 8 + 9);
assert (1L << (numBytesPerCoordinate * 8 - 1)) * factor > 180 && (1L << (numBytesPerCoordinate * 8 - 2))
* factor < 180 : numBytesPerCoordinate + " " + factor;
if (numBytes == MAX_NUM_BYTES) {
// no precision loss compared to a double
precision = new DistanceUnit.Distance(0, DistanceUnit.DEFAULT);
} else {
// factor/2 because we use Math.round instead of a cast to convert the double to a long
precision = new DistanceUnit.Distance(
GeoDistance.PLANE.calculate(0, 0, factor / 2, factor / 2, DistanceUnit.DEFAULT),
DistanceUnit.DEFAULT);
}
}
public DistanceUnit.Distance precision() {
return precision;
}
/** The number of bytes required to encode a single geo point. */
public int numBytes() {
return numBytes;
}
/** The number of bits required to encode a single coordinate of a geo point. */
public int numBitsPerCoordinate() {
return numBytesPerCoordinate << 3;
}
/** Return the bits that encode a latitude/longitude. */
public long encodeCoordinate(double lat) {
return Math.round((lat + 180) / factor);
}
/** Decode a sequence of bits into the original coordinate. */
public double decodeCoordinate(long bits) {
return bits * factor - 180;
}
private void encodeBits(long bits, byte[] out, int offset) {
for (int i = 0; i < numBytesPerCoordinate; ++i) {
out[offset++] = (byte) bits;
bits >>>= 8;
}
assert bits == 0;
}
private long decodeBits(byte [] in, int offset) {
long r = in[offset++] & 0xFFL;
for (int i = 1; i < numBytesPerCoordinate; ++i) {
r = (in[offset++] & 0xFFL) << (i * 8);
}
return r;
}
/** Encode a geo point into a byte-array, over {@link #numBytes()} bytes. */
public void encode(double lat, double lon, byte[] out, int offset) {
encodeBits(encodeCoordinate(lat), out, offset);
encodeBits(encodeCoordinate(lon), out, offset + numBytesPerCoordinate);
}
/** Decode a geo point from a byte-array, reading {@link #numBytes()} bytes. */
public GeoPoint decode(byte[] in, int offset, GeoPoint out) {
final long latBits = decodeBits(in, offset);
final long lonBits = decodeBits(in, offset + numBytesPerCoordinate);
return decode(latBits, lonBits, out);
}
/** Decode a geo point from the bits of the encoded latitude and longitudes. */
public GeoPoint decode(long latBits, long lonBits, GeoPoint out) {
final double lat = decodeCoordinate(latBits);
final double lon = decodeCoordinate(lonBits);
return out.reset(lat, lon);
}
}
protected Explicit<Boolean> coerce;
public LegacyGeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Settings indexSettings, FieldMapper latMapper, FieldMapper lonMapper, FieldMapper geoHashMapper,
MultiFields multiFields, Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields,
ignoreMalformed, copyTo);
this.coerce = coerce;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
LegacyGeoPointFieldMapper gpfmMergeWith = (LegacyGeoPointFieldMapper) mergeWith;
if (gpfmMergeWith.coerce.explicit()) {
if (coerce.explicit() && coerce.value() != gpfmMergeWith.coerce.value()) {
throw new IllegalArgumentException("mapper [" + fieldType().name() + "] has different [coerce]");
}
}
if (gpfmMergeWith.coerce.explicit()) {
this.coerce = gpfmMergeWith.coerce;
}
}
@Override
protected void parse(ParseContext context, GeoPoint point, String geoHash) throws IOException {
boolean validPoint = false;
if (coerce.value() == false && ignoreMalformed.value() == false) {
if (point.lat() > 90.0 || point.lat() < -90.0) {
throw new IllegalArgumentException("illegal latitude value [" + point.lat() + "] for " + name());
}
if (point.lon() > 180.0 || point.lon() < -180) {
throw new IllegalArgumentException("illegal longitude value [" + point.lon() + "] for " + name());
}
validPoint = true;
}
if (coerce.value() && validPoint == false) {
// by setting coerce to false we are assuming all geopoints are already in a valid coordinate system
// thus this extra step can be skipped
GeoUtils.normalizePoint(point, true, true);
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
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().name());
if (field == null) {
field = new CustomGeoPointDocValuesField(fieldType().name(), point.lat(), point.lon());
context.doc().addWithKey(fieldType().name(), field);
} else {
field.add(point.lat(), point.lon());
}
}
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || coerce.explicit()) {
builder.field(Names.COERCE, coerce.value());
}
}
@Override
public LegacyGeoPointFieldType fieldType() {
return (LegacyGeoPointFieldType) super.fieldType();
}
public static class CustomGeoPointDocValuesField extends CustomDocValuesField {
private final ObjectHashSet<GeoPoint> points;
public CustomGeoPointDocValuesField(String name, double lat, double lon) {
super(name);
points = new ObjectHashSet<>(2);
points.add(new GeoPoint(lat, lon));
}
public void add(double lat, double lon) {
points.add(new GeoPoint(lat, lon));
}
@Override
public BytesRef binaryValue() {
final byte[] bytes = new byte[points.size() * 16];
int off = 0;
for (Iterator<ObjectCursor<GeoPoint>> it = points.iterator(); it.hasNext(); ) {
final GeoPoint point = it.next().value;
ByteUtils.writeDoubleLE(point.getLat(), bytes, off);
ByteUtils.writeDoubleLE(point.getLon(), bytes, off + 8);
off += 16;
}
return new BytesRef(bytes);
}
}
}

View File

@ -1,332 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeIntegerValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyIntegerFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "integer";
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final MappedFieldType FIELD_TYPE = new IntegerFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyIntegerFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_32_BIT);
builder = this;
}
public Builder nullValue(int nullValue) {
this.fieldType.setNullValue(nullValue);
return this;
}
@Override
public LegacyIntegerFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyIntegerFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context), includeInAll,
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 32;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyIntegerFieldMapper.Builder builder = new LegacyIntegerFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(nodeIntegerValue(propNode));
iterator.remove();
}
}
return builder;
}
}
public static final class IntegerFieldType extends NumberFieldType {
public IntegerFieldType() {
super(LegacyNumericType.INT);
}
protected IntegerFieldType(IntegerFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new IntegerFieldType(this);
}
@Override
public String typeName() {
return "integer";
}
@Override
public Integer nullValue() {
return (Integer)super.nullValue();
}
@Override
public BytesRef indexedValueForSearch(Object value) {
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.intToPrefixCoded(parseValue(value), 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats.Long stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
long minValue = LegacyNumericUtils.getMinInt(terms);
long maxValue = LegacyNumericUtils.getMaxInt(terms);
return new FieldStats.Long(
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(),
isSearchable(), isAggregatable(), minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.INT);
}
}
protected LegacyIntegerFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public IntegerFieldType fieldType() {
return (IntegerFieldType) super.fieldType();
}
private static int parseValue(Object value) {
if (value instanceof Number) {
return ((Number) value).intValue();
}
if (value instanceof BytesRef) {
return Integer.parseInt(((BytesRef) value).utf8ToString());
}
return Integer.parseInt(value.toString());
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
int value;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
if (externalValue == null) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else if (externalValue instanceof String) {
String sExternalValue = (String) externalValue;
if (sExternalValue.length() == 0) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else {
value = Integer.parseInt(sExternalValue);
}
} else {
value = ((Number) externalValue).intValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), Integer.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL ||
(parser.currentToken() == XContentParser.Token.VALUE_STRING && parser.textLength() == 0)) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
Integer objValue = fieldType().nullValue();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (parser.currentToken() != XContentParser.Token.VALUE_NULL) {
objValue = parser.intValue(coerce.value());
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
if (objValue == null) {
// no value
return;
}
value = objValue;
} else {
value = parser.intValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}
addIntegerFields(context, fields, value, boost);
}
protected void addIntegerFields(ParseContext context, List<IndexableField> fields, int value, float boost) {
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomIntegerNumericField field = new CustomIntegerNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, value);
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_32_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
public static class CustomIntegerNumericField extends CustomNumericField {
private final int number;
public CustomIntegerNumericField(int number, MappedFieldType fieldType) {
super(number, fieldType);
this.number = number;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
if (fieldType().indexOptions() != IndexOptions.NONE) {
return getCachedStream().setIntValue(number);
}
return null;
}
@Override
public String numericAsString() {
return Integer.toString(number);
}
}
}

View File

@ -1,341 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.network.Cidrs;
import org.elasticsearch.common.network.InetAddresses;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.mapper.LegacyLongFieldMapper.CustomLongNumericField;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
import java.io.IOException;
import java.net.InetAddress;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyIpFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "ip";
public static final long MAX_IP = 4294967296L;
public static String longToIp(long longIp) {
int octet3 = (int) ((longIp >> 24) % 256);
int octet2 = (int) ((longIp >> 16) % 256);
int octet1 = (int) ((longIp >> 8) % 256);
int octet0 = (int) ((longIp) % 256);
return octet3 + "." + octet2 + "." + octet1 + "." + octet0;
}
private static final Pattern pattern = Pattern.compile("\\.");
public static long ipToLong(String ip) {
try {
if (!InetAddresses.isInetAddress(ip)) {
throw new IllegalArgumentException("failed to parse ip [" + ip + "], not a valid ip address");
}
String[] octets = pattern.split(ip);
if (octets.length != 4) {
throw new IllegalArgumentException("failed to parse ip [" + ip + "], not a valid ipv4 address (4 dots)");
}
return (Long.parseLong(octets[0]) << 24) + (Integer.parseInt(octets[1]) << 16) +
(Integer.parseInt(octets[2]) << 8) + Integer.parseInt(octets[3]);
} catch (Exception e) {
if (e instanceof IllegalArgumentException) {
throw (IllegalArgumentException) e;
}
throw new IllegalArgumentException("failed to parse ip [" + ip + "]", e);
}
}
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final String NULL_VALUE = null;
public static final MappedFieldType FIELD_TYPE = new IpFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyIpFieldMapper> {
protected String nullValue = Defaults.NULL_VALUE;
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_64_BIT);
builder = this;
}
@Override
public LegacyIpFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyIpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context),
coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 64;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyIpFieldMapper.Builder builder = new Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(propNode.toString());
iterator.remove();
}
}
return builder;
}
}
public static final class IpFieldType extends LegacyLongFieldMapper.LongFieldType {
public IpFieldType() {
}
protected IpFieldType(IpFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new IpFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
/**
* IPs should return as a string.
*/
@Override
public Object valueForDisplay(Object value) {
Long val = (Long) value;
if (val == null) {
return null;
}
return longToIp(val);
}
@Override
public BytesRef indexedValueForSearch(Object value) {
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.longToPrefixCoded(parseValue(value), 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query termQuery(Object value, @Nullable QueryShardContext context) {
if (value != null) {
String term;
if (value instanceof BytesRef) {
term = ((BytesRef) value).utf8ToString();
} else {
term = value.toString();
}
long[] fromTo;
// assume that the term is either a CIDR range or the
// term is a single IPv4 address; if either of these
// assumptions is wrong, the CIDR parsing will fail
// anyway, and that is okay
if (term.contains("/")) {
// treat the term as if it is in CIDR notation
fromTo = Cidrs.cidrMaskToMinMax(term);
} else {
// treat the term as if it is a single IPv4, and
// apply a CIDR mask equivalent to the host route
fromTo = Cidrs.cidrMaskToMinMax(term + "/32");
}
if (fromTo != null) {
return rangeQuery(fromTo[0] == 0 ? null : fromTo[0],
fromTo[1] == MAX_IP ? null : fromTo[1], true, false, context);
}
}
return super.termQuery(value, context);
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseValue(lowerTerm),
upperTerm == null ? null : parseValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
long minValue = LegacyNumericUtils.getMinLong(terms);
long maxValue = LegacyNumericUtils.getMaxLong(terms);
return new FieldStats.Ip(maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(),
isSearchable(), isAggregatable(),
InetAddress.getByName(longToIp(minValue)),
InetAddress.getByName(longToIp(maxValue)));
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new IndexFieldData.Builder() {
@Override
public IndexFieldData<?> build(IndexSettings indexSettings,
MappedFieldType fieldType, IndexFieldDataCache cache,
CircuitBreakerService breakerService, MapperService mapperService) {
return new LegacyIpIndexFieldData(indexSettings.getIndex(), name());
}
};
}
@Override
public DocValueFormat docValueFormat(@Nullable String format, DateTimeZone timeZone) {
if (format != null) {
throw new IllegalArgumentException("Field [" + name() + "] of type [" + typeName() + "] does not support custom formats");
}
if (timeZone != null) {
throw new IllegalArgumentException("Field [" + name() + "] of type [" + typeName()
+ "] does not support custom time zones");
}
return DocValueFormat.IP;
}
}
protected LegacyIpFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
private static long parseValue(Object value) {
if (value instanceof Number) {
return ((Number) value).longValue();
}
if (value instanceof BytesRef) {
return ipToLong(((BytesRef) value).utf8ToString());
}
return ipToLong(value.toString());
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
String ipAsString;
if (context.externalValueSet()) {
ipAsString = (String) context.externalValue();
if (ipAsString == null) {
ipAsString = fieldType().nullValueAsString();
}
} else {
if (context.parser().currentToken() == XContentParser.Token.VALUE_NULL) {
ipAsString = fieldType().nullValueAsString();
} else {
ipAsString = context.parser().text();
}
}
if (ipAsString == null) {
return;
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), ipAsString, fieldType().boost());
}
final long value = ipToLong(ipAsString);
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomLongNumericField field = new CustomLongNumericField(value, fieldType());
if (fieldType.boost() != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(fieldType().boost());
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, value);
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_64_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValueAsString() != null) {
builder.field("null_value", fieldType().nullValueAsString());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
}

View File

@ -1,145 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.document.InetAddressPoint;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.search.MultiValueMode;
import java.io.IOException;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.nio.ByteBuffer;
final class LegacyIpIndexFieldData implements IndexFieldData<AtomicFieldData> {
protected final Index index;
protected final String fieldName;
protected final Logger logger;
public LegacyIpIndexFieldData(Index index, String fieldName) {
this.index = index;
this.fieldName = fieldName;
this.logger = Loggers.getLogger(getClass());
}
public String getFieldName() {
return fieldName;
}
public void clear() {
// nothing to do
}
public void clear(IndexReader reader) {
// nothing to do
}
public Index index() {
return index;
}
@Override
public AtomicFieldData load(LeafReaderContext context) {
return new AtomicFieldData() {
@Override
public void close() {
// no-op
}
@Override
public long ramBytesUsed() {
return 0;
}
@Override
public ScriptDocValues<?> getScriptValues() {
throw new UnsupportedOperationException("Cannot run scripts on ip fields");
}
@Override
public SortedBinaryDocValues getBytesValues() {
SortedNumericDocValues values;
try {
values = DocValues.getSortedNumeric(context.reader(), fieldName);
} catch (IOException e) {
throw new IllegalStateException("Cannot load doc values", e);
}
return new SortedBinaryDocValues() {
final ByteBuffer scratch = ByteBuffer.allocate(4);
@Override
public BytesRef valueAt(int index) {
// we do not need to reorder ip addresses since both the numeric
// encoding of LegacyIpFieldMapper and the binary encoding of
// IpFieldMapper match the sort order of ip addresses
long ip = values.valueAt(index);
scratch.putInt(0, (int) ip);
InetAddress inet;
try {
inet = InetAddress.getByAddress(scratch.array());
} catch (UnknownHostException e) {
throw new IllegalStateException("Cannot happen", e);
}
byte[] encoded = InetAddressPoint.encode(inet);
return new BytesRef(encoded);
}
@Override
public void setDocument(int docId) {
values.setDocument(docId);
}
@Override
public int count() {
return values.count();
}
};
}
};
}
@Override
public AtomicFieldData loadDirect(LeafReaderContext context)
throws Exception {
return load(context);
}
@Override
public IndexFieldData.XFieldComparatorSource comparatorSource(
Object missingValue, MultiValueMode sortMode, Nested nested) {
return new BytesRefFieldComparatorSource(this, missingValue, sortMode, nested);
}
}

View File

@ -1,318 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeLongValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyLongFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "long";
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final MappedFieldType FIELD_TYPE = new LongFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyLongFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_64_BIT);
builder = this;
}
public Builder nullValue(long nullValue) {
this.fieldType.setNullValue(nullValue);
return this;
}
@Override
public LegacyLongFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyLongFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context), includeInAll, context.indexSettings(),
multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 64;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyLongFieldMapper.Builder builder = new LegacyLongFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(nodeLongValue(propNode));
iterator.remove();
}
}
return builder;
}
}
public static class LongFieldType extends NumberFieldType {
public LongFieldType() {
super(LegacyNumericType.LONG);
}
protected LongFieldType(LongFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new LongFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public Long nullValue() {
return (Long)super.nullValue();
}
@Override
public BytesRef indexedValueForSearch(Object value) {
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.longToPrefixCoded(parseLongValue(value), 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newLongRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : parseLongValue(lowerTerm),
upperTerm == null ? null : parseLongValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
long minValue = LegacyNumericUtils.getMinLong(terms);
long maxValue = LegacyNumericUtils.getMaxLong(terms);
return new FieldStats.Long(
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(),
isSearchable(), isAggregatable(), minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.LONG);
}
}
protected LegacyLongFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public LongFieldType fieldType() {
return (LongFieldType) super.fieldType();
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
long value;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
if (externalValue == null) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else if (externalValue instanceof String) {
String sExternalValue = (String) externalValue;
if (sExternalValue.length() == 0) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else {
value = Long.parseLong(sExternalValue);
}
} else {
value = ((Number) externalValue).longValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), Long.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL ||
(parser.currentToken() == XContentParser.Token.VALUE_STRING && parser.textLength() == 0)) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
Long objValue = fieldType().nullValue();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (parser.currentToken() != XContentParser.Token.VALUE_NULL) {
objValue = parser.longValue(coerce.value());
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
if (objValue == null) {
// no value
return;
}
value = objValue;
} else {
value = parser.longValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomLongNumericField field = new CustomLongNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, value);
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != Defaults.PRECISION_STEP_64_BIT) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
public static class CustomLongNumericField extends CustomNumericField {
private final long number;
public CustomLongNumericField(long number, MappedFieldType fieldType) {
super(number, fieldType);
this.number = number;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
if (fieldType().indexOptions() != IndexOptions.NONE) {
return getCachedStream().setLongValue(number);
}
return null;
}
@Override
public String numericAsString() {
return Long.toString(number);
}
}
}

View File

@ -1,321 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import java.io.IOException;
import java.io.Reader;
import java.util.List;
import org.apache.lucene.analysis.LegacyNumericTokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
public abstract class LegacyNumberFieldMapper extends FieldMapper {
// this is private since it has a different default
private static final Setting<Boolean> COERCE_SETTING =
Setting.boolSetting("index.mapping.coerce", true, Property.IndexScope);
public static class Defaults {
public static final int PRECISION_STEP_8_BIT = Integer.MAX_VALUE; // 1tpv: 256 terms at most, not useful
public static final int PRECISION_STEP_16_BIT = 8; // 2tpv
public static final int PRECISION_STEP_32_BIT = 8; // 4tpv
public static final int PRECISION_STEP_64_BIT = 16; // 4tpv
public static final Explicit<Boolean> IGNORE_MALFORMED = new Explicit<>(false, false);
public static final Explicit<Boolean> COERCE = new Explicit<>(true, false);
}
public abstract static class Builder<T extends Builder, Y extends LegacyNumberFieldMapper> extends FieldMapper.Builder<T, Y> {
private Boolean ignoreMalformed;
private Boolean coerce;
public Builder(String name, MappedFieldType fieldType, int defaultPrecisionStep) {
super(name, fieldType, fieldType);
this.fieldType.setNumericPrecisionStep(defaultPrecisionStep);
}
public T precisionStep(int precisionStep) {
fieldType.setNumericPrecisionStep(precisionStep);
return builder;
}
public T ignoreMalformed(boolean ignoreMalformed) {
this.ignoreMalformed = ignoreMalformed;
return builder;
}
protected Explicit<Boolean> ignoreMalformed(BuilderContext context) {
if (ignoreMalformed != null) {
return new Explicit<>(ignoreMalformed, true);
}
if (context.indexSettings() != null) {
return new Explicit<>(IGNORE_MALFORMED_SETTING.get(context.indexSettings()), false);
}
return Defaults.IGNORE_MALFORMED;
}
public T coerce(boolean coerce) {
this.coerce = coerce;
return builder;
}
protected Explicit<Boolean> coerce(BuilderContext context) {
if (coerce != null) {
return new Explicit<>(coerce, true);
}
if (context.indexSettings() != null) {
return new Explicit<>(COERCE_SETTING.get(context.indexSettings()), false);
}
return Defaults.COERCE;
}
protected void setupFieldType(BuilderContext context) {
super.setupFieldType(context);
int precisionStep = fieldType.numericPrecisionStep();
if (precisionStep <= 0 || precisionStep >= maxPrecisionStep()) {
fieldType.setNumericPrecisionStep(Integer.MAX_VALUE);
}
}
protected abstract int maxPrecisionStep();
}
public abstract static class NumberFieldType extends TermBasedFieldType {
public NumberFieldType(LegacyNumericType numericType) {
setTokenized(false);
setOmitNorms(true);
setIndexOptions(IndexOptions.DOCS);
setStoreTermVectors(false);
setNumericType(numericType);
}
protected NumberFieldType(NumberFieldType ref) {
super(ref);
}
@Override
public void checkCompatibility(MappedFieldType other,
List<String> conflicts, boolean strict) {
super.checkCompatibility(other, conflicts, strict);
if (numericPrecisionStep() != other.numericPrecisionStep()) {
conflicts.add("mapper [" + name() + "] has different [precision_step] values");
}
}
public abstract NumberFieldType clone();
@Override
public DocValueFormat docValueFormat(@Nullable String format, DateTimeZone timeZone) {
if (timeZone != null) {
throw new IllegalArgumentException("Field [" + name() + "] of type [" + typeName() + "] does not support custom time zones");
}
if (format == null) {
return DocValueFormat.RAW;
} else {
return new DocValueFormat.Decimal(format);
}
}
}
protected Boolean includeInAll;
protected Explicit<Boolean> ignoreMalformed;
protected Explicit<Boolean> coerce;
protected LegacyNumberFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
assert fieldType.tokenized() == false;
this.ignoreMalformed = ignoreMalformed;
this.coerce = coerce;
this.includeInAll = includeInAll;
}
@Override
protected LegacyNumberFieldMapper clone() {
return (LegacyNumberFieldMapper) super.clone();
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
RuntimeException e = null;
try {
innerParseCreateField(context, fields);
} catch (IllegalArgumentException e1) {
e = e1;
} catch (MapperParsingException e2) {
e = e2;
}
if (e != null && !ignoreMalformed.value()) {
throw e;
}
}
protected abstract void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException;
protected final void addDocValue(ParseContext context, List<IndexableField> fields, long value) {
fields.add(new SortedNumericDocValuesField(fieldType().name(), value));
}
/**
* Converts an object value into a double
*/
public static double parseDoubleValue(Object value) {
if (value instanceof Number) {
return ((Number) value).doubleValue();
}
if (value instanceof BytesRef) {
return Double.parseDouble(((BytesRef) value).utf8ToString());
}
return Double.parseDouble(value.toString());
}
/**
* Converts an object value into a long
*/
public static long parseLongValue(Object value) {
if (value instanceof Number) {
return ((Number) value).longValue();
}
if (value instanceof BytesRef) {
return Long.parseLong(((BytesRef) value).utf8ToString());
}
return Long.parseLong(value.toString());
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
LegacyNumberFieldMapper nfmMergeWith = (LegacyNumberFieldMapper) mergeWith;
this.includeInAll = nfmMergeWith.includeInAll;
if (nfmMergeWith.ignoreMalformed.explicit()) {
this.ignoreMalformed = nfmMergeWith.ignoreMalformed;
}
if (nfmMergeWith.coerce.explicit()) {
this.coerce = nfmMergeWith.coerce;
}
}
// used to we can use a numeric field in a document that is then parsed twice!
public abstract static class CustomNumericField extends Field {
private ThreadLocal<LegacyNumericTokenStream> tokenStream = new ThreadLocal<LegacyNumericTokenStream>() {
@Override
protected LegacyNumericTokenStream initialValue() {
return new LegacyNumericTokenStream(fieldType().numericPrecisionStep());
}
};
private static ThreadLocal<LegacyNumericTokenStream> tokenStream4 = new ThreadLocal<LegacyNumericTokenStream>() {
@Override
protected LegacyNumericTokenStream initialValue() {
return new LegacyNumericTokenStream(4);
}
};
private static ThreadLocal<LegacyNumericTokenStream> tokenStream8 = new ThreadLocal<LegacyNumericTokenStream>() {
@Override
protected LegacyNumericTokenStream initialValue() {
return new LegacyNumericTokenStream(8);
}
};
private static ThreadLocal<LegacyNumericTokenStream> tokenStream16 = new ThreadLocal<LegacyNumericTokenStream>() {
@Override
protected LegacyNumericTokenStream initialValue() {
return new LegacyNumericTokenStream(16);
}
};
private static ThreadLocal<LegacyNumericTokenStream> tokenStreamMax = new ThreadLocal<LegacyNumericTokenStream>() {
@Override
protected LegacyNumericTokenStream initialValue() {
return new LegacyNumericTokenStream(Integer.MAX_VALUE);
}
};
public CustomNumericField(Number value, MappedFieldType fieldType) {
super(fieldType.name(), fieldType);
if (value != null) {
this.fieldsData = value;
}
}
protected LegacyNumericTokenStream getCachedStream() {
if (fieldType().numericPrecisionStep() == 4) {
return tokenStream4.get();
} else if (fieldType().numericPrecisionStep() == 8) {
return tokenStream8.get();
} else if (fieldType().numericPrecisionStep() == 16) {
return tokenStream16.get();
} else if (fieldType().numericPrecisionStep() == Integer.MAX_VALUE) {
return tokenStreamMax.get();
}
return tokenStream.get();
}
@Override
public String stringValue() {
return null;
}
@Override
public Reader readerValue() {
return null;
}
public abstract String numericAsString();
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || ignoreMalformed.explicit()) {
builder.field("ignore_malformed", ignoreMalformed.value());
}
if (includeDefaults || coerce.explicit()) {
builder.field("coerce", coerce.value());
}
}
}

View File

@ -1,333 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LegacyNumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeShortValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
public class LegacyShortFieldMapper extends LegacyNumberFieldMapper {
public static final String CONTENT_TYPE = "short";
public static final int DEFAULT_PRECISION_STEP = 8;
public static class Defaults extends LegacyNumberFieldMapper.Defaults {
public static final MappedFieldType FIELD_TYPE = new ShortFieldType();
static {
FIELD_TYPE.freeze();
}
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyShortFieldMapper> {
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, DEFAULT_PRECISION_STEP);
builder = this;
}
@Override
public LegacyShortFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyShortFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context), includeInAll,
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 32;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyShortFieldMapper.Builder builder = new LegacyShortFieldMapper.Builder(name);
parseNumberField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(nodeShortValue(propNode));
iterator.remove();
}
}
return builder;
}
}
static final class ShortFieldType extends NumberFieldType {
public ShortFieldType() {
super(LegacyNumericType.INT);
}
protected ShortFieldType(ShortFieldType ref) {
super(ref);
}
@Override
public NumberFieldType clone() {
return new ShortFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public Short nullValue() {
return (Short)super.nullValue();
}
@Override
public Short valueForDisplay(Object value) {
if (value == null) {
return null;
}
return ((Number) value).shortValue();
}
@Override
public BytesRef indexedValueForSearch(Object value) {
BytesRefBuilder bytesRef = new BytesRefBuilder();
LegacyNumericUtils.intToPrefixCoded(parseValue(value), 0, bytesRef); // 0 because of exact match
return bytesRef.get();
}
@Override
public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower, boolean includeUpper, QueryShardContext context) {
return LegacyNumericRangeQuery.newIntRange(name(), numericPrecisionStep(),
lowerTerm == null ? null : (int)parseValue(lowerTerm),
upperTerm == null ? null : (int)parseValue(upperTerm),
includeLower, includeUpper);
}
@Override
public FieldStats.Long stats(IndexReader reader) throws IOException {
int maxDoc = reader.maxDoc();
Terms terms = org.apache.lucene.index.MultiFields.getTerms(reader, name());
if (terms == null) {
return null;
}
long minValue = LegacyNumericUtils.getMinInt(terms);
long maxValue = LegacyNumericUtils.getMaxInt(terms);
return new FieldStats.Long(
maxDoc, terms.getDocCount(), terms.getSumDocFreq(), terms.getSumTotalTermFreq(),
isSearchable(), isAggregatable(), minValue, maxValue);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
failIfNoDocValues();
return new DocValuesIndexFieldData.Builder().numericType(NumericType.SHORT);
}
}
protected LegacyShortFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
}
@Override
public ShortFieldType fieldType() {
return (ShortFieldType) super.fieldType();
}
private static short parseValue(Object value) {
if (value instanceof Number) {
return ((Number) value).shortValue();
}
if (value instanceof BytesRef) {
return Short.parseShort(((BytesRef) value).utf8ToString());
}
return Short.parseShort(value.toString());
}
@Override
protected boolean customBoost() {
return true;
}
@Override
protected void innerParseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
short value;
float boost = fieldType().boost();
if (context.externalValueSet()) {
Object externalValue = context.externalValue();
if (externalValue == null) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else if (externalValue instanceof String) {
String sExternalValue = (String) externalValue;
if (sExternalValue.length() == 0) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
} else {
value = Short.parseShort(sExternalValue);
}
} else {
value = ((Number) externalValue).shortValue();
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), Short.toString(value), boost);
}
} else {
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL ||
(parser.currentToken() == XContentParser.Token.VALUE_STRING && parser.textLength() == 0)) {
if (fieldType().nullValue() == null) {
return;
}
value = fieldType().nullValue();
if (fieldType().nullValueAsString() != null && (context.includeInAll(includeInAll, this))) {
context.allEntries().addText(fieldType().name(), fieldType().nullValueAsString(), boost);
}
} else if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
Short objValue = fieldType().nullValue();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
if (parser.currentToken() != XContentParser.Token.VALUE_NULL) {
objValue = parser.shortValue(coerce.value());
}
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
if (objValue == null) {
// no value
return;
}
value = objValue;
} else {
value = parser.shortValue(coerce.value());
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), parser.text(), boost);
}
}
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
CustomShortNumericField field = new CustomShortNumericField(value, fieldType());
if (boost != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(boost);
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
addDocValue(context, fields, value);
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
if (includeDefaults || fieldType().numericPrecisionStep() != DEFAULT_PRECISION_STEP) {
builder.field("precision_step", fieldType().numericPrecisionStep());
}
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
}
public static class CustomShortNumericField extends CustomNumericField {
private final short number;
public CustomShortNumericField(short number, NumberFieldType fieldType) {
super(number, fieldType);
this.number = number;
}
@Override
public TokenStream tokenStream(Analyzer analyzer, TokenStream previous) {
if (fieldType().indexOptions() != IndexOptions.NONE) {
return getCachedStream().setIntValue(number);
}
return null;
}
@Override
public String numericAsString() {
return Short.toString(number);
}
}
}

View File

@ -1,191 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexableField;
import org.elasticsearch.Version;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.mapper.StringFieldMapper.ValueAndBoost;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.apache.lucene.index.IndexOptions.NONE;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeIntegerValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseNumberField;
/**
* A {@link FieldMapper} that takes a string and writes a count of the tokens in that string
* to the index. In most ways the mapper acts just like an {@link LegacyIntegerFieldMapper}.
*/
public class LegacyTokenCountFieldMapper extends LegacyIntegerFieldMapper {
public static final String CONTENT_TYPE = "token_count";
public static class Defaults extends LegacyIntegerFieldMapper.Defaults {
}
public static class Builder extends LegacyNumberFieldMapper.Builder<Builder, LegacyTokenCountFieldMapper> {
private NamedAnalyzer analyzer;
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.PRECISION_STEP_32_BIT);
builder = this;
}
public Builder analyzer(NamedAnalyzer analyzer) {
this.analyzer = analyzer;
return this;
}
public NamedAnalyzer analyzer() {
return analyzer;
}
@Override
public LegacyTokenCountFieldMapper build(BuilderContext context) {
if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha2)) {
throw new IllegalStateException("Cannot use legacy numeric types after 5.0");
}
setupFieldType(context);
return new LegacyTokenCountFieldMapper(name, fieldType, defaultFieldType,
ignoreMalformed(context), coerce(context), includeInAll, context.indexSettings(),
analyzer, multiFieldsBuilder.build(this, context), copyTo);
}
@Override
protected int maxPrecisionStep() {
return 32;
}
}
public static class TypeParser implements Mapper.TypeParser {
@Override
@SuppressWarnings("unchecked")
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
LegacyTokenCountFieldMapper.Builder builder = new LegacyTokenCountFieldMapper.Builder(name);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
builder.nullValue(nodeIntegerValue(propNode));
iterator.remove();
} else if (propName.equals("analyzer")) {
NamedAnalyzer analyzer = parserContext.getIndexAnalyzers().get(propNode.toString());
if (analyzer == null) {
throw new MapperParsingException("Analyzer [" + propNode.toString() + "] not found for field [" + name + "]");
}
builder.analyzer(analyzer);
iterator.remove();
}
}
parseNumberField(builder, name, node, parserContext);
if (builder.analyzer() == null) {
throw new MapperParsingException("Analyzer must be set for field [" + name + "] but wasn't.");
}
return builder;
}
}
private NamedAnalyzer analyzer;
protected LegacyTokenCountFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit<Boolean> ignoreMalformed,
Explicit<Boolean> coerce, Boolean includeInAll, Settings indexSettings, NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
this.analyzer = analyzer;
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
ValueAndBoost valueAndBoost = StringFieldMapper.parseCreateFieldForString(context, null /* Out null value is an int so we convert*/, fieldType().boost());
if (valueAndBoost.value() == null && fieldType().nullValue() == null) {
return;
}
if (fieldType().indexOptions() != NONE || fieldType().stored() || fieldType().hasDocValues()) {
int count;
if (valueAndBoost.value() == null) {
count = fieldType().nullValue();
} else {
count = countPositions(analyzer, simpleName(), valueAndBoost.value());
}
addIntegerFields(context, fields, count, valueAndBoost.boost());
}
}
/**
* Count position increments in a token stream. Package private for testing.
* @param analyzer analyzer to create token stream
* @param fieldName field name to pass to analyzer
* @param fieldValue field value to pass to analyzer
* @return number of position increments in a token stream
* @throws IOException if tokenStream throws it
*/
static int countPositions(Analyzer analyzer, String fieldName, String fieldValue) throws IOException {
try (TokenStream tokenStream = analyzer.tokenStream(fieldName, fieldValue)) {
int count = 0;
PositionIncrementAttribute position = tokenStream.addAttribute(PositionIncrementAttribute.class);
tokenStream.reset();
while (tokenStream.incrementToken()) {
count += position.getPositionIncrement();
}
tokenStream.end();
count += position.getPositionIncrement();
return count;
}
}
/**
* Name of analyzer.
* @return name of analyzer
*/
public String analyzer() {
return analyzer.name();
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
this.analyzer = ((LegacyTokenCountFieldMapper) mergeWith).analyzer;
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
builder.field("analyzer", analyzer());
}
}

View File

@ -36,7 +36,6 @@ import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.action.fieldstats.FieldStats;
import org.elasticsearch.common.Explicit;
import org.elasticsearch.common.settings.Setting;
@ -48,7 +47,6 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData.NumericType;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.LegacyNumberFieldMapper.Defaults;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.DocValueFormat;
import org.joda.time.DateTimeZone;
@ -67,6 +65,11 @@ public class NumberFieldMapper extends FieldMapper {
static final Setting<Boolean> COERCE_SETTING =
Setting.boolSetting("index.mapping.coerce", true, Property.IndexScope);
public static class Defaults {
public static final Explicit<Boolean> IGNORE_MALFORMED = new Explicit<>(false, false);
public static final Explicit<Boolean> COERCE = new Explicit<>(true, false);
}
public static class Builder extends FieldMapper.Builder<Builder, NumberFieldMapper> {
private Boolean ignoreMalformed;
@ -131,24 +134,6 @@ public class NumberFieldMapper extends FieldMapper {
@Override
public Mapper.Builder<?,?> parse(String name, Map<String, Object> node,
ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha2)) {
switch (type) {
case BYTE:
return new LegacyByteFieldMapper.TypeParser().parse(name, node, parserContext);
case SHORT:
return new LegacyShortFieldMapper.TypeParser().parse(name, node, parserContext);
case INTEGER:
return new LegacyIntegerFieldMapper.TypeParser().parse(name, node, parserContext);
case LONG:
return new LegacyLongFieldMapper.TypeParser().parse(name, node, parserContext);
case FLOAT:
return new LegacyFloatFieldMapper.TypeParser().parse(name, node, parserContext);
case DOUBLE:
return new LegacyDoubleFieldMapper.TypeParser().parse(name, node, parserContext);
default:
throw new AssertionError();
}
}
Builder builder = new Builder(name, type);
TypeParsers.parseField(builder, name, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {

View File

@ -39,10 +39,6 @@ public class ParsedDocument {
private final String routing;
private final long timestamp;
private final long ttl;
private final List<Document> documents;
private BytesReference source;
@ -57,8 +53,6 @@ public class ParsedDocument {
String id,
String type,
String routing,
long timestamp,
long ttl,
List<Document> documents,
BytesReference source,
Mapping dynamicMappingsUpdate) {
@ -68,8 +62,6 @@ public class ParsedDocument {
this.type = type;
this.uid = Uid.createUidAsBytes(type, id);
this.routing = routing;
this.timestamp = timestamp;
this.ttl = ttl;
this.documents = documents;
this.source = source;
this.dynamicMappingsUpdate = dynamicMappingsUpdate;
@ -99,14 +91,6 @@ public class ParsedDocument {
return this.routing;
}
public long timestamp() {
return this.timestamp;
}
public long ttl() {
return this.ttl;
}
public Document rootDoc() {
return documents.get(documents.size() - 1);
}

View File

@ -49,7 +49,7 @@ import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.mapper.LegacyNumberFieldMapper.Defaults;
import org.elasticsearch.index.mapper.NumberFieldMapper.Defaults;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.search.DocValueFormat;

View File

@ -23,7 +23,6 @@ import java.util.Objects;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.unit.TimeValue;
public class SourceToParse {
@ -49,10 +48,6 @@ public class SourceToParse {
private String parentId;
private long timestamp;
private long ttl;
private SourceToParse(Origin origin, String index, String type, String id, BytesReference source) {
this.origin = Objects.requireNonNull(origin);
this.index = Objects.requireNonNull(index);
@ -101,38 +96,6 @@ public class SourceToParse {
return this;
}
public long timestamp() {
return this.timestamp;
}
public SourceToParse timestamp(String timestamp) {
this.timestamp = Long.parseLong(timestamp);
return this;
}
public SourceToParse timestamp(long timestamp) {
this.timestamp = timestamp;
return this;
}
public long ttl() {
return this.ttl;
}
public SourceToParse ttl(TimeValue ttl) {
if (ttl == null) {
this.ttl = -1;
return this;
}
this.ttl = ttl.millis();
return this;
}
public SourceToParse ttl(long ttl) {
this.ttl = ttl;
return this;
}
public enum Origin {
PRIMARY,
REPLICA

View File

@ -1,695 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
import org.elasticsearch.index.analysis.NamedAnalyzer;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import static org.apache.lucene.index.IndexOptions.NONE;
import static org.elasticsearch.index.mapper.TypeParsers.parseTextField;
public class StringFieldMapper extends FieldMapper {
public static final String CONTENT_TYPE = "string";
private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1;
// If a string field is created on 5.x and all parameters are in this list then we
// will automatically upgrade to a text/keyword field. Otherwise we will just fail
// saying that string fields are not supported anymore.
private static final Set<String> SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE_TO_KEYWORD = new HashSet<>(Arrays.asList(
"type",
// common keyword parameters, for which the upgrade is straightforward
"index", "store", "doc_values", "omit_norms", "norms", "fields", "copy_to",
"fielddata", "include_in_all", "ignore_above"));
private static final Set<String> SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE_TO_TEXT = new HashSet<>(Arrays.asList(
"type",
// common text parameters, for which the upgrade is straightforward
"index", "store", "doc_values", "omit_norms", "norms", "fields", "copy_to",
"fielddata", "include_in_all", "analyzer", "search_analyzer", "search_quote_analyzer",
"index_options", "position_increment_gap"));
public static class Defaults {
public static double FIELDDATA_MIN_FREQUENCY = 0;
public static double FIELDDATA_MAX_FREQUENCY = Integer.MAX_VALUE;
public static int FIELDDATA_MIN_SEGMENT_SIZE = 0;
public static final MappedFieldType FIELD_TYPE = new StringFieldType();
static {
FIELD_TYPE.freeze();
}
// NOTE, when adding defaults here, make sure you add them in the builder
public static final String NULL_VALUE = null;
public static final int IGNORE_ABOVE = -1;
}
public static class Builder extends FieldMapper.Builder<Builder, StringFieldMapper> {
protected String nullValue = Defaults.NULL_VALUE;
/**
* The distance between tokens from different values in the same field.
* POSITION_INCREMENT_GAP_USE_ANALYZER means default to the analyzer's
* setting which in turn defaults to Defaults.POSITION_INCREMENT_GAP.
*/
protected int positionIncrementGap = POSITION_INCREMENT_GAP_USE_ANALYZER;
protected int ignoreAbove = Defaults.IGNORE_ABOVE;
public Builder(String name) {
super(name, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE);
builder = this;
}
@Override
public StringFieldType fieldType() {
return (StringFieldType) super.fieldType();
}
@Override
public Builder searchAnalyzer(NamedAnalyzer searchAnalyzer) {
super.searchAnalyzer(searchAnalyzer);
return this;
}
public Builder positionIncrementGap(int positionIncrementGap) {
this.positionIncrementGap = positionIncrementGap;
return this;
}
public Builder ignoreAbove(int ignoreAbove) {
this.ignoreAbove = ignoreAbove;
return this;
}
public Builder fielddata(boolean fielddata) {
fieldType().setFielddata(fielddata);
return builder;
}
public Builder eagerGlobalOrdinals(boolean eagerGlobalOrdinals) {
fieldType().setEagerGlobalOrdinals(eagerGlobalOrdinals);
return builder;
}
public Builder fielddataFrequencyFilter(double minFreq, double maxFreq, int minSegmentSize) {
fieldType().setFielddataMinFrequency(minFreq);
fieldType().setFielddataMaxFrequency(maxFreq);
fieldType().setFielddataMinSegmentSize(minSegmentSize);
return builder;
}
@Override
protected void setupFieldType(BuilderContext context) {
super.setupFieldType(context);
if (fieldType().hasDocValues() && ((StringFieldType) fieldType()).fielddata()) {
((StringFieldType) fieldType()).setFielddata(false);
}
}
@Override
public StringFieldMapper build(BuilderContext context) {
// if the field is not analyzed, then by default, we should omit norms and have docs only
// index options, as probably what the user really wants
// if they are set explicitly, we will use those values
// we also change the values on the default field type so that toXContent emits what
// differs from the defaults
if (fieldType.indexOptions() != IndexOptions.NONE && !fieldType.tokenized()) {
defaultFieldType.setOmitNorms(true);
defaultFieldType.setIndexOptions(IndexOptions.DOCS);
if (!omitNormsSet && fieldType.boost() == 1.0f) {
fieldType.setOmitNorms(true);
}
if (!indexOptionsSet) {
fieldType.setIndexOptions(IndexOptions.DOCS);
}
}
if (positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
if (fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
throw new IllegalArgumentException("Cannot set position_increment_gap on field ["
+ name + "] without positions enabled");
}
fieldType.setIndexAnalyzer(new NamedAnalyzer(fieldType.indexAnalyzer(), positionIncrementGap));
fieldType.setSearchAnalyzer(new NamedAnalyzer(fieldType.searchAnalyzer(), positionIncrementGap));
fieldType.setSearchQuoteAnalyzer(new NamedAnalyzer(fieldType.searchQuoteAnalyzer(), positionIncrementGap));
}
setupFieldType(context);
return new StringFieldMapper(
name, fieldType(), defaultFieldType, positionIncrementGap, ignoreAbove, includeInAll,
context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo);
}
}
public static class TypeParser implements Mapper.TypeParser {
private final DeprecationLogger deprecationLogger;
public TypeParser() {
Logger logger = Loggers.getLogger(getClass());
this.deprecationLogger = new DeprecationLogger(logger);
}
@Override
public Mapper.Builder parse(String fieldName, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0_alpha1)) {
final Object index = node.get("index");
if (Arrays.asList(null, "no", "not_analyzed", "analyzed").contains(index) == false) {
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [no], [not_analyzed] or [analyzed]");
}
final boolean keyword = index != null && "analyzed".equals(index) == false;
// Automatically upgrade simple mappings for ease of upgrade, otherwise fail
Set<String> autoUpgradeParameters = keyword
? SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE_TO_KEYWORD
: SUPPORTED_PARAMETERS_FOR_AUTO_UPGRADE_TO_TEXT;
if (autoUpgradeParameters.containsAll(node.keySet())) {
deprecationLogger.deprecated("The [string] field is deprecated, please use [text] or [keyword] instead on [{}]",
fieldName);
{
// upgrade the index setting
node.put("index", "no".equals(index) == false);
}
{
// upgrade norms settings
Object norms = node.remove("norms");
if (norms instanceof Map) {
norms = ((Map<?,?>) norms).get("enabled");
}
if (norms != null) {
node.put("norms", TypeParsers.nodeBooleanValue("norms", norms, parserContext));
}
Object omitNorms = node.remove("omit_norms");
if (omitNorms != null) {
node.put("norms", TypeParsers.nodeBooleanValue("omit_norms", omitNorms, parserContext) == false);
}
}
{
// upgrade fielddata settings
Object fielddataO = node.get("fielddata");
if (fielddataO instanceof Map) {
Map<?,?> fielddata = (Map<?, ?>) fielddataO;
if (keyword == false) {
node.put("fielddata", "disabled".equals(fielddata.get("format")) == false);
Map<?,?> fielddataFilter = (Map<?, ?>) fielddata.get("filter");
if (fielddataFilter != null) {
Map<?,?> frequencyFilter = (Map<?, ?>) fielddataFilter.get("frequency");
frequencyFilter.keySet().retainAll(Arrays.asList("min", "max", "min_segment_size"));
node.put("fielddata_frequency_filter", frequencyFilter);
}
} else {
node.remove("fielddata");
}
final Object loading = fielddata.get("loading");
if (loading != null) {
node.put("eager_global_ordinals", "eager_global_ordinals".equals(loading));
}
}
}
if (keyword) {
return new KeywordFieldMapper.TypeParser().parse(fieldName, node, parserContext);
} else {
return new TextFieldMapper.TypeParser().parse(fieldName, node, parserContext);
}
}
Set<String> unsupportedParameters = new HashSet<>(node.keySet());
unsupportedParameters.removeAll(autoUpgradeParameters);
throw new IllegalArgumentException("The [string] type is removed in 5.0 and automatic upgrade failed because parameters "
+ unsupportedParameters + " are not supported for automatic upgrades. You should now use either a [text] "
+ "or [keyword] field instead for field [" + fieldName + "]");
}
StringFieldMapper.Builder builder = new StringFieldMapper.Builder(fieldName);
// hack for the fact that string can't just accept true/false for
// the index property and still accepts no/not_analyzed/analyzed
final Object index = node.remove("index");
if (index != null) {
final String normalizedIndex = index.toString();
switch (normalizedIndex) {
case "analyzed":
builder.tokenized(true);
node.put("index", true);
break;
case "not_analyzed":
builder.tokenized(false);
node.put("index", true);
break;
case "no":
node.put("index", false);
break;
default:
throw new IllegalArgumentException("Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [no], [not_analyzed] or [analyzed]");
}
}
final Object fielddataObject = node.get("fielddata");
if (fielddataObject instanceof Map) {
Map<?,?> fielddata = (Map<?, ?>) fielddataObject;
final Object loading = fielddata.get("loading");
if (loading != null) {
node.put("eager_global_ordinals", "eager_global_ordinals".equals(loading));
}
Map<?,?> fielddataFilter = (Map<?, ?>) fielddata.get("filter");
if (fielddataFilter != null) {
Map<?,?> frequencyFilter = (Map<?, ?>) fielddataFilter.get("frequency");
frequencyFilter.keySet().retainAll(Arrays.asList("min", "max", "min_segment_size"));
node.put("fielddata_frequency_filter", frequencyFilter);
}
node.put("fielddata", "disabled".equals(fielddata.get("format")) == false);
}
parseTextField(builder, fieldName, node, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("null_value")) {
if (propNode == null) {
throw new MapperParsingException("Property [null_value] cannot be null.");
}
builder.nullValue(propNode.toString());
iterator.remove();
} else if (propName.equals("position_increment_gap")) {
int newPositionIncrementGap = XContentMapValues.nodeIntegerValue(propNode, -1);
if (newPositionIncrementGap < 0) {
throw new MapperParsingException("positions_increment_gap less than 0 aren't allowed.");
}
builder.positionIncrementGap(newPositionIncrementGap);
// we need to update to actual analyzers if they are not set in this case...
// so we can inject the position increment gap...
if (builder.fieldType().indexAnalyzer() == null) {
builder.fieldType().setIndexAnalyzer(parserContext.getIndexAnalyzers().getDefaultIndexAnalyzer());
}
if (builder.fieldType().searchAnalyzer() == null) {
builder.fieldType().setSearchAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchAnalyzer());
}
if (builder.fieldType().searchQuoteAnalyzer() == null) {
builder.fieldType().setSearchQuoteAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchQuoteAnalyzer());
}
iterator.remove();
} else if (propName.equals("ignore_above")) {
builder.ignoreAbove(XContentMapValues.nodeIntegerValue(propNode, -1));
iterator.remove();
} else if (propName.equals("fielddata")) {
builder.fielddata(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("eager_global_ordinals")) {
builder.eagerGlobalOrdinals(XContentMapValues.nodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("fielddata_frequency_filter")) {
Map<?,?> frequencyFilter = (Map<?, ?>) propNode;
double minFrequency = XContentMapValues.nodeDoubleValue(frequencyFilter.remove("min"), 0);
double maxFrequency = XContentMapValues.nodeDoubleValue(frequencyFilter.remove("max"), Integer.MAX_VALUE);
int minSegmentSize = XContentMapValues.nodeIntegerValue(frequencyFilter.remove("min_segment_size"), 0);
builder.fielddataFrequencyFilter(minFrequency, maxFrequency, minSegmentSize);
DocumentMapperParser.checkNoRemainingFields(propName, frequencyFilter, parserContext.indexVersionCreated());
iterator.remove();
}
}
return builder;
}
}
public static final class StringFieldType extends org.elasticsearch.index.mapper.StringFieldType {
private boolean fielddata;
private double fielddataMinFrequency;
private double fielddataMaxFrequency;
private int fielddataMinSegmentSize;
public StringFieldType() {
fielddata = true;
fielddataMinFrequency = Defaults.FIELDDATA_MIN_FREQUENCY;
fielddataMaxFrequency = Defaults.FIELDDATA_MAX_FREQUENCY;
fielddataMinSegmentSize = Defaults.FIELDDATA_MIN_SEGMENT_SIZE;
}
protected StringFieldType(StringFieldType ref) {
super(ref);
this.fielddata = ref.fielddata;
this.fielddataMinFrequency = ref.fielddataMinFrequency;
this.fielddataMaxFrequency = ref.fielddataMaxFrequency;
this.fielddataMinSegmentSize = ref.fielddataMinSegmentSize;
}
@Override
public boolean equals(Object o) {
if (super.equals(o) == false) {
return false;
}
StringFieldType that = (StringFieldType) o;
return fielddata == that.fielddata
&& fielddataMinFrequency == that.fielddataMinFrequency
&& fielddataMaxFrequency == that.fielddataMaxFrequency
&& fielddataMinSegmentSize == that.fielddataMinSegmentSize;
}
@Override
public int hashCode() {
return Objects.hash(super.hashCode(), fielddata,
fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
}
public StringFieldType clone() {
return new StringFieldType(this);
}
@Override
public String typeName() {
return CONTENT_TYPE;
}
@Override
public void checkCompatibility(MappedFieldType other,
List<String> conflicts, boolean strict) {
super.checkCompatibility(other, conflicts, strict);
StringFieldType otherType = (StringFieldType) other;
if (strict) {
if (fielddata() != otherType.fielddata()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update [fielddata] "
+ "across all types.");
}
if (fielddataMinFrequency() != otherType.fielddataMinFrequency()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.min] across all types.");
}
if (fielddataMaxFrequency() != otherType.fielddataMaxFrequency()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.max] across all types.");
}
if (fielddataMinSegmentSize() != otherType.fielddataMinSegmentSize()) {
conflicts.add("mapper [" + name() + "] is used by multiple types. Set update_all_types to true to update "
+ "[fielddata_frequency_filter.min_segment_size] across all types.");
}
}
}
public boolean fielddata() {
return fielddata;
}
public void setFielddata(boolean fielddata) {
checkIfFrozen();
this.fielddata = fielddata;
}
public double fielddataMinFrequency() {
return fielddataMinFrequency;
}
public void setFielddataMinFrequency(double fielddataMinFrequency) {
checkIfFrozen();
this.fielddataMinFrequency = fielddataMinFrequency;
}
public double fielddataMaxFrequency() {
return fielddataMaxFrequency;
}
public void setFielddataMaxFrequency(double fielddataMaxFrequency) {
checkIfFrozen();
this.fielddataMaxFrequency = fielddataMaxFrequency;
}
public int fielddataMinSegmentSize() {
return fielddataMinSegmentSize;
}
public void setFielddataMinSegmentSize(int fielddataMinSegmentSize) {
checkIfFrozen();
this.fielddataMinSegmentSize = fielddataMinSegmentSize;
}
@Override
public Query nullValueQuery() {
if (nullValue() == null) {
return null;
}
return termQuery(nullValue(), null);
}
@Override
public IndexFieldData.Builder fielddataBuilder() {
if (hasDocValues()) {
return new DocValuesIndexFieldData.Builder();
} else if (fielddata) {
return new PagedBytesIndexFieldData.Builder(fielddataMinFrequency, fielddataMaxFrequency, fielddataMinSegmentSize);
} else {
throw new IllegalArgumentException("Fielddata is disabled on analyzed string fields by default. Set fielddata=true on ["
+ name() + "] in order to load fielddata in memory by uninverting the inverted index. Note that this can however "
+ "use significant memory.");
}
}
}
private Boolean includeInAll;
private int positionIncrementGap;
private int ignoreAbove;
protected StringFieldMapper(String simpleName, StringFieldType fieldType, MappedFieldType defaultFieldType,
int positionIncrementGap, int ignoreAbove, Boolean includeInAll,
Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo);
if (Version.indexCreated(indexSettings).onOrAfter(Version.V_5_0_0_alpha1)) {
throw new IllegalArgumentException("The [string] type is removed in 5.0. You should now use either a [text] "
+ "or [keyword] field instead for field [" + fieldType.name() + "]");
}
if (fieldType.tokenized() && fieldType.indexOptions() != NONE && fieldType().hasDocValues()) {
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot be analyzed and have doc values");
}
if (fieldType.hasDocValues() && (
fieldType.fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY
|| fieldType.fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY
|| fieldType.fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE)) {
throw new MapperParsingException("Field [" + fieldType.name() + "] cannot have doc values and use fielddata filtering");
}
this.positionIncrementGap = positionIncrementGap;
this.ignoreAbove = ignoreAbove;
this.includeInAll = includeInAll;
}
@Override
protected StringFieldMapper clone() {
return (StringFieldMapper) super.clone();
}
@Override
protected boolean customBoost() {
return true;
}
public int getPositionIncrementGap() {
return this.positionIncrementGap;
}
public int getIgnoreAbove() {
return ignoreAbove;
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
ValueAndBoost valueAndBoost = parseCreateFieldForString(context, fieldType().nullValueAsString(), fieldType().boost());
if (valueAndBoost.value() == null) {
return;
}
if (ignoreAbove > 0 && valueAndBoost.value().length() > ignoreAbove) {
return;
}
if (context.includeInAll(includeInAll, this)) {
context.allEntries().addText(fieldType().name(), valueAndBoost.value(), valueAndBoost.boost());
}
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
Field field = new Field(fieldType().name(), valueAndBoost.value(), fieldType());
if (valueAndBoost.boost() != 1f && Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
field.setBoost(valueAndBoost.boost());
}
fields.add(field);
}
if (fieldType().hasDocValues()) {
fields.add(new SortedSetDocValuesField(fieldType().name(), new BytesRef(valueAndBoost.value())));
}
}
/**
* Parse a field as though it were a string.
* @param context parse context used during parsing
* @param nullValue value to use for null
* @param defaultBoost default boost value returned unless overwritten in the field
* @return the parsed field and the boost either parsed or defaulted
* @throws IOException if thrown while parsing
*/
public static ValueAndBoost parseCreateFieldForString(ParseContext context, String nullValue, float defaultBoost) throws IOException {
if (context.externalValueSet()) {
return new ValueAndBoost(context.externalValue().toString(), defaultBoost);
}
XContentParser parser = context.parser();
if (parser.currentToken() == XContentParser.Token.VALUE_NULL) {
return new ValueAndBoost(nullValue, defaultBoost);
}
if (parser.currentToken() == XContentParser.Token.START_OBJECT
&& Version.indexCreated(context.indexSettings()).before(Version.V_5_0_0_alpha1)) {
XContentParser.Token token;
String currentFieldName = null;
String value = nullValue;
float boost = defaultBoost;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else {
if ("value".equals(currentFieldName) || "_value".equals(currentFieldName)) {
value = parser.textOrNull();
} else if ("boost".equals(currentFieldName) || "_boost".equals(currentFieldName)) {
boost = parser.floatValue();
} else {
throw new IllegalArgumentException("unknown property [" + currentFieldName + "]");
}
}
}
return new ValueAndBoost(value, boost);
}
return new ValueAndBoost(parser.textOrNull(), defaultBoost);
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
super.doMerge(mergeWith, updateAllTypes);
this.includeInAll = ((StringFieldMapper) mergeWith).includeInAll;
this.ignoreAbove = ((StringFieldMapper) mergeWith).ignoreAbove;
}
@Override
protected String indexTokenizeOption(boolean indexed, boolean tokenized) {
if (!indexed) {
return "no";
} else if (tokenized) {
return "analyzed";
} else {
return "not_analyzed";
}
}
@Override
public StringFieldType fieldType() {
return (StringFieldType) super.fieldType();
}
@Override
protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException {
super.doXContentBody(builder, includeDefaults, params);
doXContentAnalyzers(builder, includeDefaults);
if (includeDefaults || fieldType().nullValue() != null) {
builder.field("null_value", fieldType().nullValue());
}
if (includeInAll != null) {
builder.field("include_in_all", includeInAll);
} else if (includeDefaults) {
builder.field("include_in_all", false);
}
if (includeDefaults || positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
builder.field("position_increment_gap", positionIncrementGap);
}
if (includeDefaults || ignoreAbove != Defaults.IGNORE_ABOVE) {
builder.field("ignore_above", ignoreAbove);
}
if (includeDefaults || fieldType().fielddata() != ((StringFieldType) defaultFieldType).fielddata()) {
builder.field("fielddata", fieldType().fielddata());
}
if (fieldType().fielddata()) {
if (includeDefaults
|| fieldType().fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY
|| fieldType().fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY
|| fieldType().fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE) {
builder.startObject("fielddata_frequency_filter");
if (includeDefaults || fieldType().fielddataMinFrequency() != Defaults.FIELDDATA_MIN_FREQUENCY) {
builder.field("min", fieldType().fielddataMinFrequency());
}
if (includeDefaults || fieldType().fielddataMaxFrequency() != Defaults.FIELDDATA_MAX_FREQUENCY) {
builder.field("max", fieldType().fielddataMaxFrequency());
}
if (includeDefaults || fieldType().fielddataMinSegmentSize() != Defaults.FIELDDATA_MIN_SEGMENT_SIZE) {
builder.field("min_segment_size", fieldType().fielddataMinSegmentSize());
}
builder.endObject();
}
}
}
/**
* Parsed value and boost to be returned from {@link #parseCreateFieldForString}.
*/
public static class ValueAndBoost {
private final String value;
private final float boost;
public ValueAndBoost(String value, float boost) {
this.value = value;
this.boost = boost;
}
/**
* Value of string field.
* @return value of string field
*/
public String value() {
return value;
}
/**
* Boost either parsed from the document or defaulted.
* @return boost either parsed from the document or defaulted
*/
public float boost() {
return boost;
}
}
}

View File

@ -1,266 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.elasticsearch.Version;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.AlreadyExpiredException;
import java.io.IOException;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenientNodeBooleanValue;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeTimeValue;
public class TTLFieldMapper extends MetadataFieldMapper {
public static final String NAME = "_ttl";
public static final String CONTENT_TYPE = "_ttl";
public static class Defaults extends LegacyLongFieldMapper.Defaults {
public static final String NAME = TTLFieldMapper.CONTENT_TYPE;
public static final TTLFieldType TTL_FIELD_TYPE = new TTLFieldType();
static {
TTL_FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
TTL_FIELD_TYPE.setStored(true);
TTL_FIELD_TYPE.setTokenized(false);
TTL_FIELD_TYPE.setNumericPrecisionStep(Defaults.PRECISION_STEP_64_BIT);
TTL_FIELD_TYPE.setName(NAME);
TTL_FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
TTL_FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
TTL_FIELD_TYPE.freeze();
}
public static final EnabledAttributeMapper ENABLED_STATE = EnabledAttributeMapper.UNSET_DISABLED;
public static final long DEFAULT = -1;
}
public static class Builder extends MetadataFieldMapper.Builder<Builder, TTLFieldMapper> {
private EnabledAttributeMapper enabledState = EnabledAttributeMapper.UNSET_DISABLED;
private long defaultTTL = Defaults.DEFAULT;
public Builder() {
super(Defaults.NAME, Defaults.TTL_FIELD_TYPE, Defaults.FIELD_TYPE);
}
public Builder enabled(EnabledAttributeMapper enabled) {
this.enabledState = enabled;
return builder;
}
public Builder defaultTTL(long defaultTTL) {
this.defaultTTL = defaultTTL;
return builder;
}
@Override
public TTLFieldMapper build(BuilderContext context) {
setupFieldType(context);
fieldType.setHasDocValues(false);
return new TTLFieldMapper(fieldType, enabledState, defaultTTL, context.indexSettings());
}
}
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0_alpha4)) {
throw new IllegalArgumentException("[_ttl] is removed in 5.0. As a replacement, you should use time based indexes or cron a delete-by-query with a range query on a timestamp field.");
}
Builder builder = new Builder();
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = entry.getKey();
Object fieldNode = entry.getValue();
if (fieldName.equals("enabled")) {
EnabledAttributeMapper enabledState = lenientNodeBooleanValue(fieldNode) ? EnabledAttributeMapper.ENABLED : EnabledAttributeMapper.DISABLED;
builder.enabled(enabledState);
iterator.remove();
} else if (fieldName.equals("default")) {
TimeValue ttlTimeValue = nodeTimeValue(fieldNode, null);
if (ttlTimeValue != null) {
builder.defaultTTL(ttlTimeValue.millis());
}
iterator.remove();
}
}
return builder;
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new TTLFieldMapper(indexSettings);
}
}
public static final class TTLFieldType extends LegacyLongFieldMapper.LongFieldType {
public TTLFieldType() {
}
protected TTLFieldType(TTLFieldType ref) {
super(ref);
}
@Override
public TTLFieldType clone() {
return new TTLFieldType(this);
}
// Overrides valueForSearch to display live value of remaining ttl
@Override
public Object valueForDisplay(Object value) {
final long now = System.currentTimeMillis();
Long val = (Long) super.valueForDisplay(value);
return val - now;
}
}
private EnabledAttributeMapper enabledState;
private long defaultTTL;
private TTLFieldMapper(Settings indexSettings) {
this(Defaults.TTL_FIELD_TYPE.clone(), Defaults.ENABLED_STATE, Defaults.DEFAULT, indexSettings);
}
private TTLFieldMapper(MappedFieldType fieldType, EnabledAttributeMapper enabled, long defaultTTL,
Settings indexSettings) {
super(NAME, fieldType, Defaults.TTL_FIELD_TYPE, indexSettings);
if (enabled.enabled && Version.indexCreated(indexSettings).onOrAfter(Version.V_5_0_0_alpha4)) {
throw new IllegalArgumentException("[_ttl] is removed in 5.0. As a replacement, you should use time based indexes or cron a delete-by-query with a range query on a timestamp field.");
}
this.enabledState = enabled;
this.defaultTTL = defaultTTL;
}
public boolean enabled() {
return this.enabledState.enabled;
}
public long defaultTTL() {
return this.defaultTTL;
}
@Override
public void preParse(ParseContext context) throws IOException {
}
@Override
public void postParse(ParseContext context) throws IOException {
super.parse(context);
}
@Override
public Mapper parse(ParseContext context) throws IOException, MapperParsingException {
if (context.sourceToParse().ttl() < 0) { // no ttl has been provided externally
long ttl;
if (context.parser().currentToken() == XContentParser.Token.VALUE_STRING) {
ttl = TimeValue.parseTimeValue(context.parser().text(), null, "ttl").millis();
} else {
ttl = context.parser().longValue(true);
}
if (ttl <= 0) {
throw new MapperParsingException("TTL value must be > 0. Illegal value provided [" + ttl + "]");
}
context.sourceToParse().ttl(ttl);
}
return null;
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException, AlreadyExpiredException {
if (enabledState.enabled) {
long ttl = context.sourceToParse().ttl();
if (ttl <= 0 && defaultTTL > 0) { // no ttl provided so we use the default value
ttl = defaultTTL;
context.sourceToParse().ttl(ttl);
}
if (ttl > 0) { // a ttl has been provided either externally or in the _source
long timestamp = context.sourceToParse().timestamp();
long expire = new Date(timestamp + ttl).getTime();
long now = System.currentTimeMillis();
// there is not point indexing already expired doc
if (context.sourceToParse().origin() == SourceToParse.Origin.PRIMARY && now >= expire) {
throw new AlreadyExpiredException(context.sourceToParse().index(),
context.sourceToParse().type(), context.sourceToParse().id(), timestamp, ttl, now);
}
// the expiration timestamp (timestamp + ttl) is set as field
fields.add(new LegacyLongFieldMapper.CustomLongNumericField(expire, fieldType()));
}
}
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
boolean includeDefaults = params.paramAsBoolean("include_defaults", false);
// if all are defaults, no sense to write it at all
if (!includeDefaults && enabledState == Defaults.ENABLED_STATE && defaultTTL == Defaults.DEFAULT) {
return builder;
}
builder.startObject(CONTENT_TYPE);
if (includeDefaults || enabledState != Defaults.ENABLED_STATE) {
builder.field("enabled", enabledState.enabled);
}
if (includeDefaults || defaultTTL != Defaults.DEFAULT && enabledState.enabled) {
builder.field("default", defaultTTL);
}
builder.endObject();
return builder;
}
@Override
protected String contentType() {
return NAME;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
TTLFieldMapper ttlMergeWith = (TTLFieldMapper) mergeWith;
if (ttlMergeWith.enabledState != Defaults.ENABLED_STATE) {//only do something if actually something was set for the document mapper that we merge with
if (this.enabledState == EnabledAttributeMapper.ENABLED && ttlMergeWith.enabledState == EnabledAttributeMapper.DISABLED) {
throw new IllegalArgumentException("_ttl cannot be disabled once it was enabled.");
} else {
this.enabledState = ttlMergeWith.enabledState;
}
}
if (ttlMergeWith.defaultTTL != -1) {
// we never build the default when the field is disabled so we should also not set it
// (it does not make a difference though as everything that is not build in toXContent will also not be set in the cluster)
if (enabledState == EnabledAttributeMapper.ENABLED) {
this.defaultTTL = ttlMergeWith.defaultTTL;
}
}
}
}

View File

@ -23,7 +23,6 @@ import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.Query;
import org.elasticsearch.Version;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.support.XContentMapValues;
@ -32,15 +31,11 @@ import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import static java.util.Collections.unmodifiableList;
import static org.elasticsearch.index.mapper.TypeParsers.parseTextField;
/** A {@link FieldMapper} for full-text fields. */
@ -49,14 +44,6 @@ public class TextFieldMapper extends FieldMapper {
public static final String CONTENT_TYPE = "text";
private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1;
private static final List<String> SUPPORTED_PARAMETERS_FOR_AUTO_DOWNGRADE_TO_STRING = unmodifiableList(Arrays.asList(
"type",
// common text parameters, for which the upgrade is straightforward
"index", "store", "doc_values", "omit_norms", "norms", "boost", "fields", "copy_to",
"fielddata", "eager_global_ordinals", "fielddata_frequency_filter", "include_in_all",
"analyzer", "search_analyzer", "search_quote_analyzer",
"index_options", "position_increment_gap", "similarity"));
public static class Defaults {
public static double FIELDDATA_MIN_FREQUENCY = 0;
public static double FIELDDATA_MAX_FREQUENCY = Integer.MAX_VALUE;
@ -143,41 +130,6 @@ public class TextFieldMapper extends FieldMapper {
public static class TypeParser implements Mapper.TypeParser {
@Override
public Mapper.Builder parse(String fieldName, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha1)) {
// Downgrade "text" to "string" in indexes created in 2.x so you can use modern syntax against old indexes
Set<String> unsupportedParameters = new HashSet<>(node.keySet());
unsupportedParameters.removeAll(SUPPORTED_PARAMETERS_FOR_AUTO_DOWNGRADE_TO_STRING);
if (false == SUPPORTED_PARAMETERS_FOR_AUTO_DOWNGRADE_TO_STRING.containsAll(node.keySet())) {
throw new IllegalArgumentException("Automatic downgrade from [text] to [string] failed because parameters "
+ unsupportedParameters + " are not supported for automatic downgrades.");
}
{ // Downgrade "index"
Object index = node.get("index");
if (index == null || Boolean.TRUE.equals(index)) {
index = "analyzed";
} else if (Boolean.FALSE.equals(index)) {
index = "no";
} else {
throw new IllegalArgumentException(
"Can't parse [index] value [" + index + "] for field [" + fieldName + "], expected [true] or [false]");
}
node.put("index", index);
}
{ // Downgrade "fielddata" (default in string is true, default in text is false)
Object fielddata = node.get("fielddata");
if (fielddata == null || Boolean.FALSE.equals(fielddata)) {
fielddata = false;
} else if (Boolean.TRUE.equals(fielddata)) {
fielddata = true;
} else {
throw new IllegalArgumentException("can't parse [fielddata] value for [" + fielddata + "] for field ["
+ fieldName + "], expected [true] or [false]");
}
node.put("fielddata", fielddata);
}
return new StringFieldMapper.TypeParser().parse(fieldName, node, parserContext);
}
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(fieldName);
builder.fieldType().setIndexAnalyzer(parserContext.getIndexAnalyzers().getDefaultIndexAnalyzer());
builder.fieldType().setSearchAnalyzer(parserContext.getIndexAnalyzers().getDefaultSearchAnalyzer());

View File

@ -1,310 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.IndexableField;
import org.elasticsearch.Version;
import org.elasticsearch.action.TimestampParsingException;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenientNodeBooleanValue;
import static org.elasticsearch.index.mapper.TypeParsers.parseDateTimeFormatter;
public class TimestampFieldMapper extends MetadataFieldMapper {
public static final String NAME = "_timestamp";
public static final String CONTENT_TYPE = "_timestamp";
public static final String DEFAULT_DATE_TIME_FORMAT = "epoch_millis||strictDateOptionalTime";
public static class Defaults extends LegacyDateFieldMapper.Defaults {
public static final String NAME = "_timestamp";
// TODO: this should be removed
public static final TimestampFieldType FIELD_TYPE = new TimestampFieldType();
public static final FormatDateTimeFormatter DATE_TIME_FORMATTER = Joda.forPattern(DEFAULT_DATE_TIME_FORMAT);
static {
FIELD_TYPE.setStored(true);
FIELD_TYPE.setTokenized(false);
FIELD_TYPE.setNumericPrecisionStep(Defaults.PRECISION_STEP_64_BIT);
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setDateTimeFormatter(DATE_TIME_FORMATTER);
FIELD_TYPE.setIndexAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setSearchAnalyzer(Lucene.KEYWORD_ANALYZER);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.freeze();
}
public static final EnabledAttributeMapper ENABLED = EnabledAttributeMapper.UNSET_DISABLED;
public static final String DEFAULT_TIMESTAMP = "now";
}
public static class Builder extends MetadataFieldMapper.Builder<Builder, TimestampFieldMapper> {
private EnabledAttributeMapper enabledState = EnabledAttributeMapper.UNSET_DISABLED;
private String defaultTimestamp = Defaults.DEFAULT_TIMESTAMP;
private Boolean ignoreMissing = null;
public Builder(MappedFieldType existing) {
super(Defaults.NAME, existing == null ? Defaults.FIELD_TYPE : existing, Defaults.FIELD_TYPE);
}
@Override
public LegacyDateFieldMapper.DateFieldType fieldType() {
return (LegacyDateFieldMapper.DateFieldType)fieldType;
}
public Builder enabled(EnabledAttributeMapper enabledState) {
this.enabledState = enabledState;
return builder;
}
public Builder dateTimeFormatter(FormatDateTimeFormatter dateTimeFormatter) {
fieldType().setDateTimeFormatter(dateTimeFormatter);
return this;
}
public Builder defaultTimestamp(String defaultTimestamp) {
this.defaultTimestamp = defaultTimestamp;
return builder;
}
public Builder ignoreMissing(boolean ignoreMissing) {
this.ignoreMissing = ignoreMissing;
return builder;
}
@Override
public Builder store(boolean store) {
return super.store(store);
}
@Override
public TimestampFieldMapper build(BuilderContext context) {
setupFieldType(context);
return new TimestampFieldMapper(fieldType, defaultFieldType, enabledState, defaultTimestamp,
ignoreMissing, context.indexSettings());
}
}
public static class TypeParser implements MetadataFieldMapper.TypeParser {
@Override
public MetadataFieldMapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().onOrAfter(Version.V_5_0_0_alpha4)) {
throw new IllegalArgumentException("[_timestamp] is removed in 5.0. As a replacement, you can use an ingest pipeline to add a field with the current timestamp to your documents.");
}
Builder builder = new Builder(parserContext.mapperService().fullName(NAME));
boolean defaultSet = false;
Boolean ignoreMissing = null;
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String fieldName = entry.getKey();
Object fieldNode = entry.getValue();
if (fieldName.equals("enabled")) {
EnabledAttributeMapper enabledState = lenientNodeBooleanValue(fieldNode) ? EnabledAttributeMapper.ENABLED : EnabledAttributeMapper.DISABLED;
builder.enabled(enabledState);
iterator.remove();
} else if (fieldName.equals("format")) {
builder.dateTimeFormatter(parseDateTimeFormatter(fieldNode.toString()));
iterator.remove();
} else if (fieldName.equals("default")) {
if (fieldNode == null) {
throw new TimestampParsingException("default timestamp can not be set to null");
} else {
builder.defaultTimestamp(fieldNode.toString());
defaultSet = true;
}
iterator.remove();
} else if (fieldName.equals("ignore_missing")) {
ignoreMissing = lenientNodeBooleanValue(fieldNode);
builder.ignoreMissing(ignoreMissing);
iterator.remove();
}
}
// We can not accept a default value and rejecting null values at the same time
if (defaultSet && (ignoreMissing != null && ignoreMissing == false)) {
throw new TimestampParsingException("default timestamp can not be set with ignore_missing set to false");
}
return builder;
}
@Override
public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) {
return new TimestampFieldMapper(indexSettings, fieldType);
}
}
public static final class TimestampFieldType extends LegacyDateFieldMapper.DateFieldType {
public TimestampFieldType() {}
protected TimestampFieldType(TimestampFieldType ref) {
super(ref);
}
@Override
public TimestampFieldType clone() {
return new TimestampFieldType(this);
}
@Override
public Object valueForDisplay(Object value) {
return value;
}
}
private EnabledAttributeMapper enabledState;
private final String defaultTimestamp;
private final Boolean ignoreMissing;
private TimestampFieldMapper(Settings indexSettings, MappedFieldType existing) {
this(existing != null ? existing : Defaults.FIELD_TYPE, Defaults.FIELD_TYPE, Defaults.ENABLED, Defaults.DEFAULT_TIMESTAMP, null, indexSettings);
}
private TimestampFieldMapper(MappedFieldType fieldType, MappedFieldType defaultFieldType, EnabledAttributeMapper enabledState,
String defaultTimestamp, Boolean ignoreMissing, Settings indexSettings) {
super(NAME, fieldType, defaultFieldType, indexSettings);
if (enabledState.enabled && Version.indexCreated(indexSettings).onOrAfter(Version.V_5_0_0_alpha4)) {
throw new IllegalArgumentException("[_timestamp] is removed in 5.0. As a replacement, you can use an ingest pipeline to add a field with the current timestamp to your documents.");
}
this.enabledState = enabledState;
this.defaultTimestamp = defaultTimestamp;
this.ignoreMissing = ignoreMissing;
}
@Override
public TimestampFieldType fieldType() {
return (TimestampFieldType)super.fieldType();
}
public boolean enabled() {
return this.enabledState.enabled;
}
public String defaultTimestamp() {
return this.defaultTimestamp;
}
public Boolean ignoreMissing() {
return this.ignoreMissing;
}
@Override
public void preParse(ParseContext context) throws IOException {
super.parse(context);
}
@Override
public void postParse(ParseContext context) throws IOException {
}
@Override
public Mapper parse(ParseContext context) throws IOException {
// nothing to do here, we call the parent in preParse
return null;
}
@Override
protected void parseCreateField(ParseContext context, List<IndexableField> fields) throws IOException {
if (enabledState.enabled) {
long timestamp = context.sourceToParse().timestamp();
if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) {
fields.add(new LegacyLongFieldMapper.CustomLongNumericField(timestamp, fieldType()));
}
if (fieldType().hasDocValues()) {
fields.add(new NumericDocValuesField(fieldType().name(), timestamp));
}
}
}
@Override
protected String contentType() {
return CONTENT_TYPE;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
boolean includeDefaults = params.paramAsBoolean("include_defaults", false);
// if all are defaults, no sense to write it at all
if (!includeDefaults && enabledState == Defaults.ENABLED
&& fieldType().dateTimeFormatter().format().equals(Defaults.DATE_TIME_FORMATTER.format())
&& Defaults.DEFAULT_TIMESTAMP.equals(defaultTimestamp)) {
return builder;
}
builder.startObject(CONTENT_TYPE);
if (includeDefaults || enabledState != Defaults.ENABLED) {
builder.field("enabled", enabledState.enabled);
}
// different format handling depending on index version
String defaultDateFormat = Defaults.DATE_TIME_FORMATTER.format();
if (includeDefaults || !fieldType().dateTimeFormatter().format().equals(defaultDateFormat)) {
builder.field("format", fieldType().dateTimeFormatter().format());
}
if (includeDefaults || !Defaults.DEFAULT_TIMESTAMP.equals(defaultTimestamp)) {
builder.field("default", defaultTimestamp);
}
if (includeDefaults || ignoreMissing != null) {
builder.field("ignore_missing", ignoreMissing);
}
builder.endObject();
return builder;
}
@Override
protected void doMerge(Mapper mergeWith, boolean updateAllTypes) {
TimestampFieldMapper timestampFieldMapperMergeWith = (TimestampFieldMapper) mergeWith;
super.doMerge(mergeWith, updateAllTypes);
if (timestampFieldMapperMergeWith.enabledState != enabledState && !timestampFieldMapperMergeWith.enabledState.unset()) {
this.enabledState = timestampFieldMapperMergeWith.enabledState;
}
if (timestampFieldMapperMergeWith.defaultTimestamp() == null && defaultTimestamp == null) {
return;
}
List<String> conflicts = new ArrayList<>();
if (defaultTimestamp == null) {
conflicts.add("Cannot update default in _timestamp value. Value is null now encountering " + timestampFieldMapperMergeWith.defaultTimestamp());
} else if (timestampFieldMapperMergeWith.defaultTimestamp() == null) {
conflicts.add("Cannot update default in _timestamp value. Value is \" + defaultTimestamp.toString() + \" now encountering null");
} else if (!timestampFieldMapperMergeWith.defaultTimestamp().equals(defaultTimestamp)) {
conflicts.add("Cannot update default in _timestamp value. Value is " + defaultTimestamp.toString() + " now encountering " + timestampFieldMapperMergeWith.defaultTimestamp());
}
if (conflicts.isEmpty() == false) {
throw new IllegalArgumentException("Conflicts: " + conflicts);
}
}
}

View File

@ -40,7 +40,7 @@ import static org.elasticsearch.index.mapper.TypeParsers.parseField;
/**
* A {@link FieldMapper} that takes a string and writes a count of the tokens in that string
* to the index. In most ways the mapper acts just like an {@link LegacyIntegerFieldMapper}.
* to the index. In most ways the mapper acts just like an {@link NumberFieldMapper}.
*/
public class TokenCountFieldMapper extends FieldMapper {
public static final String CONTENT_TYPE = "token_count";
@ -78,9 +78,6 @@ public class TokenCountFieldMapper extends FieldMapper {
@Override
@SuppressWarnings("unchecked")
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
if (parserContext.indexVersionCreated().before(Version.V_5_0_0_alpha2)) {
return new LegacyTokenCountFieldMapper.TypeParser().parse(name, node, parserContext);
}
TokenCountFieldMapper.Builder builder = new TokenCountFieldMapper.Builder(name);
for (Iterator<Map.Entry<String, Object>> iterator = node.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();

View File

@ -34,7 +34,6 @@ import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
@ -257,13 +256,7 @@ public class TypeFieldMapper extends MetadataFieldMapper {
private static MappedFieldType defaultFieldType(Settings indexSettings) {
MappedFieldType defaultFieldType = Defaults.FIELD_TYPE.clone();
Version indexCreated = Version.indexCreated(indexSettings);
if (indexCreated.before(Version.V_2_1_0)) {
// enables fielddata loading, doc values was disabled on _type between 2.0 and 2.1.
((TypeFieldType) defaultFieldType).setFielddata(true);
} else {
defaultFieldType.setHasDocValues(true);
}
defaultFieldType.setHasDocValues(true);
return defaultFieldType;
}

View File

@ -71,32 +71,6 @@ public class TypeParsers {
}
}
@Deprecated // for legacy ints only
public static void parseNumberField(LegacyNumberFieldMapper.Builder builder, String name, Map<String, Object> numberNode, Mapper.TypeParser.ParserContext parserContext) {
parseField(builder, name, numberNode, parserContext);
for (Iterator<Map.Entry<String, Object>> iterator = numberNode.entrySet().iterator(); iterator.hasNext();) {
Map.Entry<String, Object> entry = iterator.next();
String propName = entry.getKey();
Object propNode = entry.getValue();
if (propName.equals("precision_step")) {
builder.precisionStep(nodeIntegerValue(propNode));
iterator.remove();
} else if (propName.equals("ignore_malformed")) {
builder.ignoreMalformed(nodeBooleanValue("ignore_malformed", propNode, parserContext));
iterator.remove();
} else if (propName.equals("coerce")) {
builder.coerce(nodeBooleanValue("coerce", propNode, parserContext));
iterator.remove();
} else if (propName.equals("similarity")) {
SimilarityProvider similarityProvider = resolveSimilarity(parserContext, name, propNode.toString());
builder.similarity(similarityProvider);
iterator.remove();
} else if (parseMultiField(builder, name, parserContext, propName, propNode)) {
iterator.remove();
}
}
}
private static void parseAnalyzersAndTermVectors(FieldMapper.Builder builder, String name, Map<String, Object> fieldNode, Mapper.TypeParser.ParserContext parserContext) {
NamedAnalyzer indexAnalyzer = null;
NamedAnalyzer searchAnalyzer = null;
@ -270,23 +244,13 @@ public class TypeParsers {
iterator.remove();
} else if (propName.equals("copy_to")) {
if (parserContext.isWithinMultiField()) {
if (indexVersionCreated.after(Version.V_2_1_0) ||
(indexVersionCreated.after(Version.V_2_0_1) && indexVersionCreated.before(Version.V_2_1_0))) {
throw new MapperParsingException("copy_to in multi fields is not allowed. Found the copy_to in field [" + name + "] which is within a multi field.");
} else {
ESLoggerFactory.getLogger("mapping [" + parserContext.type() + "]").warn("Found a copy_to in field [{}] which is within a multi field. This feature has been removed and the copy_to will be removed from the mapping.", name);
}
throw new MapperParsingException("copy_to in multi fields is not allowed. Found the copy_to in field [" + name + "] which is within a multi field.");
} else {
parseCopyFields(propNode, builder);
}
iterator.remove();
}
}
if (indexVersionCreated.before(Version.V_2_2_0)) {
// analyzer, search_analyzer, term_vectors were accepted on all fields
// before 2.2, even though it made little sense
parseAnalyzersAndTermVectors(builder, name, fieldNode, parserContext);
}
}
public static boolean parseMultiField(FieldMapper.Builder builder, String name, Mapper.TypeParser.ParserContext parserContext, String propName, Object propNode) {

View File

@ -23,8 +23,6 @@ import org.apache.lucene.document.LatLonPoint;
import org.apache.lucene.geo.Rectangle;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.apache.lucene.spatial.geopoint.search.GeoPointInBBoxQuery;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Numbers;
@ -37,13 +35,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper.LegacyGeoPointFieldType;
import org.elasticsearch.index.mapper.LatLonPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.search.geo.LegacyInMemoryGeoBoundingBoxQuery;
import org.elasticsearch.index.search.geo.LegacyIndexedGeoBoundingBoxQuery;
import java.io.IOException;
import java.util.Objects;
@ -360,34 +353,8 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
}
}
if (indexVersionCreated.onOrAfter(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
return LatLonPoint.newBoxQuery(fieldType.name(), luceneBottomRight.getLat(), luceneTopLeft.getLat(),
luceneTopLeft.getLon(), luceneBottomRight.getLon());
} else if (indexVersionCreated.onOrAfter(Version.V_2_2_0)) {
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
// if index created V_2_3 > use prefix encoded postings format
final GeoPointField.TermEncoding encoding = (indexVersionCreated.before(Version.V_2_3_0)) ?
GeoPointField.TermEncoding.NUMERIC : GeoPointField.TermEncoding.PREFIX;
return new GeoPointInBBoxQuery(fieldType.name(), encoding, luceneBottomRight.lat(), luceneTopLeft.lat(),
luceneTopLeft.lon(), luceneBottomRight.lon());
}
Query query;
switch(type) {
case INDEXED:
LegacyGeoPointFieldType geoFieldType = ((LegacyGeoPointFieldType) fieldType);
query = LegacyIndexedGeoBoundingBoxQuery.create(luceneTopLeft, luceneBottomRight, geoFieldType, context);
break;
case MEMORY:
IndexGeoPointFieldData indexFieldData = context.getForField(fieldType);
query = new LegacyInMemoryGeoBoundingBoxQuery(luceneTopLeft, luceneBottomRight, indexFieldData);
break;
default:
// Someone extended the type enum w/o adjusting this switch statement.
throw new IllegalStateException("geo bounding box type [" + type + "] not supported.");
}
return query;
return LatLonPoint.newBoxQuery(fieldType.name(), luceneBottomRight.getLat(), luceneTopLeft.getLat(),
luceneTopLeft.getLon(), luceneBottomRight.getLon());
}
@Override

View File

@ -22,8 +22,6 @@ package org.elasticsearch.index.query;
import org.apache.lucene.document.LatLonPoint;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.apache.lucene.spatial.geopoint.search.GeoPointDistanceQuery;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
@ -36,13 +34,8 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.LatLonPointFieldMapper;
import org.elasticsearch.index.mapper.LegacyGeoPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.search.geo.GeoDistanceRangeQuery;
import java.io.IOException;
import java.util.Locale;
@ -299,25 +292,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
double normDistance = geoDistance.normalize(this.distance, DistanceUnit.DEFAULT);
if (indexVersionCreated.onOrAfter(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
return LatLonPoint.newDistanceQuery(fieldType.name(), center.lat(), center.lon(), normDistance);
} else if (indexVersionCreated.before(Version.V_2_2_0)) {
LegacyGeoPointFieldMapper.LegacyGeoPointFieldType geoFieldType = (LegacyGeoPointFieldMapper.LegacyGeoPointFieldType) fieldType;
IndexGeoPointFieldData indexFieldData = shardContext.getForField(fieldType);
String bboxOptimization = Strings.isEmpty(optimizeBbox) ? DEFAULT_OPTIMIZE_BBOX : optimizeBbox;
return new GeoDistanceRangeQuery(center, null, normDistance, true, false, geoDistance,
geoFieldType, indexFieldData, bboxOptimization, shardContext);
}
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
// if index created V_2_3 > use prefix encoded postings format
final GeoPointField.TermEncoding encoding = (indexVersionCreated.before(Version.V_2_3_0)) ?
GeoPointField.TermEncoding.NUMERIC : GeoPointField.TermEncoding.PREFIX;
// Lucene 6.0 and earlier requires a radial restriction
if (indexVersionCreated.before(Version.V_5_0_0_alpha4)) {
normDistance = GeoUtils.maxRadialDistance(center, normDistance);
}
return new GeoPointDistanceQuery(fieldType.name(), encoding, center.lat(), center.lon(), normDistance);
return LatLonPoint.newDistanceQuery(fieldType.name(), center.lat(), center.lon(), normDistance);
}
@Override
@ -371,11 +346,11 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
if (token == XContentParser.Token.FIELD_NAME) {
currentName = parser.currentName();
} else if (token.isValue()) {
if (currentName.equals(GeoPointFieldMapper.Names.LAT)) {
if (currentName.equals("lat")) {
point.resetLat(parser.doubleValue());
} else if (currentName.equals(GeoPointFieldMapper.Names.LON)) {
} else if (currentName.equals("lon")) {
point.resetLon(parser.doubleValue());
} else if (currentName.equals(GeoPointFieldMapper.Names.GEOHASH)) {
} else if (currentName.equals("geohash")) {
point.resetFromGeoHash(parser.text());
} else {
throw new ParsingException(parser.getTokenLocation(),
@ -394,12 +369,12 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
unit = DistanceUnit.fromString(parser.text());
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, DISTANCE_TYPE_FIELD)) {
geoDistance = GeoDistance.fromString(parser.text());
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.LAT_SUFFIX)) {
} else if (currentFieldName.endsWith(".lat")) {
point.resetLat(parser.doubleValue());
fieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.LAT_SUFFIX.length());
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.LON_SUFFIX)) {
fieldName = currentFieldName.substring(0, currentFieldName.length() - ".lat".length());
} else if (currentFieldName.endsWith(".lon")) {
point.resetLon(parser.doubleValue());
fieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.LON_SUFFIX.length());
fieldName = currentFieldName.substring(0, currentFieldName.length() - ".lon".length());
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.NAME_FIELD)) {
queryName = parser.text();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.BOOST_FIELD)) {

View File

@ -1,630 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.query;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.apache.lucene.spatial.geopoint.search.XGeoPointDistanceRangeQuery;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper.LegacyGeoPointFieldType;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.LatLonPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.search.geo.GeoDistanceRangeQuery;
import java.io.IOException;
import java.util.Locale;
import java.util.Objects;
import java.util.Optional;
public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistanceRangeQueryBuilder> {
public static final String NAME = "geo_distance_range";
public static final boolean DEFAULT_INCLUDE_LOWER = true;
public static final boolean DEFAULT_INCLUDE_UPPER = true;
public static final GeoDistance DEFAULT_GEO_DISTANCE = GeoDistance.DEFAULT;
public static final DistanceUnit DEFAULT_UNIT = DistanceUnit.DEFAULT;
@Deprecated
public static final String DEFAULT_OPTIMIZE_BBOX = "memory";
/**
* The default value for ignore_unmapped.
*/
public static final boolean DEFAULT_IGNORE_UNMAPPED = false;
private static final ParseField FROM_FIELD = new ParseField("from");
private static final ParseField TO_FIELD = new ParseField("to");
private static final ParseField INCLUDE_LOWER_FIELD = new ParseField("include_lower");
private static final ParseField INCLUDE_UPPER_FIELD = new ParseField("include_upper");
private static final ParseField GT_FIELD = new ParseField("gt");
private static final ParseField GTE_FIELD = new ParseField("gte", "ge");
private static final ParseField LT_FIELD = new ParseField("lt");
private static final ParseField LTE_FIELD = new ParseField("lte", "le");
private static final ParseField UNIT_FIELD = new ParseField("unit");
private static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
private static final ParseField NAME_FIELD = new ParseField("_name");
private static final ParseField BOOST_FIELD = new ParseField("boost");
@Deprecated
private static final ParseField OPTIMIZE_BBOX_FIELD = new ParseField("optimize_bbox")
.withAllDeprecated("no replacement: `optimize_bbox` is no longer supported due to recent improvements");
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize")
.withAllDeprecated("use validation_method instead");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed")
.withAllDeprecated("use validation_method instead");
private static final ParseField VALIDATION_METHOD = new ParseField("validation_method");
private static final ParseField IGNORE_UNMAPPED_FIELD = new ParseField("ignore_unmapped");
private final String fieldName;
private Object from;
private Object to;
private boolean includeLower = DEFAULT_INCLUDE_LOWER;
private boolean includeUpper = DEFAULT_INCLUDE_UPPER;
private boolean ignoreUnmapped = DEFAULT_IGNORE_UNMAPPED;
private final GeoPoint point;
private GeoDistance geoDistance = DEFAULT_GEO_DISTANCE;
private DistanceUnit unit = DEFAULT_UNIT;
private String optimizeBbox = null;
private GeoValidationMethod validationMethod = GeoValidationMethod.DEFAULT;
public GeoDistanceRangeQueryBuilder(String fieldName, GeoPoint point) {
if (Strings.isEmpty(fieldName)) {
throw new IllegalArgumentException("fieldName must not be null");
}
if (point == null) {
throw new IllegalArgumentException("point must not be null");
}
this.fieldName = fieldName;
this.point = point;
}
public GeoDistanceRangeQueryBuilder(String fieldName, double lat, double lon) {
this(fieldName, new GeoPoint(lat, lon));
}
public GeoDistanceRangeQueryBuilder(String fieldName, String geohash) {
this(fieldName, geohash == null ? null : new GeoPoint().resetFromGeoHash(geohash));
}
/**
* Read from a stream.
*/
public GeoDistanceRangeQueryBuilder(StreamInput in) throws IOException {
super(in);
fieldName = in.readString();
point = in.readGeoPoint();
from = in.readGenericValue();
to = in.readGenericValue();
includeLower = in.readBoolean();
includeUpper = in.readBoolean();
unit = DistanceUnit.valueOf(in.readString());
geoDistance = GeoDistance.readFromStream(in);
optimizeBbox = in.readOptionalString();
validationMethod = GeoValidationMethod.readFromStream(in);
ignoreUnmapped = in.readBoolean();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeString(fieldName);
out.writeGeoPoint(point);
out.writeGenericValue(from);
out.writeGenericValue(to);
out.writeBoolean(includeLower);
out.writeBoolean(includeUpper);
out.writeString(unit.name());
geoDistance.writeTo(out);;
out.writeOptionalString(optimizeBbox);
validationMethod.writeTo(out);
out.writeBoolean(ignoreUnmapped);
}
public String fieldName() {
return fieldName;
}
public GeoPoint point() {
return point;
}
public GeoDistanceRangeQueryBuilder from(String from) {
if (from == null) {
throw new IllegalArgumentException("[from] must not be null");
}
this.from = from;
return this;
}
public GeoDistanceRangeQueryBuilder from(Number from) {
if (from == null) {
throw new IllegalArgumentException("[from] must not be null");
}
this.from = from;
return this;
}
public Object from() {
return from;
}
public GeoDistanceRangeQueryBuilder to(String to) {
if (to == null) {
throw new IllegalArgumentException("[to] must not be null");
}
this.to = to;
return this;
}
public GeoDistanceRangeQueryBuilder to(Number to) {
if (to == null) {
throw new IllegalArgumentException("[to] must not be null");
}
this.to = to;
return this;
}
public Object to() {
return to;
}
public GeoDistanceRangeQueryBuilder includeLower(boolean includeLower) {
this.includeLower = includeLower;
return this;
}
public boolean includeLower() {
return includeLower;
}
public GeoDistanceRangeQueryBuilder includeUpper(boolean includeUpper) {
this.includeUpper = includeUpper;
return this;
}
public boolean includeUpper() {
return includeUpper;
}
public GeoDistanceRangeQueryBuilder geoDistance(GeoDistance geoDistance) {
if (geoDistance == null) {
throw new IllegalArgumentException("geoDistance calculation mode must not be null");
}
this.geoDistance = geoDistance;
return this;
}
public GeoDistance geoDistance() {
return geoDistance;
}
public GeoDistanceRangeQueryBuilder unit(DistanceUnit unit) {
if (unit == null) {
throw new IllegalArgumentException("distance unit must not be null");
}
this.unit = unit;
return this;
}
public DistanceUnit unit() {
return unit;
}
@Deprecated
public GeoDistanceRangeQueryBuilder optimizeBbox(String optimizeBbox) {
this.optimizeBbox = optimizeBbox;
return this;
}
@Deprecated
public String optimizeBbox() {
return optimizeBbox;
}
/** Set validation method for coordinates. */
public GeoDistanceRangeQueryBuilder setValidationMethod(GeoValidationMethod method) {
this.validationMethod = method;
return this;
}
/** Returns validation method for coordinates. */
public GeoValidationMethod getValidationMethod() {
return this.validationMethod;
}
/**
* Sets whether the query builder should ignore unmapped fields (and run a
* {@link MatchNoDocsQuery} in place of this query) or throw an exception if
* the field is unmapped.
*/
public GeoDistanceRangeQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) {
this.ignoreUnmapped = ignoreUnmapped;
return this;
}
/**
* Gets whether the query builder will ignore unmapped fields (and run a
* {@link MatchNoDocsQuery} in place of this query) or throw an exception if
* the field is unmapped.
*/
public boolean ignoreUnmapped() {
return ignoreUnmapped;
}
@Override
protected Query doToQuery(QueryShardContext context) throws IOException {
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType == null) {
if (ignoreUnmapped) {
return new MatchNoDocsQuery();
} else {
throw new QueryShardException(context, "failed to find geo_point field [" + fieldName + "]");
}
}
if (!(fieldType instanceof BaseGeoPointFieldMapper.GeoPointFieldType)) {
throw new QueryShardException(context, "field [" + fieldName + "] is not a geo_point field");
}
final boolean indexCreatedBeforeV2_0 = context.indexVersionCreated().before(Version.V_2_0_0);
final boolean indexCreatedBeforeV2_2 = context.indexVersionCreated().before(Version.V_2_2_0);
// validation was not available prior to 2.x, so to support bwc
// percolation queries we only ignore_malformed on 2.x created indexes
if (!indexCreatedBeforeV2_0 && !GeoValidationMethod.isIgnoreMalformed(validationMethod)) {
if (!GeoUtils.isValidLatitude(point.lat())) {
throw new QueryShardException(context, "illegal latitude value [{}] for [{}]", point.lat(), NAME);
}
if (!GeoUtils.isValidLongitude(point.lon())) {
throw new QueryShardException(context, "illegal longitude value [{}] for [{}]", point.lon(), NAME);
}
}
GeoPoint point = new GeoPoint(this.point);
if (indexCreatedBeforeV2_2 == false || GeoValidationMethod.isCoerce(validationMethod)) {
GeoUtils.normalizePoint(point, true, true);
}
Double fromValue;
Double toValue;
if (from != null) {
if (from instanceof Number) {
fromValue = unit.toMeters(((Number) from).doubleValue());
} else {
fromValue = DistanceUnit.parse((String) from, unit, DistanceUnit.DEFAULT);
}
if (indexCreatedBeforeV2_2) {
fromValue = geoDistance.normalize(fromValue, DistanceUnit.DEFAULT);
}
} else {
fromValue = 0.0;
}
if (to != null) {
if (to instanceof Number) {
toValue = unit.toMeters(((Number) to).doubleValue());
} else {
toValue = DistanceUnit.parse((String) to, unit, DistanceUnit.DEFAULT);
}
if (indexCreatedBeforeV2_2) {
toValue = geoDistance.normalize(toValue, DistanceUnit.DEFAULT);
}
} else {
toValue = GeoUtils.maxRadialDistanceMeters(point.lat(), point.lon());
}
final Version indexVersionCreated = context.indexVersionCreated();
if (indexVersionCreated.onOrAfter(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
throw new QueryShardException(context, "[{}] queries are no longer supported for geo_point field types. "
+ "Use geo_distance sort or aggregations", NAME);
} else if (indexVersionCreated.before(Version.V_2_2_0)) {
LegacyGeoPointFieldType geoFieldType = (LegacyGeoPointFieldType) fieldType;
IndexGeoPointFieldData indexFieldData = context.getForField(fieldType);
String bboxOptimization = Strings.isEmpty(optimizeBbox) ? DEFAULT_OPTIMIZE_BBOX : optimizeBbox;
return new GeoDistanceRangeQuery(point, fromValue, toValue, includeLower, includeUpper, geoDistance, geoFieldType,
indexFieldData, bboxOptimization, context);
}
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
// if index created V_2_3 > use prefix encoded postings format
final GeoPointField.TermEncoding encoding = (indexVersionCreated.before(Version.V_2_3_0)) ?
GeoPointField.TermEncoding.NUMERIC : GeoPointField.TermEncoding.PREFIX;
return new XGeoPointDistanceRangeQuery(fieldType.name(), encoding, point.lat(), point.lon(),
(includeLower) ? fromValue : fromValue + GeoUtils.TOLERANCE,
(includeUpper) ? toValue : toValue - GeoUtils.TOLERANCE);
}
@Override
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(NAME);
builder.startArray(fieldName).value(point.lon()).value(point.lat()).endArray();
builder.field(FROM_FIELD.getPreferredName(), from);
builder.field(TO_FIELD.getPreferredName(), to);
builder.field(INCLUDE_LOWER_FIELD.getPreferredName(), includeLower);
builder.field(INCLUDE_UPPER_FIELD.getPreferredName(), includeUpper);
builder.field(UNIT_FIELD.getPreferredName(), unit);
builder.field(DISTANCE_TYPE_FIELD.getPreferredName(), geoDistance.name().toLowerCase(Locale.ROOT));
if (Strings.isEmpty(optimizeBbox) == false) {
builder.field(OPTIMIZE_BBOX_FIELD.getPreferredName(), optimizeBbox);
}
builder.field(VALIDATION_METHOD.getPreferredName(), validationMethod);
builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
printBoostAndQueryName(builder);
builder.endObject();
}
public static Optional<GeoDistanceRangeQueryBuilder> fromXContent(QueryParseContext parseContext) throws IOException {
XContentParser parser = parseContext.parser();
XContentParser.Token token;
Float boost = null;
String queryName = null;
String currentFieldName = null;
GeoPoint point = null;
String fieldName = null;
Object vFrom = null;
Object vTo = null;
Boolean includeLower = null;
Boolean includeUpper = null;
DistanceUnit unit = null;
GeoDistance geoDistance = null;
String optimizeBbox = null;
boolean coerce = GeoValidationMethod.DEFAULT_LENIENT_PARSING;
boolean ignoreMalformed = GeoValidationMethod.DEFAULT_LENIENT_PARSING;
GeoValidationMethod validationMethod = null;
boolean ignoreUnmapped = DEFAULT_IGNORE_UNMAPPED;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (parseContext.isDeprecatedSetting(currentFieldName)) {
// skip
} else if (token == XContentParser.Token.START_ARRAY) {
if (fieldName == null) {
if (point == null) {
point = new GeoPoint();
}
GeoUtils.parseGeoPoint(parser, point);
fieldName = currentFieldName;
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + GeoDistanceRangeQueryBuilder.NAME +
"] field name already set to [" + fieldName + "] but found [" + currentFieldName + "]");
}
} else if (token == XContentParser.Token.START_OBJECT) {
// the json in the format of -> field : { lat : 30, lon : 12 }
if (fieldName == null) {
fieldName = currentFieldName;
if (point == null) {
point = new GeoPoint();
}
GeoUtils.parseGeoPoint(parser, point);
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + GeoDistanceRangeQueryBuilder.NAME +
"] field name already set to [" + fieldName + "] but found [" + currentFieldName + "]");
}
} else if (token.isValue()) {
if (parseContext.getParseFieldMatcher().match(currentFieldName, FROM_FIELD)) {
if (token == XContentParser.Token.VALUE_NULL) {
} else if (token == XContentParser.Token.VALUE_STRING) {
vFrom = parser.text(); // a String
} else {
vFrom = parser.numberValue(); // a Number
}
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, TO_FIELD)) {
if (token == XContentParser.Token.VALUE_NULL) {
} else if (token == XContentParser.Token.VALUE_STRING) {
vTo = parser.text(); // a String
} else {
vTo = parser.numberValue(); // a Number
}
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, INCLUDE_LOWER_FIELD)) {
includeLower = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, INCLUDE_UPPER_FIELD)) {
includeUpper = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, IGNORE_UNMAPPED_FIELD)) {
ignoreUnmapped = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, GT_FIELD)) {
if (token == XContentParser.Token.VALUE_NULL) {
} else if (token == XContentParser.Token.VALUE_STRING) {
vFrom = parser.text(); // a String
} else {
vFrom = parser.numberValue(); // a Number
}
includeLower = false;
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, GTE_FIELD)) {
if (token == XContentParser.Token.VALUE_NULL) {
} else if (token == XContentParser.Token.VALUE_STRING) {
vFrom = parser.text(); // a String
} else {
vFrom = parser.numberValue(); // a Number
}
includeLower = true;
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, LT_FIELD)) {
if (token == XContentParser.Token.VALUE_NULL) {
} else if (token == XContentParser.Token.VALUE_STRING) {
vTo = parser.text(); // a String
} else {
vTo = parser.numberValue(); // a Number
}
includeUpper = false;
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, LTE_FIELD)) {
if (token == XContentParser.Token.VALUE_NULL) {
} else if (token == XContentParser.Token.VALUE_STRING) {
vTo = parser.text(); // a String
} else {
vTo = parser.numberValue(); // a Number
}
includeUpper = true;
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, UNIT_FIELD)) {
unit = DistanceUnit.fromString(parser.text());
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, DISTANCE_TYPE_FIELD)) {
geoDistance = GeoDistance.fromString(parser.text());
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.LAT_SUFFIX)) {
String maybeFieldName = currentFieldName.substring(0,
currentFieldName.length() - GeoPointFieldMapper.Names.LAT_SUFFIX.length());
if (fieldName == null || fieldName.equals(maybeFieldName)) {
fieldName = maybeFieldName;
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + GeoDistanceRangeQueryBuilder.NAME +
"] field name already set to [" + fieldName + "] but found [" + currentFieldName + "]");
}
if (point == null) {
point = new GeoPoint();
}
point.resetLat(parser.doubleValue());
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.LON_SUFFIX)) {
String maybeFieldName = currentFieldName.substring(0,
currentFieldName.length() - GeoPointFieldMapper.Names.LON_SUFFIX.length());
if (fieldName == null || fieldName.equals(maybeFieldName)) {
fieldName = maybeFieldName;
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + GeoDistanceRangeQueryBuilder.NAME +
"] field name already set to [" + fieldName + "] but found [" + currentFieldName + "]");
}
if (point == null) {
point = new GeoPoint();
}
point.resetLon(parser.doubleValue());
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
queryName = parser.text();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, BOOST_FIELD)) {
boost = parser.floatValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, OPTIMIZE_BBOX_FIELD)) {
optimizeBbox = parser.textOrNull();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, COERCE_FIELD)) {
coerce = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, IGNORE_MALFORMED_FIELD)) {
ignoreMalformed = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, VALIDATION_METHOD)) {
validationMethod = GeoValidationMethod.fromString(parser.text());
} else {
if (fieldName == null) {
if (point == null) {
point = new GeoPoint();
}
point.resetFromString(parser.text());
fieldName = currentFieldName;
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + GeoDistanceRangeQueryBuilder.NAME +
"] field name already set to [" + fieldName + "] but found [" + currentFieldName + "]");
}
}
}
}
GeoDistanceRangeQueryBuilder queryBuilder = new GeoDistanceRangeQueryBuilder(fieldName, point);
if (boost != null) {
queryBuilder.boost(boost);
}
if (queryName != null) {
queryBuilder.queryName(queryName);
}
if (vFrom != null) {
if (vFrom instanceof Number) {
queryBuilder.from((Number) vFrom);
} else {
queryBuilder.from((String) vFrom);
}
}
if (vTo != null) {
if (vTo instanceof Number) {
queryBuilder.to((Number) vTo);
} else {
queryBuilder.to((String) vTo);
}
}
if (includeUpper != null) {
queryBuilder.includeUpper(includeUpper);
}
if (includeLower != null) {
queryBuilder.includeLower(includeLower);
}
if (unit != null) {
queryBuilder.unit(unit);
}
if (geoDistance != null) {
queryBuilder.geoDistance(geoDistance);
}
if (optimizeBbox != null) {
queryBuilder.optimizeBbox(optimizeBbox);
}
if (validationMethod != null) {
// if validation method is set explicitly ignore deprecated coerce/ignore malformed fields if any
queryBuilder.setValidationMethod(validationMethod);
} else {
queryBuilder.setValidationMethod(GeoValidationMethod.infer(coerce, ignoreMalformed));
}
queryBuilder.ignoreUnmapped(ignoreUnmapped);
return Optional.of(queryBuilder);
}
@Override
protected boolean doEquals(GeoDistanceRangeQueryBuilder other) {
return ((Objects.equals(fieldName, other.fieldName)) &&
(Objects.equals(point, other.point)) &&
(Objects.equals(from, other.from)) &&
(Objects.equals(to, other.to)) &&
(Objects.equals(includeUpper, other.includeUpper)) &&
(Objects.equals(includeLower, other.includeLower)) &&
(Objects.equals(geoDistance, other.geoDistance)) &&
(Objects.equals(optimizeBbox, other.optimizeBbox)) &&
(Objects.equals(validationMethod, other.validationMethod))) &&
Objects.equals(ignoreUnmapped, other.ignoreUnmapped);
}
@Override
protected int doHashCode() {
return Objects.hash(fieldName, point, from, to, includeUpper, includeLower, geoDistance, optimizeBbox, validationMethod,
ignoreUnmapped);
}
@Override
public String getWriteableName() {
return NAME;
}
}

View File

@ -23,9 +23,6 @@ import org.apache.lucene.document.LatLonPoint;
import org.apache.lucene.geo.Polygon;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.apache.lucene.spatial.geopoint.search.GeoPointInPolygonQuery;
import org.elasticsearch.Version;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
@ -36,11 +33,8 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.LatLonPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.search.geo.GeoPolygonQuery;
import java.io.IOException;
import java.util.ArrayList;
@ -181,10 +175,9 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
}
final int shellSize = shell.size();
final boolean indexCreatedBeforeV2_0 = context.indexVersionCreated().before(Version.V_2_0_0);
// validation was not available prior to 2.x, so to support bwc
// percolation queries we only ignore_malformed on 2.x created indexes
if (!indexCreatedBeforeV2_0 && !GeoValidationMethod.isIgnoreMalformed(validationMethod)) {
if (!GeoValidationMethod.isIgnoreMalformed(validationMethod)) {
for (GeoPoint point : shell) {
if (!GeoUtils.isValidLatitude(point.lat())) {
throw new QueryShardException(context, "illegal latitude value [{}] for [{}]", point.lat(),
@ -197,18 +190,12 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
}
}
final Version indexVersionCreated = context.indexVersionCreated();
if (indexVersionCreated.onOrAfter(Version.V_2_2_0) || GeoValidationMethod.isCoerce(validationMethod)) {
if (GeoValidationMethod.isCoerce(validationMethod)) {
for (GeoPoint point : shell) {
GeoUtils.normalizePoint(point, true, true);
}
}
if (indexVersionCreated.before(Version.V_2_2_0)) {
IndexGeoPointFieldData indexFieldData = context.getForField(fieldType);
return new GeoPolygonQuery(indexFieldData, shell.toArray(new GeoPoint[shellSize]));
}
double[] lats = new double[shellSize];
double[] lons = new double[shellSize];
GeoPoint p;
@ -218,14 +205,7 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
lons[i] = p.lon();
}
if (indexVersionCreated.onOrAfter(LatLonPointFieldMapper.LAT_LON_FIELD_VERSION)) {
return LatLonPoint.newPolygonQuery(fieldType.name(), new Polygon(lats, lons));
}
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
// if index created V_2_3 > use prefix encoded postings format
final GeoPointField.TermEncoding encoding = (indexVersionCreated.before(Version.V_2_3_0)) ?
GeoPointField.TermEncoding.NUMERIC : GeoPointField.TermEncoding.PREFIX;
return new GeoPointInPolygonQuery(fieldType.name(), encoding, lats, lons);
return LatLonPoint.newPolygonQuery(fieldType.name(), new Polygon(lats, lons));
}
@Override

View File

@ -1,390 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.query;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.LatLonPointFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
/**
* A geohash cell filter that filters {@link GeoPoint}s by their geohashes. Basically the a
* Geohash prefix is defined by the filter and all geohashes that are matching this
* prefix will be returned. The <code>neighbors</code> flag allows to filter
* geohashes that surround the given geohash. In general the neighborhood of a
* geohash is defined by its eight adjacent cells.<br>
* The structure of the {@link GeohashCellQuery} is defined as:
* <pre>
* &quot;geohash_bbox&quot; {
* &quot;field&quot;:&quot;location&quot;,
* &quot;geohash&quot;:&quot;u33d8u5dkx8k&quot;,
* &quot;neighbors&quot;:false
* }
* </pre>
*/
public class GeohashCellQuery {
public static final String NAME = "geohash_cell";
public static final boolean DEFAULT_NEIGHBORS = false;
/**
* The default value for ignore_unmapped.
*/
public static final boolean DEFAULT_IGNORE_UNMAPPED = false;
private static final ParseField NEIGHBORS_FIELD = new ParseField("neighbors");
private static final ParseField PRECISION_FIELD = new ParseField("precision");
private static final ParseField IGNORE_UNMAPPED_FIELD = new ParseField("ignore_unmapped");
/**
* Create a new geohash filter for a given set of geohashes. In general this method
* returns a boolean filter combining the geohashes OR-wise.
*
* @param context Context of the filter
* @param fieldType field mapper for geopoints
* @param geohash mandatory geohash
* @param geohashes optional array of additional geohashes
* @return a new GeoBoundinboxfilter
*/
public static Query create(QueryShardContext context, BaseGeoPointFieldMapper.LegacyGeoPointFieldType fieldType,
String geohash, @Nullable List<CharSequence> geohashes) {
MappedFieldType geoHashMapper = fieldType.geoHashFieldType();
if (geoHashMapper == null) {
throw new IllegalArgumentException("geohash filter needs geohash_prefix to be enabled");
}
if (geohashes == null || geohashes.size() == 0) {
return geoHashMapper.termQuery(geohash, context);
} else {
geohashes.add(geohash);
return geoHashMapper.termsQuery(geohashes, context);
}
}
/**
* Builder for a geohashfilter. It needs the fields <code>fieldname</code> and
* <code>geohash</code> to be set. the default for a neighbor filteing is
* <code>false</code>.
*/
public static class Builder extends AbstractQueryBuilder<Builder> {
// we need to store the geohash rather than the corresponding point,
// because a transformation from a geohash to a point an back to the
// geohash will extend the accuracy of the hash to max precision
// i.e. by filing up with z's.
private String fieldName;
private String geohash;
private Integer levels = null;
private boolean neighbors = DEFAULT_NEIGHBORS;
private boolean ignoreUnmapped = DEFAULT_IGNORE_UNMAPPED;
public Builder(String field, GeoPoint point) {
this(field, point == null ? null : point.geohash(), false);
}
public Builder(String field, String geohash) {
this(field, geohash, false);
}
public Builder(String field, String geohash, boolean neighbors) {
if (Strings.isEmpty(field)) {
throw new IllegalArgumentException("fieldName must not be null");
}
if (Strings.isEmpty(geohash)) {
throw new IllegalArgumentException("geohash or point must be defined");
}
this.fieldName = field;
this.geohash = geohash;
this.neighbors = neighbors;
}
/**
* Read from a stream.
*/
public Builder(StreamInput in) throws IOException {
super(in);
fieldName = in.readString();
geohash = in.readString();
levels = in.readOptionalVInt();
neighbors = in.readBoolean();
ignoreUnmapped = in.readBoolean();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeString(fieldName);
out.writeString(geohash);
out.writeOptionalVInt(levels);
out.writeBoolean(neighbors);
out.writeBoolean(ignoreUnmapped);
}
public Builder point(GeoPoint point) {
this.geohash = point.getGeohash();
return this;
}
public Builder point(double lat, double lon) {
this.geohash = GeoHashUtils.stringEncode(lon, lat);
return this;
}
public Builder geohash(String geohash) {
this.geohash = geohash;
return this;
}
public String geohash() {
return geohash;
}
public Builder precision(int levels) {
if (levels <= 0) {
throw new IllegalArgumentException("precision must be greater than 0. Found [" + levels + "]");
}
this.levels = levels;
return this;
}
public Integer precision() {
return levels;
}
public Builder precision(String precision) {
double meters = DistanceUnit.parse(precision, DistanceUnit.DEFAULT, DistanceUnit.METERS);
return precision(GeoUtils.geoHashLevelsForPrecision(meters));
}
public Builder neighbors(boolean neighbors) {
this.neighbors = neighbors;
return this;
}
public boolean neighbors() {
return neighbors;
}
public Builder fieldName(String fieldName) {
this.fieldName = fieldName;
return this;
}
public String fieldName() {
return fieldName;
}
/**
* Sets whether the query builder should ignore unmapped fields (and run
* a {@link MatchNoDocsQuery} in place of this query) or throw an
* exception if the field is unmapped.
*/
public GeohashCellQuery.Builder ignoreUnmapped(boolean ignoreUnmapped) {
this.ignoreUnmapped = ignoreUnmapped;
return this;
}
/**
* Gets whether the query builder will ignore unmapped fields (and run a
* {@link MatchNoDocsQuery} in place of this query) or throw an
* exception if the field is unmapped.
*/
public boolean ignoreUnmapped() {
return ignoreUnmapped;
}
@Override
protected Query doToQuery(QueryShardContext context) throws IOException {
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType == null) {
if (ignoreUnmapped) {
return new MatchNoDocsQuery();
} else {
throw new QueryShardException(context, "failed to parse [{}] query. missing [{}] field [{}]", NAME,
BaseGeoPointFieldMapper.CONTENT_TYPE, fieldName);
}
}
if (fieldType instanceof LatLonPointFieldMapper.LatLonPointFieldType) {
throw new QueryShardException(context, "failed to parse [{}] query. "
+ "geo_point field no longer supports geohash_cell queries", NAME);
} else if (!(fieldType instanceof BaseGeoPointFieldMapper.LegacyGeoPointFieldType)) {
throw new QueryShardException(context, "failed to parse [{}] query. field [{}] is not a geo_point field", NAME, fieldName);
}
BaseGeoPointFieldMapper.LegacyGeoPointFieldType geoFieldType = ((BaseGeoPointFieldMapper.LegacyGeoPointFieldType) fieldType);
if (!geoFieldType.isGeoHashPrefixEnabled()) {
throw new QueryShardException(context, "failed to parse [{}] query. [geohash_prefix] is not enabled for field [{}]", NAME,
fieldName);
}
String geohash = this.geohash;
if (levels != null) {
int len = Math.min(levels, geohash.length());
geohash = geohash.substring(0, len);
}
Query query;
if (neighbors) {
query = create(context, geoFieldType, geohash, GeoHashUtils.addNeighbors(geohash, new ArrayList<CharSequence>(8)));
} else {
query = create(context, geoFieldType, geohash, null);
}
return query;
}
@Override
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(NAME);
builder.field(NEIGHBORS_FIELD.getPreferredName(), neighbors);
if (levels != null) {
builder.field(PRECISION_FIELD.getPreferredName(), levels);
}
builder.field(fieldName, geohash);
builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
printBoostAndQueryName(builder);
builder.endObject();
}
public static Optional<Builder> fromXContent(QueryParseContext parseContext) throws IOException {
XContentParser parser = parseContext.parser();
String fieldName = null;
String geohash = null;
Integer levels = null;
Boolean neighbors = null;
String queryName = null;
Float boost = null;
boolean ignoreUnmapped = DEFAULT_IGNORE_UNMAPPED;
XContentParser.Token token;
if ((token = parser.currentToken()) != Token.START_OBJECT) {
throw new ElasticsearchParseException("failed to parse [{}] query. expected an object but found [{}] instead", NAME, token);
}
while ((token = parser.nextToken()) != Token.END_OBJECT) {
if (token == Token.FIELD_NAME) {
String field = parser.currentName();
if (parseContext.isDeprecatedSetting(field)) {
// skip
} else if (parseContext.getParseFieldMatcher().match(field, PRECISION_FIELD)) {
token = parser.nextToken();
if (token == Token.VALUE_NUMBER) {
levels = parser.intValue();
} else if (token == Token.VALUE_STRING) {
double meters = DistanceUnit.parse(parser.text(), DistanceUnit.DEFAULT, DistanceUnit.METERS);
levels = GeoUtils.geoHashLevelsForPrecision(meters);
}
} else if (parseContext.getParseFieldMatcher().match(field, NEIGHBORS_FIELD)) {
parser.nextToken();
neighbors = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(field, AbstractQueryBuilder.NAME_FIELD)) {
parser.nextToken();
queryName = parser.text();
} else if (parseContext.getParseFieldMatcher().match(field, IGNORE_UNMAPPED_FIELD)) {
parser.nextToken();
ignoreUnmapped = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(field, AbstractQueryBuilder.BOOST_FIELD)) {
parser.nextToken();
boost = parser.floatValue();
} else {
if (fieldName == null) {
fieldName = field;
token = parser.nextToken();
if (token == Token.VALUE_STRING) {
// A string indicates either a geohash or a
// lat/lon
// string
String location = parser.text();
if (location.indexOf(",") > 0) {
geohash = GeoUtils.parseGeoPoint(parser).geohash();
} else {
geohash = location;
}
} else {
geohash = GeoUtils.parseGeoPoint(parser).geohash();
}
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + NAME +
"] field name already set to [" + fieldName + "] but found [" + field + "]");
}
}
} else {
throw new ElasticsearchParseException("failed to parse [{}] query. unexpected token [{}]", NAME, token);
}
}
Builder builder = new Builder(fieldName, geohash);
if (levels != null) {
builder.precision(levels);
}
if (neighbors != null) {
builder.neighbors(neighbors);
}
if (queryName != null) {
builder.queryName(queryName);
}
if (boost != null) {
builder.boost(boost);
}
builder.ignoreUnmapped(ignoreUnmapped);
return Optional.of(builder);
}
@Override
protected boolean doEquals(Builder other) {
return Objects.equals(fieldName, other.fieldName)
&& Objects.equals(geohash, other.geohash)
&& Objects.equals(levels, other.levels)
&& Objects.equals(neighbors, other.neighbors)
&& Objects.equals(ignoreUnmapped, other.ignoreUnmapped);
}
@Override
protected int doHashCode() {
return Objects.hash(fieldName, geohash, levels, neighbors, ignoreUnmapped);
}
@Override
public String getWriteableName() {
return NAME;
}
}
}

View File

@ -55,7 +55,6 @@ import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType;
import org.elasticsearch.index.mapper.StringFieldMapper.StringFieldType;
import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType;
import java.io.IOException;
@ -93,7 +92,7 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQ
public static final boolean DEFAULT_FAIL_ON_UNSUPPORTED_FIELDS = true;
private static final Set<Class<? extends MappedFieldType>> SUPPORTED_FIELD_TYPES = new HashSet<>(
Arrays.asList(StringFieldType.class, TextFieldType.class, KeywordFieldType.class));
Arrays.asList(TextFieldType.class, KeywordFieldType.class));
private interface Field {
ParseField FIELDS = new ParseField("fields");

View File

@ -651,37 +651,6 @@ public abstract class QueryBuilders {
return new GeoDistanceQueryBuilder(name);
}
/**
* A filter to filter based on a specific range from a specific geo location / point.
*
* @param name The location field name.
* @param point The point
*/
public static GeoDistanceRangeQueryBuilder geoDistanceRangeQuery(String name, GeoPoint point) {
return new GeoDistanceRangeQueryBuilder(name, point);
}
/**
* A filter to filter based on a specific range from a specific geo location / point.
*
* @param name The location field name.
* @param geohash The point as geohash
*/
public static GeoDistanceRangeQueryBuilder geoDistanceRangeQuery(String name, String geohash) {
return new GeoDistanceRangeQueryBuilder(name, geohash);
}
/**
* A filter to filter based on a specific range from a specific geo location / point.
*
* @param name The location field name.
* @param lat The points latitude
* @param lon The points longitude
*/
public static GeoDistanceRangeQueryBuilder geoDistanceRangeQuery(String name, double lat, double lon) {
return new GeoDistanceRangeQueryBuilder(name, lat, lon);
}
/**
* A filter to filter based on a bounding box defined by top left and bottom right locations / points
*
@ -691,43 +660,6 @@ public abstract class QueryBuilders {
return new GeoBoundingBoxQueryBuilder(name);
}
/**
* A filter based on a bounding box defined by geohash. The field this filter is applied to
* must have <code>{&quot;type&quot;:&quot;geo_point&quot;, &quot;geohash&quot;:true}</code>
* to work.
*
* @param name The geo point field name.
* @param geohash The Geohash to filter
*/
public static GeohashCellQuery.Builder geoHashCellQuery(String name, String geohash) {
return new GeohashCellQuery.Builder(name, geohash);
}
/**
* A filter based on a bounding box defined by geohash. The field this filter is applied to
* must have <code>{&quot;type&quot;:&quot;geo_point&quot;, &quot;geohash&quot;:true}</code>
* to work.
*
* @param name The geo point field name.
* @param point a geo point within the geohash bucket
*/
public static GeohashCellQuery.Builder geoHashCellQuery(String name, GeoPoint point) {
return new GeohashCellQuery.Builder(name, point);
}
/**
* A filter based on a bounding box defined by geohash. The field this filter is applied to
* must have <code>{&quot;type&quot;:&quot;geo_point&quot;, &quot;geohash&quot;:true}</code>
* to work.
*
* @param name The geo point field name
* @param geohash The Geohash to filter
* @param neighbors should the neighbor cell also be filtered
*/
public static GeohashCellQuery.Builder geoHashCellQuery(String name, String geohash, boolean neighbors) {
return new GeohashCellQuery.Builder(name, geohash, neighbors);
}
/**
* A filter to filter based on a polygon defined by a set of locations / points.
*

View File

@ -44,9 +44,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.index.mapper.ScaledFloatFieldMapper;
import org.elasticsearch.index.mapper.StringFieldMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import org.elasticsearch.index.query.support.QueryParsers;
import org.joda.time.DateTimeZone;
@ -129,9 +127,7 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder<QueryStringQue
ALLOWED_QUERY_MAPPER_TYPES.add(nt.typeName());
}
ALLOWED_QUERY_MAPPER_TYPES.add(ScaledFloatFieldMapper.CONTENT_TYPE);
ALLOWED_QUERY_MAPPER_TYPES.add(StringFieldMapper.CONTENT_TYPE);
ALLOWED_QUERY_MAPPER_TYPES.add(TextFieldMapper.CONTENT_TYPE);
ALLOWED_QUERY_MAPPER_TYPES.add(TimestampFieldMapper.CONTENT_TYPE);
}
private final String queryString;

View File

@ -37,7 +37,6 @@ import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.LegacyDateFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.RangeFieldMapper;
@ -493,11 +492,7 @@ public class RangeQueryBuilder extends AbstractQueryBuilder<RangeQueryBuilder> i
Query query = null;
MappedFieldType mapper = context.fieldMapper(this.fieldName);
if (mapper != null) {
if (mapper instanceof LegacyDateFieldMapper.DateFieldType) {
query = ((LegacyDateFieldMapper.DateFieldType) mapper).rangeQuery(from, to, includeLower, includeUpper,
timeZone, getForceDateParser(), context);
} else if (mapper instanceof DateFieldMapper.DateFieldType) {
if (mapper instanceof DateFieldMapper.DateFieldType) {
query = ((DateFieldMapper.DateFieldType) mapper).rangeQuery(from, to, includeLower, includeUpper,
timeZone, getForceDateParser(), context);

View File

@ -44,11 +44,8 @@ import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.LegacyDateFieldMapper;
import org.elasticsearch.index.mapper.LegacyNumberFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.NumberFieldMapper;
import org.elasticsearch.index.query.QueryRewriteContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.MultiValueMode;
@ -205,13 +202,11 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
// dates and time and geo need special handling
parser.nextToken();
if (fieldType instanceof LegacyDateFieldMapper.DateFieldType
|| fieldType instanceof DateFieldMapper.DateFieldType) {
if (fieldType instanceof DateFieldMapper.DateFieldType) {
return parseDateVariable(parser, context, fieldType, mode);
} else if (fieldType instanceof BaseGeoPointFieldMapper.GeoPointFieldType) {
return parseGeoVariable(parser, context, fieldType, mode);
} else if (fieldType instanceof LegacyNumberFieldMapper.NumberFieldType
|| fieldType instanceof NumberFieldMapper.NumberFieldType) {
} else if (fieldType instanceof NumberFieldMapper.NumberFieldType) {
return parseNumberVariable(parser, context, fieldType, mode);
} else {
throw new ParsingException(parser.getTokenLocation(), "field [{}] is of type [{}], but only numeric types are supported.",
@ -315,12 +310,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
if (originString == null) {
origin = context.nowInMillis();
} else {
if (dateFieldType instanceof LegacyDateFieldMapper.DateFieldType) {
origin = ((LegacyDateFieldMapper.DateFieldType) dateFieldType).parseToMilliseconds(originString, false, null, null,
context);
} else {
origin = ((DateFieldMapper.DateFieldType) dateFieldType).parseToMilliseconds(originString, false, null, null, context);
}
origin = ((DateFieldMapper.DateFieldType) dateFieldType).parseToMilliseconds(originString, false, null, null, context);
}
if (scaleString == null) {

View File

@ -1,236 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.search.geo;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.ConstantScoreScorer;
import org.apache.lucene.search.ConstantScoreWeight;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.TwoPhaseIterator;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.mapper.LegacyGeoPointFieldMapper;
import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException;
/**
* Query geo_point fields by distance ranges. Used for indexes created prior to 2.2
* @deprecated
*/
@Deprecated
public class GeoDistanceRangeQuery extends Query {
private final double lat;
private final double lon;
private final double inclusiveLowerPoint; // in meters
private final double inclusiveUpperPoint; // in meters
private final GeoDistance geoDistance;
private final GeoDistance.FixedSourceDistance fixedSourceDistance;
private GeoDistance.DistanceBoundingCheck distanceBoundingCheck;
private final Query boundingBoxFilter;
private final IndexGeoPointFieldData indexFieldData;
public GeoDistanceRangeQuery(GeoPoint point, Double lowerVal, Double upperVal, boolean includeLower,
boolean includeUpper, GeoDistance geoDistance,
LegacyGeoPointFieldMapper.LegacyGeoPointFieldType fieldType,
IndexGeoPointFieldData indexFieldData, String optimizeBbox, QueryShardContext context) {
this.lat = point.lat();
this.lon = point.lon();
this.geoDistance = geoDistance;
this.indexFieldData = indexFieldData;
this.fixedSourceDistance = geoDistance.fixedSourceDistance(lat, lon, DistanceUnit.DEFAULT);
if (lowerVal != null) {
double f = lowerVal.doubleValue();
long i = NumericUtils.doubleToSortableLong(f);
inclusiveLowerPoint = NumericUtils.sortableLongToDouble(includeLower ? i : (i + 1L));
} else {
inclusiveLowerPoint = Double.NEGATIVE_INFINITY;
}
if (upperVal != null) {
double f = upperVal.doubleValue();
long i = NumericUtils.doubleToSortableLong(f);
inclusiveUpperPoint = NumericUtils.sortableLongToDouble(includeUpper ? i : (i - 1L));
} else {
inclusiveUpperPoint = Double.POSITIVE_INFINITY;
// we disable bounding box in this case, since the upper point is all and we create bounding box up to the
// upper point it will effectively include all
// TODO we can create a bounding box up to from and "not" it
optimizeBbox = null;
}
if (optimizeBbox != null && !"none".equals(optimizeBbox)) {
distanceBoundingCheck = GeoDistance.distanceBoundingCheck(lat, lon, inclusiveUpperPoint, DistanceUnit.DEFAULT);
if ("memory".equals(optimizeBbox)) {
boundingBoxFilter = null;
} else if ("indexed".equals(optimizeBbox)) {
boundingBoxFilter = LegacyIndexedGeoBoundingBoxQuery.create(distanceBoundingCheck.topLeft(),
distanceBoundingCheck.bottomRight(), fieldType, context);
distanceBoundingCheck = GeoDistance.ALWAYS_INSTANCE; // fine, we do the bounding box check using the filter
} else {
throw new IllegalArgumentException("type [" + optimizeBbox + "] for bounding box optimization not supported");
}
} else {
distanceBoundingCheck = GeoDistance.ALWAYS_INSTANCE;
boundingBoxFilter = null;
}
}
public double lat() {
return lat;
}
public double lon() {
return lon;
}
public GeoDistance geoDistance() {
return geoDistance;
}
public double minInclusiveDistance() {
return inclusiveLowerPoint;
}
public double maxInclusiveDistance() {
return inclusiveUpperPoint;
}
public String fieldName() {
return indexFieldData.getFieldName();
}
@Override
public Query rewrite(IndexReader reader) throws IOException {
return super.rewrite(reader);
}
@Override
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
final Weight boundingBoxWeight;
if (boundingBoxFilter != null) {
boundingBoxWeight = searcher.createNormalizedWeight(boundingBoxFilter, false);
} else {
boundingBoxWeight = null;
}
return new ConstantScoreWeight(this) {
@Override
public Scorer scorer(LeafReaderContext context) throws IOException {
final DocIdSetIterator approximation;
if (boundingBoxWeight != null) {
Scorer s = boundingBoxWeight.scorer(context);
if (s == null) {
// if the approximation does not match anything, we're done
return null;
}
approximation = s.iterator();
} else {
approximation = DocIdSetIterator.all(context.reader().maxDoc());
}
final MultiGeoPointValues values = indexFieldData.load(context).getGeoPointValues();
final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(approximation) {
@Override
public boolean matches() throws IOException {
final int doc = approximation.docID();
values.setDocument(doc);
final int length = values.count();
for (int i = 0; i < length; i++) {
GeoPoint point = values.valueAt(i);
if (distanceBoundingCheck.isWithin(point.lat(), point.lon())) {
double d = fixedSourceDistance.calculate(point.lat(), point.lon());
if (d >= inclusiveLowerPoint && d <= inclusiveUpperPoint) {
return true;
}
}
}
return false;
}
@Override
public float matchCost() {
if (distanceBoundingCheck == GeoDistance.ALWAYS_INSTANCE) {
return 0.0f;
} else {
// TODO: is this right (up to 4 comparisons from GeoDistance.SimpleDistanceBoundingCheck)?
return 4.0f;
}
}
};
return new ConstantScoreScorer(this, score(), twoPhaseIterator);
}
};
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (sameClassAs(o) == false) return false;
GeoDistanceRangeQuery filter = (GeoDistanceRangeQuery) o;
if (Double.compare(filter.inclusiveLowerPoint, inclusiveLowerPoint) != 0) return false;
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.getFieldName().equals(filter.indexFieldData.getFieldName()))
return false;
if (geoDistance != filter.geoDistance) return false;
return true;
}
@Override
public String toString(String field) {
return "GeoDistanceRangeQuery(" + indexFieldData.getFieldName() + ", " + geoDistance + ", ["
+ inclusiveLowerPoint + " - " + inclusiveUpperPoint + "], " + lat + ", " + lon + ")";
}
@Override
public int hashCode() {
int result = classHash();
long temp;
temp = lat != +0.0d ? Double.doubleToLongBits(lat) : 0L;
result = 31 * result + Long.hashCode(temp);
temp = lon != +0.0d ? Double.doubleToLongBits(lon) : 0L;
result = 31 * result + Long.hashCode(temp);
temp = inclusiveLowerPoint != +0.0d ? Double.doubleToLongBits(inclusiveLowerPoint) : 0L;
result = 31 * result + Long.hashCode(temp);
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.getFieldName().hashCode();
return result;
}
}

View File

@ -1,126 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.search.geo;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RandomAccessWeight;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.Bits;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import java.io.IOException;
import java.util.Arrays;
public class GeoPolygonQuery extends Query {
private final GeoPoint[] points;
private final IndexGeoPointFieldData indexFieldData;
public GeoPolygonQuery(IndexGeoPointFieldData indexFieldData, GeoPoint...points) {
this.points = points;
this.indexFieldData = indexFieldData;
}
public GeoPoint[] points() {
return points;
}
public String fieldName() {
return indexFieldData.getFieldName();
}
@Override
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
return new RandomAccessWeight(this) {
@Override
protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
final int maxDoc = context.reader().maxDoc();
final MultiGeoPointValues values = indexFieldData.load(context).getGeoPointValues();
return new Bits() {
private boolean pointInPolygon(GeoPoint[] points, double lat, double lon) {
boolean inPoly = false;
for (int i = 1; i < points.length; i++) {
if (points[i].lon() < lon && points[i-1].lon() >= lon
|| points[i-1].lon() < lon && points[i].lon() >= lon) {
if (points[i].lat() + (lon - points[i].lon()) /
(points[i-1].lon() - points[i].lon()) * (points[i-1].lat() - points[i].lat()) < lat) {
inPoly = !inPoly;
}
}
}
return inPoly;
}
@Override
public boolean get(int doc) {
values.setDocument(doc);
final int length = values.count();
for (int i = 0; i < length; i++) {
GeoPoint point = values.valueAt(i);
if (pointInPolygon(points, point.lat(), point.lon())) {
return true;
}
}
return false;
}
@Override
public int length() {
return maxDoc;
}
};
}
};
}
@Override
public String toString(String field) {
StringBuilder sb = new StringBuilder("GeoPolygonQuery(");
sb.append(indexFieldData.getFieldName());
sb.append(", ").append(Arrays.toString(points)).append(')');
return sb.toString();
}
@Override
public boolean equals(Object obj) {
if (sameClassAs(obj) == false) {
return false;
}
GeoPolygonQuery that = (GeoPolygonQuery) obj;
return indexFieldData.getFieldName().equals(that.indexFieldData.getFieldName())
&& Arrays.equals(points, that.points);
}
@Override
public int hashCode() {
int h = classHash();
h = 31 * h + indexFieldData.getFieldName().hashCode();
h = 31 * h + Arrays.hashCode(points);
return h;
}
}

View File

@ -1,168 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.search.geo;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.RandomAccessWeight;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.Bits;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import java.io.IOException;
import java.util.Objects;
/**
*
* @deprecated This query is no longer used for geo_point indexes created after version 2.1
*/
@Deprecated
public class LegacyInMemoryGeoBoundingBoxQuery extends Query {
private final GeoPoint topLeft;
private final GeoPoint bottomRight;
private final IndexGeoPointFieldData indexFieldData;
public LegacyInMemoryGeoBoundingBoxQuery(GeoPoint topLeft, GeoPoint bottomRight, IndexGeoPointFieldData indexFieldData) {
this.topLeft = topLeft;
this.bottomRight = bottomRight;
this.indexFieldData = indexFieldData;
}
public GeoPoint topLeft() {
return topLeft;
}
public GeoPoint bottomRight() {
return bottomRight;
}
public String fieldName() {
return indexFieldData.getFieldName();
}
@Override
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
return new RandomAccessWeight(this) {
@Override
protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
final int maxDoc = context.reader().maxDoc();
final MultiGeoPointValues values = indexFieldData.load(context).getGeoPointValues();
// checks to see if bounding box crosses 180 degrees
if (topLeft.lon() > bottomRight.lon()) {
return new Meridian180GeoBoundingBoxBits(maxDoc, values, topLeft, bottomRight);
} else {
return new GeoBoundingBoxBits(maxDoc, values, topLeft, bottomRight);
}
}
};
}
@Override
public String toString(String field) {
return "GeoBoundingBoxFilter(" + indexFieldData.getFieldName() + ", " + topLeft + ", " + bottomRight + ")";
}
@Override
public boolean equals(Object obj) {
if (sameClassAs(obj) == false) {
return false;
}
LegacyInMemoryGeoBoundingBoxQuery other = (LegacyInMemoryGeoBoundingBoxQuery) obj;
return fieldName().equalsIgnoreCase(other.fieldName())
&& topLeft.equals(other.topLeft)
&& bottomRight.equals(other.bottomRight);
}
@Override
public int hashCode() {
return Objects.hash(classHash(), fieldName(), topLeft, bottomRight);
}
private static class Meridian180GeoBoundingBoxBits implements Bits {
private final int maxDoc;
private final MultiGeoPointValues values;
private final GeoPoint topLeft;
private final GeoPoint bottomRight;
public Meridian180GeoBoundingBoxBits(int maxDoc, MultiGeoPointValues values, GeoPoint topLeft, GeoPoint bottomRight) {
this.maxDoc = maxDoc;
this.values = values;
this.topLeft = topLeft;
this.bottomRight = bottomRight;
}
@Override
public boolean get(int doc) {
values.setDocument(doc);
final int length = values.count();
for (int i = 0; i < length; i++) {
GeoPoint point = values.valueAt(i);
if (((topLeft.lon() <= point.lon() || bottomRight.lon() >= point.lon())) &&
(topLeft.lat() >= point.lat() && bottomRight.lat() <= point.lat())) {
return true;
}
}
return false;
}
@Override
public int length() {
return maxDoc;
}
}
private static class GeoBoundingBoxBits implements Bits {
private final int maxDoc;
private final MultiGeoPointValues values;
private final GeoPoint topLeft;
private final GeoPoint bottomRight;
public GeoBoundingBoxBits(int maxDoc, MultiGeoPointValues values, GeoPoint topLeft, GeoPoint bottomRight) {
this.maxDoc = maxDoc;
this.values = values;
this.topLeft = topLeft;
this.bottomRight = bottomRight;
}
@Override
public boolean get(int doc) {
values.setDocument(doc);
final int length = values.count();
for (int i = 0; i < length; i++) {
GeoPoint point = values.valueAt(i);
if (topLeft.lon() <= point.lon() && bottomRight.lon() >= point.lon()
&& topLeft.lat() >= point.lat() && bottomRight.lat() <= point.lat()) {
return true;
}
}
return false;
}
@Override
public int length() {
return maxDoc;
}
}
}

View File

@ -1,68 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.search.geo;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.index.mapper.LegacyGeoPointFieldMapper;
import org.elasticsearch.index.query.QueryShardContext;
/**
*
* @deprecated This query is no longer used for geo_point indexes created after version 2.1
*/
@Deprecated
public class LegacyIndexedGeoBoundingBoxQuery {
public static Query create(GeoPoint topLeft, GeoPoint bottomRight,
LegacyGeoPointFieldMapper.LegacyGeoPointFieldType fieldType, QueryShardContext context) {
if (!fieldType.isLatLonEnabled()) {
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()) {
return westGeoBoundingBoxFilter(topLeft, bottomRight, fieldType, context);
} else {
return eastGeoBoundingBoxFilter(topLeft, bottomRight, fieldType, context);
}
}
private static Query westGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight,
LegacyGeoPointFieldMapper.LegacyGeoPointFieldType fieldType, QueryShardContext context) {
BooleanQuery.Builder filter = new BooleanQuery.Builder();
filter.setMinimumNumberShouldMatch(1);
filter.add(fieldType.lonFieldType().rangeQuery(null, bottomRight.lon(), true, true, context), Occur.SHOULD);
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), null, true, true, context), Occur.SHOULD);
filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true, context), Occur.MUST);
return new ConstantScoreQuery(filter.build());
}
private static Query eastGeoBoundingBoxFilter(GeoPoint topLeft, GeoPoint bottomRight,
LegacyGeoPointFieldMapper.LegacyGeoPointFieldType fieldType, QueryShardContext context) {
BooleanQuery.Builder filter = new BooleanQuery.Builder();
filter.add(fieldType.lonFieldType().rangeQuery(topLeft.lon(), bottomRight.lon(), true, true, context), Occur.MUST);
filter.add(fieldType.latFieldType().rangeQuery(bottomRight.lat(), topLeft.lat(), true, true, context), Occur.MUST);
return new ConstantScoreQuery(filter.build());
}
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.shard;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.IndexCommit;
@ -125,7 +124,6 @@ import org.elasticsearch.repositories.RepositoriesService;
import org.elasticsearch.repositories.Repository;
import org.elasticsearch.search.suggest.completion.CompletionFieldStats;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.FileNotFoundException;
@ -753,10 +751,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
CompletionStats completionStats = new CompletionStats();
try (final Engine.Searcher currentSearcher = acquireSearcher("completion_stats")) {
completionStats.add(CompletionFieldStats.completionStats(currentSearcher.reader(), fields));
// Necessary for 2.x shards:
Completion090PostingsFormat postingsFormat = ((Completion090PostingsFormat)
PostingsFormat.forName(Completion090PostingsFormat.CODEC_NAME));
completionStats.add(postingsFormat.completionStats(currentSearcher.reader(), fields));
}
return completionStats;
}

View File

@ -155,7 +155,7 @@ public class TranslogRecoveryPerformer {
// we set canHaveDuplicates to true all the time such that we de-optimze the translog case and ensure that all
// autoGeneratedID docs that are coming from the primary are updated correctly.
Engine.Index engineIndex = IndexShard.prepareIndex(docMapper(index.type()), source(shardId.getIndexName(), index.type(), index.id(), index.source())
.routing(index.routing()).parent(index.parent()).timestamp(index.timestamp()).ttl(index.ttl()), index.seqNo(),
.routing(index.routing()).parent(index.parent()), index.seqNo(),
index.version(), index.versionType().versionTypeForReplicationAndRecovery(), origin, index.getAutoGeneratedIdTimestamp(), true);
maybeAddMappingUpdate(engineIndex.type(), engineIndex.parsedDoc().dynamicMappingsUpdate(), engineIndex.id(), allowMappingUpdates);
if (logger.isTraceEnabled()) {

View File

@ -47,7 +47,6 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.StringFieldMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.UidFieldMapper;
@ -163,8 +162,7 @@ public class TermVectorsService {
private static boolean isValidField(MappedFieldType fieldType) {
// must be a string
if (fieldType instanceof StringFieldMapper.StringFieldType == false
&& fieldType instanceof KeywordFieldMapper.KeywordFieldType == false
if (fieldType instanceof KeywordFieldMapper.KeywordFieldType == false
&& fieldType instanceof TextFieldMapper.TextFieldType == false) {
return false;
}

View File

@ -808,15 +808,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
public final BytesReference source;
public final String routing;
public final String parent;
public final long timestamp;
public final long ttl;
public Source(BytesReference source, String routing, String parent, long timestamp, long ttl) {
public Source(BytesReference source, String routing, String parent) {
this.source = source;
this.routing = routing;
this.parent = parent;
this.timestamp = timestamp;
this.ttl = ttl;
}
}
@ -834,8 +830,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
private final BytesReference source;
private final String routing;
private final String parent;
private final long timestamp;
private final long ttl;
public Index(StreamInput in) throws IOException {
final int format = in.readVInt(); // SERIALIZATION_FORMAT
@ -846,8 +840,10 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
routing = in.readOptionalString();
parent = in.readOptionalString();
this.version = in.readLong();
this.timestamp = in.readLong();
this.ttl = in.readLong();
if (format < FORMAT_SEQ_NO) {
in.readLong(); // timestamp
in.readLong(); // ttl
}
this.versionType = VersionType.fromValue(in.readByte());
assert versionType.validateVersionForWrites(this.version);
if (format >= FORMAT_AUTO_GENERATED_IDS) {
@ -868,8 +864,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
this.parent = index.parent();
this.seqNo = indexResult.getSeqNo();
this.version = indexResult.getVersion();
this.timestamp = index.timestamp();
this.ttl = index.ttl();
this.versionType = index.versionType();
this.autoGeneratedIdTimestamp = index.getAutoGeneratedIdTimestamp();
}
@ -883,8 +877,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
versionType = VersionType.INTERNAL;
routing = null;
parent = null;
timestamp = 0;
ttl = 0;
autoGeneratedIdTimestamp = -1;
}
@ -914,14 +906,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
return this.parent;
}
public long timestamp() {
return this.timestamp;
}
public long ttl() {
return this.ttl;
}
public BytesReference source() {
return this.source;
}
@ -940,7 +924,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
@Override
public Source getSource() {
return new Source(source, routing, parent, timestamp, ttl);
return new Source(source, routing, parent);
}
@Override
@ -952,8 +936,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
out.writeOptionalString(routing);
out.writeOptionalString(parent);
out.writeLong(version);
out.writeLong(timestamp);
out.writeLong(ttl);
out.writeByte(versionType.getValue());
out.writeLong(autoGeneratedIdTimestamp);
out.writeVLong(seqNo);
@ -972,8 +955,6 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
if (version != index.version ||
seqNo != index.seqNo ||
timestamp != index.timestamp ||
ttl != index.ttl ||
id.equals(index.id) == false ||
type.equals(index.type) == false ||
versionType != index.versionType ||
@ -998,9 +979,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
result = 31 * result + source.hashCode();
result = 31 * result + (routing != null ? routing.hashCode() : 0);
result = 31 * result + (parent != null ? parent.hashCode() : 0);
result = 31 * result + Long.hashCode(timestamp);
result = 31 * result + Long.hashCode(autoGeneratedIdTimestamp);
result = 31 * result + Long.hashCode(ttl);
return result;
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.mapper.BooleanFieldMapper;
import org.elasticsearch.index.mapper.CompletionFieldMapper;
import org.elasticsearch.index.mapper.DateFieldMapper;
import org.elasticsearch.index.mapper.FieldNamesFieldMapper;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.GeoShapeFieldMapper;
import org.elasticsearch.index.mapper.IdFieldMapper;
import org.elasticsearch.index.mapper.IndexFieldMapper;
@ -47,10 +46,7 @@ import org.elasticsearch.index.mapper.RangeFieldMapper;
import org.elasticsearch.index.mapper.RoutingFieldMapper;
import org.elasticsearch.index.mapper.ScaledFloatFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.StringFieldMapper;
import org.elasticsearch.index.mapper.TTLFieldMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import org.elasticsearch.index.mapper.TokenCountFieldMapper;
import org.elasticsearch.index.mapper.TypeFieldMapper;
import org.elasticsearch.index.mapper.UidFieldMapper;
@ -65,7 +61,6 @@ import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.store.TransportNodesListShardStoreMetaData;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.plugins.MapperPlugin;
import java.util.ArrayList;
@ -110,14 +105,12 @@ public class IndicesModule extends AbstractModule {
mappers.put(DateFieldMapper.CONTENT_TYPE, new DateFieldMapper.TypeParser());
mappers.put(IpFieldMapper.CONTENT_TYPE, new IpFieldMapper.TypeParser());
mappers.put(ScaledFloatFieldMapper.CONTENT_TYPE, new ScaledFloatFieldMapper.TypeParser());
mappers.put(StringFieldMapper.CONTENT_TYPE, new StringFieldMapper.TypeParser());
mappers.put(TextFieldMapper.CONTENT_TYPE, new TextFieldMapper.TypeParser());
mappers.put(KeywordFieldMapper.CONTENT_TYPE, new KeywordFieldMapper.TypeParser());
mappers.put(TokenCountFieldMapper.CONTENT_TYPE, new TokenCountFieldMapper.TypeParser());
mappers.put(ObjectMapper.CONTENT_TYPE, new ObjectMapper.TypeParser());
mappers.put(ObjectMapper.NESTED_CONTENT_TYPE, new ObjectMapper.TypeParser());
mappers.put(CompletionFieldMapper.CONTENT_TYPE, new CompletionFieldMapper.TypeParser());
mappers.put(GeoPointFieldMapper.CONTENT_TYPE, new GeoPointFieldMapper.TypeParser());
mappers.put(LatLonPointFieldMapper.CONTENT_TYPE, new LatLonPointFieldMapper.TypeParser());
if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) {
mappers.put(GeoShapeFieldMapper.CONTENT_TYPE, new GeoShapeFieldMapper.TypeParser());
@ -147,8 +140,6 @@ public class IndicesModule extends AbstractModule {
metadataMappers.put(SourceFieldMapper.NAME, new SourceFieldMapper.TypeParser());
metadataMappers.put(TypeFieldMapper.NAME, new TypeFieldMapper.TypeParser());
metadataMappers.put(AllFieldMapper.NAME, new AllFieldMapper.TypeParser());
metadataMappers.put(TimestampFieldMapper.NAME, new TimestampFieldMapper.TypeParser());
metadataMappers.put(TTLFieldMapper.NAME, new TTLFieldMapper.TypeParser());
metadataMappers.put(VersionFieldMapper.NAME, new VersionFieldMapper.TypeParser());
metadataMappers.put(ParentFieldMapper.NAME, new ParentFieldMapper.TypeParser());
metadataMappers.put(SeqNoFieldMapper.NAME, new SeqNoFieldMapper.TypeParser());
@ -176,7 +167,6 @@ public class IndicesModule extends AbstractModule {
bind(IndicesClusterStateService.class).asEagerSingleton();
bind(SyncedFlushService.class).asEagerSingleton();
bind(TransportNodesListShardStoreMetaData.class).asEagerSingleton();
bind(IndicesTTLService.class).asEagerSingleton();
bind(GlobalCheckpointSyncAction.class).asEagerSingleton();
}

View File

@ -1,359 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.indices.ttl;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SimpleCollector;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.bulk.TransportBulkAction;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.fieldvisitor.FieldsVisitor;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.TTLFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.mapper.VersionFieldMapper;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.indices.IndicesService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
/**
* A node level service that delete expired docs on node primary shards.
*/
public class IndicesTTLService extends AbstractLifecycleComponent {
public static final Setting<TimeValue> INDICES_TTL_INTERVAL_SETTING =
Setting.positiveTimeSetting("indices.ttl.interval", TimeValue.timeValueSeconds(60),
Property.Dynamic, Property.NodeScope);
private final ClusterService clusterService;
private final IndicesService indicesService;
private final TransportBulkAction bulkAction;
private final int bulkSize;
private PurgerThread purgerThread;
@Inject
public IndicesTTLService(Settings settings, ClusterService clusterService, IndicesService indicesService, ClusterSettings clusterSettings, TransportBulkAction bulkAction) {
super(settings);
this.clusterService = clusterService;
this.indicesService = indicesService;
TimeValue interval = INDICES_TTL_INTERVAL_SETTING.get(settings);
this.bulkAction = bulkAction;
this.bulkSize = this.settings.getAsInt("indices.ttl.bulk_size", 10000);
this.purgerThread = new PurgerThread(EsExecutors.threadName(settings, "[ttl_expire]"), interval);
clusterSettings.addSettingsUpdateConsumer(INDICES_TTL_INTERVAL_SETTING, this.purgerThread::resetInterval);
}
@Override
protected void doStart() {
this.purgerThread.start();
}
@Override
protected void doStop() {
try {
this.purgerThread.shutdown();
} catch (InterruptedException e) {
// we intentionally do not want to restore the interruption flag, we're about to shutdown anyway
}
}
@Override
protected void doClose() {
}
private class PurgerThread extends Thread {
private final AtomicBoolean running = new AtomicBoolean(true);
private final Notifier notifier;
private final CountDownLatch shutdownLatch = new CountDownLatch(1);
public PurgerThread(String name, TimeValue interval) {
super(name);
setDaemon(true);
this.notifier = new Notifier(interval);
}
public void shutdown() throws InterruptedException {
if (running.compareAndSet(true, false)) {
notifier.doNotify();
shutdownLatch.await();
}
}
public void resetInterval(TimeValue interval) {
notifier.setTimeout(interval);
}
@Override
public void run() {
try {
while (running.get()) {
try {
List<IndexShard> shardsToPurge = getShardsToPurge();
purgeShards(shardsToPurge);
} catch (Exception e) {
if (running.get()) {
logger.warn("failed to execute ttl purge", e);
}
}
if (running.get()) {
notifier.await();
}
}
} finally {
shutdownLatch.countDown();
}
}
/**
* Returns the shards to purge, i.e. the local started primary shards that have ttl enabled and disable_purge to false
*/
private List<IndexShard> getShardsToPurge() {
List<IndexShard> shardsToPurge = new ArrayList<>();
MetaData metaData = clusterService.state().metaData();
for (IndexService indexService : indicesService) {
// check the value of disable_purge for this index
IndexMetaData indexMetaData = metaData.index(indexService.index());
if (indexMetaData == null) {
continue;
}
if (indexService.getIndexSettings().isTTLPurgeDisabled()) {
continue;
}
// check if ttl is enabled for at least one type of this index
boolean hasTTLEnabled = false;
for (String type : indexService.mapperService().types()) {
DocumentMapper documentType = indexService.mapperService().documentMapper(type);
if (documentType.TTLFieldMapper().enabled()) {
hasTTLEnabled = true;
break;
}
}
if (hasTTLEnabled) {
for (IndexShard indexShard : indexService) {
if (indexShard.state() == IndexShardState.STARTED && indexShard.routingEntry().primary() && indexShard.routingEntry().started()) {
shardsToPurge.add(indexShard);
}
}
}
}
return shardsToPurge;
}
public TimeValue getInterval() {
return notifier.getTimeout();
}
}
private void purgeShards(List<IndexShard> shardsToPurge) {
for (IndexShard shardToPurge : shardsToPurge) {
Query query = shardToPurge.mapperService().fullName(TTLFieldMapper.NAME).rangeQuery(null, System.currentTimeMillis(), false,
true, null);
Engine.Searcher searcher = shardToPurge.acquireSearcher("indices_ttl");
try {
logger.debug("[{}][{}] purging shard", shardToPurge.routingEntry().index(), shardToPurge.routingEntry().id());
ExpiredDocsCollector expiredDocsCollector = new ExpiredDocsCollector();
searcher.searcher().search(query, expiredDocsCollector);
List<DocToPurge> docsToPurge = expiredDocsCollector.getDocsToPurge();
BulkRequest bulkRequest = new BulkRequest();
for (DocToPurge docToPurge : docsToPurge) {
bulkRequest.add(new DeleteRequest().index(shardToPurge.routingEntry().getIndexName()).type(docToPurge.type).id(docToPurge.id).version(docToPurge.version).routing(docToPurge.routing));
bulkRequest = processBulkIfNeeded(bulkRequest, false);
}
processBulkIfNeeded(bulkRequest, true);
} catch (Exception e) {
logger.warn("failed to purge", e);
} finally {
searcher.close();
}
}
}
private static class DocToPurge {
public final String type;
public final String id;
public final long version;
public final String routing;
public DocToPurge(String type, String id, long version, String routing) {
this.type = type;
this.id = id;
this.version = version;
this.routing = routing;
}
}
private class ExpiredDocsCollector extends SimpleCollector {
private LeafReaderContext context;
private List<DocToPurge> docsToPurge = new ArrayList<>();
private NumericDocValues versions;
public ExpiredDocsCollector() {
}
@Override
public void setScorer(Scorer scorer) {
}
@Override
public boolean needsScores() {
return false;
}
@Override
public void collect(int doc) {
try {
FieldsVisitor fieldsVisitor = new FieldsVisitor(false);
context.reader().document(doc, fieldsVisitor);
Uid uid = fieldsVisitor.uid();
final long version = versions == null ? Versions.NOT_FOUND : versions.get(doc);
docsToPurge.add(new DocToPurge(uid.type(), uid.id(), version, fieldsVisitor.routing()));
} catch (Exception e) {
logger.trace("failed to collect doc", e);
}
}
@Override
public void doSetNextReader(LeafReaderContext context) throws IOException {
this.context = context;
this.versions = context.reader().getNumericDocValues(VersionFieldMapper.NAME);
}
public List<DocToPurge> getDocsToPurge() {
return this.docsToPurge;
}
}
private BulkRequest processBulkIfNeeded(BulkRequest bulkRequest, boolean force) {
if ((force && bulkRequest.numberOfActions() > 0) || bulkRequest.numberOfActions() >= bulkSize) {
try {
bulkAction.executeBulk(bulkRequest, new ActionListener<BulkResponse>() {
@Override
public void onResponse(BulkResponse bulkResponse) {
if (bulkResponse.hasFailures()) {
int failedItems = 0;
for (BulkItemResponse response : bulkResponse) {
if (response.isFailed()) failedItems++;
}
if (logger.isTraceEnabled()) {
logger.trace("bulk deletion failures for [{}]/[{}] items, failure message: [{}]", failedItems, bulkResponse.getItems().length, bulkResponse.buildFailureMessage());
} else {
logger.error("bulk deletion failures for [{}]/[{}] items", failedItems, bulkResponse.getItems().length);
}
} else {
logger.trace("bulk deletion took {}ms", bulkResponse.getTookInMillis());
}
}
@Override
public void onFailure(Exception e) {
if (logger.isTraceEnabled()) {
logger.trace("failed to execute bulk", e);
} else {
logger.warn("failed to execute bulk: ", e);
}
}
});
} catch (Exception e) {
logger.warn("failed to process bulk", e);
}
bulkRequest = new BulkRequest();
}
return bulkRequest;
}
private static final class Notifier {
private final ReentrantLock lock = new ReentrantLock();
private final Condition condition = lock.newCondition();
private volatile TimeValue timeout;
public Notifier(TimeValue timeout) {
assert timeout != null;
this.timeout = timeout;
}
public void await() {
lock.lock();
try {
condition.await(timeout.millis(), TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
// we intentionally do not want to restore the interruption flag, we're about to shutdown anyway
} finally {
lock.unlock();
}
}
public void setTimeout(TimeValue timeout) {
assert timeout != null;
this.timeout = timeout;
doNotify();
}
public TimeValue getTimeout() {
return timeout;
}
public void doNotify() {
lock.lock();
try {
condition.signalAll();
} finally {
lock.unlock();
}
}
}
}

View File

@ -25,8 +25,6 @@ import org.elasticsearch.index.mapper.IndexFieldMapper;
import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.mapper.RoutingFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.TTLFieldMapper;
import org.elasticsearch.index.mapper.TimestampFieldMapper;
import org.elasticsearch.index.mapper.TypeFieldMapper;
import java.text.DateFormat;
@ -56,8 +54,7 @@ public final class IngestDocument {
private final Map<String, Object> sourceAndMetadata;
private final Map<String, Object> ingestMetadata;
public IngestDocument(String index, String type, String id, String routing, String parent, String timestamp,
String ttl, Map<String, Object> source) {
public IngestDocument(String index, String type, String id, String routing, String parent, Map<String, Object> source) {
this.sourceAndMetadata = new HashMap<>();
this.sourceAndMetadata.putAll(source);
this.sourceAndMetadata.put(MetaData.INDEX.getFieldName(), index);
@ -69,12 +66,6 @@ public final class IngestDocument {
if (parent != null) {
this.sourceAndMetadata.put(MetaData.PARENT.getFieldName(), parent);
}
if (timestamp != null) {
this.sourceAndMetadata.put(MetaData.TIMESTAMP.getFieldName(), timestamp);
}
if (ttl != null) {
this.sourceAndMetadata.put(MetaData.TTL.getFieldName(), ttl);
}
this.ingestMetadata = new HashMap<>();
DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.ROOT);
@ -639,9 +630,7 @@ public final class IngestDocument {
TYPE(TypeFieldMapper.NAME),
ID(IdFieldMapper.NAME),
ROUTING(RoutingFieldMapper.NAME),
PARENT(ParentFieldMapper.NAME),
TIMESTAMP(TimestampFieldMapper.NAME),
TTL(TTLFieldMapper.NAME);
PARENT(ParentFieldMapper.NAME);
private final String fieldName;

View File

@ -159,10 +159,8 @@ public class PipelineExecutionService implements ClusterStateListener {
String id = indexRequest.id();
String routing = indexRequest.routing();
String parent = indexRequest.parent();
String timestamp = indexRequest.timestamp();
String ttl = indexRequest.ttl() == null ? null : indexRequest.ttl().toString();
Map<String, Object> sourceAsMap = indexRequest.sourceAsMap();
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, parent, timestamp, ttl, sourceAsMap);
IngestDocument ingestDocument = new IngestDocument(index, type, id, routing, parent, sourceAsMap);
pipeline.execute(ingestDocument);
Map<IngestDocument.MetaData, String> metadataMap = ingestDocument.extractMetadata();
@ -173,8 +171,6 @@ public class PipelineExecutionService implements ClusterStateListener {
indexRequest.id(metadataMap.get(IngestDocument.MetaData.ID));
indexRequest.routing(metadataMap.get(IngestDocument.MetaData.ROUTING));
indexRequest.parent(metadataMap.get(IngestDocument.MetaData.PARENT));
indexRequest.timestamp(metadataMap.get(IngestDocument.MetaData.TIMESTAMP));
indexRequest.ttl(metadataMap.get(IngestDocument.MetaData.TTL));
indexRequest.source(ingestDocument.getSourceAndMetadata());
} catch (Exception e) {
totalStats.ingestFailed();

View File

@ -98,7 +98,6 @@ import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.ingest.IngestService;
import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.monitor.jvm.JvmInfo;
@ -542,7 +541,6 @@ public class Node implements Closeable {
injector.getInstance(MappingUpdatedAction.class).setClient(client);
injector.getInstance(IndicesService.class).start();
injector.getInstance(IndicesClusterStateService.class).start();
injector.getInstance(IndicesTTLService.class).start();
injector.getInstance(SnapshotsService.class).start();
injector.getInstance(SnapshotShardsService.class).start();
injector.getInstance(RoutingService.class).start();
@ -666,7 +664,6 @@ public class Node implements Closeable {
// This can confuse other nodes and delay things - mostly if we're the master and we're running tests.
injector.getInstance(Discovery.class).stop();
// we close indices first, so operations won't be allowed on it
injector.getInstance(IndicesTTLService.class).stop();
injector.getInstance(RoutingService.class).stop();
injector.getInstance(ClusterService.class).stop();
injector.getInstance(NodeConnectionsService.class).stop();
@ -717,7 +714,6 @@ public class Node implements Closeable {
toClose.add(() -> stopWatch.stop().start("indices_cluster"));
toClose.add(injector.getInstance(IndicesClusterStateService.class));
toClose.add(() -> stopWatch.stop().start("indices"));
toClose.add(injector.getInstance(IndicesTTLService.class));
toClose.add(injector.getInstance(IndicesService.class));
// close filter/fielddata caches after indices
toClose.add(injector.getInstance(IndicesStore.class));

View File

@ -66,10 +66,6 @@ public class RestIndexAction extends BaseRestHandler {
IndexRequest indexRequest = new IndexRequest(request.param("index"), request.param("type"), request.param("id"));
indexRequest.routing(request.param("routing"));
indexRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
indexRequest.timestamp(request.param("timestamp"));
if (request.hasParam("ttl")) {
indexRequest.ttl(request.param("ttl"));
}
indexRequest.setPipeline(request.param("pipeline"));
indexRequest.source(request.content());
indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT));

View File

@ -87,10 +87,6 @@ public class RestUpdateAction extends BaseRestHandler {
if (upsertRequest != null) {
upsertRequest.routing(request.param("routing"));
upsertRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
upsertRequest.timestamp(request.param("timestamp"));
if (request.hasParam("ttl")) {
upsertRequest.ttl(request.param("ttl"));
}
upsertRequest.version(RestActions.parseVersion(request));
upsertRequest.versionType(VersionType.fromString(request.param("version_type"), upsertRequest.versionType()));
}
@ -98,10 +94,6 @@ public class RestUpdateAction extends BaseRestHandler {
if (doc != null) {
doc.routing(request.param("routing"));
doc.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
doc.timestamp(request.param("timestamp"));
if (request.hasParam("ttl")) {
doc.ttl(request.param("ttl"));
}
doc.version(RestActions.parseVersion(request));
doc.versionType(VersionType.fromString(request.param("version_type"), doc.versionType()));
}

View File

@ -38,10 +38,8 @@ import org.elasticsearch.index.query.FieldMaskingSpanQueryBuilder;
import org.elasticsearch.index.query.FuzzyQueryBuilder;
import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder;
import org.elasticsearch.index.query.GeoDistanceQueryBuilder;
import org.elasticsearch.index.query.GeoDistanceRangeQueryBuilder;
import org.elasticsearch.index.query.GeoPolygonQueryBuilder;
import org.elasticsearch.index.query.GeoShapeQueryBuilder;
import org.elasticsearch.index.query.GeohashCellQuery;
import org.elasticsearch.index.query.HasChildQueryBuilder;
import org.elasticsearch.index.query.HasParentQueryBuilder;
import org.elasticsearch.index.query.IdsQueryBuilder;
@ -775,11 +773,8 @@ public class SearchModule {
registerQuery(new QuerySpec<>(TypeQueryBuilder.NAME, TypeQueryBuilder::new, TypeQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(ScriptQueryBuilder.NAME, ScriptQueryBuilder::new, ScriptQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeoDistanceQueryBuilder.NAME, GeoDistanceQueryBuilder::new, GeoDistanceQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeoDistanceRangeQueryBuilder.NAME, GeoDistanceRangeQueryBuilder::new,
GeoDistanceRangeQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeoBoundingBoxQueryBuilder.NAME, GeoBoundingBoxQueryBuilder::new,
GeoBoundingBoxQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeohashCellQuery.NAME, GeohashCellQuery.Builder::new, GeohashCellQuery.Builder::fromXContent));
registerQuery(new QuerySpec<>(GeoPolygonQueryBuilder.NAME, GeoPolygonQueryBuilder::new, GeoPolygonQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(ExistsQueryBuilder.NAME, ExistsQueryBuilder::new, ExistsQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MatchNoneQueryBuilder.NAME, MatchNoneQueryBuilder::new, MatchNoneQueryBuilder::fromXContent));

View File

@ -27,7 +27,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.SourceFieldMapper;
import org.elasticsearch.index.mapper.StringFieldMapper;
import org.elasticsearch.index.mapper.TextFieldMapper;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.internal.SearchContext;
@ -89,8 +88,7 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
// what they were doing and try to highlight anyway.
if (fieldNameContainsWildcards) {
if (fieldMapper.fieldType().typeName().equals(TextFieldMapper.CONTENT_TYPE) == false &&
fieldMapper.fieldType().typeName().equals(KeywordFieldMapper.CONTENT_TYPE) == false &&
fieldMapper.fieldType().typeName().equals(StringFieldMapper.CONTENT_TYPE) == false) {
fieldMapper.fieldType().typeName().equals(KeywordFieldMapper.CONTENT_TYPE) == false) {
continue;
}
}

View File

@ -123,9 +123,8 @@ public class Suggest implements Iterable<Suggest.Suggestion<? extends Entry<? ex
case CompletionSuggestion.TYPE:
suggestion = new CompletionSuggestion();
break;
case org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.TYPE:
suggestion = new org.elasticsearch.search.suggest.completion2x.CompletionSuggestion();
break;
case 2: // CompletionSuggestion.TYPE
throw new IllegalArgumentException("Completion suggester 2.x is not supported anymore");
case PhraseSuggestion.TYPE:
suggestion = new PhraseSuggestion();
break;

View File

@ -18,11 +18,7 @@
*/
package org.elasticsearch.search.suggest.completion;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.BulkScorer;
import org.apache.lucene.search.CollectionTerminatedException;
import org.apache.lucene.search.IndexSearcher;
@ -32,28 +28,19 @@ import org.apache.lucene.search.suggest.document.CompletionQuery;
import org.apache.lucene.search.suggest.document.TopSuggestDocs;
import org.apache.lucene.search.suggest.document.TopSuggestDocsCollector;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.PriorityQueue;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.fielddata.AtomicFieldData;
import org.elasticsearch.index.fielddata.ScriptDocValues;
import org.elasticsearch.index.mapper.CompletionFieldMapper;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.suggest.Suggest;
import org.elasticsearch.search.suggest.Suggester;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@ -93,73 +80,11 @@ public class CompletionSuggester extends Suggester<CompletionSuggestionContext>
break;
}
}
return completionSuggestion;
} else if (suggestionContext.getFieldType2x() != null) {
final IndexReader indexReader = searcher.getIndexReader();
org.elasticsearch.search.suggest.completion2x.CompletionSuggestion completionSuggestion =
new org.elasticsearch.search.suggest.completion2x.CompletionSuggestion(name, suggestionContext.getSize());
spare.copyUTF8Bytes(suggestionContext.getText());
org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry completionSuggestEntry =
new org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry(new Text(spare.toString()), 0, spare.length());
completionSuggestion.addTerm(completionSuggestEntry);
String fieldName = suggestionContext.getField();
Map<String, org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option> results =
new HashMap<>(indexReader.leaves().size() * suggestionContext.getSize());
for (LeafReaderContext atomicReaderContext : indexReader.leaves()) {
LeafReader atomicReader = atomicReaderContext.reader();
Terms terms = atomicReader.fields().terms(fieldName);
if (terms instanceof Completion090PostingsFormat.CompletionTerms) {
final Completion090PostingsFormat.CompletionTerms lookupTerms = (Completion090PostingsFormat.CompletionTerms) terms;
final Lookup lookup = lookupTerms.getLookup(suggestionContext.getFieldType2x(), suggestionContext);
if (lookup == null) {
// we don't have a lookup for this segment.. this might be possible if a merge dropped all
// docs from the segment that had a value in this segment.
continue;
}
List<Lookup.LookupResult> lookupResults = lookup.lookup(spare.get(), false, suggestionContext.getSize());
for (Lookup.LookupResult res : lookupResults) {
final String key = res.key.toString();
final float score = res.value;
final org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option value = results.get(key);
if (value == null) {
final org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option option =
new org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option(new Text(key), score,
res.payload == null ? null : new BytesArray(res.payload));
results.put(key, option);
} else if (value.getScore() < score) {
value.setScore(score);
value.setPayload(res.payload == null ? null : new BytesArray(res.payload));
}
}
}
}
final List<org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option> options =
new ArrayList<>(results.values());
CollectionUtil.introSort(options, scoreComparator);
int optionCount = Math.min(suggestionContext.getSize(), options.size());
for (int i = 0; i < optionCount; i++) {
completionSuggestEntry.addOption(options.get(i));
}
return completionSuggestion;
}
return null;
}
private static final ScoreComparator scoreComparator = new ScoreComparator();
public static class ScoreComparator implements
Comparator<org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option> {
@Override
public int compare(org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option o1,
org.elasticsearch.search.suggest.completion2x.CompletionSuggestion.Entry.Option o2) {
return Float.compare(o2.getScore(), o1.getScore());
}
}
private static void suggest(IndexSearcher searcher, CompletionQuery query, TopSuggestDocsCollector collector) throws IOException {
query = (CompletionQuery) query.rewrite(searcher.getIndexReader());
Weight weight = query.createWeight(searcher, collector.needsScores());

View File

@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.mapper.CompletionFieldMapper;
import org.elasticsearch.index.mapper.CompletionFieldMapper2x;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryParseContext;
@ -41,9 +40,6 @@ import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.SuggestionSearchContext.SuggestionContext;
import org.elasticsearch.search.suggest.completion.context.ContextMapping;
import org.elasticsearch.search.suggest.completion.context.ContextMappings;
import org.elasticsearch.search.suggest.completion2x.context.CategoryContextMapping;
import org.elasticsearch.search.suggest.completion2x.context.ContextMapping.ContextQuery;
import org.elasticsearch.search.suggest.completion2x.context.GeolocationContextMapping;
import java.io.IOException;
import java.util.ArrayList;
@ -215,105 +211,6 @@ public class CompletionSuggestionBuilder extends SuggestionBuilder<CompletionSug
return this;
}
public CompletionSuggestionBuilder contexts(Contexts2x contexts2x) {
Objects.requireNonNull(contexts2x, "contexts must not be null");
try {
XContentBuilder contentBuilder = XContentFactory.jsonBuilder();
contentBuilder.startObject();
for (ContextQuery contextQuery : contexts2x.contextQueries) {
contextQuery.toXContent(contentBuilder, EMPTY_PARAMS);
}
contentBuilder.endObject();
return contexts(contentBuilder);
} catch (IOException e) {
throw new IllegalArgumentException(e);
}
}
// for 2.x context support
public static class Contexts2x {
private List<ContextQuery> contextQueries = new ArrayList<>();
@SuppressWarnings("unchecked")
private Contexts2x addContextQuery(ContextQuery ctx) {
this.contextQueries.add(ctx);
return this;
}
/**
* Setup a Geolocation for suggestions. See {@link GeolocationContextMapping}.
* @param lat Latitude of the location
* @param lon Longitude of the Location
* @return this
*/
@Deprecated
public Contexts2x addGeoLocation(String name, double lat, double lon, int ... precisions) {
return addContextQuery(GeolocationContextMapping.query(name, lat, lon, precisions));
}
/**
* Setup a Geolocation for suggestions. See {@link GeolocationContextMapping}.
* @param lat Latitude of the location
* @param lon Longitude of the Location
* @param precisions precisions as string var-args
* @return this
*/
@Deprecated
public Contexts2x addGeoLocationWithPrecision(String name, double lat, double lon, String ... precisions) {
return addContextQuery(GeolocationContextMapping.query(name, lat, lon, precisions));
}
/**
* Setup a Geolocation for suggestions. See {@link GeolocationContextMapping}.
* @param geohash Geohash of the location
* @return this
*/
@Deprecated
public Contexts2x addGeoLocation(String name, String geohash) {
return addContextQuery(GeolocationContextMapping.query(name, geohash));
}
/**
* Setup a Category for suggestions. See {@link CategoryContextMapping}.
* @param categories name of the category
* @return this
*/
@Deprecated
public Contexts2x addCategory(String name, CharSequence...categories) {
return addContextQuery(CategoryContextMapping.query(name, categories));
}
/**
* Setup a Category for suggestions. See {@link CategoryContextMapping}.
* @param categories name of the category
* @return this
*/
@Deprecated
public Contexts2x addCategory(String name, Iterable<? extends CharSequence> categories) {
return addContextQuery(CategoryContextMapping.query(name, categories));
}
/**
* Setup a Context Field for suggestions. See {@link CategoryContextMapping}.
* @param fieldvalues name of the category
* @return this
*/
@Deprecated
public Contexts2x addContextField(String name, CharSequence...fieldvalues) {
return addContextQuery(CategoryContextMapping.query(name, fieldvalues));
}
/**
* Setup a Context Field for suggestions. See {@link CategoryContextMapping}.
* @param fieldvalues name of the category
* @return this
*/
@Deprecated
public Contexts2x addContextField(String name, Iterable<? extends CharSequence> fieldvalues) {
return addContextQuery(CategoryContextMapping.query(name, fieldvalues));
}
}
private static class InnerBuilder extends CompletionSuggestionBuilder {
private String field;
@ -366,8 +263,7 @@ public class CompletionSuggestionBuilder extends SuggestionBuilder<CompletionSug
suggestionContext.setRegexOptions(regexOptions);
MappedFieldType mappedFieldType = mapperService.fullName(suggestionContext.getField());
if (mappedFieldType == null ||
(mappedFieldType instanceof CompletionFieldMapper.CompletionFieldType == false
&& mappedFieldType instanceof CompletionFieldMapper2x.CompletionFieldType == false)) {
mappedFieldType instanceof CompletionFieldMapper.CompletionFieldType == false) {
throw new IllegalArgumentException("Field [" + suggestionContext.getField() + "] is not a completion suggest field");
}
if (mappedFieldType instanceof CompletionFieldMapper.CompletionFieldType) {
@ -395,23 +291,8 @@ public class CompletionSuggestionBuilder extends SuggestionBuilder<CompletionSug
} else if (contextBytes != null) {
throw new IllegalArgumentException("suggester [" + type.name() + "] doesn't expect any context");
}
} else if (mappedFieldType instanceof CompletionFieldMapper2x.CompletionFieldType) {
CompletionFieldMapper2x.CompletionFieldType type = ((CompletionFieldMapper2x.CompletionFieldType) mappedFieldType);
suggestionContext.setFieldType2x(type);
if (type.requiresContext()) {
if (contextBytes != null) {
try (XContentParser contextParser = XContentFactory.xContent(contextBytes).createParser(contextBytes)) {
contextParser.nextToken();
suggestionContext.setContextQueries(ContextQuery.parseQueries(type.getContextMapping(), contextParser));
}
} else {
throw new IllegalArgumentException("suggester [completion] requires context to be setup");
}
} else if (contextBytes != null) {
throw new IllegalArgumentException("suggester [completion] doesn't expect any context");
}
}
assert suggestionContext.getFieldType() != null || suggestionContext.getFieldType2x() != null : "no completion field type set";
assert suggestionContext.getFieldType() != null : "no completion field type set";
return suggestionContext;
}

View File

@ -21,12 +21,10 @@ package org.elasticsearch.search.suggest.completion;
import org.apache.lucene.search.suggest.document.CompletionQuery;
import org.elasticsearch.common.unit.Fuzziness;
import org.elasticsearch.index.mapper.CompletionFieldMapper;
import org.elasticsearch.index.mapper.CompletionFieldMapper2x;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.suggest.SuggestionSearchContext;
import org.elasticsearch.search.suggest.completion.context.ContextMapping;
import org.elasticsearch.search.suggest.completion.context.ContextMappings;
import org.elasticsearch.search.suggest.completion2x.context.ContextMapping.ContextQuery;
import java.util.Collections;
import java.util.List;
@ -42,17 +40,11 @@ public class CompletionSuggestionContext extends SuggestionSearchContext.Suggest
private FuzzyOptions fuzzyOptions;
private RegexOptions regexOptions;
private Map<String, List<ContextMapping.InternalQueryContext>> queryContexts = Collections.emptyMap();
private CompletionFieldMapper2x.CompletionFieldType fieldType2x;
private List<ContextQuery> contextQueries;
CompletionFieldMapper.CompletionFieldType getFieldType() {
return this.fieldType;
}
CompletionFieldMapper2x.CompletionFieldType getFieldType2x() {
return this.fieldType2x;
}
void setFieldType(CompletionFieldMapper.CompletionFieldType fieldType) {
this.fieldType = fieldType;
}
@ -113,15 +105,4 @@ public class CompletionSuggestionContext extends SuggestionSearchContext.Suggest
return query;
}
public void setFieldType2x(CompletionFieldMapper2x.CompletionFieldType type) {
this.fieldType2x = type;
}
public void setContextQueries(List<ContextQuery> contextQueries) {
this.contextQueries = contextQueries;
}
public List<ContextQuery> getContextQueries() {
return contextQueries;
}
}

View File

@ -29,8 +29,8 @@ import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.mapper.BaseGeoPointFieldMapper;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.query.QueryParseContext;
@ -140,7 +140,7 @@ public class GeoContextMapping extends ContextMapping<GeoQueryContext> {
public Set<CharSequence> parseContext(ParseContext parseContext, XContentParser parser) throws IOException, ElasticsearchParseException {
if (fieldName != null) {
FieldMapper mapper = parseContext.docMapper().mappers().getMapper(fieldName);
if (!(mapper instanceof GeoPointFieldMapper)) {
if (!(mapper instanceof BaseGeoPointFieldMapper)) {
throw new ElasticsearchParseException("referenced field must be mapped to geo_point");
}
}

View File

@ -1,413 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x;
import com.carrotsearch.hppc.ObjectLongHashMap;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.search.suggest.analyzing.XAnalyzingSuggester;
import org.apache.lucene.search.suggest.analyzing.XFuzzySuggester;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PairOutputs;
import org.apache.lucene.util.fst.PairOutputs.Pair;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.index.mapper.CompletionFieldMapper2x;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionContext;
import org.elasticsearch.search.suggest.completion.FuzzyOptions;
import org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat.CompletionLookupProvider;
import org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat.LookupFactory;
import org.elasticsearch.search.suggest.completion2x.context.ContextMapping.ContextQuery;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
public class AnalyzingCompletionLookupProvider extends CompletionLookupProvider {
// for serialization
public static final int SERIALIZE_PRESERVE_SEPARATORS = 1;
public static final int SERIALIZE_HAS_PAYLOADS = 2;
public static final int SERIALIZE_PRESERVE_POSITION_INCREMENTS = 4;
private static final int MAX_SURFACE_FORMS_PER_ANALYZED_FORM = 256;
private static final int MAX_GRAPH_EXPANSIONS = -1;
public static final String CODEC_NAME = "analyzing";
public static final int CODEC_VERSION_START = 1;
public static final int CODEC_VERSION_SERIALIZED_LABELS = 2;
public static final int CODEC_VERSION_CHECKSUMS = 3;
public static final int CODEC_VERSION_LATEST = CODEC_VERSION_CHECKSUMS;
private final boolean preserveSep;
private final boolean preservePositionIncrements;
private final int maxSurfaceFormsPerAnalyzedForm;
private final int maxGraphExpansions;
private final boolean hasPayloads;
private final XAnalyzingSuggester prototype;
public AnalyzingCompletionLookupProvider(boolean preserveSep, boolean preservePositionIncrements, boolean hasPayloads) {
this.preserveSep = preserveSep;
this.preservePositionIncrements = preservePositionIncrements;
this.hasPayloads = hasPayloads;
this.maxSurfaceFormsPerAnalyzedForm = MAX_SURFACE_FORMS_PER_ANALYZED_FORM;
this.maxGraphExpansions = MAX_GRAPH_EXPANSIONS;
int options = preserveSep ? XAnalyzingSuggester.PRESERVE_SEP : 0;
// needs to fixed in the suggester first before it can be supported
//options |= exactFirst ? XAnalyzingSuggester.EXACT_FIRST : 0;
prototype = new XAnalyzingSuggester(null, null, null, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions,
preservePositionIncrements, null, false, 1, XAnalyzingSuggester.SEP_LABEL, XAnalyzingSuggester.PAYLOAD_SEP,
XAnalyzingSuggester.END_BYTE, XAnalyzingSuggester.HOLE_CHARACTER);
}
@Override
public String getName() {
return "analyzing";
}
public boolean getPreserveSep() {
return preserveSep;
}
public boolean getPreservePositionsIncrements() {
return preservePositionIncrements;
}
public boolean hasPayloads() {
return hasPayloads;
}
@Override
public FieldsConsumer consumer(final IndexOutput output) throws IOException {
CodecUtil.writeHeader(output, CODEC_NAME, CODEC_VERSION_LATEST);
return new FieldsConsumer() {
private Map<String, Long> fieldOffsets = new HashMap<>();
@Override
public void close() throws IOException {
try {
/*
* write the offsets per field such that we know where
* we need to load the FSTs from
*/
long pointer = output.getFilePointer();
output.writeVInt(fieldOffsets.size());
for (Map.Entry<String, Long> entry : fieldOffsets.entrySet()) {
output.writeString(entry.getKey());
output.writeVLong(entry.getValue());
}
output.writeLong(pointer);
CodecUtil.writeFooter(output);
} finally {
IOUtils.close(output);
}
}
@Override
public void write(Fields fields) throws IOException {
for(String field : fields) {
Terms terms = fields.terms(field);
if (terms == null) {
continue;
}
TermsEnum termsEnum = terms.iterator();
PostingsEnum docsEnum = null;
final SuggestPayload spare = new SuggestPayload();
int maxAnalyzedPathsForOneInput = 0;
final XAnalyzingSuggester.XBuilder builder = new XAnalyzingSuggester.XBuilder(
maxSurfaceFormsPerAnalyzedForm, hasPayloads, XAnalyzingSuggester.PAYLOAD_SEP);
int docCount = 0;
while (true) {
BytesRef term = termsEnum.next();
if (term == null) {
break;
}
docsEnum = termsEnum.postings(docsEnum, PostingsEnum.PAYLOADS);
builder.startTerm(term);
int docFreq = 0;
while (docsEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
for (int i = 0; i < docsEnum.freq(); i++) {
final int position = docsEnum.nextPosition();
AnalyzingCompletionLookupProvider.this.parsePayload(docsEnum.getPayload(), spare);
builder.addSurface(spare.surfaceForm.get(), spare.payload.get(), spare.weight);
// multi fields have the same surface form so we sum up here
maxAnalyzedPathsForOneInput = Math.max(maxAnalyzedPathsForOneInput, position + 1);
}
docFreq++;
docCount = Math.max(docCount, docsEnum.docID()+1);
}
builder.finishTerm(docFreq);
}
/*
* Here we are done processing the field and we can
* buid the FST and write it to disk.
*/
FST<Pair<Long, BytesRef>> build = builder.build();
assert build != null || docCount == 0: "the FST is null but docCount is != 0 actual value: [" + docCount + "]";
/*
* it's possible that the FST is null if we have 2 segments that get merged
* and all docs that have a value in this field are deleted. This will cause
* a consumer to be created but it doesn't consume any values causing the FSTBuilder
* to return null.
*/
if (build != null) {
fieldOffsets.put(field, output.getFilePointer());
build.save(output);
/* write some more meta-info */
output.writeVInt(maxAnalyzedPathsForOneInput);
output.writeVInt(maxSurfaceFormsPerAnalyzedForm);
output.writeInt(maxGraphExpansions); // can be negative
int options = 0;
options |= preserveSep ? SERIALIZE_PRESERVE_SEPARATORS : 0;
options |= hasPayloads ? SERIALIZE_HAS_PAYLOADS : 0;
options |= preservePositionIncrements ? SERIALIZE_PRESERVE_POSITION_INCREMENTS : 0;
output.writeVInt(options);
output.writeVInt(XAnalyzingSuggester.SEP_LABEL);
output.writeVInt(XAnalyzingSuggester.END_BYTE);
output.writeVInt(XAnalyzingSuggester.PAYLOAD_SEP);
output.writeVInt(XAnalyzingSuggester.HOLE_CHARACTER);
}
}
}
};
}
@Override
public LookupFactory load(IndexInput input) throws IOException {
long sizeInBytes = 0;
int version = CodecUtil.checkHeader(input, CODEC_NAME, CODEC_VERSION_START, CODEC_VERSION_LATEST);
if (version >= CODEC_VERSION_CHECKSUMS) {
CodecUtil.checksumEntireFile(input);
}
final long metaPointerPosition = input.length() - (version >= CODEC_VERSION_CHECKSUMS? 8 + CodecUtil.footerLength() : 8);
final Map<String, AnalyzingSuggestHolder> lookupMap = new HashMap<>();
input.seek(metaPointerPosition);
long metaPointer = input.readLong();
input.seek(metaPointer);
int numFields = input.readVInt();
Map<Long, String> meta = new TreeMap<>();
for (int i = 0; i < numFields; i++) {
String name = input.readString();
long offset = input.readVLong();
meta.put(offset, name);
}
for (Map.Entry<Long, String> entry : meta.entrySet()) {
input.seek(entry.getKey());
FST<Pair<Long, BytesRef>> fst = new FST<>(input, new PairOutputs<>(
PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
int maxAnalyzedPathsForOneInput = input.readVInt();
int maxSurfaceFormsPerAnalyzedForm = input.readVInt();
int maxGraphExpansions = input.readInt();
int options = input.readVInt();
boolean preserveSep = (options & SERIALIZE_PRESERVE_SEPARATORS) != 0;
boolean hasPayloads = (options & SERIALIZE_HAS_PAYLOADS) != 0;
boolean preservePositionIncrements = (options & SERIALIZE_PRESERVE_POSITION_INCREMENTS) != 0;
// first version did not include these three fields, so fall back to old default (before the analyzingsuggester
// was updated in Lucene, so we cannot use the suggester defaults)
int sepLabel, payloadSep, endByte, holeCharacter;
switch (version) {
case CODEC_VERSION_START:
sepLabel = 0xFF;
payloadSep = '\u001f';
endByte = 0x0;
holeCharacter = '\u001E';
break;
default:
sepLabel = input.readVInt();
endByte = input.readVInt();
payloadSep = input.readVInt();
holeCharacter = input.readVInt();
}
AnalyzingSuggestHolder holder = new AnalyzingSuggestHolder(preserveSep, preservePositionIncrements,
maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions, hasPayloads, maxAnalyzedPathsForOneInput,
fst, sepLabel, payloadSep, endByte, holeCharacter);
sizeInBytes += fst.ramBytesUsed();
lookupMap.put(entry.getValue(), holder);
}
final long ramBytesUsed = sizeInBytes;
return new LookupFactory() {
@Override
public Lookup getLookup(CompletionFieldMapper2x.CompletionFieldType fieldType, CompletionSuggestionContext suggestionContext) {
AnalyzingSuggestHolder analyzingSuggestHolder = lookupMap.get(fieldType.name());
if (analyzingSuggestHolder == null) {
return null;
}
int flags = analyzingSuggestHolder.getPreserveSeparator() ? XAnalyzingSuggester.PRESERVE_SEP : 0;
final XAnalyzingSuggester suggester;
final Automaton queryPrefix = fieldType.requiresContext() ?
ContextQuery.toAutomaton(analyzingSuggestHolder.getPreserveSeparator(), suggestionContext.getContextQueries()) : null;
final FuzzyOptions fuzzyOptions = suggestionContext.getFuzzyOptions();
if (fuzzyOptions != null) {
suggester = new XFuzzySuggester(fieldType.indexAnalyzer(), queryPrefix, fieldType.searchAnalyzer(), flags,
analyzingSuggestHolder.maxSurfaceFormsPerAnalyzedForm, analyzingSuggestHolder.maxGraphExpansions,
fuzzyOptions.getEditDistance(), fuzzyOptions.isTranspositions(),
fuzzyOptions.getFuzzyPrefixLength(), fuzzyOptions.getFuzzyMinLength(), fuzzyOptions.isUnicodeAware(),
analyzingSuggestHolder.fst, analyzingSuggestHolder.hasPayloads,
analyzingSuggestHolder.maxAnalyzedPathsForOneInput, analyzingSuggestHolder.sepLabel,
analyzingSuggestHolder.payloadSep, analyzingSuggestHolder.endByte,
analyzingSuggestHolder.holeCharacter);
} else {
suggester = new XAnalyzingSuggester(fieldType.indexAnalyzer(), queryPrefix, fieldType.searchAnalyzer(), flags,
analyzingSuggestHolder.maxSurfaceFormsPerAnalyzedForm, analyzingSuggestHolder.maxGraphExpansions,
analyzingSuggestHolder.preservePositionIncrements, analyzingSuggestHolder.fst, analyzingSuggestHolder.hasPayloads,
analyzingSuggestHolder.maxAnalyzedPathsForOneInput, analyzingSuggestHolder.sepLabel,
analyzingSuggestHolder.payloadSep, analyzingSuggestHolder.endByte, analyzingSuggestHolder.holeCharacter);
}
return suggester;
}
@Override
public CompletionStats stats(String... fields) {
long sizeInBytes = 0;
ObjectLongHashMap<String> completionFields = null;
if (fields != null && fields.length > 0) {
completionFields = new ObjectLongHashMap<>(fields.length);
}
for (Map.Entry<String, AnalyzingSuggestHolder> entry : lookupMap.entrySet()) {
sizeInBytes += entry.getValue().fst.ramBytesUsed();
if (fields == null || fields.length == 0) {
continue;
}
if (Regex.simpleMatch(fields, entry.getKey())) {
long fstSize = entry.getValue().fst.ramBytesUsed();
completionFields.addTo(entry.getKey(), fstSize);
}
}
return new CompletionStats(sizeInBytes, completionFields);
}
@Override
AnalyzingSuggestHolder getAnalyzingSuggestHolder(MappedFieldType fieldType) {
return lookupMap.get(fieldType.name());
}
@Override
public long ramBytesUsed() {
return ramBytesUsed;
}
@Override
public Collection<Accountable> getChildResources() {
return Accountables.namedAccountables("field", lookupMap);
}
};
}
static class AnalyzingSuggestHolder implements Accountable {
final boolean preserveSep;
final boolean preservePositionIncrements;
final int maxSurfaceFormsPerAnalyzedForm;
final int maxGraphExpansions;
final boolean hasPayloads;
final int maxAnalyzedPathsForOneInput;
final FST<Pair<Long, BytesRef>> fst;
final int sepLabel;
final int payloadSep;
final int endByte;
final int holeCharacter;
public AnalyzingSuggestHolder(boolean preserveSep, boolean preservePositionIncrements,
int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions,
boolean hasPayloads, int maxAnalyzedPathsForOneInput,
FST<Pair<Long, BytesRef>> fst, int sepLabel, int payloadSep,
int endByte, int holeCharacter) {
this.preserveSep = preserveSep;
this.preservePositionIncrements = preservePositionIncrements;
this.maxSurfaceFormsPerAnalyzedForm = maxSurfaceFormsPerAnalyzedForm;
this.maxGraphExpansions = maxGraphExpansions;
this.hasPayloads = hasPayloads;
this.maxAnalyzedPathsForOneInput = maxAnalyzedPathsForOneInput;
this.fst = fst;
this.sepLabel = sepLabel;
this.payloadSep = payloadSep;
this.endByte = endByte;
this.holeCharacter = holeCharacter;
}
public boolean getPreserveSeparator() {
return preserveSep;
}
public boolean getPreservePositionIncrements() {
return preservePositionIncrements;
}
public boolean hasPayloads() {
return hasPayloads;
}
@Override
public long ramBytesUsed() {
if (fst != null) {
return fst.ramBytesUsed();
} else {
return 0;
}
}
@Override
public Collection<Accountable> getChildResources() {
if (fst != null) {
return Collections.singleton(Accountables.namedAccountable("fst", fst));
} else {
return Collections.emptyList();
}
}
}
@Override
public Set<IntsRef> toFiniteStrings(TokenStream stream) throws IOException {
return prototype.toFiniteStrings(stream);
}
}

View File

@ -1,360 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FilterLeafReader.FilterTerms;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.search.suggest.Lookup;
import org.apache.lucene.store.IOContext.Context;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.Version;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.index.mapper.CompletionFieldMapper2x;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionContext;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
* This {@link PostingsFormat} is basically a T-Sink for a default postings
* format that is used to store postings on disk fitting the lucene APIs and
* builds a suggest FST as an auxiliary data structure next to the actual
* postings format. It uses the delegate postings format for simplicity to
* handle all the merge operations. The auxiliary suggest FST data structure is
* only loaded if a FieldsProducer is requested for reading, for merging it uses
* the low memory delegate postings format.
*/
public class Completion090PostingsFormat extends PostingsFormat {
public static final String CODEC_NAME = "completion090";
public static final int SUGGEST_CODEC_VERSION = 1;
public static final int SUGGEST_VERSION_CURRENT = SUGGEST_CODEC_VERSION;
public static final String EXTENSION = "cmp";
private static final Logger logger = Loggers.getLogger(Completion090PostingsFormat.class);
private PostingsFormat delegatePostingsFormat;
private static final Map<String, CompletionLookupProvider> providers;
private CompletionLookupProvider writeProvider;
static {
final CompletionLookupProvider provider = new AnalyzingCompletionLookupProvider(true, true, false);
providers = Collections.singletonMap(provider.getName(), provider);
}
public Completion090PostingsFormat(PostingsFormat delegatePostingsFormat, CompletionLookupProvider provider) {
super(CODEC_NAME);
this.delegatePostingsFormat = delegatePostingsFormat;
this.writeProvider = provider;
assert delegatePostingsFormat != null && writeProvider != null;
}
/*
* Used only by core Lucene at read-time via Service Provider instantiation
* do not use at Write-time in application code.
*/
public Completion090PostingsFormat() {
super(CODEC_NAME);
}
@Override
public CompletionFieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
if (delegatePostingsFormat == null) {
throw new UnsupportedOperationException("Error - " + getClass().getName()
+ " has been constructed without a choice of PostingsFormat");
}
assert writeProvider != null;
return new CompletionFieldsConsumer(state);
}
@Override
public CompletionFieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
return new CompletionFieldsProducer(state);
}
private class CompletionFieldsConsumer extends FieldsConsumer {
private FieldsConsumer delegatesFieldsConsumer;
private FieldsConsumer suggestFieldsConsumer;
public CompletionFieldsConsumer(SegmentWriteState state) throws IOException {
this.delegatesFieldsConsumer = delegatePostingsFormat.fieldsConsumer(state);
String suggestFSTFile = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
IndexOutput output = null;
boolean success = false;
try {
output = state.directory.createOutput(suggestFSTFile, state.context);
CodecUtil.writeIndexHeader(output, CODEC_NAME, SUGGEST_VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
/*
* we write the delegate postings format name so we can load it
* without getting an instance in the ctor
*/
output.writeString(delegatePostingsFormat.getName());
output.writeString(writeProvider.getName());
this.suggestFieldsConsumer = writeProvider.consumer(output);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(output);
}
}
}
@Override
public void write(Fields fields) throws IOException {
delegatesFieldsConsumer.write(fields);
suggestFieldsConsumer.write(fields);
}
@Override
public void close() throws IOException {
IOUtils.close(delegatesFieldsConsumer, suggestFieldsConsumer);
}
}
private static class CompletionFieldsProducer extends FieldsProducer {
// TODO make this class lazyload all the things in order to take advantage of the new merge instance API
// today we just load everything up-front
private final FieldsProducer delegateProducer;
private final LookupFactory lookupFactory;
private final int version;
public CompletionFieldsProducer(SegmentReadState state) throws IOException {
String suggestFSTFile = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
IndexInput input = state.directory.openInput(suggestFSTFile, state.context);
if (state.segmentInfo.getVersion().onOrAfter(Version.LUCENE_6_2_0)) {
// Lucene 6.2.0+ requires all index files to use index header, but prior to that we used an ordinary codec header:
version = CodecUtil.checkIndexHeader(input, CODEC_NAME, SUGGEST_CODEC_VERSION, SUGGEST_VERSION_CURRENT,
state.segmentInfo.getId(), state.segmentSuffix);
} else {
version = CodecUtil.checkHeader(input, CODEC_NAME, SUGGEST_CODEC_VERSION, SUGGEST_VERSION_CURRENT);
}
FieldsProducer delegateProducer = null;
boolean success = false;
try {
PostingsFormat delegatePostingsFormat = PostingsFormat.forName(input.readString());
String providerName = input.readString();
CompletionLookupProvider completionLookupProvider = providers.get(providerName);
if (completionLookupProvider == null) {
throw new IllegalStateException("no provider with name [" + providerName + "] registered");
}
// TODO: we could clone the ReadState and make it always forward IOContext.MERGE to prevent unecessary heap usage?
delegateProducer = delegatePostingsFormat.fieldsProducer(state);
/*
* If we are merging we don't load the FSTs at all such that we
* don't consume so much memory during merge
*/
if (state.context.context != Context.MERGE) {
// TODO: maybe we can do this in a fully lazy fashion based on some configuration
// eventually we should have some kind of curciut breaker that prevents us from going OOM here
// with some configuration
this.lookupFactory = completionLookupProvider.load(input);
} else {
this.lookupFactory = null;
}
this.delegateProducer = delegateProducer;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(delegateProducer, input);
} else {
IOUtils.close(input);
}
}
}
@Override
public void close() throws IOException {
IOUtils.close(delegateProducer);
}
@Override
public Iterator<String> iterator() {
return delegateProducer.iterator();
}
@Override
public Terms terms(String field) throws IOException {
final Terms terms = delegateProducer.terms(field);
if (terms == null || lookupFactory == null) {
return terms;
}
return new CompletionTerms(terms, lookupFactory);
}
@Override
public int size() {
return delegateProducer.size();
}
@Override
public long ramBytesUsed() {
return (lookupFactory == null ? 0 : lookupFactory.ramBytesUsed()) + delegateProducer.ramBytesUsed();
}
@Override
public Collection<Accountable> getChildResources() {
List<Accountable> resources = new ArrayList<>();
if (lookupFactory != null) {
resources.add(Accountables.namedAccountable("lookup", lookupFactory));
}
resources.add(Accountables.namedAccountable("delegate", delegateProducer));
return Collections.unmodifiableList(resources);
}
@Override
public void checkIntegrity() throws IOException {
delegateProducer.checkIntegrity();
}
@Override
public FieldsProducer getMergeInstance() throws IOException {
return delegateProducer.getMergeInstance();
}
}
public static final class CompletionTerms extends FilterTerms {
private final LookupFactory lookup;
public CompletionTerms(Terms delegate, LookupFactory lookup) {
super(delegate);
this.lookup = lookup;
}
public Lookup getLookup(CompletionFieldMapper2x.CompletionFieldType mapper, CompletionSuggestionContext suggestionContext) {
return lookup.getLookup(mapper, suggestionContext);
}
public CompletionStats stats(String ... fields) {
return lookup.stats(fields);
}
}
public abstract static class CompletionLookupProvider implements PayloadProcessor, CompletionTokenStream.ToFiniteStrings {
public static final char UNIT_SEPARATOR = '\u001f';
public abstract FieldsConsumer consumer(IndexOutput output) throws IOException;
public abstract String getName();
public abstract LookupFactory load(IndexInput input) throws IOException;
@Override
public BytesRef buildPayload(BytesRef surfaceForm, long weight, BytesRef payload) throws IOException {
if (weight < -1 || weight > Integer.MAX_VALUE) {
throw new IllegalArgumentException("weight must be >= -1 && <= Integer.MAX_VALUE");
}
for (int i = 0; i < surfaceForm.length; i++) {
if (surfaceForm.bytes[i] == UNIT_SEPARATOR) {
throw new IllegalArgumentException(
"surface form cannot contain unit separator character U+001F; this character is reserved");
}
}
ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
OutputStreamDataOutput output = new OutputStreamDataOutput(byteArrayOutputStream);
output.writeVLong(weight + 1);
output.writeVInt(surfaceForm.length);
output.writeBytes(surfaceForm.bytes, surfaceForm.offset, surfaceForm.length);
output.writeVInt(payload.length);
output.writeBytes(payload.bytes, 0, payload.length);
output.close();
return new BytesRef(byteArrayOutputStream.toByteArray());
}
@Override
public void parsePayload(BytesRef payload, SuggestPayload ref) throws IOException {
ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(payload.bytes, payload.offset, payload.length);
InputStreamDataInput input = new InputStreamDataInput(byteArrayInputStream);
ref.weight = input.readVLong() - 1;
int len = input.readVInt();
ref.surfaceForm.grow(len);
ref.surfaceForm.setLength(len);
input.readBytes(ref.surfaceForm.bytes(), 0, ref.surfaceForm.length());
len = input.readVInt();
ref.payload.grow(len);
ref.payload.setLength(len);
input.readBytes(ref.payload.bytes(), 0, ref.payload.length());
input.close();
}
}
/**
* Returns total in-heap bytes used by all suggesters. This method has CPU cost <code>O(numIndexedFields)</code>.
*
* @param fieldNamePatterns if non-null, any completion field name matching any of these patterns will break out its in-heap bytes
* separately in the returned {@link CompletionStats}
*/
public CompletionStats completionStats(IndexReader indexReader, String ... fieldNamePatterns) {
CompletionStats completionStats = new CompletionStats();
for (LeafReaderContext atomicReaderContext : indexReader.leaves()) {
LeafReader atomicReader = atomicReaderContext.reader();
try {
Fields fields = atomicReader.fields();
for (String fieldName : fields) {
Terms terms = fields.terms(fieldName);
if (terms instanceof CompletionTerms) {
CompletionTerms completionTerms = (CompletionTerms) terms;
completionStats.add(completionTerms.stats(fieldNamePatterns));
}
}
} catch (IOException ioe) {
logger.error("Could not get completion stats", ioe);
}
}
return completionStats;
}
public abstract static class LookupFactory implements Accountable {
public abstract Lookup getLookup(CompletionFieldMapper2x.CompletionFieldType fieldType,
CompletionSuggestionContext suggestionContext);
public abstract CompletionStats stats(String ... fields);
abstract AnalyzingCompletionLookupProvider.AnalyzingSuggestHolder getAnalyzingSuggestHolder(MappedFieldType fieldType);
}
}

View File

@ -1,141 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.search.suggest.Suggest;
import java.io.IOException;
import java.util.Map;
public class CompletionSuggestion extends Suggest.Suggestion<CompletionSuggestion.Entry> {
public static final int TYPE = 2;
public CompletionSuggestion() {
}
public CompletionSuggestion(String name, int size) {
super(name, size);
}
@Override
public int getType() {
return TYPE;
}
@Override
protected Entry newEntry() {
return new Entry();
}
public static class Entry extends org.elasticsearch.search.suggest.Suggest.Suggestion.Entry<CompletionSuggestion.Entry.Option> {
public Entry(Text text, int offset, int length) {
super(text, offset, length);
}
protected Entry() {
super();
}
@Override
protected Option newOption() {
return new Option();
}
public static class Option extends org.elasticsearch.search.suggest.Suggest.Suggestion.Entry.Option {
private BytesReference payload;
public Option(Text text, float score, BytesReference payload) {
super(text, score);
this.payload = payload;
}
protected Option() {
super();
}
public void setPayload(BytesReference payload) {
this.payload = payload;
}
public BytesReference getPayload() {
return payload;
}
public String getPayloadAsString() {
return payload.utf8ToString();
}
public long getPayloadAsLong() {
return Long.parseLong(payload.utf8ToString());
}
public double getPayloadAsDouble() {
return Double.parseDouble(payload.utf8ToString());
}
public Map<String, Object> getPayloadAsMap() {
return XContentHelper.convertToMap(payload, false).v2();
}
@Override
public void setScore(float score) {
super.setScore(score);
}
@Override
protected XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException {
super.innerToXContent(builder, params);
if (payload != null && payload.length() > 0) {
XContentType contentType = XContentFactory.xContentType(payload);
if (contentType == null) {
// must be a string or number
builder.field("payload", payload.utf8ToString());
} else {
builder.rawField("payload", payload);
}
}
return builder;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
payload = in.readBytesReference();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBytesReference(payload);
}
}
}
}

View File

@ -1,173 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeReflector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.CharsRefBuilder;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.Util;
import java.io.IOException;
import java.util.Iterator;
import java.util.Set;
public final class CompletionTokenStream extends TokenStream {
private final PayloadAttribute payloadAttr = addAttribute(PayloadAttribute.class);
private final PositionIncrementAttribute posAttr = addAttribute(PositionIncrementAttribute.class);
private final ByteTermAttribute bytesAtt = addAttribute(ByteTermAttribute.class);;
private final TokenStream input;
private BytesRef payload;
private Iterator<IntsRef> finiteStrings;
private ToFiniteStrings toFiniteStrings;
private int posInc = -1;
private static final int MAX_PATHS = 256;
private CharTermAttribute charTermAttribute;
public CompletionTokenStream(TokenStream input, BytesRef payload, ToFiniteStrings toFiniteStrings) {
// Don't call the super(input) ctor - this is a true delegate and has a new attribute source since we consume
// the input stream entirely in toFiniteStrings(input)
this.input = input;
this.payload = payload;
this.toFiniteStrings = toFiniteStrings;
}
@Override
public boolean incrementToken() throws IOException {
clearAttributes();
if (finiteStrings == null) {
Set<IntsRef> strings = toFiniteStrings.toFiniteStrings(input);
if (strings.size() > MAX_PATHS) {
throw new IllegalArgumentException("TokenStream expanded to " + strings.size() + " finite strings. Only <= " + MAX_PATHS
+ " finite strings are supported");
}
posInc = strings.size();
finiteStrings = strings.iterator();
}
if (finiteStrings.hasNext()) {
posAttr.setPositionIncrement(posInc);
/*
* this posInc encodes the number of paths that this surface form
* produced. Multi Fields have the same surface form and therefore sum up
*/
posInc = 0;
Util.toBytesRef(finiteStrings.next(), bytesAtt.builder()); // now we have UTF-8
if (charTermAttribute != null) {
charTermAttribute.setLength(0);
charTermAttribute.append(bytesAtt.toUTF16());
}
if (payload != null) {
payloadAttr.setPayload(this.payload);
}
return true;
}
return false;
}
@Override
public void end() throws IOException {
super.end();
if (posInc == -1) {
input.end();
}
}
@Override
public void close() throws IOException {
input.close();
}
public interface ToFiniteStrings {
Set<IntsRef> toFiniteStrings(TokenStream stream) throws IOException;
}
@Override
public void reset() throws IOException {
super.reset();
if (hasAttribute(CharTermAttribute.class)) {
// we only create this if we really need it to safe the UTF-8 to UTF-16 conversion
charTermAttribute = getAttribute(CharTermAttribute.class);
}
finiteStrings = null;
posInc = -1;
}
public interface ByteTermAttribute extends TermToBytesRefAttribute {
// marker interface
/**
* Return the builder from which the term is derived.
*/
BytesRefBuilder builder();
CharSequence toUTF16();
}
public static final class ByteTermAttributeImpl extends AttributeImpl implements ByteTermAttribute, TermToBytesRefAttribute {
private final BytesRefBuilder bytes = new BytesRefBuilder();
private CharsRefBuilder charsRef;
@Override
public BytesRefBuilder builder() {
return bytes;
}
@Override
public BytesRef getBytesRef() {
return bytes.get();
}
@Override
public void clear() {
bytes.clear();
}
@Override
public void reflectWith(AttributeReflector reflector) {
}
@Override
public void copyTo(AttributeImpl target) {
ByteTermAttributeImpl other = (ByteTermAttributeImpl) target;
other.bytes.copyBytes(bytes);
}
@Override
public CharSequence toUTF16() {
if (charsRef == null) {
charsRef = new CharsRefBuilder();
}
charsRef.copyUTF8Bytes(getBytesRef());
return charsRef.get();
}
}
}

View File

@ -1,38 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import java.io.IOException;
interface PayloadProcessor {
BytesRef buildPayload(BytesRef surfaceForm, long weight, BytesRef payload) throws IOException;
void parsePayload(BytesRef payload, SuggestPayload ref) throws IOException;
static class SuggestPayload {
final BytesRefBuilder payload = new BytesRefBuilder();
long weight = 0;
final BytesRefBuilder surfaceForm = new BytesRefBuilder();
}
}

View File

@ -1,374 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x.context;
import org.apache.lucene.analysis.PrefixAnalyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.util.automaton.Automata;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.Operations;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* The {@link CategoryContextMapping} is used to define a {@link ContextMapping} that
* references a field within a document. The value of the field in turn will be
* used to setup the suggestions made by the completion suggester.
*/
public class CategoryContextMapping extends ContextMapping {
protected static final String TYPE = "category";
private static final String FIELD_FIELDNAME = "path";
private static final String DEFAULT_FIELDNAME = "_type";
private static final Iterable<String> EMPTY_VALUES = Collections.emptyList();
private final String fieldName;
private final Iterable<String> defaultValues;
private final FieldConfig defaultConfig;
/**
* Create a new {@link CategoryContextMapping} with the default field
* <code>[_type]</code>
*/
public CategoryContextMapping(String name) {
this(name, DEFAULT_FIELDNAME, EMPTY_VALUES);
}
/**
* Create a new {@link CategoryContextMapping} with the default field
* <code>[_type]</code>
*/
public CategoryContextMapping(String name, String fieldName) {
this(name, fieldName, EMPTY_VALUES);
}
/**
* Create a new {@link CategoryContextMapping} with the default field
* <code>[_type]</code>
*/
public CategoryContextMapping(String name, Iterable<String> defaultValues) {
this(name, DEFAULT_FIELDNAME, defaultValues);
}
/**
* Create a new {@link CategoryContextMapping} with the default field
* <code>[_type]</code>
*/
public CategoryContextMapping(String name, String fieldName, Iterable<String> defaultValues) {
super(TYPE, name);
this.fieldName = fieldName;
this.defaultValues = defaultValues;
this.defaultConfig = new FieldConfig(fieldName, defaultValues, null);
}
/**
* Name of the field used by this {@link CategoryContextMapping}
*/
public String getFieldName() {
return fieldName;
}
public Iterable<? extends CharSequence> getDefaultValues() {
return defaultValues;
}
@Override
public FieldConfig defaultConfig() {
return defaultConfig;
}
/**
* Load the specification of a {@link CategoryContextMapping}
*
* @param name
* name of the field to use. If <code>null</code> default field
* will be used
* @return new {@link CategoryContextMapping}
*/
protected static CategoryContextMapping load(String name, Map<String, Object> config) throws ElasticsearchParseException {
CategoryContextMapping.Builder mapping = new CategoryContextMapping.Builder(name);
Object fieldName = config.get(FIELD_FIELDNAME);
Object defaultValues = config.get(FIELD_MISSING);
if (fieldName != null) {
mapping.fieldName(fieldName.toString());
config.remove(FIELD_FIELDNAME);
}
if (defaultValues != null) {
if (defaultValues instanceof Iterable) {
for (Object value : (Iterable) defaultValues) {
mapping.addDefaultValue(value.toString());
}
} else {
mapping.addDefaultValue(defaultValues.toString());
}
config.remove(FIELD_MISSING);
}
return mapping.build();
}
@Override
protected XContentBuilder toInnerXContent(XContentBuilder builder, Params params) throws IOException {
if (fieldName != null) {
builder.field(FIELD_FIELDNAME, fieldName);
}
builder.startArray(FIELD_MISSING);
for (CharSequence value : defaultValues) {
builder.value(value);
}
builder.endArray();
return builder;
}
@Override
public ContextConfig parseContext(ParseContext parseContext, XContentParser parser) throws IOException, ElasticsearchParseException {
Token token = parser.currentToken();
if (token == Token.VALUE_NULL) {
return new FieldConfig(fieldName, defaultValues, null);
} else if (token == Token.VALUE_STRING) {
return new FieldConfig(fieldName, null, Collections.singleton(parser.text()));
} else if (token == Token.VALUE_NUMBER) {
return new FieldConfig(fieldName, null, Collections.singleton(parser.text()));
} else if (token == Token.VALUE_BOOLEAN) {
return new FieldConfig(fieldName, null, Collections.singleton(parser.text()));
} else if (token == Token.START_ARRAY) {
ArrayList<String> values = new ArrayList<>();
while((token = parser.nextToken()) != Token.END_ARRAY) {
values.add(parser.text());
}
if(values.isEmpty()) {
throw new ElasticsearchParseException("FieldConfig must contain a least one category");
}
return new FieldConfig(fieldName, null, values);
} else {
throw new ElasticsearchParseException("FieldConfig must be either [null], a string or a list of strings");
}
}
@Override
public FieldQuery parseQuery(String name, XContentParser parser) throws IOException, ElasticsearchParseException {
Iterable<? extends CharSequence> values;
Token token = parser.currentToken();
if (token == Token.START_ARRAY) {
ArrayList<String> list = new ArrayList<>();
while ((token = parser.nextToken()) != Token.END_ARRAY) {
list.add(parser.text());
}
values = list;
} else if (token == Token.VALUE_NULL) {
values = defaultValues;
} else {
values = Collections.singleton(parser.text());
}
return new FieldQuery(name, values);
}
public static FieldQuery query(String name, CharSequence... fieldvalues) {
return query(name, Arrays.asList(fieldvalues));
}
public static FieldQuery query(String name, Iterable<? extends CharSequence> fieldvalues) {
return new FieldQuery(name, fieldvalues);
}
@Override
public boolean equals(Object obj) {
if (obj instanceof CategoryContextMapping) {
CategoryContextMapping other = (CategoryContextMapping) obj;
if (this.fieldName.equals(other.fieldName)) {
return Objects.deepEquals(this.defaultValues, other.defaultValues);
}
}
return false;
}
@Override
public int hashCode() {
int hashCode = fieldName.hashCode();
for (CharSequence seq : defaultValues) {
hashCode = 31 * hashCode + seq.hashCode();
}
return hashCode;
}
private static class FieldConfig extends ContextConfig {
private final String fieldname;
private final Iterable<String> defaultValues;
private final Iterable<String> values;
public FieldConfig(String fieldname, Iterable<String> defaultValues, Iterable<String> values) {
this.fieldname = fieldname;
this.defaultValues = defaultValues;
this.values = values;
}
@Override
protected TokenStream wrapTokenStream(Document doc, TokenStream stream) {
if (values != null) {
return new PrefixAnalyzer.PrefixTokenFilter(stream, ContextMapping.SEPARATOR, values);
// if fieldname is default, BUT our default values are set, we take that one
} else if ((doc.getFields(fieldname).length == 0
|| fieldname.equals(DEFAULT_FIELDNAME)) && defaultValues.iterator().hasNext()) {
return new PrefixAnalyzer.PrefixTokenFilter(stream, ContextMapping.SEPARATOR, defaultValues);
} else {
IndexableField[] fields = doc.getFields(fieldname);
ArrayList<CharSequence> values = new ArrayList<>(fields.length);
for (int i = 0; i < fields.length; i++) {
values.add(fields[i].stringValue());
}
return new PrefixAnalyzer.PrefixTokenFilter(stream, ContextMapping.SEPARATOR, values);
}
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("FieldConfig(" + fieldname + " = [");
if (this.values != null && this.values.iterator().hasNext()) {
final Iterator<String> valuesIterator = this.values.iterator();
sb.append("(");
while (valuesIterator.hasNext()) {
sb.append(valuesIterator.next());
if (valuesIterator.hasNext()) {
sb.append(", ");
}
}
sb.append(")");
}
if (this.defaultValues != null && this.defaultValues.iterator().hasNext()) {
final Iterator<String> defaultValuesIterator = this.defaultValues.iterator();
sb.append(" default(");
while (defaultValuesIterator.hasNext()) {
sb.append(defaultValuesIterator.next());
if (defaultValuesIterator.hasNext()) {
sb.append(", ");
}
}
sb.append(")");
}
return sb.append("])").toString();
}
}
private static class FieldQuery extends ContextQuery {
private final Iterable<? extends CharSequence> values;
public FieldQuery(String name, Iterable<? extends CharSequence> values) {
super(name);
this.values = values;
}
@Override
public Automaton toAutomaton() {
List<Automaton> automatons = new ArrayList<>();
for (CharSequence value : values) {
automatons.add(Automata.makeString(value.toString()));
}
return Operations.union(automatons);
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startArray(name);
for (CharSequence value : values) {
builder.value(value);
}
builder.endArray();
return builder;
}
}
public static class Builder extends ContextBuilder<CategoryContextMapping> {
private String fieldname;
private List<String> defaultValues = new ArrayList<>();
public Builder(String name) {
this(name, DEFAULT_FIELDNAME);
}
public Builder(String name, String fieldname) {
super(name);
this.fieldname = fieldname;
}
/**
* Set the name of the field to use
*/
public Builder fieldName(String fieldname) {
this.fieldname = fieldname;
return this;
}
/**
* Add value to the default values of the mapping
*/
public Builder addDefaultValue(String defaultValue) {
this.defaultValues.add(defaultValue);
return this;
}
/**
* Add set of default values to the mapping
*/
public Builder addDefaultValues(String... defaultValues) {
Collections.addAll(this.defaultValues, defaultValues);
return this;
}
/**
* Add set of default values to the mapping
*/
public Builder addDefaultValues(Iterable<String> defaultValues) {
for (String defaultValue : defaultValues) {
this.defaultValues.add(defaultValue);
}
return this;
}
@Override
public CategoryContextMapping build() {
return new CategoryContextMapping(name, fieldname, defaultValues);
}
}
}

View File

@ -1,136 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x.context;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.MapperParsingException;
import java.util.Map;
import java.util.Map.Entry;
import java.util.SortedMap;
import java.util.TreeMap;
public abstract class ContextBuilder<E extends ContextMapping> {
protected String name;
public ContextBuilder(String name) {
this.name = name;
}
public abstract E build();
/**
* Create a new {@link GeolocationContextMapping}
*/
public static GeolocationContextMapping.Builder location(String name) {
return new GeolocationContextMapping.Builder(name);
}
/**
* Create a new {@link GeolocationContextMapping} with given precision and
* neighborhood usage
*
* @param precision geohash length
* @param neighbors use neighbor cells
*/
public static GeolocationContextMapping.Builder location(String name, int precision, boolean neighbors) {
return new GeolocationContextMapping.Builder(name, neighbors, precision);
}
/**
* Create a new {@link CategoryContextMapping}
*/
public static CategoryContextMapping.Builder category(String name) {
return new CategoryContextMapping.Builder(name, null);
}
/**
* Create a new {@link CategoryContextMapping} with default category
*
* @param defaultCategory category to use, if it is not provided
*/
public static CategoryContextMapping.Builder category(String name, String defaultCategory) {
return new CategoryContextMapping.Builder(name, null).addDefaultValue(defaultCategory);
}
/**
* Create a new {@link CategoryContextMapping}
*
* @param fieldname
* name of the field to use
*/
public static CategoryContextMapping.Builder reference(String name, String fieldname) {
return new CategoryContextMapping.Builder(name, fieldname);
}
/**
* Create a new {@link CategoryContextMapping}
*
* @param fieldname name of the field to use
* @param defaultValues values to use, if the document not provides
* a field with the given name
*/
public static CategoryContextMapping.Builder reference(String name, String fieldname, Iterable<String> defaultValues) {
return new CategoryContextMapping.Builder(name, fieldname).addDefaultValues(defaultValues);
}
public static SortedMap<String, ContextMapping> loadMappings(Object configuration, Version indexVersionCreated)
throws ElasticsearchParseException {
if (configuration instanceof Map) {
Map<String, Object> configurations = (Map<String, Object>)configuration;
SortedMap<String, ContextMapping> mappings = new TreeMap<>();
for (Entry<String,Object> config : configurations.entrySet()) {
String name = config.getKey();
mappings.put(name, loadMapping(name, (Map<String, Object>) config.getValue(), indexVersionCreated));
}
return mappings;
} else if (configuration == null) {
return ContextMapping.EMPTY_MAPPING;
} else {
throw new ElasticsearchParseException("no valid context configuration");
}
}
protected static ContextMapping loadMapping(String name, Map<String, Object> config, Version indexVersionCreated)
throws ElasticsearchParseException {
final Object argType = config.get(ContextMapping.FIELD_TYPE);
if (argType == null) {
throw new ElasticsearchParseException("missing [{}] in context mapping", ContextMapping.FIELD_TYPE);
}
final String type = argType.toString();
ContextMapping contextMapping;
if (GeolocationContextMapping.TYPE.equals(type)) {
contextMapping = GeolocationContextMapping.load(name, config);
} else if (CategoryContextMapping.TYPE.equals(type)) {
contextMapping = CategoryContextMapping.load(name, config);
} else {
throw new ElasticsearchParseException("unknown context type [{}]", type);
}
config.remove(ContextMapping.FIELD_TYPE);
DocumentMapperParser.checkNoRemainingFields(name, config, indexVersionCreated);
return contextMapping;
}
}

View File

@ -1,319 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x.context;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.search.suggest.analyzing.XAnalyzingSuggester;
import org.apache.lucene.util.automaton.Automata;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.Operations;
import org.apache.lucene.util.fst.FST;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.SortedMap;
/**
* A {@link ContextMapping} is used t define a context that may used
* in conjunction with a suggester. To define a suggester that depends on a
* specific context derived class of {@link ContextMapping} will be
* used to specify the kind of additional information required in order to make
* suggestions.
*/
public abstract class ContextMapping implements ToXContent {
/** Character used to separate several contexts */
public static final char SEPARATOR = '\u001D';
/** Dummy Context Mapping that should be used if no context is used*/
public static final SortedMap<String, ContextMapping> EMPTY_MAPPING = Collections.emptySortedMap();
/** Dummy Context Config matching the Dummy Mapping by providing an empty context*/
public static final SortedMap<String, ContextConfig> EMPTY_CONFIG = Collections.emptySortedMap();
/** Dummy Context matching the Dummy Mapping by not wrapping a {@link TokenStream} */
public static final Context EMPTY_CONTEXT = new Context(EMPTY_CONFIG, null);
public static final String FIELD_VALUE = "value";
public static final String FIELD_MISSING = "default";
public static final String FIELD_TYPE = "type";
protected final String type; // Type of the Contextmapping
protected final String name;
/**
* Define a new context mapping of a specific type
*
* @param type
* name of the new context mapping
*/
protected ContextMapping(String type, String name) {
super();
this.type = type;
this.name = name;
}
/**
* @return the type name of the context
*/
protected String type() {
return type;
}
/**
* @return the name/id of the context
*/
public String name() {
return name;
}
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(name);
builder.field(FIELD_TYPE, type);
toInnerXContent(builder, params);
builder.endObject();
return builder;
}
/**
* A {@link ContextMapping} combined with the information provided by a document
* form a {@link ContextConfig} which is used to build the underlying FST.
*
* @param parseContext context of parsing phase
* @param parser {@link XContentParser} used to read and setup the configuration
* @return A {@link ContextConfig} related to <b>this</b> mapping
*
*/
public abstract ContextConfig parseContext(ParseContext parseContext, XContentParser parser)
throws IOException, ElasticsearchParseException;
public abstract ContextConfig defaultConfig();
/**
* Parse a query according to the context. Parsing starts at parsers <b>current</b> position
*
* @param name name of the context
* @param parser {@link XContentParser} providing the data of the query
*
* @return {@link ContextQuery} according to this mapping
*
*/
public abstract ContextQuery parseQuery(String name, XContentParser parser) throws IOException, ElasticsearchParseException;
/**
* Since every context mapping is assumed to have a name given by the field name of an context object, this
* method is used to build the value used to serialize the mapping
*
* @param builder builder to append the mapping to
* @param params parameters passed to the builder
*
* @return the builder used
*
*/
protected abstract XContentBuilder toInnerXContent(XContentBuilder builder, Params params) throws IOException;
/**
* Test equality of two mapping
*
* @param thisMappings first mapping
* @param otherMappings second mapping
*
* @return true if both arguments are equal
*/
public static boolean mappingsAreEqual(SortedMap<String, ? extends ContextMapping> thisMappings,
SortedMap<String, ? extends ContextMapping> otherMappings) {
return Objects.equals(thisMappings, otherMappings);
}
@Override
public String toString() {
try {
return toXContent(JsonXContent.contentBuilder(), ToXContent.EMPTY_PARAMS).string();
} catch (IOException e) {
return super.toString();
}
}
/**
* A collection of {@link ContextMapping}s, their {@link ContextConfig}uration and a
* Document form a complete {@link Context}. Since this Object provides all information used
* to setup a suggestion, it can be used to wrap the entire {@link TokenStream} used to build a
* path within the {@link FST}.
*/
public static class Context {
final SortedMap<String, ContextConfig> contexts;
final Document doc;
public Context(SortedMap<String, ContextConfig> contexts, Document doc) {
super();
this.contexts = contexts;
this.doc = doc;
}
/**
* Wrap the {@link TokenStream} according to the provided informations of {@link ContextConfig}
* and a related {@link Document}.
*
* @param tokenStream {@link TokenStream} to wrap
*
* @return wrapped token stream
*/
public TokenStream wrapTokenStream(TokenStream tokenStream) {
for (ContextConfig context : contexts.values()) {
tokenStream = context.wrapTokenStream(doc, tokenStream);
}
return tokenStream;
}
}
/**
* A {@link ContextMapping} combined with the information provided by a document
* form a {@link ContextConfig} which is used to build the underlying {@link FST}. This class hold
* a simple method wrapping a {@link TokenStream} by provided document informations.
*/
public abstract static class ContextConfig {
/**
* Wrap a {@link TokenStream} for building suggestions to use context informations
* provided by a document or a {@link ContextMapping}
*
* @param doc document related to the stream
* @param stream original stream used to build the underlying {@link FST}
*
* @return A new {@link TokenStream} providing additional context information
*/
protected abstract TokenStream wrapTokenStream(Document doc, TokenStream stream);
}
/**
* A {@link ContextQuery} defines the context information for a specific {@link ContextMapping}
* defined within a suggestion request. According to the parameters set in the request and the
* {@link ContextMapping} such a query is used to wrap the {@link TokenStream} of the actual
* suggestion request into a {@link TokenStream} with the context settings
*/
public abstract static class ContextQuery implements ToXContent {
protected final String name;
protected ContextQuery(String name) {
this.name = name;
}
public String name() {
return name;
}
/**
* Create a automaton for a given context query this automaton will be used
* to find the matching paths with the fst
*
* @param preserveSep set an additional char (<code>XAnalyzingSuggester.SEP_LABEL</code>) between each context query
* @param queries list of {@link ContextQuery} defining the lookup context
*
* @return Automaton matching the given Query
*/
public static Automaton toAutomaton(boolean preserveSep, Iterable<ContextQuery> queries) {
Automaton a = Automata.makeEmptyString();
Automaton gap = Automata.makeChar(ContextMapping.SEPARATOR);
if (preserveSep) {
// if separators are preserved the fst contains a SEP_LABEL
// behind each gap. To have a matching automaton, we need to
// include the SEP_LABEL in the query as well
gap = Operations.concatenate(gap, Automata.makeChar(XAnalyzingSuggester.SEP_LABEL));
}
for (ContextQuery query : queries) {
a = Operations.concatenate(Arrays.asList(query.toAutomaton(), gap, a));
}
// TODO: should we limit this? Do any of our ContextQuery impls really create exponential regexps?
// GeoQuery looks safe (union of strings).
return Operations.determinize(a, Integer.MAX_VALUE);
}
/**
* Build a LookUp Automaton for this context.
* @return LookUp Automaton
*/
protected abstract Automaton toAutomaton();
/**
* Parse a set of {@link ContextQuery} according to a given mapping
* @param mappings List of mapping defined y the suggest field
* @param parser parser holding the settings of the queries. The parsers
* current token is assumed hold an array. The number of elements
* in this array must match the number of elements in the mappings.
* @return List of context queries
*
* @throws IOException if something unexpected happened on the underlying stream
* @throws ElasticsearchParseException if the list of queries could not be parsed
*/
public static List<ContextQuery> parseQueries(Map<String, ContextMapping> mappings, XContentParser parser)
throws IOException, ElasticsearchParseException {
Map<String, ContextQuery> querySet = new HashMap<>();
Token token = parser.currentToken();
if(token == Token.START_OBJECT) {
while ((token = parser.nextToken()) != Token.END_OBJECT) {
String name = parser.currentName();
ContextMapping mapping = mappings.get(name);
if (mapping == null) {
throw new ElasticsearchParseException("no mapping defined for [{}]", name);
}
parser.nextToken();
querySet.put(name, mapping.parseQuery(name, parser));
}
}
List<ContextQuery> queries = new ArrayList<>(mappings.size());
for (ContextMapping mapping : mappings.values()) {
queries.add(querySet.get(mapping.name));
}
return queries;
}
@Override
public String toString() {
try {
return toXContent(JsonXContent.contentBuilder(), ToXContent.EMPTY_PARAMS).string();
} catch (IOException e) {
return super.toString();
}
}
}
}

View File

@ -1,750 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.suggest.completion2x.context;
import com.carrotsearch.hppc.IntHashSet;
import org.apache.lucene.analysis.PrefixAnalyzer.PrefixTokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.spatial.geopoint.document.GeoPointField;
import org.apache.lucene.util.automaton.Automata;
import org.apache.lucene.util.automaton.Automaton;
import org.apache.lucene.util.automaton.Operations;
import org.apache.lucene.util.fst.FST;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.geo.GeoHashUtils;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.GeoPointFieldMapper;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.ParseContext.Document;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
/**
* The {@link GeolocationContextMapping} allows to take GeoInfomation into account
* during building suggestions. The mapping itself works with geohashes
* explicitly and is configured by three parameters:
* <ul>
* <li><code>precision</code>: length of the geohash indexed as prefix of the
* completion field</li>
* <li><code>neighbors</code>: Should the neighbor cells of the deepest geohash
* level also be indexed as alternatives to the actual geohash</li>
* <li><code>location</code>: (optional) location assumed if it is not provided</li>
* </ul>
* Internally this mapping wraps the suggestions into a form
* <code>[geohash][suggestion]</code>. If the neighbor option is set the cells
* next to the cell on the deepest geohash level ( <code>precision</code>) will
* be indexed as well. The {@link TokenStream} used to build the {@link FST} for
* suggestion will be wrapped into a {@link PrefixTokenFilter} managing these
* geohases as prefixes.
*/
public class GeolocationContextMapping extends ContextMapping {
public static final String TYPE = "geo";
public static final String FIELD_PRECISION = "precision";
public static final String FIELD_NEIGHBORS = "neighbors";
public static final String FIELD_FIELDNAME = "path";
private final Collection<String> defaultLocations;
private final int[] precision;
private final boolean neighbors;
private final String fieldName;
private final GeoConfig defaultConfig;
/**
* Create a new {@link GeolocationContextMapping} with a given precision
*
* @param precision
* length of the geohashes
* @param neighbors
* should neighbors be indexed
* @param defaultLocations
* location to use, if it is not provided by the document
*/
protected GeolocationContextMapping(String name, int[] precision, boolean neighbors,
Collection<String> defaultLocations, String fieldName) {
super(TYPE, name);
this.precision = precision;
this.neighbors = neighbors;
this.defaultLocations = defaultLocations;
this.fieldName = fieldName;
this.defaultConfig = new GeoConfig(this, defaultLocations);
}
/**
* load a {@link GeolocationContextMapping} by configuration. Such a configuration
* can set the parameters
* <ul>
* <li>precision [<code>String</code>, <code>Double</code>,
* <code>Float</code> or <code>Integer</code>] defines the length of the
* underlying geohash</li>
* <li>defaultLocation [<code>String</code>] defines the location to use if
* it is not provided by the document</li>
* <li>neighbors [<code>Boolean</code>] defines if the last level of the
* geohash should be extended by neighbor cells</li>
* </ul>
*
* @param config
* Configuration for {@link GeolocationContextMapping}
* @return new {@link GeolocationContextMapping} configured by the parameters of
* <code>config</code>
*/
protected static GeolocationContextMapping load(String name, Map<String, Object> config) {
if (!config.containsKey(FIELD_PRECISION)) {
throw new ElasticsearchParseException("field [precision] is missing");
}
final GeolocationContextMapping.Builder builder = new GeolocationContextMapping.Builder(name);
if (config != null) {
final Object configPrecision = config.get(FIELD_PRECISION);
if (configPrecision == null) {
// ignore precision
} else if (configPrecision instanceof Integer) {
builder.precision((Integer) configPrecision);
config.remove(FIELD_PRECISION);
} else if (configPrecision instanceof Long) {
builder.precision((Long) configPrecision);
config.remove(FIELD_PRECISION);
} else if (configPrecision instanceof Double) {
builder.precision((Double) configPrecision);
config.remove(FIELD_PRECISION);
} else if (configPrecision instanceof Float) {
builder.precision((Float) configPrecision);
config.remove(FIELD_PRECISION);
} else if (configPrecision instanceof Iterable) {
for (Object precision : (Iterable)configPrecision) {
if (precision instanceof Integer) {
builder.precision((Integer) precision);
} else if (precision instanceof Long) {
builder.precision((Long) precision);
} else if (precision instanceof Double) {
builder.precision((Double) precision);
} else if (precision instanceof Float) {
builder.precision((Float) precision);
} else {
builder.precision(precision.toString());
}
}
config.remove(FIELD_PRECISION);
} else {
builder.precision(configPrecision.toString());
config.remove(FIELD_PRECISION);
}
final Object configNeighbors = config.get(FIELD_NEIGHBORS);
if (configNeighbors != null) {
builder.neighbors((Boolean) configNeighbors);
config.remove(FIELD_NEIGHBORS);
}
final Object def = config.get(FIELD_MISSING);
if (def != null) {
if (def instanceof Iterable) {
for (Object location : (Iterable)def) {
builder.addDefaultLocation(location.toString());
}
} else if (def instanceof String) {
builder.addDefaultLocation(def.toString());
} else if (def instanceof Map) {
Map<String, Object> latlonMap = (Map<String, Object>) def;
if (!latlonMap.containsKey("lat") || !(latlonMap.get("lat") instanceof Double)) {
throw new ElasticsearchParseException(
"field [{}] map must have field lat and a valid latitude", FIELD_MISSING);
}
if (!latlonMap.containsKey("lon") || !(latlonMap.get("lon") instanceof Double)) {
throw new ElasticsearchParseException(
"field [{}] map must have field lon and a valid longitude", FIELD_MISSING);
}
builder.addDefaultLocation(
Double.valueOf(latlonMap.get("lat").toString()), Double.valueOf(latlonMap.get("lon").toString()));
} else {
throw new ElasticsearchParseException("field [{}] must be of type string or list", FIELD_MISSING);
}
config.remove(FIELD_MISSING);
}
final Object fieldName = config.get(FIELD_FIELDNAME);
if (fieldName != null) {
builder.field(fieldName.toString());
config.remove(FIELD_FIELDNAME);
}
}
return builder.build();
}
@Override
protected XContentBuilder toInnerXContent(XContentBuilder builder, Params params) throws IOException {
builder.array(FIELD_PRECISION, precision);
builder.field(FIELD_NEIGHBORS, neighbors);
if (defaultLocations != null) {
builder.startArray(FIELD_MISSING);
for (String defaultLocation : defaultLocations) {
builder.value(defaultLocation);
}
builder.endArray();
}
if (fieldName != null) {
builder.field(FIELD_FIELDNAME, fieldName);
}
return builder;
}
protected static Collection<String> parseSinglePointOrList(XContentParser parser) throws IOException {
Token token = parser.currentToken();
if(token == Token.START_ARRAY) {
token = parser.nextToken();
// Test if value is a single point in <code>[lon, lat]</code> format
if(token == Token.VALUE_NUMBER) {
double lon = parser.doubleValue();
if(parser.nextToken() == Token.VALUE_NUMBER) {
double lat = parser.doubleValue();
if(parser.nextToken() == Token.END_ARRAY) {
return Collections.singleton(GeoHashUtils.stringEncode(lon, lat));
} else {
throw new ElasticsearchParseException("only two values expected");
}
} else {
throw new ElasticsearchParseException("latitue must be a numeric value");
}
} else {
// otherwise it's a list of locations
ArrayList<String> result = new ArrayList<>();
while (token != Token.END_ARRAY) {
result.add(GeoUtils.parseGeoPoint(parser).geohash());
token = parser.nextToken(); //infinite loop without this line
}
return result;
}
} else {
// or a single location
return Collections.singleton(GeoUtils.parseGeoPoint(parser).geohash());
}
}
@Override
public ContextConfig defaultConfig() {
return defaultConfig;
}
@Override
public ContextConfig parseContext(ParseContext parseContext, XContentParser parser)
throws IOException, ElasticsearchParseException {
if(fieldName != null) {
FieldMapper mapper = parseContext.docMapper().mappers().getMapper(fieldName);
if(!(mapper instanceof GeoPointFieldMapper)) {
throw new ElasticsearchParseException("referenced field must be mapped to geo_point");
}
}
Collection<String> locations;
if(parser.currentToken() == Token.VALUE_NULL) {
locations = null;
} else {
locations = parseSinglePointOrList(parser);
}
return new GeoConfig(this, locations);
}
/**
* Create a new geolocation query from a given GeoPoint
*
* @param point
* query location
* @return new geolocation query
*/
public static GeoQuery query(String name, GeoPoint point) {
return query(name, point.getGeohash());
}
/**
* Create a new geolocation query from a given geocoordinate
*
* @param lat
* latitude of the location
* @param lon
* longitude of the location
* @return new geolocation query
*/
public static GeoQuery query(String name, double lat, double lon, int ... precisions) {
return query(name, GeoHashUtils.stringEncode(lon, lat), precisions);
}
public static GeoQuery query(String name, double lat, double lon, String ... precisions) {
int precisionInts[] = new int[precisions.length];
for (int i = 0 ; i < precisions.length; i++) {
precisionInts[i] = GeoUtils.geoHashLevelsForPrecision(precisions[i]);
}
return query(name, GeoHashUtils.stringEncode(lon, lat), precisionInts);
}
/**
* Create a new geolocation query from a given geohash
*
* @param geohash
* geohash of the location
* @return new geolocation query
*/
public static GeoQuery query(String name, String geohash, int ... precisions) {
return new GeoQuery(name, geohash, precisions);
}
private static int parsePrecision(XContentParser parser) throws IOException, ElasticsearchParseException {
switch (parser.currentToken()) {
case VALUE_STRING:
return GeoUtils.geoHashLevelsForPrecision(parser.text());
case VALUE_NUMBER:
switch (parser.numberType()) {
case INT:
case LONG:
return parser.intValue();
default:
return GeoUtils.geoHashLevelsForPrecision(parser.doubleValue());
}
default:
throw new ElasticsearchParseException("invalid precision value");
}
}
@Override
public GeoQuery parseQuery(String name, XContentParser parser) throws IOException, ElasticsearchParseException {
if (parser.currentToken() == Token.START_OBJECT) {
double lat = Double.NaN;
double lon = Double.NaN;
GeoPoint point = null;
int[] precision = null;
while (parser.nextToken() != Token.END_OBJECT) {
final String fieldName = parser.currentName();
if("lat".equals(fieldName)) {
if(point == null) {
parser.nextToken();
switch (parser.currentToken()) {
case VALUE_NUMBER:
case VALUE_STRING:
lat = parser.doubleValue(true);
break;
default:
throw new ElasticsearchParseException("latitude must be a number");
}
} else {
throw new ElasticsearchParseException("only lat/lon or [{}] is allowed", FIELD_VALUE);
}
} else if ("lon".equals(fieldName)) {
if(point == null) {
parser.nextToken();
switch (parser.currentToken()) {
case VALUE_NUMBER:
case VALUE_STRING:
lon = parser.doubleValue(true);
break;
default:
throw new ElasticsearchParseException("longitude must be a number");
}
} else {
throw new ElasticsearchParseException("only lat/lon or [{}] is allowed", FIELD_VALUE);
}
} else if (FIELD_PRECISION.equals(fieldName)) {
if(parser.nextToken() == Token.START_ARRAY) {
IntHashSet precisions = new IntHashSet();
while(parser.nextToken() != Token.END_ARRAY) {
precisions.add(parsePrecision(parser));
}
precision = precisions.toArray();
} else {
precision = new int[] { parsePrecision(parser) };
}
} else if (FIELD_VALUE.equals(fieldName)) {
if(Double.isNaN(lon) && Double.isNaN(lat)) {
parser.nextToken();
point = GeoUtils.parseGeoPoint(parser);
} else {
throw new ElasticsearchParseException("only lat/lon or [{}] is allowed", FIELD_VALUE);
}
} else {
throw new ElasticsearchParseException("unexpected fieldname [{}]", fieldName);
}
}
if (point == null) {
if (Double.isNaN(lat) || Double.isNaN(lon)) {
throw new ElasticsearchParseException("location is missing");
} else {
point = new GeoPoint(lat, lon);
}
}
if (precision == null || precision.length == 0) {
precision = this.precision;
}
return new GeoQuery(name, point.geohash(), precision);
} else {
return new GeoQuery(name, GeoUtils.parseGeoPoint(parser).getGeohash(), precision);
}
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + ((defaultLocations == null) ? 0 : defaultLocations.hashCode());
result = prime * result + ((fieldName == null) ? 0 : fieldName.hashCode());
result = prime * result + (neighbors ? 1231 : 1237);
result = prime * result + Arrays.hashCode(precision);
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
GeolocationContextMapping other = (GeolocationContextMapping) obj;
if (defaultLocations == null) {
if (other.defaultLocations != null)
return false;
} else if (!defaultLocations.equals(other.defaultLocations))
return false;
if (fieldName == null) {
if (other.fieldName != null)
return false;
} else if (!fieldName.equals(other.fieldName))
return false;
if (neighbors != other.neighbors)
return false;
if (!Arrays.equals(precision, other.precision))
return false;
return true;
}
public static class Builder extends ContextBuilder<GeolocationContextMapping> {
private IntHashSet precisions = new IntHashSet();
private boolean neighbors; // take neighbor cell on the lowest level into account
private HashSet<String> defaultLocations = new HashSet<>();
private String fieldName = null;
protected Builder(String name) {
this(name, true, null);
}
protected Builder(String name, boolean neighbors, int...levels) {
super(name);
neighbors(neighbors);
if (levels != null) {
for (int level : levels) {
precision(level);
}
}
}
/**
* Set the precision use o make suggestions
*
* @param precision
* precision as distance with {@link DistanceUnit}. Default:
* meters
* @return this
*/
public Builder precision(String precision) {
return precision(DistanceUnit.parse(precision, DistanceUnit.METERS, DistanceUnit.METERS));
}
/**
* Set the precision use o make suggestions
*
* @param precision
* precision value
* @param unit
* {@link DistanceUnit} to use
* @return this
*/
public Builder precision(double precision, DistanceUnit unit) {
return precision(unit.toMeters(precision));
}
/**
* Set the precision use o make suggestions
*
* @param meters
* precision as distance in meters
* @return this
*/
public Builder precision(double meters) {
int level = GeoUtils.geoHashLevelsForPrecision(meters);
// Ceiling precision: we might return more results
if (GeoUtils.geoHashCellSize(level) < meters) {
level = Math.max(1, level - 1);
}
return precision(level);
}
/**
* Set the precision use o make suggestions
*
* @param level
* maximum length of geohashes
* @return this
*/
public Builder precision(int level) {
this.precisions.add(level);
return this;
}
/**
* Set neighborhood usage
*
* @param neighbors
* should neighbor cells also be valid
* @return this
*/
public Builder neighbors(boolean neighbors) {
this.neighbors = neighbors;
return this;
}
/**
* Set a default location that should be used, if no location is
* provided by the query
*
* @param geohash
* geohash of the default location
* @return this
*/
public Builder addDefaultLocation(String geohash) {
this.defaultLocations.add(geohash);
return this;
}
/**
* Set a default location that should be used, if no location is
* provided by the query
*
* @param geohashes
* geohash of the default location
* @return this
*/
public Builder addDefaultLocations(Collection<String> geohashes) {
this.defaultLocations.addAll(geohashes);
return this;
}
/**
* Set a default location that should be used, if no location is
* provided by the query
*
* @param lat
* latitude of the default location
* @param lon
* longitude of the default location
* @return this
*/
public Builder addDefaultLocation(double lat, double lon) {
this.defaultLocations.add(GeoHashUtils.stringEncode(lon, lat));
return this;
}
/**
* Set a default location that should be used, if no location is
* provided by the query
*
* @param point
* location
* @return this
*/
public Builder defaultLocation(GeoPoint point) {
this.defaultLocations.add(point.geohash());
return this;
}
/**
* Set the name of the field containing a geolocation to use
* @param fieldName name of the field
* @return this
*/
public Builder field(String fieldName) {
this.fieldName = fieldName;
return this;
}
@Override
public GeolocationContextMapping build() {
if(precisions.isEmpty()) {
precisions.add(GeoHashUtils.PRECISION);
}
int[] precisionArray = precisions.toArray();
Arrays.sort(precisionArray);
return new GeolocationContextMapping(name, precisionArray, neighbors, defaultLocations, fieldName);
}
}
private static class GeoConfig extends ContextConfig {
private final GeolocationContextMapping mapping;
private final Collection<String> locations;
public GeoConfig(GeolocationContextMapping mapping, Collection<String> locations) {
this.locations = locations;
this.mapping = mapping;
}
@Override
protected TokenStream wrapTokenStream(Document doc, TokenStream stream) {
Collection<String> geohashes;
if (locations == null || locations.size() == 0) {
if(mapping.fieldName != null) {
IndexableField[] fields = doc.getFields(mapping.fieldName);
if(fields.length == 0) {
IndexableField[] lonFields = doc.getFields(mapping.fieldName + ".lon");
IndexableField[] latFields = doc.getFields(mapping.fieldName + ".lat");
if (lonFields.length > 0 && latFields.length > 0) {
geohashes = new ArrayList<>(fields.length);
GeoPoint spare = new GeoPoint();
for (int i = 0 ; i < lonFields.length ; i++) {
IndexableField lonField = lonFields[i];
IndexableField latField = latFields[i];
assert lonField.fieldType().docValuesType() == latField.fieldType().docValuesType();
// we write doc values fields differently: one field for all values,
// so we need to only care about indexed fields
if (lonField.fieldType().docValuesType() == DocValuesType.NONE) {
spare.reset(latField.numericValue().doubleValue(), lonField.numericValue().doubleValue());
geohashes.add(spare.geohash());
}
}
} else {
geohashes = mapping.defaultLocations;
}
} else {
geohashes = new ArrayList<>(fields.length);
GeoPoint spare = new GeoPoint();
for (IndexableField field : fields) {
if (field instanceof StringField) {
spare.resetFromString(field.stringValue());
} else if (field instanceof GeoPointField) {
GeoPointField geoPointField = (GeoPointField) field;
spare.reset(geoPointField.getLat(), geoPointField.getLon());
} else {
spare.resetFromString(field.stringValue());
}
geohashes.add(spare.geohash());
}
}
} else {
geohashes = mapping.defaultLocations;
}
} else {
geohashes = locations;
}
Collection<String> locations = new HashSet<>();
for (String geohash : geohashes) {
for (int p : mapping.precision) {
int precision = Math.min(p, geohash.length());
String truncatedGeohash = geohash.substring(0, precision);
if(mapping.neighbors) {
GeoHashUtils.addNeighbors(truncatedGeohash, precision, locations);
}
locations.add(truncatedGeohash);
}
}
return new PrefixTokenFilter(stream, ContextMapping.SEPARATOR, locations);
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder("GeoConfig(location = [");
Iterator<? extends CharSequence> location = this.locations.iterator();
if (location.hasNext()) {
sb.append(location.next());
while (location.hasNext()) {
sb.append(", ").append(location.next());
}
}
return sb.append("])").toString();
}
}
private static class GeoQuery extends ContextQuery {
private final String location;
private final int[] precisions;
public GeoQuery(String name, String location, int...precisions) {
super(name);
this.location = location;
this.precisions = precisions;
}
@Override
public Automaton toAutomaton() {
Automaton automaton;
if(precisions == null || precisions.length == 0) {
automaton = Automata.makeString(location);
} else {
automaton = Automata.makeString(
location.substring(0, Math.max(1, Math.min(location.length(), precisions[0]))));
for (int i = 1; i < precisions.length; i++) {
final String cell = location.substring(0, Math.max(1, Math.min(location.length(), precisions[i])));
automaton = Operations.union(automaton, Automata.makeString(cell));
}
}
return automaton;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if(precisions == null || precisions.length == 0) {
builder.field(name, location);
} else {
builder.startObject(name);
builder.field(FIELD_VALUE, location);
builder.array(FIELD_PRECISION, precisions);
builder.endObject();
}
return builder;
}
}
}

View File

@ -1,23 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
/**
* Support for completion suggesters with contexts built on 2.x indices.
*/
package org.elasticsearch.search.suggest.completion2x.context;

View File

@ -1,23 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
/**
* Support for completion suggesters built on 2.x indices.
*/
package org.elasticsearch.search.suggest.completion2x;

View File

@ -1,2 +1 @@
org.apache.lucene.search.suggest.document.Completion50PostingsFormat
org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat

View File

@ -36,7 +36,7 @@
"enabled" : false
},
"description": {
"type": "string"
"type": "text"
}
}
},

View File

@ -111,7 +111,7 @@ public class MetaDataIndexTemplateServiceTests extends ESSingleNodeTestCase {
PutRequest request = new PutRequest("api", "validate_template");
request.patterns(Collections.singletonList("te*"));
request.putMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("properties")
.startObject("field2").field("type", "string").field("analyzer", "custom_1").endObject()
.startObject("field2").field("type", "text").field("analyzer", "custom_1").endObject()
.endObject().endObject().endObject().string());
List<Throwable> errors = putTemplateDetail(request);

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