mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-02-22 04:45:37 +00:00
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:
parent
235e6acd73
commit
6231009a8f
@ -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);
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
*/
|
||||
|
@ -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());
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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(),
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -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)
|
||||
|
@ -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,
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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];
|
||||
}
|
||||
|
||||
};
|
||||
}
|
||||
}
|
@ -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));
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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();) {
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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 {
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
@ -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();) {
|
||||
|
@ -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();) {
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
@ -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();) {
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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());
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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
|
||||
|
@ -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)) {
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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
|
||||
|
@ -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>
|
||||
* "geohash_bbox" {
|
||||
* "field":"location",
|
||||
* "geohash":"u33d8u5dkx8k",
|
||||
* "neighbors":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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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");
|
||||
|
@ -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>{"type":"geo_point", "geohash":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>{"type":"geo_point", "geohash":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>{"type":"geo_point", "geohash":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.
|
||||
*
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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) {
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
|
@ -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()) {
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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));
|
||||
|
@ -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));
|
||||
|
@ -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()));
|
||||
}
|
||||
|
@ -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));
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
@ -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;
|
@ -1,2 +1 @@
|
||||
org.apache.lucene.search.suggest.document.Completion50PostingsFormat
|
||||
org.elasticsearch.search.suggest.completion2x.Completion090PostingsFormat
|
@ -36,7 +36,7 @@
|
||||
"enabled" : false
|
||||
},
|
||||
"description": {
|
||||
"type": "string"
|
||||
"type": "text"
|
||||
}
|
||||
}
|
||||
},
|
||||
|
@ -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
Loading…
x
Reference in New Issue
Block a user