From 3bcc6546d9b4f7b267758bbbb98a74f23cf131ef Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 2 May 2025 15:09:38 -0400 Subject: [PATCH 01/42] Initial commit of match_phrase --- .../index/query/MatchPhraseQueryBuilder.java | 5 + .../index/query/ZeroTermsQueryOption.java | 9 + .../xpack/esql/action/EsqlCapabilities.java | 5 + .../function/EsqlFunctionRegistry.java | 2 + .../function/fulltext/FullTextFunction.java | 8 + .../function/fulltext/MatchPhrase.java | 380 ++++++++++++++++++ .../esql/querydsl/query/MatchPhraseQuery.java | 109 +++++ .../function/fulltext/MatchPhraseTests.java | 102 +++++ 8 files changed, 620 insertions(+) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java diff --git a/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java index c0e5758de81b4..86f5988135d79 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java @@ -129,6 +129,11 @@ public MatchPhraseQueryBuilder zeroTermsQuery(ZeroTermsQueryOption zeroTermsQuer return this; } + public MatchPhraseQueryBuilder zeroTermsQuery(String zeroTermsQueryString) { + ZeroTermsQueryOption zeroTermsQueryOption = ZeroTermsQueryOption.readFromString(zeroTermsQueryString); + return zeroTermsQuery(zeroTermsQueryOption); + } + public ZeroTermsQueryOption zeroTermsQuery() { return this.zeroTermsQuery; } diff --git a/server/src/main/java/org/elasticsearch/index/query/ZeroTermsQueryOption.java b/server/src/main/java/org/elasticsearch/index/query/ZeroTermsQueryOption.java index 2b07d40ab43a0..792347b08ecb6 100644 --- a/server/src/main/java/org/elasticsearch/index/query/ZeroTermsQueryOption.java +++ b/server/src/main/java/org/elasticsearch/index/query/ZeroTermsQueryOption.java @@ -55,6 +55,15 @@ public static ZeroTermsQueryOption readFromStream(StreamInput in) throws IOExcep throw new ElasticsearchException("unknown serialized type [" + ord + "]"); } + public static ZeroTermsQueryOption readFromString(String input) { + for (ZeroTermsQueryOption zeroTermsQuery : ZeroTermsQueryOption.values()) { + if (zeroTermsQuery.name().equalsIgnoreCase(input)) { + return zeroTermsQuery; + } + } + throw new ElasticsearchException("unknown serialized type [" + input + "]"); + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(this.ordinal); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index b201ab7cb4afe..8e34c290b72b2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -649,6 +649,11 @@ public enum Cap { */ MATCH_FUNCTION, + /** + * MATCH PHRASE function + */ + MATCH_PHRASE_FUNCTION(Build.current().isSnapshot()), + /** * KQL function */ diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java index 0bd4e4bda7c5b..91af50de4006f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java @@ -39,6 +39,7 @@ import org.elasticsearch.xpack.esql.expression.function.aggregate.WeightedAvg; import org.elasticsearch.xpack.esql.expression.function.fulltext.Kql; import org.elasticsearch.xpack.esql.expression.function.fulltext.Match; +import org.elasticsearch.xpack.esql.expression.function.fulltext.MatchPhrase; import org.elasticsearch.xpack.esql.expression.function.fulltext.MultiMatch; import org.elasticsearch.xpack.esql.expression.function.fulltext.QueryString; import org.elasticsearch.xpack.esql.expression.function.fulltext.Term; @@ -431,6 +432,7 @@ private static FunctionDefinition[][] functions() { new FunctionDefinition[] { def(Kql.class, uni(Kql::new), "kql"), def(Match.class, tri(Match::new), "match"), + def(MatchPhrase.class, tri(MatchPhrase::new), "match_phrase"), def(MultiMatch.class, MultiMatch::new, "multi_match"), def(QueryString.class, bi(QueryString::new), "qstr") } }; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 2d86d7a604b36..77e125a1038b2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -209,6 +209,14 @@ private static void checkFullTextQueryFunctions(LogicalPlan plan, Failures failu m -> "[" + m.functionName() + "] " + m.functionType(), failures ); + checkCommandsBeforeExpression( + plan, + condition, + MatchPhrase.class, + lp -> (lp instanceof Limit == false) && (lp instanceof Aggregate == false), + m -> "[" + m.functionName() + "] " + m.functionType(), + failures + ); checkCommandsBeforeExpression( plan, condition, diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java new file mode 100644 index 0000000000000..f3017a40d304b --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -0,0 +1,380 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.fulltext; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.esql.capabilities.PostAnalysisPlanVerificationAware; +import org.elasticsearch.xpack.esql.common.Failure; +import org.elasticsearch.xpack.esql.common.Failures; +import org.elasticsearch.xpack.esql.core.InvalidArgumentException; +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.FoldContext; +import org.elasticsearch.xpack.esql.core.expression.MapExpression; +import org.elasticsearch.xpack.esql.core.querydsl.query.Query; +import org.elasticsearch.xpack.esql.core.tree.NodeInfo; +import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; +import org.elasticsearch.xpack.esql.core.util.Check; +import org.elasticsearch.xpack.esql.expression.function.Example; +import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesTo; +import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesToLifecycle; +import org.elasticsearch.xpack.esql.expression.function.FunctionInfo; +import org.elasticsearch.xpack.esql.expression.function.MapParam; +import org.elasticsearch.xpack.esql.expression.function.OptionalArgument; +import org.elasticsearch.xpack.esql.expression.function.Param; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.TranslatorHandler; +import org.elasticsearch.xpack.esql.querydsl.query.MatchPhraseQuery; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.BiConsumer; + +import static java.util.Map.entry; +import static org.elasticsearch.index.query.AbstractQueryBuilder.BOOST_FIELD; +import static org.elasticsearch.index.query.MatchPhraseQueryBuilder.SLOP_FIELD; +import static org.elasticsearch.index.query.MatchPhraseQueryBuilder.ZERO_TERMS_QUERY_FIELD; +import static org.elasticsearch.index.query.MatchQueryBuilder.ANALYZER_FIELD; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; +import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; +import static org.elasticsearch.xpack.esql.core.type.DataType.FLOAT; +import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; +import static org.elasticsearch.xpack.esql.core.type.DataType.IP; +import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; +import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; +import static org.elasticsearch.xpack.esql.core.type.DataType.VERSION; +import static org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison.formatIncompatibleTypesMessage; + +/** + * Full text function that performs a {@link org.elasticsearch.xpack.esql.querydsl.query.MatchPhraseQuery} . + */ +public class MatchPhrase extends FullTextFunction implements OptionalArgument, PostAnalysisPlanVerificationAware { + + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Expression.class, + "MatchPhrase", + MatchPhrase::readFrom + ); + public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); + public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); + + protected final Expression field; + + // Options for match_phrase function. They don’t need to be serialized as the data nodes will retrieve them from the query builder + private final transient Expression options; + + public static final Map ALLOWED_OPTIONS = Map.ofEntries( + entry(ANALYZER_FIELD.getPreferredName(), KEYWORD), + entry(BOOST_FIELD.getPreferredName(), FLOAT), + entry(SLOP_FIELD.getPreferredName(), INTEGER), + entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), KEYWORD) + ); + + @FunctionInfo( + returnType = "boolean", + preview = true, + description = """ + Use `MATCH_PHRASE` to perform a <> on the specified field. + Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. + + MatchPhrase can be used on <> fields, as well as other field types like keyword, boolean, or date types. + MatchPhrase is not supported for <> or numeric types. + + MatchPhrase can use <> to specify additional options for the match_phrase query. + All <> are supported. + + For a simplified syntax, you can use the <> `:` operator instead of `MATCH_PHRASE`. + + `MATCH_PHRASE` returns true if the provided query matches the row.""", + examples = { + @Example(file = "match-phrase-function", tag = "match-phrase-with-field"), + @Example(file = "match-phrase-function", tag = "match-phrase-with-named-function-params") }, + appliesTo = { + @FunctionAppliesTo( + lifeCycle = FunctionAppliesToLifecycle.COMING, + description = "Support for optional named parameters is only available in serverless, or in a future {{es}} release" + ) } + ) + public MatchPhrase( + Source source, + @Param( + name = "field", + type = { "keyword", "text", "boolean", "date", "date_nanos", "ip", "version" }, + description = "Field that the query will target." + ) Expression field, + @Param( + name = "query", + type = { "keyword", "boolean", "date", "date_nanos", "ip", "version" }, + description = "Value to find in the provided field." + ) Expression matchPhraseQuery, + @MapParam( + name = "options", + params = { + @MapParam.MapParamEntry( + name = "analyzer", + type = "keyword", + valueHint = { "standard" }, + description = "Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer" + + " mapped for the field. If no analyzer is mapped, the index’s default analyzer is used." + ), + @MapParam.MapParamEntry( + name = "slop", + type = "integer", + valueHint = { "1" }, + description = "Maximum number of positions allowed between matching tokens. Defaults to 0." + + " Transposed terms have a slop of 2." + ), + @MapParam.MapParamEntry( + name = "zero_terms_query", + type = "keyword", + valueHint = { "none", "all" }, + description = "Indicates whether all documents or none are returned if the analyzer removes all tokens, such as " + + "when using a stop filter. Defaults to none." + ) }, + description = "(Optional) MatchPhrase additional options as <>." + + " See <> for more information.", + optional = true + ) Expression options + ) { + this(source, field, matchPhraseQuery, options, null); + } + + public MatchPhrase(Source source, Expression field, Expression matchPhraseQuery, Expression options, QueryBuilder queryBuilder) { + super( + source, + matchPhraseQuery, + options == null ? List.of(field, matchPhraseQuery) : List.of(field, matchPhraseQuery, options), + queryBuilder + ); + this.field = field; + this.options = options; + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + private static MatchPhrase readFrom(StreamInput in) throws IOException { + Source source = Source.readFrom((PlanStreamInput) in); + Expression field = in.readNamedWriteable(Expression.class); + Expression query = in.readNamedWriteable(Expression.class); + QueryBuilder queryBuilder = null; + queryBuilder = in.readOptionalNamedWriteable(QueryBuilder.class); + return new MatchPhrase(source, field, query, null, queryBuilder); + } + + @Override + public final void writeTo(StreamOutput out) throws IOException { + source().writeTo(out); + out.writeNamedWriteable(field()); + out.writeNamedWriteable(query()); + out.writeOptionalNamedWriteable(queryBuilder()); + } + + @Override + protected TypeResolution resolveParams() { + return resolveField().and(resolveQuery()).and(resolveOptions()).and(checkParamCompatibility()); + } + + private TypeResolution resolveField() { + return isNotNull(field, sourceText(), FIRST).and( + isType(field, FIELD_DATA_TYPES::contains, sourceText(), FIRST, "keyword, text, boolean, date, date_nanos, ip, version") + ); + } + + private TypeResolution resolveQuery() { + return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword, boolean, date, date_nanos, ip, version").and( + isNotNullAndFoldable(query(), sourceText(), SECOND) + ); + } + + private TypeResolution checkParamCompatibility() { + DataType fieldType = field().dataType(); + DataType queryType = query().dataType(); + + // Field and query types should match. If the query is a string, then it can match any field type. + if ((fieldType == queryType) || (queryType == KEYWORD)) { + return TypeResolution.TYPE_RESOLVED; + } + + return new TypeResolution(formatIncompatibleTypesMessage(fieldType, queryType, sourceText())); + } + + private TypeResolution resolveOptions() { + if (options() != null) { + TypeResolution resolution = isNotNull(options(), sourceText(), THIRD); + if (resolution.unresolved()) { + return resolution; + } + // MapExpression does not have a DataType associated with it + resolution = isMapExpression(options(), sourceText(), THIRD); + if (resolution.unresolved()) { + return resolution; + } + + try { + matchPhraseQueryOptions(); + } catch (InvalidArgumentException e) { + return new TypeResolution(e.getMessage()); + } + } + return TypeResolution.TYPE_RESOLVED; + } + + private Map matchPhraseQueryOptions() throws InvalidArgumentException { + if (options() == null) { + return Map.of(); + } + + Map matchPhraseOptions = new HashMap<>(); + populateOptionsMap((MapExpression) options(), matchPhraseOptions, SECOND, sourceText(), ALLOWED_OPTIONS); + return matchPhraseOptions; + } + + public Expression field() { + return field; + } + + public Expression options() { + return options; + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, MatchPhrase::new, field(), query(), options(), queryBuilder()); + } + + @Override + public Expression replaceChildren(List newChildren) { + return new MatchPhrase( + source(), + newChildren.get(0), + newChildren.get(1), + newChildren.size() > 2 ? newChildren.get(2) : null, + queryBuilder() + ); + } + + @Override + public Expression replaceQueryBuilder(QueryBuilder queryBuilder) { + return new MatchPhrase(source(), field, query(), options(), queryBuilder); + } + + @Override + public BiConsumer postAnalysisPlanVerification() { + return (plan, failures) -> { + super.postAnalysisPlanVerification().accept(plan, failures); + plan.forEachExpression(MatchPhrase.class, mp -> { + if (mp.fieldAsFieldAttribute() == null) { + failures.add( + Failure.fail( + mp.field(), + "[{}] {} cannot operate on [{}], which is not a field from an index mapping", + functionName(), + functionType(), + mp.field().sourceText() + ) + ); + } + }); + }; + } + + @Override + public Object queryAsObject() { + Object queryAsObject = query().fold(FoldContext.small() /* TODO remove me */); + + // Convert BytesRef to string for string-based values + if (queryAsObject instanceof BytesRef bytesRef) { + return switch (query().dataType()) { + case IP -> EsqlDataTypeConverter.ipToString(bytesRef); + case VERSION -> EsqlDataTypeConverter.versionToString(bytesRef); + default -> bytesRef.utf8ToString(); + }; + } + + // Converts specific types to the correct type for the query + if (query().dataType() == DataType.DATETIME && queryAsObject instanceof Long) { + // When casting to date and datetime, we get a long back. But MatchPhrase query needs a date string + return EsqlDataTypeConverter.dateTimeToString((Long) queryAsObject); + } else if (query().dataType() == DATE_NANOS && queryAsObject instanceof Long) { + return EsqlDataTypeConverter.nanoTimeToString((Long) queryAsObject); + } + + return queryAsObject; + } + + @Override + protected Query translate(TranslatorHandler handler) { + var fieldAttribute = fieldAsFieldAttribute(); + Check.notNull(fieldAttribute, "MatchPhrase must have a field attribute as the first argument"); + String fieldName = getNameFromFieldAttribute(fieldAttribute); + return new MatchPhraseQuery(source(), fieldName, queryAsObject(), matchPhraseQueryOptions()); + } + + public static String getNameFromFieldAttribute(FieldAttribute fieldAttribute) { + String fieldName = fieldAttribute.name(); + if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { + // If we have multiple field types, we allow the query to be done, but getting the underlying field name + fieldName = multiTypeEsField.getName(); + } + return fieldName; + } + + public static FieldAttribute fieldAsFieldAttribute(Expression field) { + Expression fieldExpression = field; + // Field may be converted to other data type (field_name :: data_type), so we need to check the original field + if (fieldExpression instanceof AbstractConvertFunction convertFunction) { + fieldExpression = convertFunction.field(); + } + return fieldExpression instanceof FieldAttribute fieldAttribute ? fieldAttribute : null; + } + + private FieldAttribute fieldAsFieldAttribute() { + return fieldAsFieldAttribute(field); + } + + @Override + public boolean equals(Object o) { + // MatchPhrase does not serialize options, as they get included in the query builder. We need to override equals and hashcode to + // ignore options when comparing two Match functions + if (o == null || getClass() != o.getClass()) return false; + MatchPhrase match = (MatchPhrase) o; + return Objects.equals(field(), match.field()) + && Objects.equals(query(), match.query()) + && Objects.equals(queryBuilder(), match.queryBuilder()); + } + + @Override + public int hashCode() { + return Objects.hash(field(), query(), queryBuilder()); + } + +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java new file mode 100644 index 0000000000000..942559b8b7bc6 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -0,0 +1,109 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.esql.querydsl.query; + +import org.elasticsearch.index.query.MatchPhraseQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.xpack.esql.core.querydsl.query.Query; +import org.elasticsearch.xpack.esql.core.tree.Source; + +import java.util.Map; +import java.util.Objects; +import java.util.function.BiConsumer; + +import static java.util.Map.entry; +import static org.elasticsearch.index.query.MatchPhraseQueryBuilder.SLOP_FIELD; +import static org.elasticsearch.index.query.MatchPhraseQueryBuilder.ZERO_TERMS_QUERY_FIELD; +import static org.elasticsearch.index.query.MatchQueryBuilder.ANALYZER_FIELD; + +public class MatchPhraseQuery extends Query { + + private static final Map> BUILDER_APPLIERS; + + static { + BUILDER_APPLIERS = Map.ofEntries( + entry(ANALYZER_FIELD.getPreferredName(), (qb, s) -> qb.analyzer(s.toString())), + entry(SLOP_FIELD.getPreferredName(), (qb, i) -> qb.slop((Integer) i)), + entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)) + ); + } + + private final String name; + private final Object text; + private final Double boost; + private final Map options; + + public MatchPhraseQuery(Source source, String name, Object text) { + this(source, name, text, Map.of()); + } + + public MatchPhraseQuery(Source source, String name, Object text, Map options) { + super(source); + assert options != null; + this.name = name; + this.text = text; + this.options = options; + this.boost = null; + } + + @Override + protected QueryBuilder asBuilder() { + final MatchPhraseQueryBuilder queryBuilder = QueryBuilders.matchPhraseQuery(name, text); + options.forEach((k, v) -> { + if (BUILDER_APPLIERS.containsKey(k)) { + BUILDER_APPLIERS.get(k).accept(queryBuilder, v); + } else { + throw new IllegalArgumentException("illegal match_phrase option [" + k + "]"); + } + }); + if (boost != null) { + queryBuilder.boost(boost.floatValue()); + } + return queryBuilder; + } + + public String name() { + return name; + } + + public Object text() { + return text; + } + + @Override + public int hashCode() { + return Objects.hash(text, name, options, boost); + } + + @Override + public boolean equals(Object obj) { + if (false == super.equals(obj)) { + return false; + } + + MatchPhraseQuery other = (MatchPhraseQuery) obj; + return Objects.equals(text, other.text) + && Objects.equals(name, other.name) + && Objects.equals(options, other.options) + && Objects.equals(boost, other.boost); + } + + @Override + protected String innerToString() { + return name + ":" + text; + } + + public Map options() { + return options; + } + + @Override + public boolean scorable() { + return true; + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java new file mode 100644 index 0000000000000..37d1588c82964 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -0,0 +1,102 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.fulltext; + +import com.carrotsearch.randomizedtesting.annotations.Name; +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.Literal; +import org.elasticsearch.xpack.esql.core.expression.MapExpression; +import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.expression.function.FunctionName; +import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; +import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.esql.SerializationTestUtils.serializeDeserialize; +import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; +import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; +import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; +import static org.elasticsearch.xpack.esql.core.type.DataType.UNSUPPORTED; +import static org.elasticsearch.xpack.esql.planner.TranslatorHandler.TRANSLATOR_HANDLER; +import static org.hamcrest.Matchers.equalTo; + +@FunctionName("match_phrase") +public class MatchPhraseTests extends AbstractMatchFullTextFunctionTests { + + public MatchPhraseTests(@Name("TestCase") Supplier testCaseSupplier) { + this.testCase = testCaseSupplier.get(); + } + + @ParametersFactory + public static Iterable parameters() { + return parameterSuppliersFromTypedData(addFunctionNamedParams(testCaseSuppliers())); + } + + /** + * Adds function named parameters to all the test case suppliers provided + */ + private static List addFunctionNamedParams(List suppliers) { + List result = new ArrayList<>(); + for (TestCaseSupplier supplier : suppliers) { + List dataTypes = new ArrayList<>(supplier.types()); + dataTypes.add(UNSUPPORTED); + result.add(new TestCaseSupplier(supplier.name() + ", options", dataTypes, () -> { + List values = new ArrayList<>(supplier.get().getData()); + values.add( + new TestCaseSupplier.TypedData( + new MapExpression( + Source.EMPTY, + List.of(new Literal(Source.EMPTY, "slop", INTEGER), new Literal(Source.EMPTY, randomAlphaOfLength(10), KEYWORD)) + ), + UNSUPPORTED, + "options" + ).forceLiteral() + ); + + return new TestCaseSupplier.TestCase(values, equalTo("MatchPhraseEvaluator"), BOOLEAN, equalTo(true)); + })); + } + return result; + } + + @Override + protected Expression build(Source source, List args) { + MatchPhrase matchPhrase = new MatchPhrase(source, args.get(0), args.get(1), args.size() > 2 ? args.get(2) : null); + // We need to add the QueryBuilder to the match_phrase expression, as it is used to implement equals() and hashCode() and + // thus test the serialization methods. But we can only do this if the parameters make sense . + if (args.get(0) instanceof FieldAttribute && args.get(1).foldable()) { + QueryBuilder queryBuilder = TRANSLATOR_HANDLER.asQuery(LucenePushdownPredicates.DEFAULT, matchPhrase).toQueryBuilder(); + matchPhrase.replaceQueryBuilder(queryBuilder); + } + return matchPhrase; + } + + /** + * Copy of the overridden method that doesn't check for children size, as the {@code options} child isn't serialized in MatchPhrase. + */ + @Override + protected Expression serializeDeserializeExpression(Expression expression) { + Expression newExpression = serializeDeserialize( + expression, + PlanStreamOutput::writeNamedWriteable, + in -> in.readNamedWriteable(Expression.class), + testCase.getConfiguration() // The configuration query should be == to the source text of the function for this to work + ); + // Fields use synthetic sources, which can't be serialized. So we use the originals instead. + return newExpression.replaceChildren(expression.children()); + } +} From baee81bd7a68e08cf7b22c1c32746c7da6192a5a Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 2 May 2025 16:03:05 -0400 Subject: [PATCH 02/42] Add MatchPhraseQueryTests --- .../function/fulltext/MatchPhrase.java | 6 +- .../esql/querydsl/query/MatchPhraseQuery.java | 2 +- .../querydsl/query/MatchPhraseQueryTests.java | 74 +++++ .../rest-api-spec/test/esql/60_usage.yml | 304 +++++++++--------- 4 files changed, 231 insertions(+), 155 deletions(-) create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQueryTests.java diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index f3017a40d304b..0405aee0a07f0 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -106,10 +106,12 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P MatchPhrase can be used on <> fields, as well as other field types like keyword, boolean, or date types. MatchPhrase is not supported for <> or numeric types. - MatchPhrase can use <> to specify additional options for the match_phrase query. + MatchPhrase can use <> to specify additional options for the + match_phrase query. All <> are supported. - For a simplified syntax, you can use the <> `:` operator instead of `MATCH_PHRASE`. + For a simplified syntax, you can use the <> `:` operator instead + of `MATCH_PHRASE`. `MATCH_PHRASE` returns true if the provided query matches the row.""", examples = { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java index 942559b8b7bc6..6f6735efa7482 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -28,7 +28,7 @@ public class MatchPhraseQuery extends Query { static { BUILDER_APPLIERS = Map.ofEntries( entry(ANALYZER_FIELD.getPreferredName(), (qb, s) -> qb.analyzer(s.toString())), - entry(SLOP_FIELD.getPreferredName(), (qb, i) -> qb.slop((Integer) i)), + entry(SLOP_FIELD.getPreferredName(), (qb, s) -> qb.slop(Integer.parseInt(s.toString()))), entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQueryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQueryTests.java new file mode 100644 index 0000000000000..6b81cda4ebb28 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQueryTests.java @@ -0,0 +1,74 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.esql.querydsl.query; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.index.query.MatchPhraseQueryBuilder; +import org.elasticsearch.index.query.ZeroTermsQueryOption; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.core.tree.SourceTests; +import org.elasticsearch.xpack.esql.core.util.StringUtils; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Function; + +import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; +import static org.hamcrest.Matchers.equalTo; + +public class MatchPhraseQueryTests extends ESTestCase { + static MatchPhraseQuery randomMatchPhraseQuery() { + return new MatchPhraseQuery(SourceTests.randomSource(), randomAlphaOfLength(5), randomAlphaOfLength(5)); + } + + public void testEqualsAndHashCode() { + checkEqualsAndHashCode(randomMatchPhraseQuery(), MatchPhraseQueryTests::copy, MatchPhraseQueryTests::mutate); + } + + private static MatchPhraseQuery copy(MatchPhraseQuery query) { + return new MatchPhraseQuery(query.source(), query.name(), query.text(), query.options()); + } + + private static MatchPhraseQuery mutate(MatchPhraseQuery query) { + List> options = Arrays.asList( + q -> new MatchPhraseQuery(SourceTests.mutate(q.source()), q.name(), q.text(), q.options()), + q -> new MatchPhraseQuery(q.source(), randomValueOtherThan(q.name(), () -> randomAlphaOfLength(5)), q.text(), q.options()), + q -> new MatchPhraseQuery(q.source(), q.name(), randomValueOtherThan(q.text(), () -> randomAlphaOfLength(5)), q.options()) + ); + return randomFrom(options).apply(query); + } + + public void testQueryBuilding() { + + MatchPhraseQueryBuilder qb = getBuilder(Map.of("slop", 2, "zero_terms_query", "none")); + assertThat(qb.slop(), equalTo(2)); + assertThat(qb.zeroTermsQuery(), equalTo(ZeroTermsQueryOption.NONE)); + + Exception e = expectThrows(IllegalArgumentException.class, () -> getBuilder(Map.of("pizza", "yummy"))); + assertThat(e.getMessage(), equalTo("illegal match_phrase option [pizza]")); + + e = expectThrows(NumberFormatException.class, () -> getBuilder(Map.of("slop", "mushrooms"))); + assertThat(e.getMessage(), equalTo("For input string: \"mushrooms\"")); + + e = expectThrows(ElasticsearchException.class, () -> getBuilder(Map.of("zero_terms_query", "pepperoni"))); + assertThat(e.getMessage(), equalTo("unknown serialized type [pepperoni]")); + } + + private static MatchPhraseQueryBuilder getBuilder(Map options) { + final Source source = new Source(1, 1, StringUtils.EMPTY); + final MatchPhraseQuery mpq = new MatchPhraseQuery(source, "eggplant", "foo bar", options); + return (MatchPhraseQueryBuilder) mpq.asBuilder(); + } + + public void testToString() { + final Source source = new Source(1, 1, StringUtils.EMPTY); + final MatchPhraseQuery mpq = new MatchPhraseQuery(source, "eggplant", "foo bar"); + assertEquals("MatchPhraseQuery@1:2[eggplant:foo bar]", mpq.toString()); + } +} diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml index f4a6f48e27ddf..69e60f22e3273 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml @@ -7,11 +7,11 @@ setup: parameters: [ method, path, parameters, capabilities ] capabilities: [ function_stats ] reason: "META command removed which changes the count of the data returned" - test_runner_features: [capabilities] + test_runner_features: [ capabilities ] - do: indices.create: - index: test + index: test body: settings: number_of_shards: 1 @@ -27,58 +27,58 @@ setup: --- "Basic ESQL usage output (telemetry) snapshot version": - requires: - test_runner_features: [capabilities] + test_runner_features: [ capabilities ] capabilities: - method: POST path: /_query - parameters: [] + parameters: [ ] # A snapshot function was removed in match_function_options, it can't work on mixed cluster tests otherwise. - capabilities: [ snapshot_test_for_telemetry, fn_byte_length, match_function_options, last_over_time] + capabilities: [ snapshot_test_for_telemetry, fn_byte_length, match_function_options, last_over_time ] reason: "Test that should only be executed on snapshot versions" - - do: {xpack.usage: {}} + - do: { xpack.usage: { } } - match: { esql.available: true } - match: { esql.enabled: true } - length: { esql.features: 26 } - - set: {esql.features.dissect: dissect_counter} - - set: {esql.features.drop: drop_counter} - - set: {esql.features.eval: eval_counter} - - set: {esql.features.enrich: enrich_counter} - - set: {esql.features.from: from_counter} - - set: {esql.features.grok: grok_counter} - - set: {esql.features.keep: keep_counter} - - set: {esql.features.limit: limit_counter} - - set: {esql.features.mv_expand: mv_expand_counter} - - set: {esql.features.rename: rename_counter} - - set: {esql.features.row: row_counter} - - set: {esql.features.show: show_counter} - - set: {esql.features.sort: sort_counter} - - set: {esql.features.stats: stats_counter} - - set: {esql.features.where: where_counter} - - set: {esql.features.lookup_join: lookup_join_counter} - - set: {esql.features.lookup: lookup_counter} - - set: {esql.features.change_point: change_point_counter} - - set: {esql.features.inlinestats: inlinestats_counter} - - set: {esql.features.rerank: rerank_counter} - - set: {esql.features.dedup: dedup_counter} - - set: {esql.features.insist: insist_counter} - - set: {esql.features.fork: fork_counter} - - set: {esql.features.rrf: rrf_counter} - - set: {esql.features.completion: completion_counter} - - set: {esql.features.sample: sample_counter} + - set: { esql.features.dissect: dissect_counter } + - set: { esql.features.drop: drop_counter } + - set: { esql.features.eval: eval_counter } + - set: { esql.features.enrich: enrich_counter } + - set: { esql.features.from: from_counter } + - set: { esql.features.grok: grok_counter } + - set: { esql.features.keep: keep_counter } + - set: { esql.features.limit: limit_counter } + - set: { esql.features.mv_expand: mv_expand_counter } + - set: { esql.features.rename: rename_counter } + - set: { esql.features.row: row_counter } + - set: { esql.features.show: show_counter } + - set: { esql.features.sort: sort_counter } + - set: { esql.features.stats: stats_counter } + - set: { esql.features.where: where_counter } + - set: { esql.features.lookup_join: lookup_join_counter } + - set: { esql.features.lookup: lookup_counter } + - set: { esql.features.change_point: change_point_counter } + - set: { esql.features.inlinestats: inlinestats_counter } + - set: { esql.features.rerank: rerank_counter } + - set: { esql.features.dedup: dedup_counter } + - set: { esql.features.insist: insist_counter } + - set: { esql.features.fork: fork_counter } + - set: { esql.features.rrf: rrf_counter } + - set: { esql.features.completion: completion_counter } + - set: { esql.features.sample: sample_counter } - length: { esql.queries: 3 } - - set: {esql.queries.rest.total: rest_total_counter} - - set: {esql.queries.rest.failed: rest_failed_counter} - - set: {esql.queries.kibana.total: kibana_total_counter} - - set: {esql.queries.kibana.failed: kibana_failed_counter} - - set: {esql.queries._all.total: all_total_counter} - - set: {esql.queries._all.failed: all_failed_counter} - - set: {esql.functions.max: functions_max} - - set: {esql.functions.min: functions_min} - - set: {esql.functions.cos: functions_cos} - - set: {esql.functions.to_long: functions_to_long} - - set: {esql.functions.coalesce: functions_coalesce} - - set: {esql.functions.categorize: functions_categorize} + - set: { esql.queries.rest.total: rest_total_counter } + - set: { esql.queries.rest.failed: rest_failed_counter } + - set: { esql.queries.kibana.total: kibana_total_counter } + - set: { esql.queries.kibana.failed: kibana_failed_counter } + - set: { esql.queries._all.total: all_total_counter } + - set: { esql.queries._all.failed: all_failed_counter } + - set: { esql.functions.max: functions_max } + - set: { esql.functions.min: functions_min } + - set: { esql.functions.cos: functions_cos } + - set: { esql.functions.to_long: functions_to_long } + - set: { esql.functions.coalesce: functions_coalesce } + - set: { esql.functions.categorize: functions_categorize } - do: esql.query: @@ -89,96 +89,96 @@ setup: | limit 5 | stats m = max(data) by categorize(message)' - - do: {xpack.usage: {}} + - do: { xpack.usage: { } } - match: { esql.available: true } - match: { esql.enabled: true } - - match: {esql.features.dissect: $dissect_counter} - - match: {esql.features.eval: $eval_counter} - - match: {esql.features.grok: $grok_counter} - - gt: {esql.features.limit: $limit_counter} - - gt: {esql.features.sort: $sort_counter} - - gt: {esql.features.stats: $stats_counter} - - gt: {esql.features.where: $where_counter} - - match: {esql.features.lookup_join: $lookup_join_counter} - - match: {esql.features.lookup: $lookup_counter} - - match: {esql.features.change_point: $change_point_counter} - - match: {esql.features.inlinestats: $inlinestats_counter} - - match: {esql.features.rerank: $rerank_counter} - - match: {esql.features.dedup: $dedup_counter} - - match: {esql.features.insist: $insist_counter} - - match: {esql.features.fork: $fork_counter} - - match: {esql.features.rrf: $rrf_counter} - - match: {esql.features.completion: $completion_counter} - - match: {esql.features.sample: $sample_counter} - - gt: {esql.queries.rest.total: $rest_total_counter} - - match: {esql.queries.rest.failed: $rest_failed_counter} - - match: {esql.queries.kibana.total: $kibana_total_counter} - - match: {esql.queries.kibana.failed: $kibana_failed_counter} - - gt: {esql.queries._all.total: $all_total_counter} - - match: {esql.queries._all.failed: $all_failed_counter} - - gt: {esql.functions.max: $functions_max} - - match: {esql.functions.min: $functions_min} - - match: {esql.functions.cos: $functions_cos} - - gt: {esql.functions.to_long: $functions_to_long} - - match: {esql.functions.coalesce: $functions_coalesce} - - gt: {esql.functions.categorize: $functions_categorize} + - match: { esql.features.dissect: $dissect_counter } + - match: { esql.features.eval: $eval_counter } + - match: { esql.features.grok: $grok_counter } + - gt: { esql.features.limit: $limit_counter } + - gt: { esql.features.sort: $sort_counter } + - gt: { esql.features.stats: $stats_counter } + - gt: { esql.features.where: $where_counter } + - match: { esql.features.lookup_join: $lookup_join_counter } + - match: { esql.features.lookup: $lookup_counter } + - match: { esql.features.change_point: $change_point_counter } + - match: { esql.features.inlinestats: $inlinestats_counter } + - match: { esql.features.rerank: $rerank_counter } + - match: { esql.features.dedup: $dedup_counter } + - match: { esql.features.insist: $insist_counter } + - match: { esql.features.fork: $fork_counter } + - match: { esql.features.rrf: $rrf_counter } + - match: { esql.features.completion: $completion_counter } + - match: { esql.features.sample: $sample_counter } + - gt: { esql.queries.rest.total: $rest_total_counter } + - match: { esql.queries.rest.failed: $rest_failed_counter } + - match: { esql.queries.kibana.total: $kibana_total_counter } + - match: { esql.queries.kibana.failed: $kibana_failed_counter } + - gt: { esql.queries._all.total: $all_total_counter } + - match: { esql.queries._all.failed: $all_failed_counter } + - gt: { esql.functions.max: $functions_max } + - match: { esql.functions.min: $functions_min } + - match: { esql.functions.cos: $functions_cos } + - gt: { esql.functions.to_long: $functions_to_long } + - match: { esql.functions.coalesce: $functions_coalesce } + - gt: { esql.functions.categorize: $functions_categorize } # Testing for the entire function set isn't feasbile, so we just check that we return the correct count as an approximation. - - length: {esql.functions: 138} # check the "sister" test below for a likely update to the same esql.functions length check + - length: { esql.functions: 139 } # check the "sister" test below for a likely update to the same esql.functions length check --- "Basic ESQL usage output (telemetry) non-snapshot version": - requires: - test_runner_features: [capabilities] + test_runner_features: [ capabilities ] capabilities: - method: POST path: /_query - parameters: [] + parameters: [ ] capabilities: [ non_snapshot_test_for_telemetry, fn_byte_length ] reason: "Test that should only be executed on release versions" - - do: {xpack.usage: {}} + - do: { xpack.usage: { } } - match: { esql.available: true } - match: { esql.enabled: true } - length: { esql.features: 26 } - - set: {esql.features.dissect: dissect_counter} - - set: {esql.features.drop: drop_counter} - - set: {esql.features.eval: eval_counter} - - set: {esql.features.enrich: enrich_counter} - - set: {esql.features.from: from_counter} - - set: {esql.features.grok: grok_counter} - - set: {esql.features.keep: keep_counter} - - set: {esql.features.limit: limit_counter} - - set: {esql.features.mv_expand: mv_expand_counter} - - set: {esql.features.rename: rename_counter} - - set: {esql.features.row: row_counter} - - set: {esql.features.show: show_counter} - - set: {esql.features.sort: sort_counter} - - set: {esql.features.stats: stats_counter} - - set: {esql.features.where: where_counter} - - set: {esql.features.lookup_join: lookup_join_counter} - - set: {esql.features.lookup: lookup_counter} - - set: {esql.features.change_point: change_point_counter} - - set: {esql.features.inlinestats: inlinestats_counter} - - set: {esql.features.rerank: rerank_counter} - - set: {esql.features.dedup: dedup_counter} - - set: {esql.features.insist: insist_counter} - - set: {esql.features.fork: fork_counter} - - set: {esql.features.rrf: rrf_counter} - - set: {esql.features.completion: completion_counter} - - set: {esql.features.sample: sample_counter} + - set: { esql.features.dissect: dissect_counter } + - set: { esql.features.drop: drop_counter } + - set: { esql.features.eval: eval_counter } + - set: { esql.features.enrich: enrich_counter } + - set: { esql.features.from: from_counter } + - set: { esql.features.grok: grok_counter } + - set: { esql.features.keep: keep_counter } + - set: { esql.features.limit: limit_counter } + - set: { esql.features.mv_expand: mv_expand_counter } + - set: { esql.features.rename: rename_counter } + - set: { esql.features.row: row_counter } + - set: { esql.features.show: show_counter } + - set: { esql.features.sort: sort_counter } + - set: { esql.features.stats: stats_counter } + - set: { esql.features.where: where_counter } + - set: { esql.features.lookup_join: lookup_join_counter } + - set: { esql.features.lookup: lookup_counter } + - set: { esql.features.change_point: change_point_counter } + - set: { esql.features.inlinestats: inlinestats_counter } + - set: { esql.features.rerank: rerank_counter } + - set: { esql.features.dedup: dedup_counter } + - set: { esql.features.insist: insist_counter } + - set: { esql.features.fork: fork_counter } + - set: { esql.features.rrf: rrf_counter } + - set: { esql.features.completion: completion_counter } + - set: { esql.features.sample: sample_counter } - length: { esql.queries: 3 } - - set: {esql.queries.rest.total: rest_total_counter} - - set: {esql.queries.rest.failed: rest_failed_counter} - - set: {esql.queries.kibana.total: kibana_total_counter} - - set: {esql.queries.kibana.failed: kibana_failed_counter} - - set: {esql.queries._all.total: all_total_counter} - - set: {esql.queries._all.failed: all_failed_counter} - - set: {esql.functions.max: functions_max} - - set: {esql.functions.min: functions_min} - - set: {esql.functions.cos: functions_cos} - - set: {esql.functions.to_long: functions_to_long} - - set: {esql.functions.coalesce: functions_coalesce} - - set: {esql.functions.categorize: functions_categorize} + - set: { esql.queries.rest.total: rest_total_counter } + - set: { esql.queries.rest.failed: rest_failed_counter } + - set: { esql.queries.kibana.total: kibana_total_counter } + - set: { esql.queries.kibana.failed: kibana_failed_counter } + - set: { esql.queries._all.total: all_total_counter } + - set: { esql.queries._all.failed: all_failed_counter } + - set: { esql.functions.max: functions_max } + - set: { esql.functions.min: functions_min } + - set: { esql.functions.cos: functions_cos } + - set: { esql.functions.to_long: functions_to_long } + - set: { esql.functions.coalesce: functions_coalesce } + - set: { esql.functions.categorize: functions_categorize } - do: esql.query: @@ -189,52 +189,52 @@ setup: | limit 5 | stats m = max(data) by categorize(message)' - - do: {xpack.usage: {}} + - do: { xpack.usage: { } } - match: { esql.available: true } - match: { esql.enabled: true } - - match: {esql.features.dissect: $dissect_counter} - - match: {esql.features.eval: $eval_counter} - - match: {esql.features.grok: $grok_counter} - - gt: {esql.features.limit: $limit_counter} - - gt: {esql.features.sort: $sort_counter} - - gt: {esql.features.stats: $stats_counter} - - gt: {esql.features.where: $where_counter} - - match: {esql.features.lookup_join: $lookup_join_counter} - - match: {esql.features.lookup: $lookup_counter} - - match: {esql.features.change_point: $change_point_counter} - - match: {esql.features.inlinestats: $inlinestats_counter} - - match: {esql.features.rerank: $rerank_counter} - - match: {esql.features.dedup: $dedup_counter} - - match: {esql.features.insist: $insist_counter} - - match: {esql.features.fork: $fork_counter} - - match: {esql.features.rrf: $rrf_counter} - - match: {esql.features.completion: $completion_counter} - - gt: {esql.queries.rest.total: $rest_total_counter} - - match: {esql.queries.rest.failed: $rest_failed_counter} - - match: {esql.queries.kibana.total: $kibana_total_counter} - - match: {esql.queries.kibana.failed: $kibana_failed_counter} - - gt: {esql.queries._all.total: $all_total_counter} - - match: {esql.queries._all.failed: $all_failed_counter} - - gt: {esql.functions.max: $functions_max} - - match: {esql.functions.min: $functions_min} - - match: {esql.functions.cos: $functions_cos} - - gt: {esql.functions.to_long: $functions_to_long} - - match: {esql.functions.coalesce: $functions_coalesce} - - gt: {esql.functions.categorize: $functions_categorize} - - length: {esql.functions: 132} # check the "sister" test above for a likely update to the same esql.functions length check + - match: { esql.features.dissect: $dissect_counter } + - match: { esql.features.eval: $eval_counter } + - match: { esql.features.grok: $grok_counter } + - gt: { esql.features.limit: $limit_counter } + - gt: { esql.features.sort: $sort_counter } + - gt: { esql.features.stats: $stats_counter } + - gt: { esql.features.where: $where_counter } + - match: { esql.features.lookup_join: $lookup_join_counter } + - match: { esql.features.lookup: $lookup_counter } + - match: { esql.features.change_point: $change_point_counter } + - match: { esql.features.inlinestats: $inlinestats_counter } + - match: { esql.features.rerank: $rerank_counter } + - match: { esql.features.dedup: $dedup_counter } + - match: { esql.features.insist: $insist_counter } + - match: { esql.features.fork: $fork_counter } + - match: { esql.features.rrf: $rrf_counter } + - match: { esql.features.completion: $completion_counter } + - gt: { esql.queries.rest.total: $rest_total_counter } + - match: { esql.queries.rest.failed: $rest_failed_counter } + - match: { esql.queries.kibana.total: $kibana_total_counter } + - match: { esql.queries.kibana.failed: $kibana_failed_counter } + - gt: { esql.queries._all.total: $all_total_counter } + - match: { esql.queries._all.failed: $all_failed_counter } + - gt: { esql.functions.max: $functions_max } + - match: { esql.functions.min: $functions_min } + - match: { esql.functions.cos: $functions_cos } + - gt: { esql.functions.to_long: $functions_to_long } + - match: { esql.functions.coalesce: $functions_coalesce } + - gt: { esql.functions.categorize: $functions_categorize } + - length: { esql.functions: 132 } # check the "sister" test above for a likely update to the same esql.functions length check --- took: - requires: - test_runner_features: [capabilities] + test_runner_features: [ capabilities ] capabilities: - method: POST path: /_query - parameters: [] - capabilities: [usage_contains_took] + parameters: [ ] + capabilities: [ usage_contains_took ] reason: "Test that should only be executed on snapshot versions" - - do: {xpack.usage: {}} + - do: { xpack.usage: { } } - exists: esql.took.lt_10ms - exists: esql.took.lt_1s - exists: esql.took.lt_10s From 13ccda7e43e20af36bef4bd17c2631bedd10c335 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 6 May 2025 09:06:09 -0400 Subject: [PATCH 03/42] First pass at CSV specs --- .../functions/examples/match_phrase.md | 26 ++ .../functions/layout/match_phrase.md | 42 +++ .../_snippets/functions/types/match_phrase.md | 16 + .../esql/_snippets/lists/search-functions.md | 5 +- .../functions-operators/search-functions.md | 26 +- .../resources/match-phrase-function.csv-spec | 318 ++++++++++++++++++ .../function/fulltext/FullTextWritables.java | 1 + .../function/fulltext/MatchPhrase.java | 16 +- .../esql/querydsl/query/MatchPhraseQuery.java | 2 +- .../elasticsearch/xpack/esql/CsvTests.java | 4 + 10 files changed, 436 insertions(+), 20 deletions(-) create mode 100644 docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md create mode 100644 docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md create mode 100644 docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md create mode 100644 x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec diff --git a/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md new file mode 100644 index 0000000000000..893ff30421008 --- /dev/null +++ b/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md @@ -0,0 +1,26 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See +../README.md for how to regenerate it. + +**Examples** + +```esql +FROM books +| WHERE MATCH_PHRASE(author, "William Faulkner") +``` + +| book_no:keyword | author:text | +|-----------------|------------------| +| 2713 | William Faulkner | +| 2883 | William Faulkner | + +```esql +FROM books +| WHERE MATCH_PHRASE(title, "Hobbit There", {"slop": "5"}) +| KEEP title; +``` + +| title:text | +|------------------------------------| +| The Hobbit or There and Back Again | + + diff --git a/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md new file mode 100644 index 0000000000000..6b170e2876c12 --- /dev/null +++ b/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md @@ -0,0 +1,42 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See +../README.md for how to regenerate it. + +## `MATCH_PHRASE` [esql-match-phrase] + +:::{warning} +Do not use on production environments. This functionality is in technical +preview and +may be changed or removed in a future release. Elastic will work to fix any +issues, but features in technical preview +are not subject to the support SLA of official GA features. +::: + +:::{note} + +###### Serverless: GA, Elastic Stack: COMING + +Support for optional named parameters is only available in serverless, or in a +future {{es}} release +::: + +**Syntax** + +:::{image} ../../../images/functions/match_phrase.svg +:alt: Embedded +:class: text-center +::: + +:::{include} ../parameters/match_phrase.md +::: + +:::{include} ../description/match_phrase.md +::: + +:::{include} ../types/match_phrase.md +::: + +:::{include} ../functionNamedParams/match_phrase.md +::: + +:::{include} ../examples/match_phrase.md +::: diff --git a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md new file mode 100644 index 0000000000000..ebc04831d673f --- /dev/null +++ b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md @@ -0,0 +1,16 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See +../README.md for how to regenerate it. + +**Supported types** + +| field | query | options | result | +|------------|---------|------------------|---------| +| boolean | keyword | named parameters | boolean | +| date | keyword | named parameters | boolean | +| date_nanos | keyword | named parameters | boolean | +| ip | keyword | named parameters | boolean | +| keyword | keyword | named parameters | boolean | +| text | keyword | named parameters | boolean | +| version | keyword | named parameters | boolean | +| version | version | named parameters | boolean | + diff --git a/docs/reference/query-languages/esql/_snippets/lists/search-functions.md b/docs/reference/query-languages/esql/_snippets/lists/search-functions.md index f8deaf752414f..a3eb5e79b17f0 100644 --- a/docs/reference/query-languages/esql/_snippets/lists/search-functions.md +++ b/docs/reference/query-languages/esql/_snippets/lists/search-functions.md @@ -1,4 +1,7 @@ * [preview] [`KQL`](../../functions-operators/search-functions.md#esql-kql) * [preview] [`MATCH`](../../functions-operators/search-functions.md#esql-match) +* [preview] [ + `MATCH_PHRASE`](../../functions-operators/search-functions.md#esql-match-phrase) * [preview] [`QSTR`](../../functions-operators/search-functions.md#esql-qstr) -% * [preview] [`TERM`](../../functions-operators/search-functions.md#esql-term) + % * [preview] [ + `TERM`](../../functions-operators/search-functions.md#esql-term) diff --git a/docs/reference/query-languages/esql/functions-operators/search-functions.md b/docs/reference/query-languages/esql/functions-operators/search-functions.md index 2f5e4731d4ffa..44207fbf30ce7 100644 --- a/docs/reference/query-languages/esql/functions-operators/search-functions.md +++ b/docs/reference/query-languages/esql/functions-operators/search-functions.md @@ -6,33 +6,45 @@ mapped_pages: # {{esql}} Search functions [esql-search-functions] -Use these functions for [full-text search](docs-content://solutions/search/full-text.md) and [semantic search](docs-content://solutions/search/semantic-search/semantic-search-semantic-text.md). +Use these functions +for [full-text search](docs-content://solutions/search/full-text.md) +and [semantic search](docs-content://solutions/search/semantic-search/semantic-search-semantic-text.md). -Get started with {{esql}} for search use cases with our [hands-on tutorial](docs-content://solutions/search/esql-search-tutorial.md). +Get started with {{esql}} for search use cases with +our [hands-on tutorial](docs-content://solutions/search/esql-search-tutorial.md). -Full text functions can be used to match [multivalued fields](/reference/query-languages/esql/esql-multivalued-fields.md). A multivalued field that contains a value that matches a full text query is considered to match the query. +Full text functions can be used to +match [multivalued fields](/reference/query-languages/esql/esql-multivalued-fields.md). +A multivalued field that contains a value that matches a full text query is +considered to match the query. -Full text functions are significantly more performant for text search use cases on large data sets than using pattern matching or regular expressions with `LIKE` or `RLIKE` +Full text functions are significantly more performant for text search use cases +on large data sets than using pattern matching or regular expressions with +`LIKE` or `RLIKE` -See [full text search limitations](/reference/query-languages/esql/limitations.md#esql-limitations-full-text-search) for information on the limitations of full text search. +See [full text search limitations](/reference/query-languages/esql/limitations.md#esql-limitations-full-text-search) +for information on the limitations of full text search. {{esql}} supports these full-text search functions: :::{include} ../_snippets/lists/search-functions.md ::: - :::{include} ../_snippets/functions/layout/kql.md ::: :::{include} ../_snippets/functions/layout/match.md ::: +:::{include} ../_snippets/functions/layout/match_phrase.md +::: + :::{include} ../_snippets/functions/layout/qstr.md ::: % TERM is currently a hidden feature -% To make it visible again, uncomment this and the line in lists/search-functions.md +% To make it visible again, uncomment this and the line in +lists/search-functions.md % :::{include} ../_snippets/functions/layout/term.md % ::: diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec new file mode 100644 index 0000000000000..2d3ea62453ff1 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -0,0 +1,318 @@ +############################################### +# Tests for MatchPhrase function +# + +matchPhraseWithField +required_capability: match_phrase_function + +// tag::match-phrase-with-field[] +FROM books +| WHERE MATCH_PHRASE(author, "William Faulkner") +// end::match-phrase-with-field[] +| KEEP book_no, author +| SORT book_no +| LIMIT 5 +; + +// tag::match-phrase-with-field-result[] +book_no:keyword | author:text +2713 | William Faulkner +2883 | William Faulkner +// end::match-phrase-with-field-result[] +; + +matchPhraseWithMultipleFunctions +required_capability: match_phrase_function + +from books +| where match_phrase(title, "Return of the King") AND match_phrase(author, "J.R.R. Tolkien") +| keep book_no, title; +ignoreOrder:true + +book_no:keyword | title:text +2714 | Return of the King Being the Third Part of The Lord of the Rings +7350 | Return of the Shadow +; + +matchPhraseWithQueryExpressions +required_capability: match_phrase_function + +from books +| where match_phrase(title, CONCAT("Return of the", " King")) +| keep book_no, title; +ignoreOrder:true + +book_no:keyword | title:text +2714 | Return of the King Being the Third Part of The Lord of the Rings +; + +matchPhraseAfterKeep +required_capability: match_phrase_function + +from books +| keep book_no, author +| where match_phrase(author, "William Faulkner") +| sort book_no +| limit 5; + +book_no:keyword | author:text +2713 | William Faulkner +2883 | William Faulkner +; + +matchPhraseAfterDrop +required_capability: match_phrase_function + +from books +| drop ratings, description, year, publisher, title, author.keyword +| where match_phrase(author, "William Faulkner") +| keep book_no, author +| sort book_no +| limit 5; + +book_no:keyword | author:text +2713 | William Faulkner +2883 | William Faulkner +; + +matchPhraseAfterEval +required_capability: match_phrase_function + +from books +| eval stars = to_long(ratings / 2.0) +| where match_phrase(author, "William Faulkner") +| sort book_no +| keep book_no, author, stars +| limit 5; + +book_no:keyword | author:text | stars:long +2713 | William Faulkner | 2 +2883 | William Faulkner | 2 +; + +matchPhraseWithConjunction +required_capability: match_phrase_function + +from books +| where match_phrase(title, "Lord of the Rings") and ratings > 4.6 +| keep book_no, title; +ignoreOrder:true + +book_no:keyword | title:text +4023 |A Tolkien Compass: Including J. R. R. Tolkien's Guide to the Names in The Lord of the Rings +7140 |The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) +; + +matchPhraseWithDisjunction +required_capability: match_phrase_function +required_capability: full_text_functions_disjunctions + +from books +| where match_phrase(author, "Kurt Vonnegut") or match_phrase(author, "Carole Guinane") +| keep book_no, author; +ignoreOrder:true + +book_no:keyword | author:text +2464 | Kurt Vonnegut +8956 | Kurt Vonnegut +3950 | Kurt Vonnegut +4382 | Carole Guinane +; + +matchPhraseWithDisjunctionAndFiltersConjunction +required_capability: match_phrase_function +required_capability: full_text_functions_disjunctions + +from books +| where (match_phrase(author, "Edith Vonnegut") or match_phrase(author, "Carole Guinane")) and year > 1997 +| keep book_no, author, year; +ignoreOrder:true + +book_no:keyword | author:text | year:integer +6970 | Edith Vonnegut | 1998 +4382 | Carole Guinane | 2001 +; + +matchPhraseWithDisjunctionAndConjunction +required_capability: match_phrase_function +required_capability: full_text_functions_disjunctions + +from books +| where (match_phrase(author, "Kurt Vonnegut") or match_phrase(author, "Gabriel Garcia Marquez")) and match_phrase(description, "realism") +| keep book_no; + +book_no:keyword +4814 +; + +matchPhraseWithMoreComplexDisjunctionAndConjunction +required_capability: match_phrase_function +required_capability: full_text_functions_disjunctions + +from books +| where (match_phrase(author, "Kurt Vonnegut") and match_phrase(description, "charming")) or (match_phrase(author, "Gabriel Garcia Marquez") and match_phrase(description, "realism")) +| keep book_no; +ignoreOrder:true + +book_no:keyword +6970 +4814 +; + +matchPhraseWithDisjunctionIncludingConjunction +required_capability: match_phrase_function +required_capability: full_text_functions_disjunctions + +from books +| where match_phrase(author, "Kurt Vonnegut") or (match_phrase(author, "Gabriel Garcia Marquez") and match_phrase(description, "realism")) +| keep book_no; +ignoreOrder:true + +book_no:keyword +2464 +6970 +4814 +8956 +3950 +; + +matcPhraseWithFunctionPushedToLucene +required_capability: match_phrase_function + +from hosts +| where match_phrase(host, "beta") and cidr_match(ip1, "127.0.0.2/32", "127.0.0.3/32") +| keep card, host, ip0, ip1; +ignoreOrder:true + +card:keyword |host:keyword |ip0:ip |ip1:ip +eth1 |beta |127.0.0.1 |127.0.0.2 +; + +matchPhraseWithNonPushableConjunction +required_capability: match_phrase_function + +from books +| where match_phrase(title, "Lord of the Rings") and length(title) > 75 +| keep book_no, title; +ignoreOrder:true + +book_no:keyword | title:text +4023 | A Tolkien Compass: Including J. R. R. Tolkien's Guide to the Names in The Lord of the Rings +; + +matchPhraseWithMultipleWhereClauses +required_capability: match_phrase_function + +from books +| where match_phrase(title, "Lord of") +| where match_phrase(title, "the Rings") +| keep book_no, title; +ignoreOrder:true + +book_no:keyword | title:text +2675 | The Lord of the Rings - Boxed Set +2714 | Return of the King Being the Third Part of The Lord of the Rings +4023 | A Tolkien Compass: Including J. R. R. Tolkien's Guide to the Names in The Lord of the Rings +7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) +; + +matchPhraseMultivaluedField +required_capability: match_phrase_function + +from employees +| where match_phrase(job_positions, "Tech Lead") and match_phrase(job_positions, "Reporting Analyst") +| keep emp_no, first_name, last_name; +ignoreOrder:true + +emp_no:integer | first_name:keyword | last_name:keyword +10004 | Chirstian | Koblick +10010 | Duangkaew | Piveteau +10011 | Mary | Sluis +10088 | Jungsoon | Syrzycki +10093 | Sailaja | Desikan +10097 | Remzi | Waschkowski +; + +testMultiValuedFieldWithConjunction +required_capability: match_phrase_function + +from employees +| where match_phrase(job_positions, "Data Scientist") and match_phrase(job_positions, "Support Engineer") +| keep emp_no, first_name, last_name; +ignoreOrder:true + +emp_no:integer | first_name:keyword | last_name:keyword +10043 | Yishay | Tzvieli +; + +testMatchPhraseAndQueryStringFunctions +required_capability: match_phrase_function +required_capability: qstr_function + +from employees +| where match_phrase(job_positions, "Data Scientist") and qstr("job_positions: (Support Engineer) and gender: F") +| keep emp_no, first_name, last_name; +ignoreOrder:true + +emp_no:integer | first_name:keyword | last_name:keyword +10041 | Uri | Lenart +10043 | Yishay | Tzvieli +; + +testMatchPhraseVersionField +required_capability: match_phrase_function + +from apps +| where match_phrase(version, "2.1"::VERSION) +| keep name, version; + +name:keyword | version:version +bbbbb | 2.1 +; + +testMatchPhraseIpField +required_capability: match_phrase_function + +from sample_data +| where match(client_ip, "172.21.0.5") +| keep client_ip, message; + +client_ip:ip | message:keyword +172.21.0.5 | Disconnected +; + +testMatchPhraseDateFieldAsString +required_capability: match_phrase_function + +from date_nanos +| where match_phrase(millis, "2023-10-23T13:55:01.543Z") +| keep millis; + +millis:date +2023-10-23T13:55:01.543Z +; + +testMatchPhraseDateNanosFieldAsString +required_capability: match_phrase_function + +from date_nanos +| where match(nanos, "2023-10-23T13:55:01.543123456Z") +| keep nanos; + +nanos:date_nanos +2023-10-23T13:55:01.543123456Z +; + +testMatchPhraseWithOptionsSlop +required_capability: match_function +from books +| where match_phrase(title, "Lord of Rings", {"slop": 5}) +| keep book_no; +ignoreOrder:true + +book_no:keyword +2714 +2675 +4023 +7140 +; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java index 7ef632c66d44b..313b42088b962 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java @@ -21,6 +21,7 @@ public static List getNamedWriteables() { entries.add(QueryString.ENTRY); entries.add(Match.ENTRY); + entries.add(MatchPhrase.ENTRY); entries.add(MultiMatch.ENTRY); entries.add(Kql.ENTRY); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index 0405aee0a07f0..6483b6d7665e2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -82,7 +82,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P MatchPhrase::readFrom ); public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); - public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); + public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD); protected final Expression field; @@ -99,6 +99,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P @FunctionInfo( returnType = "boolean", preview = true, + // TODO link to match-phrase-field-params description = """ Use `MATCH_PHRASE` to perform a <> on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. @@ -108,10 +109,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P MatchPhrase can use <> to specify additional options for the match_phrase query. - All <> are supported. - - For a simplified syntax, you can use the <> `:` operator instead - of `MATCH_PHRASE`. + All match_phrase query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row.""", examples = { @@ -130,11 +128,7 @@ public MatchPhrase( type = { "keyword", "text", "boolean", "date", "date_nanos", "ip", "version" }, description = "Field that the query will target." ) Expression field, - @Param( - name = "query", - type = { "keyword", "boolean", "date", "date_nanos", "ip", "version" }, - description = "Value to find in the provided field." - ) Expression matchPhraseQuery, + @Param(name = "query", type = { "keyword" }, description = "Value to find in the provided field.") Expression matchPhraseQuery, @MapParam( name = "options", params = { @@ -212,7 +206,7 @@ private TypeResolution resolveField() { } private TypeResolution resolveQuery() { - return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword, boolean, date, date_nanos, ip, version").and( + return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword").and( isNotNullAndFoldable(query(), sourceText(), SECOND) ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java index 6f6735efa7482..431ea54ad64f8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -95,7 +95,7 @@ public boolean equals(Object obj) { @Override protected String innerToString() { - return name + ":" + text; + return name + ":\"" + text + "\""; } public Map options() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index 7ab45f805c754..bc978cf15ddc6 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -284,6 +284,10 @@ public final void test() throws Throwable { "can't use MATCH function in csv tests", testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.MATCH_FUNCTION.capabilityName()) ); + assumeFalse( + "can't use MATCH_PHRASE function in csv tests", + testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.capabilityName()) + ); assumeFalse( "can't use KQL function in csv tests", testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.KQL_FUNCTION.capabilityName()) From c8e67703b1befc07d0595675c0f9bcc94c30256f Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 7 May 2025 14:16:41 -0400 Subject: [PATCH 04/42] Update docs/changelog/127661.yaml --- docs/changelog/127661.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/127661.yaml diff --git a/docs/changelog/127661.yaml b/docs/changelog/127661.yaml new file mode 100644 index 0000000000000..ed2e41646e0dd --- /dev/null +++ b/docs/changelog/127661.yaml @@ -0,0 +1,5 @@ +pr: 127661 +summary: Add MATCH_PHRASE +area: Relevance +type: enhancement +issues: [] From d02864e2e5538b6e4e704136ef0867fe5f7a61b0 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Thu, 8 May 2025 13:02:58 -0400 Subject: [PATCH 05/42] Refactor so MatchPhrase doesn't use all fulltext test cases, just text only --- .../functions/description/match_phrase.md | 6 + .../functions/examples/match_phrase.md | 19 +- .../functionNamedParams/match_phrase.md | 13 + .../functions/layout/match_phrase.md | 18 +- .../functions/parameters/match_phrase.md | 13 + .../_snippets/functions/types/match_phrase.md | 22 +- .../esql/images/functions/match_phrase.svg | 1 + .../definition/functions/match_phrase.json | 239 ++++++++++++++++++ .../kibana/docs/functions/match_phrase.md | 19 ++ .../function/fulltext/MatchPhrase.java | 12 +- .../esql/querydsl/query/MatchPhraseQuery.java | 2 +- .../function/fulltext/MatchPhraseTests.java | 111 +++++++- 12 files changed, 433 insertions(+), 42 deletions(-) create mode 100644 docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md create mode 100644 docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md create mode 100644 docs/reference/query-languages/esql/_snippets/functions/parameters/match_phrase.md create mode 100644 docs/reference/query-languages/esql/images/functions/match_phrase.svg create mode 100644 docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json create mode 100644 docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md diff --git a/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md new file mode 100644 index 0000000000000..a3983a7a5c6d0 --- /dev/null +++ b/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md @@ -0,0 +1,6 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. + +**Description** + +Use `MATCH_PHRASE` to perform a [match_phrase query](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. MatchPhrase can be used on [text](/reference/elasticsearch/mapping-reference/text.md) fields, as well as other field types like keyword, boolean, or date types. MatchPhrase is not supported for [semantic_text](/reference/elasticsearch/mapping-reference/semantic-text.md) or numeric types. MatchPhrase can use [function named parameters](/reference/query-languages/esql/esql-syntax.md#esql-function-named-params) to specify additional options for the match_phrase query. All match_phrase query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. + diff --git a/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md index 893ff30421008..57a09b3c82b11 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md @@ -1,5 +1,4 @@ -% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See -../README.md for how to regenerate it. +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. **Examples** @@ -8,19 +7,13 @@ FROM books | WHERE MATCH_PHRASE(author, "William Faulkner") ``` -| book_no:keyword | author:text | -|-----------------|------------------| -| 2713 | William Faulkner | -| 2883 | William Faulkner | +| book_no:keyword | author:text | +| --- | --- | +| 2713 | William Faulkner | +| 2883 | William Faulkner | ```esql -FROM books -| WHERE MATCH_PHRASE(title, "Hobbit There", {"slop": "5"}) -| KEEP title; +null ``` -| title:text | -|------------------------------------| -| The Hobbit or There and Back Again | - diff --git a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md new file mode 100644 index 0000000000000..c0af0cf068e82 --- /dev/null +++ b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md @@ -0,0 +1,13 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. + +**Supported function named parameters** + +`zero_terms_query` +: (keyword) Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none. + +`analyzer` +: (keyword) Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used. + +`slop` +: (integer) Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2. + diff --git a/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md index 6b170e2876c12..5506c2cd6b25d 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/layout/match_phrase.md @@ -1,22 +1,15 @@ -% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See -../README.md for how to regenerate it. - -## `MATCH_PHRASE` [esql-match-phrase] +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. +## `MATCH_PHRASE` [esql-match_phrase] :::{warning} -Do not use on production environments. This functionality is in technical -preview and -may be changed or removed in a future release. Elastic will work to fix any -issues, but features in technical preview +Do not use on production environments. This functionality is in technical preview and +may be changed or removed in a future release. Elastic will work to fix any issues, but features in technical preview are not subject to the support SLA of official GA features. ::: :::{note} - ###### Serverless: GA, Elastic Stack: COMING - -Support for optional named parameters is only available in serverless, or in a -future {{es}} release +Support for optional named parameters is only available in serverless, or in a future {{es}} release ::: **Syntax** @@ -26,6 +19,7 @@ future {{es}} release :class: text-center ::: + :::{include} ../parameters/match_phrase.md ::: diff --git a/docs/reference/query-languages/esql/_snippets/functions/parameters/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/parameters/match_phrase.md new file mode 100644 index 0000000000000..baf89625dc51c --- /dev/null +++ b/docs/reference/query-languages/esql/_snippets/functions/parameters/match_phrase.md @@ -0,0 +1,13 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. + +**Parameters** + +`field` +: Field that the query will target. + +`query` +: Value to find in the provided field. + +`options` +: (Optional) MatchPhrase additional options as [function named parameters](/reference/query-languages/esql/esql-syntax.md#esql-function-named-params). See [match_phrase query](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) for more information. + diff --git a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md index ebc04831d673f..b49384a4e211f 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md @@ -1,16 +1,16 @@ -% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See -../README.md for how to regenerate it. +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. **Supported types** -| field | query | options | result | -|------------|---------|------------------|---------| -| boolean | keyword | named parameters | boolean | -| date | keyword | named parameters | boolean | +| field | query | options | result | +| --- | --- | --- | --- | +| boolean | keyword | named parameters | boolean | +| date | keyword | named parameters | boolean | | date_nanos | keyword | named parameters | boolean | -| ip | keyword | named parameters | boolean | -| keyword | keyword | named parameters | boolean | -| text | keyword | named parameters | boolean | -| version | keyword | named parameters | boolean | -| version | version | named parameters | boolean | +| ip | keyword | named parameters | boolean | +| keyword | keyword | named parameters | boolean | +| keyword | text | named parameters | boolean | +| text | keyword | named parameters | boolean | +| text | text | named parameters | boolean | +| version | keyword | named parameters | boolean | diff --git a/docs/reference/query-languages/esql/images/functions/match_phrase.svg b/docs/reference/query-languages/esql/images/functions/match_phrase.svg new file mode 100644 index 0000000000000..a636b25d26016 --- /dev/null +++ b/docs/reference/query-languages/esql/images/functions/match_phrase.svg @@ -0,0 +1 @@ +MATCH_PHRASE(field,query,options) \ No newline at end of file diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json new file mode 100644 index 0000000000000..4e569d7b5813c --- /dev/null +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -0,0 +1,239 @@ +{ + "comment" : "This is generated by ESQL’s AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it.", + "type" : "scalar", + "name" : "match_phrase", + "description" : "Use `MATCH_PHRASE` to perform a match_phrase query on the specified field.\nUsing `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL.\n\nMatchPhrase can be used on text fields, as well as other field types like keyword, boolean, or date types.\nMatchPhrase is not supported for semantic_text or numeric types.\n\nMatchPhrase can use function named parameters to specify additional options for the\nmatch_phrase query.\nAll match_phrase query parameters are supported.\n\n`MATCH_PHRASE` returns true if the provided query matches the row.", + "signatures" : [ + { + "params" : [ + { + "name" : "field", + "type" : "boolean", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "date", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "date_nanos", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "ip", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "keyword", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "keyword", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "text", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "text", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "text", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "text", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "version", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "keyword", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + } + ], + "examples" : [ + "FROM books\n| WHERE MATCH_PHRASE(author, \"William Faulkner\")", + null + ], + "preview" : true, + "snapshot_only" : false +} diff --git a/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md b/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md new file mode 100644 index 0000000000000..cb90729565bfe --- /dev/null +++ b/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md @@ -0,0 +1,19 @@ +% This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. + +### MATCH PHRASE +Use `MATCH_PHRASE` to perform a [match_phrase query](https://www.elastic.co/docs/reference/query-languages/query-dsl/query-dsl-match-query#query-dsl-match-query-phrase) on the specified field. +Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. + +MatchPhrase can be used on [text](https://www.elastic.co/docs/reference/elasticsearch/mapping-reference/text) fields, as well as other field types like keyword, boolean, or date types. +MatchPhrase is not supported for [semantic_text](https://www.elastic.co/docs/reference/elasticsearch/mapping-reference/semantic-text) or numeric types. + +MatchPhrase can use [function named parameters](https://www.elastic.co/docs/reference/query-languages/esql/esql-syntax#esql-function-named-params) to specify additional options for the +match_phrase query. +All match_phrase query parameters are supported. + +`MATCH_PHRASE` returns true if the provided query matches the row. + +```esql +FROM books +| WHERE MATCH_PHRASE(author, "William Faulkner") +``` diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index 6483b6d7665e2..660da70ff8417 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -82,7 +82,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P MatchPhrase::readFrom ); public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); - public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD); + public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD, TEXT); protected final Expression field; @@ -128,7 +128,11 @@ public MatchPhrase( type = { "keyword", "text", "boolean", "date", "date_nanos", "ip", "version" }, description = "Field that the query will target." ) Expression field, - @Param(name = "query", type = { "keyword" }, description = "Value to find in the provided field.") Expression matchPhraseQuery, + @Param( + name = "query", + type = { "keyword", "text" }, + description = "Value to find in the provided field." + ) Expression matchPhraseQuery, @MapParam( name = "options", params = { @@ -206,7 +210,7 @@ private TypeResolution resolveField() { } private TypeResolution resolveQuery() { - return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword").and( + return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword, text").and( isNotNullAndFoldable(query(), sourceText(), SECOND) ); } @@ -216,7 +220,7 @@ private TypeResolution checkParamCompatibility() { DataType queryType = query().dataType(); // Field and query types should match. If the query is a string, then it can match any field type. - if ((fieldType == queryType) || (queryType == KEYWORD)) { + if ((fieldType == queryType) || (queryType == KEYWORD) || (queryType == TEXT)) { return TypeResolution.TYPE_RESOLVED; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java index 431ea54ad64f8..6f6735efa7482 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -95,7 +95,7 @@ public boolean equals(Object obj) { @Override protected String innerToString() { - return name + ":\"" + text + "\""; + return name + ":" + text; } public Map options() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java index 37d1588c82964..adc36af5a9328 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.xpack.esql.core.expression.MapExpression; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase; import org.elasticsearch.xpack.esql.expression.function.FunctionName; import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier; import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; @@ -31,11 +32,12 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; import static org.elasticsearch.xpack.esql.core.type.DataType.UNSUPPORTED; +import static org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier.stringCases; import static org.elasticsearch.xpack.esql.planner.TranslatorHandler.TRANSLATOR_HANDLER; import static org.hamcrest.Matchers.equalTo; @FunctionName("match_phrase") -public class MatchPhraseTests extends AbstractMatchFullTextFunctionTests { +public class MatchPhraseTests extends AbstractFunctionTestCase { public MatchPhraseTests(@Name("TestCase") Supplier testCaseSupplier) { this.testCase = testCaseSupplier.get(); @@ -46,6 +48,113 @@ public static Iterable parameters() { return parameterSuppliersFromTypedData(addFunctionNamedParams(testCaseSuppliers())); } + private static List testCaseSuppliers() { + List suppliers = new ArrayList<>(); + addQueryAsStringTestCases(suppliers); + addStringTestCases(suppliers); + return suppliers; + } + + public static void addQueryAsStringTestCases(List suppliers) { + + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.booleanCases(), + TestCaseSupplier.stringCases(DataType.KEYWORD), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.ipCases(), + TestCaseSupplier.stringCases(DataType.KEYWORD), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.versionCases(""), + TestCaseSupplier.stringCases(DataType.KEYWORD), + List.of(), + false + ) + ); + // Datetime + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.dateCases(), + TestCaseSupplier.stringCases(DataType.KEYWORD), + List.of(), + false + ) + ); + + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.dateNanosCases(), + TestCaseSupplier.stringCases(DataType.KEYWORD), + List.of(), + false + ) + ); + } + + public static void addStringTestCases(List suppliers) { + for (DataType fieldType : DataType.stringTypes()) { + if (DataType.UNDER_CONSTRUCTION.containsKey(fieldType)) { + continue; + } + for (TestCaseSupplier.TypedDataSupplier queryDataSupplier : stringCases(fieldType)) { + suppliers.add( + TestCaseSupplier.testCaseSupplier( + queryDataSupplier, + new TestCaseSupplier.TypedDataSupplier(fieldType.typeName(), () -> randomAlphaOfLength(10), DataType.KEYWORD), + (d1, d2) -> equalTo("string"), + DataType.BOOLEAN, + (o1, o2) -> true + ) + ); + + suppliers.add( + TestCaseSupplier.testCaseSupplier( + queryDataSupplier, + new TestCaseSupplier.TypedDataSupplier(fieldType.typeName(), () -> randomAlphaOfLength(10), DataType.TEXT), + (d1, d2) -> equalTo("string"), + DataType.BOOLEAN, + (o1, o2) -> true + ) + ); + } + } + } + /** * Adds function named parameters to all the test case suppliers provided */ From 49ae77f12e682e7b2d511233c27366a1cfa5c0d7 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Thu, 8 May 2025 17:00:38 -0400 Subject: [PATCH 06/42] Fix tests --- .../resources/match-phrase-function.csv-spec | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index 2d3ea62453ff1..c1b8b416ce638 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -18,6 +18,9 @@ FROM books book_no:keyword | author:text 2713 | William Faulkner 2883 | William Faulkner +4724 | William Faulkner +4977 | William Faulkner +5119 | William Faulkner // end::match-phrase-with-field-result[] ; @@ -25,13 +28,12 @@ matchPhraseWithMultipleFunctions required_capability: match_phrase_function from books -| where match_phrase(title, "Return of the King") AND match_phrase(author, "J.R.R. Tolkien") +| where match_phrase(title, "Return of the King") AND match_phrase(author, "J. R. R. Tolkien") | keep book_no, title; ignoreOrder:true book_no:keyword | title:text 2714 | Return of the King Being the Third Part of The Lord of the Rings -7350 | Return of the Shadow ; matchPhraseWithQueryExpressions @@ -58,6 +60,9 @@ from books book_no:keyword | author:text 2713 | William Faulkner 2883 | William Faulkner +4724 | William Faulkner +4977 | William Faulkner +5119 | William Faulkner ; matchPhraseAfterDrop @@ -73,6 +78,9 @@ from books book_no:keyword | author:text 2713 | William Faulkner 2883 | William Faulkner +4724 | William Faulkner +4977 | William Faulkner +5119 | William Faulkner ; matchPhraseAfterEval @@ -88,6 +96,9 @@ from books book_no:keyword | author:text | stars:long 2713 | William Faulkner | 2 2883 | William Faulkner | 2 +4724 | William Faulkner | 2 +4977 | William Faulkner | 2 +5119 | William Faulkner | 2 ; matchPhraseWithConjunction @@ -150,7 +161,7 @@ required_capability: match_phrase_function required_capability: full_text_functions_disjunctions from books -| where (match_phrase(author, "Kurt Vonnegut") and match_phrase(description, "charming")) or (match_phrase(author, "Gabriel Garcia Marquez") and match_phrase(description, "realism")) +| where (match_phrase(author, "Edith Vonnegut") and match_phrase(description, "charming and insightful")) or (match_phrase(author, "Gabriel Garcia Marquez") and match_phrase(description, "realism")) | keep book_no; ignoreOrder:true @@ -170,13 +181,12 @@ ignoreOrder:true book_no:keyword 2464 -6970 4814 8956 3950 ; -matcPhraseWithFunctionPushedToLucene +matchPhraseWithFunctionPushedToLucene required_capability: match_phrase_function from hosts @@ -263,7 +273,7 @@ testMatchPhraseVersionField required_capability: match_phrase_function from apps -| where match_phrase(version, "2.1"::VERSION) +| where match_phrase(version, "2.1") | keep name, version; name:keyword | version:version From 5f14d2e3f6e0b1f0b472d72813c19564cfab8a4c Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 9 May 2025 15:18:58 -0400 Subject: [PATCH 07/42] Add some CSV test cases --- .../resources/match-phrase-function.csv-spec | 58 ++++++++++++++++++- 1 file changed, 57 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index c1b8b416ce638..eddc27540fdd2 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -314,7 +314,7 @@ nanos:date_nanos ; testMatchPhraseWithOptionsSlop -required_capability: match_function +required_capability: match_phrase_function from books | where match_phrase(title, "Lord of Rings", {"slop": 5}) | keep book_no; @@ -326,3 +326,59 @@ book_no:keyword 4023 7140 ; + +testMatchPhraseWithOptionsZeroTermsNone +required_capability: match_phrase_function +from books +| where match_phrase(title, "", {"zero_terms_query": "none"}) +| keep book_no; +ignoreOrder:true + +book_no:keyword +; + +testMatchPhraseWithOptionsZeroTermsAll +required_capability: match_phrase_function +from books +| where match_phrase(title, "", {"zero_terms_query": "all"}) +| sort book_no +| keep book_no +| limit 5; + +book_no:keyword +1211 +1463 +1502 +1937 +1985 +; + + +testMatchPhraseWithOptionsAnalyzer +required_capability: match_phrase_function +from books +| where match_phrase(title, "Lord of the Rings", {"analyzer": "standard"}) +| keep book_no; +ignoreOrder:true + +book_no:keyword +2714 +2675 +4023 +7140 +; + + +testMatchPhraseWithOptionsSlop +required_capability: match_phrase_function +from books +| where match_phrase(title, "Lord of Rings", {"slop": 3, "analyzer": "standard", "zero_terms_query": "none"}) +| keep book_no; +ignoreOrder:true + +book_no:keyword +2714 +2675 +4023 +7140 +; From 8095a28ab18ffaafce74f831d1f8939be034ddc5 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 9 May 2025 15:20:47 -0400 Subject: [PATCH 08/42] Fix test --- .../rest-api-spec/test/esql/60_usage.yml | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml index 280b5d647dcb3..1e4781adc2204 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml @@ -33,7 +33,7 @@ setup: path: /_query parameters: [ ] # A snapshot function was removed in match_function_options, it can't work on mixed cluster tests otherwise. - capabilities: [ snapshot_test_for_telemetry, fn_byte_length, match_function_options, first_over_time] + capabilities: [ snapshot_test_for_telemetry, fn_byte_length, match_function_options, first_over_time ] reason: "Test that should only be executed on snapshot versions" - do: { xpack.usage: { } } @@ -92,38 +92,38 @@ setup: - do: { xpack.usage: { } } - match: { esql.available: true } - match: { esql.enabled: true } - - match: {esql.features.dissect: $dissect_counter} - - match: {esql.features.eval: $eval_counter} - - match: {esql.features.grok: $grok_counter} - - gt: {esql.features.limit: $limit_counter} - - gt: {esql.features.sort: $sort_counter} - - gt: {esql.features.stats: $stats_counter} - - gt: {esql.features.where: $where_counter} - - match: {esql.features.lookup_join: $lookup_join_counter} - - match: {esql.features.lookup: $lookup_counter} - - match: {esql.features.change_point: $change_point_counter} - - match: {esql.features.inlinestats: $inlinestats_counter} - - match: {esql.features.rerank: $rerank_counter} - - match: {esql.features.dedup: $dedup_counter} - - match: {esql.features.insist: $insist_counter} - - match: {esql.features.fork: $fork_counter} - - match: {esql.features.rrf: $rrf_counter} - - match: {esql.features.completion: $completion_counter} - - match: {esql.features.sample: $sample_counter} - - gt: {esql.queries.rest.total: $rest_total_counter} - - match: {esql.queries.rest.failed: $rest_failed_counter} - - match: {esql.queries.kibana.total: $kibana_total_counter} - - match: {esql.queries.kibana.failed: $kibana_failed_counter} - - gt: {esql.queries._all.total: $all_total_counter} - - match: {esql.queries._all.failed: $all_failed_counter} - - gt: {esql.functions.max: $functions_max} - - match: {esql.functions.min: $functions_min} - - match: {esql.functions.cos: $functions_cos} - - gt: {esql.functions.to_long: $functions_to_long} - - match: {esql.functions.coalesce: $functions_coalesce} - - gt: {esql.functions.categorize: $functions_categorize} + - match: { esql.features.dissect: $dissect_counter } + - match: { esql.features.eval: $eval_counter } + - match: { esql.features.grok: $grok_counter } + - gt: { esql.features.limit: $limit_counter } + - gt: { esql.features.sort: $sort_counter } + - gt: { esql.features.stats: $stats_counter } + - gt: { esql.features.where: $where_counter } + - match: { esql.features.lookup_join: $lookup_join_counter } + - match: { esql.features.lookup: $lookup_counter } + - match: { esql.features.change_point: $change_point_counter } + - match: { esql.features.inlinestats: $inlinestats_counter } + - match: { esql.features.rerank: $rerank_counter } + - match: { esql.features.dedup: $dedup_counter } + - match: { esql.features.insist: $insist_counter } + - match: { esql.features.fork: $fork_counter } + - match: { esql.features.rrf: $rrf_counter } + - match: { esql.features.completion: $completion_counter } + - match: { esql.features.sample: $sample_counter } + - gt: { esql.queries.rest.total: $rest_total_counter } + - match: { esql.queries.rest.failed: $rest_failed_counter } + - match: { esql.queries.kibana.total: $kibana_total_counter } + - match: { esql.queries.kibana.failed: $kibana_failed_counter } + - gt: { esql.queries._all.total: $all_total_counter } + - match: { esql.queries._all.failed: $all_failed_counter } + - gt: { esql.functions.max: $functions_max } + - match: { esql.functions.min: $functions_min } + - match: { esql.functions.cos: $functions_cos } + - gt: { esql.functions.to_long: $functions_to_long } + - match: { esql.functions.coalesce: $functions_coalesce } + - gt: { esql.functions.categorize: $functions_categorize } # Testing for the entire function set isn't feasible, so we just check that we return the correct count as an approximation. - - length: {esql.functions: 141} # check the "sister" test below for a likely update to the same esql.functions length check + - length: { esql.functions: 142 } # check the "sister" test below for a likely update to the same esql.functions length check --- "Basic ESQL usage output (telemetry) non-snapshot version": From 42208437e396553a580626358bfd0ef51027706a Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 9 May 2025 15:31:04 -0400 Subject: [PATCH 09/42] Update changelog --- docs/changelog/127661.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/changelog/127661.yaml b/docs/changelog/127661.yaml index ed2e41646e0dd..66c8336963271 100644 --- a/docs/changelog/127661.yaml +++ b/docs/changelog/127661.yaml @@ -1,5 +1,5 @@ pr: 127661 summary: Add MATCH_PHRASE -area: Relevance +area: ES|QL type: enhancement issues: [] From 4b5f4fa5f460ef9b25be35b1da7a37286de44cb9 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 9 May 2025 15:43:15 -0400 Subject: [PATCH 10/42] Update tests --- .../esql/_snippets/functions/description/match_phrase.md | 2 +- .../esql/_snippets/functions/examples/match_phrase.md | 3 +++ .../query-languages/esql/kibana/docs/functions/match_phrase.md | 2 +- .../xpack/esql/expression/function/fulltext/MatchPhrase.java | 3 +-- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md index a3983a7a5c6d0..168c1fed49a8b 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md @@ -2,5 +2,5 @@ **Description** -Use `MATCH_PHRASE` to perform a [match_phrase query](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. MatchPhrase can be used on [text](/reference/elasticsearch/mapping-reference/text.md) fields, as well as other field types like keyword, boolean, or date types. MatchPhrase is not supported for [semantic_text](/reference/elasticsearch/mapping-reference/semantic-text.md) or numeric types. MatchPhrase can use [function named parameters](/reference/query-languages/esql/esql-syntax.md#esql-function-named-params) to specify additional options for the match_phrase query. All match_phrase query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. +Use `MATCH_PHRASE` to perform a [match_phrase query](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. MatchPhrase can be used on [text](/reference/elasticsearch/mapping-reference/text.md) fields, as well as other field types like keyword, boolean, or date types. MatchPhrase is not supported for [semantic_text](/reference/elasticsearch/mapping-reference/semantic-text.md) or numeric types. MatchPhrase can use [function named parameters](/reference/query-languages/esql/esql-syntax.md#esql-function-named-params) to specify additional options for the match_phrase query. All [match_phrase](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. diff --git a/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md index 57a09b3c82b11..c93c22aab37e0 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/examples/match_phrase.md @@ -11,6 +11,9 @@ FROM books | --- | --- | | 2713 | William Faulkner | | 2883 | William Faulkner | +| 4724 | William Faulkner | +| 4977 | William Faulkner | +| 5119 | William Faulkner | ```esql null diff --git a/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md b/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md index cb90729565bfe..ac27ba2da06f9 100644 --- a/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md +++ b/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md @@ -9,7 +9,7 @@ MatchPhrase is not supported for [semantic_text](https://www.elastic.co/docs/ref MatchPhrase can use [function named parameters](https://www.elastic.co/docs/reference/query-languages/esql/esql-syntax#esql-function-named-params) to specify additional options for the match_phrase query. -All match_phrase query parameters are supported. +All [match_phrase](https://www.elastic.co/docs/reference/query-languages/query-dsl/query-dsl-match-query#query-dsl-match-query-phrase) query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index 660da70ff8417..c846e858d3b35 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -99,7 +99,6 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P @FunctionInfo( returnType = "boolean", preview = true, - // TODO link to match-phrase-field-params description = """ Use `MATCH_PHRASE` to perform a <> on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. @@ -109,7 +108,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P MatchPhrase can use <> to specify additional options for the match_phrase query. - All match_phrase query parameters are supported. + All <> query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row.""", examples = { From 641f65b1925eeae3ef077dcfcf5ab67184e9d0e8 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 14 May 2025 06:59:25 -0700 Subject: [PATCH 11/42] Comment out MATCH_PHRASE in search-functions Markdown --- .../query-languages/esql/_snippets/lists/search-functions.md | 2 +- .../esql/functions-operators/search-functions.md | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/lists/search-functions.md b/docs/reference/query-languages/esql/_snippets/lists/search-functions.md index a3eb5e79b17f0..8ceb0d2f99b90 100644 --- a/docs/reference/query-languages/esql/_snippets/lists/search-functions.md +++ b/docs/reference/query-languages/esql/_snippets/lists/search-functions.md @@ -1,6 +1,6 @@ * [preview] [`KQL`](../../functions-operators/search-functions.md#esql-kql) * [preview] [`MATCH`](../../functions-operators/search-functions.md#esql-match) -* [preview] [ + % * [preview] [ `MATCH_PHRASE`](../../functions-operators/search-functions.md#esql-match-phrase) * [preview] [`QSTR`](../../functions-operators/search-functions.md#esql-qstr) % * [preview] [ diff --git a/docs/reference/query-languages/esql/functions-operators/search-functions.md b/docs/reference/query-languages/esql/functions-operators/search-functions.md index 44207fbf30ce7..7406aee8f737d 100644 --- a/docs/reference/query-languages/esql/functions-operators/search-functions.md +++ b/docs/reference/query-languages/esql/functions-operators/search-functions.md @@ -36,8 +36,9 @@ for information on the limitations of full text search. :::{include} ../_snippets/functions/layout/match.md ::: -:::{include} ../_snippets/functions/layout/match_phrase.md -::: +% MATCH_PHRASE is currently hidden +% :::{include} ../_snippets/functions/layout/match_phrase.md +% ::: :::{include} ../_snippets/functions/layout/qstr.md ::: From c58c4a8bf8e4cf6c40db2c9b9cca35d7d50e8c16 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 20 May 2025 13:12:19 -0400 Subject: [PATCH 12/42] Minor PR feedback --- .../xpack/esql/action/EsqlCapabilities.java | 12 +- .../rest-api-spec/test/esql/60_usage.yml | 304 +++++++++--------- 2 files changed, 158 insertions(+), 158 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index afdfa1a5c3015..c7a0ad8645aae 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -649,11 +649,6 @@ public enum Cap { */ MATCH_FUNCTION, - /** - * MATCH PHRASE function - */ - MATCH_PHRASE_FUNCTION(Build.current().isSnapshot()), - /** * KQL function */ @@ -1084,7 +1079,12 @@ public enum Cap { /** * Full text functions in STATS */ - FULL_TEXT_FUNCTIONS_IN_STATS_WHERE; + FULL_TEXT_FUNCTIONS_IN_STATS_WHERE, + + /** + * MATCH PHRASE function + */ + MATCH_PHRASE_FUNCTION(Build.current().isSnapshot()); private final boolean enabled; diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml index 1e4781adc2204..99a412ac83f9f 100644 --- a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/60_usage.yml @@ -7,11 +7,11 @@ setup: parameters: [ method, path, parameters, capabilities ] capabilities: [ function_stats ] reason: "META command removed which changes the count of the data returned" - test_runner_features: [ capabilities ] + test_runner_features: [capabilities] - do: indices.create: - index: test + index: test body: settings: number_of_shards: 1 @@ -27,58 +27,58 @@ setup: --- "Basic ESQL usage output (telemetry) snapshot version": - requires: - test_runner_features: [ capabilities ] + test_runner_features: [capabilities] capabilities: - method: POST path: /_query - parameters: [ ] + parameters: [] # A snapshot function was removed in match_function_options, it can't work on mixed cluster tests otherwise. - capabilities: [ snapshot_test_for_telemetry, fn_byte_length, match_function_options, first_over_time ] + capabilities: [ snapshot_test_for_telemetry, fn_byte_length, match_function_options, first_over_time] reason: "Test that should only be executed on snapshot versions" - - do: { xpack.usage: { } } + - do: {xpack.usage: {}} - match: { esql.available: true } - match: { esql.enabled: true } - length: { esql.features: 26 } - - set: { esql.features.dissect: dissect_counter } - - set: { esql.features.drop: drop_counter } - - set: { esql.features.eval: eval_counter } - - set: { esql.features.enrich: enrich_counter } - - set: { esql.features.from: from_counter } - - set: { esql.features.grok: grok_counter } - - set: { esql.features.keep: keep_counter } - - set: { esql.features.limit: limit_counter } - - set: { esql.features.mv_expand: mv_expand_counter } - - set: { esql.features.rename: rename_counter } - - set: { esql.features.row: row_counter } - - set: { esql.features.show: show_counter } - - set: { esql.features.sort: sort_counter } - - set: { esql.features.stats: stats_counter } - - set: { esql.features.where: where_counter } - - set: { esql.features.lookup_join: lookup_join_counter } - - set: { esql.features.lookup: lookup_counter } - - set: { esql.features.change_point: change_point_counter } - - set: { esql.features.inlinestats: inlinestats_counter } - - set: { esql.features.rerank: rerank_counter } - - set: { esql.features.dedup: dedup_counter } - - set: { esql.features.insist: insist_counter } - - set: { esql.features.fork: fork_counter } - - set: { esql.features.rrf: rrf_counter } - - set: { esql.features.completion: completion_counter } - - set: { esql.features.sample: sample_counter } + - set: {esql.features.dissect: dissect_counter} + - set: {esql.features.drop: drop_counter} + - set: {esql.features.eval: eval_counter} + - set: {esql.features.enrich: enrich_counter} + - set: {esql.features.from: from_counter} + - set: {esql.features.grok: grok_counter} + - set: {esql.features.keep: keep_counter} + - set: {esql.features.limit: limit_counter} + - set: {esql.features.mv_expand: mv_expand_counter} + - set: {esql.features.rename: rename_counter} + - set: {esql.features.row: row_counter} + - set: {esql.features.show: show_counter} + - set: {esql.features.sort: sort_counter} + - set: {esql.features.stats: stats_counter} + - set: {esql.features.where: where_counter} + - set: {esql.features.lookup_join: lookup_join_counter} + - set: {esql.features.lookup: lookup_counter} + - set: {esql.features.change_point: change_point_counter} + - set: {esql.features.inlinestats: inlinestats_counter} + - set: {esql.features.rerank: rerank_counter} + - set: {esql.features.dedup: dedup_counter} + - set: {esql.features.insist: insist_counter} + - set: {esql.features.fork: fork_counter} + - set: {esql.features.rrf: rrf_counter} + - set: {esql.features.completion: completion_counter} + - set: {esql.features.sample: sample_counter} - length: { esql.queries: 3 } - - set: { esql.queries.rest.total: rest_total_counter } - - set: { esql.queries.rest.failed: rest_failed_counter } - - set: { esql.queries.kibana.total: kibana_total_counter } - - set: { esql.queries.kibana.failed: kibana_failed_counter } - - set: { esql.queries._all.total: all_total_counter } - - set: { esql.queries._all.failed: all_failed_counter } - - set: { esql.functions.max: functions_max } - - set: { esql.functions.min: functions_min } - - set: { esql.functions.cos: functions_cos } - - set: { esql.functions.to_long: functions_to_long } - - set: { esql.functions.coalesce: functions_coalesce } - - set: { esql.functions.categorize: functions_categorize } + - set: {esql.queries.rest.total: rest_total_counter} + - set: {esql.queries.rest.failed: rest_failed_counter} + - set: {esql.queries.kibana.total: kibana_total_counter} + - set: {esql.queries.kibana.failed: kibana_failed_counter} + - set: {esql.queries._all.total: all_total_counter} + - set: {esql.queries._all.failed: all_failed_counter} + - set: {esql.functions.max: functions_max} + - set: {esql.functions.min: functions_min} + - set: {esql.functions.cos: functions_cos} + - set: {esql.functions.to_long: functions_to_long} + - set: {esql.functions.coalesce: functions_coalesce} + - set: {esql.functions.categorize: functions_categorize} - do: esql.query: @@ -89,96 +89,96 @@ setup: | limit 5 | stats m = max(data) by categorize(message)' - - do: { xpack.usage: { } } + - do: {xpack.usage: {}} - match: { esql.available: true } - match: { esql.enabled: true } - - match: { esql.features.dissect: $dissect_counter } - - match: { esql.features.eval: $eval_counter } - - match: { esql.features.grok: $grok_counter } - - gt: { esql.features.limit: $limit_counter } - - gt: { esql.features.sort: $sort_counter } - - gt: { esql.features.stats: $stats_counter } - - gt: { esql.features.where: $where_counter } - - match: { esql.features.lookup_join: $lookup_join_counter } - - match: { esql.features.lookup: $lookup_counter } - - match: { esql.features.change_point: $change_point_counter } - - match: { esql.features.inlinestats: $inlinestats_counter } - - match: { esql.features.rerank: $rerank_counter } - - match: { esql.features.dedup: $dedup_counter } - - match: { esql.features.insist: $insist_counter } - - match: { esql.features.fork: $fork_counter } - - match: { esql.features.rrf: $rrf_counter } - - match: { esql.features.completion: $completion_counter } - - match: { esql.features.sample: $sample_counter } - - gt: { esql.queries.rest.total: $rest_total_counter } - - match: { esql.queries.rest.failed: $rest_failed_counter } - - match: { esql.queries.kibana.total: $kibana_total_counter } - - match: { esql.queries.kibana.failed: $kibana_failed_counter } - - gt: { esql.queries._all.total: $all_total_counter } - - match: { esql.queries._all.failed: $all_failed_counter } - - gt: { esql.functions.max: $functions_max } - - match: { esql.functions.min: $functions_min } - - match: { esql.functions.cos: $functions_cos } - - gt: { esql.functions.to_long: $functions_to_long } - - match: { esql.functions.coalesce: $functions_coalesce } - - gt: { esql.functions.categorize: $functions_categorize } + - match: {esql.features.dissect: $dissect_counter} + - match: {esql.features.eval: $eval_counter} + - match: {esql.features.grok: $grok_counter} + - gt: {esql.features.limit: $limit_counter} + - gt: {esql.features.sort: $sort_counter} + - gt: {esql.features.stats: $stats_counter} + - gt: {esql.features.where: $where_counter} + - match: {esql.features.lookup_join: $lookup_join_counter} + - match: {esql.features.lookup: $lookup_counter} + - match: {esql.features.change_point: $change_point_counter} + - match: {esql.features.inlinestats: $inlinestats_counter} + - match: {esql.features.rerank: $rerank_counter} + - match: {esql.features.dedup: $dedup_counter} + - match: {esql.features.insist: $insist_counter} + - match: {esql.features.fork: $fork_counter} + - match: {esql.features.rrf: $rrf_counter} + - match: {esql.features.completion: $completion_counter} + - match: {esql.features.sample: $sample_counter} + - gt: {esql.queries.rest.total: $rest_total_counter} + - match: {esql.queries.rest.failed: $rest_failed_counter} + - match: {esql.queries.kibana.total: $kibana_total_counter} + - match: {esql.queries.kibana.failed: $kibana_failed_counter} + - gt: {esql.queries._all.total: $all_total_counter} + - match: {esql.queries._all.failed: $all_failed_counter} + - gt: {esql.functions.max: $functions_max} + - match: {esql.functions.min: $functions_min} + - match: {esql.functions.cos: $functions_cos} + - gt: {esql.functions.to_long: $functions_to_long} + - match: {esql.functions.coalesce: $functions_coalesce} + - gt: {esql.functions.categorize: $functions_categorize} # Testing for the entire function set isn't feasible, so we just check that we return the correct count as an approximation. - - length: { esql.functions: 142 } # check the "sister" test below for a likely update to the same esql.functions length check + - length: {esql.functions: 142} # check the "sister" test below for a likely update to the same esql.functions length check --- "Basic ESQL usage output (telemetry) non-snapshot version": - requires: - test_runner_features: [ capabilities ] + test_runner_features: [capabilities] capabilities: - method: POST path: /_query - parameters: [ ] + parameters: [] capabilities: [ non_snapshot_test_for_telemetry, fn_byte_length ] reason: "Test that should only be executed on release versions" - - do: { xpack.usage: { } } + - do: {xpack.usage: {}} - match: { esql.available: true } - match: { esql.enabled: true } - length: { esql.features: 26 } - - set: { esql.features.dissect: dissect_counter } - - set: { esql.features.drop: drop_counter } - - set: { esql.features.eval: eval_counter } - - set: { esql.features.enrich: enrich_counter } - - set: { esql.features.from: from_counter } - - set: { esql.features.grok: grok_counter } - - set: { esql.features.keep: keep_counter } - - set: { esql.features.limit: limit_counter } - - set: { esql.features.mv_expand: mv_expand_counter } - - set: { esql.features.rename: rename_counter } - - set: { esql.features.row: row_counter } - - set: { esql.features.show: show_counter } - - set: { esql.features.sort: sort_counter } - - set: { esql.features.stats: stats_counter } - - set: { esql.features.where: where_counter } - - set: { esql.features.lookup_join: lookup_join_counter } - - set: { esql.features.lookup: lookup_counter } - - set: { esql.features.change_point: change_point_counter } - - set: { esql.features.inlinestats: inlinestats_counter } - - set: { esql.features.rerank: rerank_counter } - - set: { esql.features.dedup: dedup_counter } - - set: { esql.features.insist: insist_counter } - - set: { esql.features.fork: fork_counter } - - set: { esql.features.rrf: rrf_counter } - - set: { esql.features.completion: completion_counter } - - set: { esql.features.sample: sample_counter } + - set: {esql.features.dissect: dissect_counter} + - set: {esql.features.drop: drop_counter} + - set: {esql.features.eval: eval_counter} + - set: {esql.features.enrich: enrich_counter} + - set: {esql.features.from: from_counter} + - set: {esql.features.grok: grok_counter} + - set: {esql.features.keep: keep_counter} + - set: {esql.features.limit: limit_counter} + - set: {esql.features.mv_expand: mv_expand_counter} + - set: {esql.features.rename: rename_counter} + - set: {esql.features.row: row_counter} + - set: {esql.features.show: show_counter} + - set: {esql.features.sort: sort_counter} + - set: {esql.features.stats: stats_counter} + - set: {esql.features.where: where_counter} + - set: {esql.features.lookup_join: lookup_join_counter} + - set: {esql.features.lookup: lookup_counter} + - set: {esql.features.change_point: change_point_counter} + - set: {esql.features.inlinestats: inlinestats_counter} + - set: {esql.features.rerank: rerank_counter} + - set: {esql.features.dedup: dedup_counter} + - set: {esql.features.insist: insist_counter} + - set: {esql.features.fork: fork_counter} + - set: {esql.features.rrf: rrf_counter} + - set: {esql.features.completion: completion_counter} + - set: {esql.features.sample: sample_counter} - length: { esql.queries: 3 } - - set: { esql.queries.rest.total: rest_total_counter } - - set: { esql.queries.rest.failed: rest_failed_counter } - - set: { esql.queries.kibana.total: kibana_total_counter } - - set: { esql.queries.kibana.failed: kibana_failed_counter } - - set: { esql.queries._all.total: all_total_counter } - - set: { esql.queries._all.failed: all_failed_counter } - - set: { esql.functions.max: functions_max } - - set: { esql.functions.min: functions_min } - - set: { esql.functions.cos: functions_cos } - - set: { esql.functions.to_long: functions_to_long } - - set: { esql.functions.coalesce: functions_coalesce } - - set: { esql.functions.categorize: functions_categorize } + - set: {esql.queries.rest.total: rest_total_counter} + - set: {esql.queries.rest.failed: rest_failed_counter} + - set: {esql.queries.kibana.total: kibana_total_counter} + - set: {esql.queries.kibana.failed: kibana_failed_counter} + - set: {esql.queries._all.total: all_total_counter} + - set: {esql.queries._all.failed: all_failed_counter} + - set: {esql.functions.max: functions_max} + - set: {esql.functions.min: functions_min} + - set: {esql.functions.cos: functions_cos} + - set: {esql.functions.to_long: functions_to_long} + - set: {esql.functions.coalesce: functions_coalesce} + - set: {esql.functions.categorize: functions_categorize} - do: esql.query: @@ -189,52 +189,52 @@ setup: | limit 5 | stats m = max(data) by categorize(message)' - - do: { xpack.usage: { } } + - do: {xpack.usage: {}} - match: { esql.available: true } - match: { esql.enabled: true } - - match: { esql.features.dissect: $dissect_counter } - - match: { esql.features.eval: $eval_counter } - - match: { esql.features.grok: $grok_counter } - - gt: { esql.features.limit: $limit_counter } - - gt: { esql.features.sort: $sort_counter } - - gt: { esql.features.stats: $stats_counter } - - gt: { esql.features.where: $where_counter } - - match: { esql.features.lookup_join: $lookup_join_counter } - - match: { esql.features.lookup: $lookup_counter } - - match: { esql.features.change_point: $change_point_counter } - - match: { esql.features.inlinestats: $inlinestats_counter } - - match: { esql.features.rerank: $rerank_counter } - - match: { esql.features.dedup: $dedup_counter } - - match: { esql.features.insist: $insist_counter } - - match: { esql.features.fork: $fork_counter } - - match: { esql.features.rrf: $rrf_counter } - - match: { esql.features.completion: $completion_counter } - - gt: { esql.queries.rest.total: $rest_total_counter } - - match: { esql.queries.rest.failed: $rest_failed_counter } - - match: { esql.queries.kibana.total: $kibana_total_counter } - - match: { esql.queries.kibana.failed: $kibana_failed_counter } - - gt: { esql.queries._all.total: $all_total_counter } - - match: { esql.queries._all.failed: $all_failed_counter } - - gt: { esql.functions.max: $functions_max } - - match: { esql.functions.min: $functions_min } - - match: { esql.functions.cos: $functions_cos } - - gt: { esql.functions.to_long: $functions_to_long } - - match: { esql.functions.coalesce: $functions_coalesce } - - gt: { esql.functions.categorize: $functions_categorize } - - length: { esql.functions: 132 } # check the "sister" test above for a likely update to the same esql.functions length check + - match: {esql.features.dissect: $dissect_counter} + - match: {esql.features.eval: $eval_counter} + - match: {esql.features.grok: $grok_counter} + - gt: {esql.features.limit: $limit_counter} + - gt: {esql.features.sort: $sort_counter} + - gt: {esql.features.stats: $stats_counter} + - gt: {esql.features.where: $where_counter} + - match: {esql.features.lookup_join: $lookup_join_counter} + - match: {esql.features.lookup: $lookup_counter} + - match: {esql.features.change_point: $change_point_counter} + - match: {esql.features.inlinestats: $inlinestats_counter} + - match: {esql.features.rerank: $rerank_counter} + - match: {esql.features.dedup: $dedup_counter} + - match: {esql.features.insist: $insist_counter} + - match: {esql.features.fork: $fork_counter} + - match: {esql.features.rrf: $rrf_counter} + - match: {esql.features.completion: $completion_counter} + - gt: {esql.queries.rest.total: $rest_total_counter} + - match: {esql.queries.rest.failed: $rest_failed_counter} + - match: {esql.queries.kibana.total: $kibana_total_counter} + - match: {esql.queries.kibana.failed: $kibana_failed_counter} + - gt: {esql.queries._all.total: $all_total_counter} + - match: {esql.queries._all.failed: $all_failed_counter} + - gt: {esql.functions.max: $functions_max} + - match: {esql.functions.min: $functions_min} + - match: {esql.functions.cos: $functions_cos} + - gt: {esql.functions.to_long: $functions_to_long} + - match: {esql.functions.coalesce: $functions_coalesce} + - gt: {esql.functions.categorize: $functions_categorize} + - length: {esql.functions: 132} # check the "sister" test above for a likely update to the same esql.functions length check --- took: - requires: - test_runner_features: [ capabilities ] + test_runner_features: [capabilities] capabilities: - method: POST path: /_query - parameters: [ ] - capabilities: [ usage_contains_took ] + parameters: [] + capabilities: [usage_contains_took] reason: "Test that should only be executed on snapshot versions" - - do: { xpack.usage: { } } + - do: {xpack.usage: {}} - exists: esql.took.lt_10ms - exists: esql.took.lt_1s - exists: esql.took.lt_10s From 7f8d66344cf222c811e0174309ac371a29fba6dd Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 20 May 2025 13:26:31 -0400 Subject: [PATCH 13/42] PR feedback - refactor/consolidate code --- .../function/fulltext/FullTextFunction.java | 50 +++++++++++++++++++ .../expression/function/fulltext/Match.java | 46 ++--------------- .../function/fulltext/MatchPhrase.java | 46 ++--------------- .../function/fulltext/QueryString.java | 26 ++-------- 4 files changed, 62 insertions(+), 106 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 7f4b3c2be7787..3310bc0183822 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -21,6 +21,7 @@ import org.elasticsearch.xpack.esql.core.InvalidArgumentException; import org.elasticsearch.xpack.esql.core.expression.EntryExpression; import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.FoldContext; import org.elasticsearch.xpack.esql.core.expression.Literal; import org.elasticsearch.xpack.esql.core.expression.MapExpression; @@ -31,7 +32,9 @@ import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.DataTypeConverter; +import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.evaluator.mapper.EvaluatorMapper; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; import org.elasticsearch.xpack.esql.expression.predicate.logical.BinaryLogic; import org.elasticsearch.xpack.esql.expression.predicate.logical.Not; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; @@ -56,7 +59,10 @@ import static org.elasticsearch.common.logging.LoggerMessageFormat.format; import static org.elasticsearch.xpack.esql.common.Failure.fail; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.DEFAULT; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isFoldable; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; @@ -389,4 +395,48 @@ protected static void populateOptionsMap( } } } + + protected TypeResolution resolveOptions(Expression options) { + if (options != null) { + TypeResolution resolution = isNotNull(options, sourceText(), THIRD); + if (resolution.unresolved()) { + return resolution; + } + // MapExpression does not have a DataType associated with it + resolution = isMapExpression(options, sourceText(), THIRD); + if (resolution.unresolved()) { + return resolution; + } + + try { + resolvedOptions(); + } catch (InvalidArgumentException e) { + return new TypeResolution(e.getMessage()); + } + } + return TypeResolution.TYPE_RESOLVED; + } + + protected Map resolvedOptions() throws InvalidArgumentException { + return Map.of(); + } + + public static String getNameFromFieldAttribute(FieldAttribute fieldAttribute) { + String fieldName = fieldAttribute.name(); + if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { + // If we have multiple field types, we allow the query to be done, but getting the underlying field name + fieldName = multiTypeEsField.getName(); + } + return fieldName; + } + + public static FieldAttribute fieldAsFieldAttribute(Expression field) { + Expression fieldExpression = field; + // Field may be converted to other data type (field_name :: data_type), so we need to check the original field + if (fieldExpression instanceof AbstractConvertFunction convertFunction) { + fieldExpression = convertFunction.field(); + } + return fieldExpression instanceof FieldAttribute fieldAttribute ? fieldAttribute : null; + } + } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java index bd5efb7279a99..b46342cecc825 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java @@ -26,7 +26,6 @@ import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; -import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.core.util.Check; import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.Example; @@ -36,7 +35,6 @@ import org.elasticsearch.xpack.esql.expression.function.MapParam; import org.elasticsearch.xpack.esql.expression.function.OptionalArgument; import org.elasticsearch.xpack.esql.expression.function.Param; -import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; @@ -65,8 +63,6 @@ import static org.elasticsearch.index.query.MatchQueryBuilder.ZERO_TERMS_QUERY_FIELD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; @@ -303,7 +299,7 @@ public final void writeTo(StreamOutput out) throws IOException { @Override protected TypeResolution resolveParams() { - return resolveField().and(resolveQuery()).and(resolveOptions()).and(checkParamCompatibility()); + return resolveField().and(resolveQuery()).and(resolveOptions(options())).and(checkParamCompatibility()); } private TypeResolution resolveField() { @@ -347,25 +343,9 @@ private TypeResolution checkParamCompatibility() { return new TypeResolution(formatIncompatibleTypesMessage(fieldType, queryType, sourceText())); } - private TypeResolution resolveOptions() { - if (options() != null) { - TypeResolution resolution = isNotNull(options(), sourceText(), THIRD); - if (resolution.unresolved()) { - return resolution; - } - // MapExpression does not have a DataType associated with it - resolution = isMapExpression(options(), sourceText(), THIRD); - if (resolution.unresolved()) { - return resolution; - } - - try { - matchQueryOptions(); - } catch (InvalidArgumentException e) { - return new TypeResolution(e.getMessage()); - } - } - return TypeResolution.TYPE_RESOLVED; + @Override + protected Map resolvedOptions() { + return matchQueryOptions(); } private Map matchQueryOptions() throws InvalidArgumentException { @@ -465,24 +445,6 @@ protected Query translate(TranslatorHandler handler) { return new MatchQuery(source(), fieldName, queryAsObject(), matchQueryOptions()); } - public static String getNameFromFieldAttribute(FieldAttribute fieldAttribute) { - String fieldName = fieldAttribute.name(); - if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { - // If we have multiple field types, we allow the query to be done, but getting the underlying field name - fieldName = multiTypeEsField.getName(); - } - return fieldName; - } - - public static FieldAttribute fieldAsFieldAttribute(Expression field) { - Expression fieldExpression = field; - // Field may be converted to other data type (field_name :: data_type), so we need to check the original field - if (fieldExpression instanceof AbstractConvertFunction convertFunction) { - fieldExpression = convertFunction.field(); - } - return fieldExpression instanceof FieldAttribute fieldAttribute ? fieldAttribute : null; - } - private FieldAttribute fieldAsFieldAttribute() { return fieldAsFieldAttribute(field); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index c846e858d3b35..c8039e421ecd1 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -24,7 +24,6 @@ import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; -import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.core.util.Check; import org.elasticsearch.xpack.esql.expression.function.Example; import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesTo; @@ -33,7 +32,6 @@ import org.elasticsearch.xpack.esql.expression.function.MapParam; import org.elasticsearch.xpack.esql.expression.function.OptionalArgument; import org.elasticsearch.xpack.esql.expression.function.Param; -import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; @@ -55,8 +53,6 @@ import static org.elasticsearch.index.query.MatchQueryBuilder.ANALYZER_FIELD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; @@ -199,7 +195,7 @@ public final void writeTo(StreamOutput out) throws IOException { @Override protected TypeResolution resolveParams() { - return resolveField().and(resolveQuery()).and(resolveOptions()).and(checkParamCompatibility()); + return resolveField().and(resolveQuery()).and(resolveOptions(options())).and(checkParamCompatibility()); } private TypeResolution resolveField() { @@ -226,25 +222,9 @@ private TypeResolution checkParamCompatibility() { return new TypeResolution(formatIncompatibleTypesMessage(fieldType, queryType, sourceText())); } - private TypeResolution resolveOptions() { - if (options() != null) { - TypeResolution resolution = isNotNull(options(), sourceText(), THIRD); - if (resolution.unresolved()) { - return resolution; - } - // MapExpression does not have a DataType associated with it - resolution = isMapExpression(options(), sourceText(), THIRD); - if (resolution.unresolved()) { - return resolution; - } - - try { - matchPhraseQueryOptions(); - } catch (InvalidArgumentException e) { - return new TypeResolution(e.getMessage()); - } - } - return TypeResolution.TYPE_RESOLVED; + @Override + protected Map resolvedOptions() throws InvalidArgumentException { + return matchPhraseQueryOptions(); } private Map matchPhraseQueryOptions() throws InvalidArgumentException { @@ -338,24 +318,6 @@ protected Query translate(TranslatorHandler handler) { return new MatchPhraseQuery(source(), fieldName, queryAsObject(), matchPhraseQueryOptions()); } - public static String getNameFromFieldAttribute(FieldAttribute fieldAttribute) { - String fieldName = fieldAttribute.name(); - if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { - // If we have multiple field types, we allow the query to be done, but getting the underlying field name - fieldName = multiTypeEsField.getName(); - } - return fieldName; - } - - public static FieldAttribute fieldAsFieldAttribute(Expression field) { - Expression fieldExpression = field; - // Field may be converted to other data type (field_name :: data_type), so we need to check the original field - if (fieldExpression instanceof AbstractConvertFunction convertFunction) { - fieldExpression = convertFunction.field(); - } - return fieldExpression instanceof FieldAttribute fieldAttribute ? fieldAttribute : null; - } - private FieldAttribute fieldAsFieldAttribute() { return fieldAsFieldAttribute(field); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java index 63a463d1e25a0..6f702ecc43484 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java @@ -60,8 +60,6 @@ import static org.elasticsearch.index.query.QueryStringQueryBuilder.TIME_ZONE_FIELD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; @@ -328,30 +326,14 @@ private Map queryStringOptions() throws InvalidArgumentException return matchOptions; } - private TypeResolution resolveOptions() { - if (options() != null) { - TypeResolution resolution = isNotNull(options(), sourceText(), SECOND); - if (resolution.unresolved()) { - return resolution; - } - // MapExpression does not have a DataType associated with it - resolution = isMapExpression(options(), sourceText(), SECOND); - if (resolution.unresolved()) { - return resolution; - } - - try { - queryStringOptions(); - } catch (InvalidArgumentException e) { - return new TypeResolution(e.getMessage()); - } - } - return TypeResolution.TYPE_RESOLVED; + @Override + protected Map resolvedOptions() { + return queryStringOptions(); } @Override protected TypeResolution resolveParams() { - return resolveQuery().and(resolveOptions()); + return resolveQuery().and(resolveOptions(options())); } @Override From e0ae8891594a71c6fefe025381c1021a86f16430 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 20 May 2025 14:15:32 -0400 Subject: [PATCH 14/42] Add some more tests --- .../esql/plugin/MatchPhraseFunctionIT.java | 315 ++++++++++++++++++ .../xpack/esql/analysis/VerifierTests.java | 26 ++ 2 files changed, 341 insertions(+) create mode 100644 x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java new file mode 100644 index 0000000000000..0758402dafe7b --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java @@ -0,0 +1,315 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plugin; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.xpack.esql.VerificationException; +import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.hamcrest.Matchers; +import org.junit.Before; + +import java.util.List; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.xpack.esql.EsqlTestUtils.getValuesList; +import static org.hamcrest.CoreMatchers.containsString; + +//@TestLogging(value = "org.elasticsearch.xpack.esql:TRACE,org.elasticsearch.compute:TRACE", reason = "debug") +public class MatchPhraseFunctionIT extends AbstractEsqlIntegTestCase { + + @Before + public void setupIndex() { + createAndPopulateIndex(); + } + + public void testSimpleWhereMatchPhrase() { + var query = """ + FROM test + | WHERE match_phrase(content, "brown fox") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(6))); + } + } + + public void testCombinedWhereMatchPhrase() { + var query = """ + FROM test + | WHERE match_phrase(content, "brown fox") AND id > 5 + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(6))); + } + } + + public void testMultipleMatchPhrase() { + var query = """ + FROM test + | WHERE match_phrase(content, "the quick") AND match_phrase(content, "brown fox") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(6))); + } + } + + public void testMultipleWhereMatchPhrase() { + var query = """ + FROM test + | WHERE match_phrase(content, "the quick") AND match_phrase(content, "brown fox") + | EVAL summary = CONCAT("document with id: ", to_str(id), "and content: ", content) + | SORT summary + | LIMIT 4 + | WHERE match_phrase(content, "lazy dog") + | KEEP id + """; + + var error = expectThrows(ElasticsearchException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("[MATCH_PHRASE] function cannot be used after LIMIT")); + } + + public void testNotWhereMatchPhrase() { + var query = """ + FROM test + | WHERE NOT match_phrase(content, "brown fox") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(5))); + } + } + + public void testWhereMatchPhraseWithScoring() { + var query = """ + FROM test + METADATA _score + | WHERE match_phrase(content, "brown fox") + | KEEP id, _score + | SORT id ASC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testWhereMatchPhraseWithScoringDifferentSort() { + + var query = """ + FROM test + METADATA _score + | WHERE match_phrase(content, "brown fox") + | KEEP id, _score + | SORT id DESC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(6, 0.9114001989364624), List.of(1, 1.156558871269226))); + } + } + + public void testWhereMatchPhraseWithScoringSortScore() { + var query = """ + FROM test + METADATA _score + | WHERE match_phrase(content, "brown fox") + | KEEP id, _score + | SORT _score DESC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testWhereMatchPhraseWithScoringNoSort() { + var query = """ + FROM test + METADATA _score + | WHERE match_phrase(content, "brown fox") + | KEEP id, _score + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValuesInAnyOrder(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testNonExistingColumn() { + var query = """ + FROM test + | WHERE match_phrase(something, "brown fox") + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("Unknown column [something]")); + } + + public void testWhereMatchPhraseEvalColumn() { + var query = """ + FROM test + | EVAL upper_content = to_upper(content) + | WHERE match_phrase(upper_content, "BROWN FOX") + | KEEP id + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString("[MATCH_PHRASE] function cannot operate on [upper_content], which is not a field from an index mapping") + ); + } + + public void testWhereMatchPhraseOverWrittenColumn() { + var query = """ + FROM test + | DROP content + | EVAL content = CONCAT("document with ID ", to_str(id)) + | WHERE match_phrase(content, "document content") + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString("[MATCH_PHRASE] function cannot operate on [content], which is not a field from an index mapping") + ); + } + + public void testWhereMatchPhraseAfterStats() { + var query = """ + FROM test + | STATS count(*) + | WHERE match_phrase(content, "brown fox") + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("Unknown column [content]")); + } + + public void testWhereMatchPhraseNotPushedDown() { + var query = """ + FROM test + | WHERE match(content, "brown fox") OR length(content) < 20 + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(2), List.of(6))); + } + } + + public void testWhereMatchPhraseWithRow() { + var query = """ + ROW content = "a brown fox" + | WHERE match(content, "brown fox") + """; + + var error = expectThrows(ElasticsearchException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString("line 2:15: [MATCH_PHRASE] function cannot operate on [content], which is not a field from an index mapping") + ); + } + + public void testMatchPhraseithStats() { + var errorQuery = """ + FROM test + | STATS c = count(*) BY match_phrase(content, "brown fox") + """; + + var error = expectThrows(ElasticsearchException.class, () -> run(errorQuery)); + assertThat(error.getMessage(), containsString("[MATCH_PHRASE] function is only supported in WHERE and STATS commands")); + + var query = """ + FROM test + | STATS c = count(*) WHERE match_phrase(content, "brown fox"), d = count(*) WHERE match_phrase(content, "lazy dog") + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("c", "d")); + assertColumnTypes(resp.columns(), List.of("long", "long")); + assertValues(resp.values(), List.of(List.of(2L, 4L))); + } + + query = """ + FROM test METADATA _score + | WHERE match_phrase(content, "brown fox") + | STATS m = max(_score), n = min(_score) + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("m", "n")); + assertColumnTypes(resp.columns(), List.of("double", "double")); + List> valuesList = getValuesList(resp.values()); + assertEquals(1, valuesList.size()); + assertThat((double) valuesList.get(0).get(0), Matchers.greaterThan(1.0)); + assertThat((double) valuesList.get(0).get(1), Matchers.greaterThan(0.0)); + } + } + + public void testMatchPhraseWithinEval() { + var query = """ + FROM test + | EVAL matches_query = match_phrase(content, "brown fox") + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("[MATCH_PHRASE] function is only supported in WHERE and STATS commands")); + } + + private void createAndPopulateIndex() { + var indexName = "test"; + var client = client().admin().indices(); + var CreateRequest = client.prepareCreate(indexName) + .setSettings(Settings.builder().put("index.number_of_shards", 1)) + .setMapping("id", "type=integer", "content", "type=text"); + assertAcked(CreateRequest); + client().prepareBulk() + .add(new IndexRequest(indexName).id("1").source("id", 1, "content", "This is a brown fox")) + .add(new IndexRequest(indexName).id("2").source("id", 2, "content", "This is a brown dog")) + .add(new IndexRequest(indexName).id("3").source("id", 3, "content", "This dog is really brown")) + .add(new IndexRequest(indexName).id("4").source("id", 4, "content", "The dog is brown but this document is very very long")) + .add(new IndexRequest(indexName).id("5").source("id", 5, "content", "There is also a white cat")) + .add(new IndexRequest(indexName).id("6").source("id", 6, "content", "The quick brown fox jumps over the lazy dog")) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + ensureYellow(indexName); + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index d76a355a6c9a9..c39f6a7a9f81a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1215,6 +1215,14 @@ public void testMatchInsideEval() throws Exception { ); } + public void testMatchPhraseInsideEval() throws Exception { + assertEquals( + "1:36: [:] operator is only supported in WHERE and STATS commands\n" + + "line 1:36: [:] operator cannot operate on [title], which is not a field from an index mapping", + error("row title = \"brown fox\" | eval x = match_phrase(title:\"fox\") ") + ); + } + public void testMatchFunctionNotAllowedAfterCommands() throws Exception { assertEquals( "1:24: [MATCH] function cannot be used after LIMIT", @@ -1251,6 +1259,17 @@ public void testMatchWithNonIndexedColumnCurrentlyUnsupported() { ); } + public void testMatchPhraseWithNonIndexedColumnCurrentlyUnsupported() { + assertEquals( + "1:67: [MATCH_PHRASE] function cannot operate on [initial], which is not a field from an index mapping", + error("from test | eval initial = substring(first_name, 1) | where match_phrase(initial, \"A\")") + ); + assertEquals( + "1:67: [MATCH_PHRASE] function cannot operate on [text], which is not a field from an index mapping", + error("from test | eval text=concat(first_name, last_name) | where match_phrase(text, \"cat\")") + ); + } + public void testMatchFunctionIsNotNullable() { assertEquals( "1:48: [MATCH] function cannot operate on [text::keyword], which is not a field from an index mapping", @@ -1258,6 +1277,13 @@ public void testMatchFunctionIsNotNullable() { ); } + public void testMatchPhraseFunctionIsNotNullable() { + assertEquals( + "1:48: [MATCH_PHRASE] function cannot operate on [text::keyword], which is not a field from an index mapping", + error("row n = null | eval text = n + 5 | where match_phrase(text::keyword, \"Anna\")") + ); + } + public void testQueryStringFunctionsNotAllowedAfterCommands() throws Exception { // Source commands assertEquals("1:13: [QSTR] function cannot be used after SHOW", error("show info | where qstr(\"8.16.0\")")); From 9ebdda22830a2481b9b74a3e2d17ca8b910222c6 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 20 May 2025 15:58:10 -0400 Subject: [PATCH 15/42] Fix some tests --- .../function/fulltext/FullTextFunction.java | 6 +++--- .../esql/expression/function/fulltext/Match.java | 3 ++- .../expression/function/fulltext/MatchPhrase.java | 8 +++++++- .../expression/function/fulltext/QueryString.java | 2 +- .../xpack/esql/analysis/VerifierTests.java | 12 ++++++------ 5 files changed, 19 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 3310bc0183822..2f757ab9af5f8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -396,14 +396,14 @@ protected static void populateOptionsMap( } } - protected TypeResolution resolveOptions(Expression options) { + protected TypeResolution resolveOptions(Expression options, TypeResolutions.ParamOrdinal paramOrdinal) { if (options != null) { - TypeResolution resolution = isNotNull(options, sourceText(), THIRD); + TypeResolution resolution = isNotNull(options, sourceText(), paramOrdinal); if (resolution.unresolved()) { return resolution; } // MapExpression does not have a DataType associated with it - resolution = isMapExpression(options, sourceText(), THIRD); + resolution = isMapExpression(options, sourceText(), paramOrdinal); if (resolution.unresolved()) { return resolution; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java index b46342cecc825..da42e7e60af94 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java @@ -63,6 +63,7 @@ import static org.elasticsearch.index.query.MatchQueryBuilder.ZERO_TERMS_QUERY_FIELD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; @@ -299,7 +300,7 @@ public final void writeTo(StreamOutput out) throws IOException { @Override protected TypeResolution resolveParams() { - return resolveField().and(resolveQuery()).and(resolveOptions(options())).and(checkParamCompatibility()); + return resolveField().and(resolveQuery()).and(resolveOptions(options(), THIRD)).and(checkParamCompatibility()); } private TypeResolution resolveField() { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index c8039e421ecd1..56fc29695cc41 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -53,6 +53,7 @@ import static org.elasticsearch.index.query.MatchQueryBuilder.ANALYZER_FIELD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; @@ -176,6 +177,11 @@ public String getWriteableName() { return ENTRY.name; } + @Override + public String functionName() { + return ENTRY.name; + } + private static MatchPhrase readFrom(StreamInput in) throws IOException { Source source = Source.readFrom((PlanStreamInput) in); Expression field = in.readNamedWriteable(Expression.class); @@ -195,7 +201,7 @@ public final void writeTo(StreamOutput out) throws IOException { @Override protected TypeResolution resolveParams() { - return resolveField().and(resolveQuery()).and(resolveOptions(options())).and(checkParamCompatibility()); + return resolveField().and(resolveQuery()).and(resolveOptions(options(), THIRD)).and(checkParamCompatibility()); } private TypeResolution resolveField() { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java index 6f702ecc43484..4b5a7547d3203 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryString.java @@ -333,7 +333,7 @@ protected Map resolvedOptions() { @Override protected TypeResolution resolveParams() { - return resolveQuery().and(resolveOptions(options())); + return resolveQuery().and(resolveOptions(options(), SECOND)); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index c39f6a7a9f81a..b13a72b7c1545 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1217,9 +1217,9 @@ public void testMatchInsideEval() throws Exception { public void testMatchPhraseInsideEval() throws Exception { assertEquals( - "1:36: [:] operator is only supported in WHERE and STATS commands\n" - + "line 1:36: [:] operator cannot operate on [title], which is not a field from an index mapping", - error("row title = \"brown fox\" | eval x = match_phrase(title:\"fox\") ") + "1:36: [MatchPhrase] function is only supported in WHERE and STATS commands\n" + + "line 1:49: [MatchPhrase] function cannot operate on [title], which is not a field from an index mapping", + error("row title = \"brown fox\" | eval x = match_phrase(title, \"brown fox\") ") ); } @@ -1261,11 +1261,11 @@ public void testMatchWithNonIndexedColumnCurrentlyUnsupported() { public void testMatchPhraseWithNonIndexedColumnCurrentlyUnsupported() { assertEquals( - "1:67: [MATCH_PHRASE] function cannot operate on [initial], which is not a field from an index mapping", + "1:74: [MatchPhrase] function cannot operate on [initial], which is not a field from an index mapping", error("from test | eval initial = substring(first_name, 1) | where match_phrase(initial, \"A\")") ); assertEquals( - "1:67: [MATCH_PHRASE] function cannot operate on [text], which is not a field from an index mapping", + "1:74: [MatchPhrase] function cannot operate on [text], which is not a field from an index mapping", error("from test | eval text=concat(first_name, last_name) | where match_phrase(text, \"cat\")") ); } @@ -1279,7 +1279,7 @@ public void testMatchFunctionIsNotNullable() { public void testMatchPhraseFunctionIsNotNullable() { assertEquals( - "1:48: [MATCH_PHRASE] function cannot operate on [text::keyword], which is not a field from an index mapping", + "1:55: [MatchPhrase] function cannot operate on [text::keyword], which is not a field from an index mapping", error("row n = null | eval text = n + 5 | where match_phrase(text::keyword, \"Anna\")") ); } From 45d6810330a345dcd379b85af4191b286e68b10f Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Tue, 20 May 2025 20:07:40 +0000 Subject: [PATCH 16/42] [CI] Auto commit changes from spotless --- .../esql/expression/function/fulltext/FullTextFunction.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 2f757ab9af5f8..7ea7ff0684a6d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -59,7 +59,6 @@ import static org.elasticsearch.common.logging.LoggerMessageFormat.format; import static org.elasticsearch.xpack.esql.common.Failure.fail; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.DEFAULT; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; From f71620c56e0ad647722867c67ff694891db0b7f0 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 21 May 2025 09:24:20 -0400 Subject: [PATCH 17/42] Fix tests --- .../esql/plugin/MatchPhraseFunctionIT.java | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java index 0758402dafe7b..80876d06ca6ac 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java @@ -71,7 +71,7 @@ public void testMultipleMatchPhrase() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("id")); assertColumnTypes(resp.columns(), List.of("integer")); - assertValues(resp.values(), List.of(List.of(1), List.of(6))); + assertValues(resp.values(), List.of(List.of(6))); } } @@ -87,7 +87,7 @@ public void testMultipleWhereMatchPhrase() { """; var error = expectThrows(ElasticsearchException.class, () -> run(query)); - assertThat(error.getMessage(), containsString("[MATCH_PHRASE] function cannot be used after LIMIT")); + assertThat(error.getMessage(), containsString("[MatchPhrase] function cannot be used after LIMIT")); } public void testNotWhereMatchPhrase() { @@ -101,7 +101,7 @@ public void testNotWhereMatchPhrase() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("id")); assertColumnTypes(resp.columns(), List.of("integer")); - assertValues(resp.values(), List.of(List.of(5))); + assertValues(resp.values(), List.of(List.of(2), List.of(3), List.of(4), List.of(5))); } } @@ -117,7 +117,7 @@ public void testWhereMatchPhraseWithScoring() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("id", "_score")); assertColumnTypes(resp.columns(), List.of("integer", "double")); - assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + assertValues(resp.values(), List.of(List.of(1, 1.4274532794952393), List.of(6, 1.1248723268508911))); } } @@ -134,7 +134,7 @@ public void testWhereMatchPhraseWithScoringDifferentSort() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("id", "_score")); assertColumnTypes(resp.columns(), List.of("integer", "double")); - assertValues(resp.values(), List.of(List.of(6, 0.9114001989364624), List.of(1, 1.156558871269226))); + assertValues(resp.values(), List.of(List.of(6, 1.1248723268508911), List.of(1, 1.4274532794952393))); } } @@ -150,7 +150,7 @@ public void testWhereMatchPhraseWithScoringSortScore() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("id", "_score")); assertColumnTypes(resp.columns(), List.of("integer", "double")); - assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + assertValues(resp.values(), List.of(List.of(1, 1.4274532794952393), List.of(6, 1.1248723268508911))); } } @@ -165,7 +165,7 @@ public void testWhereMatchPhraseWithScoringNoSort() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("id", "_score")); assertColumnTypes(resp.columns(), List.of("integer", "double")); - assertValuesInAnyOrder(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + assertValuesInAnyOrder(resp.values(), List.of(List.of(1, 1.4274532794952393), List.of(6, 1.1248723268508911))); } } @@ -190,7 +190,7 @@ public void testWhereMatchPhraseEvalColumn() { var error = expectThrows(VerificationException.class, () -> run(query)); assertThat( error.getMessage(), - containsString("[MATCH_PHRASE] function cannot operate on [upper_content], which is not a field from an index mapping") + containsString("[MatchPhrase] function cannot operate on [upper_content], which is not a field from an index mapping") ); } @@ -205,7 +205,7 @@ public void testWhereMatchPhraseOverWrittenColumn() { var error = expectThrows(VerificationException.class, () -> run(query)); assertThat( error.getMessage(), - containsString("[MATCH_PHRASE] function cannot operate on [content], which is not a field from an index mapping") + containsString("[MatchPhrase] function cannot operate on [content], which is not a field from an index mapping") ); } @@ -223,7 +223,7 @@ public void testWhereMatchPhraseAfterStats() { public void testWhereMatchPhraseNotPushedDown() { var query = """ FROM test - | WHERE match(content, "brown fox") OR length(content) < 20 + | WHERE match_phrase(content, "brown fox") OR length(content) < 20 | KEEP id | SORT id """; @@ -238,24 +238,24 @@ public void testWhereMatchPhraseNotPushedDown() { public void testWhereMatchPhraseWithRow() { var query = """ ROW content = "a brown fox" - | WHERE match(content, "brown fox") + | WHERE match_phrase(content, "brown fox") """; var error = expectThrows(ElasticsearchException.class, () -> run(query)); assertThat( error.getMessage(), - containsString("line 2:15: [MATCH_PHRASE] function cannot operate on [content], which is not a field from an index mapping") + containsString("line 2:22: [MatchPhrase] function cannot operate on [content], which is not a field from an index mapping") ); } - public void testMatchPhraseithStats() { + public void testMatchPhraseWithStats() { var errorQuery = """ FROM test | STATS c = count(*) BY match_phrase(content, "brown fox") """; var error = expectThrows(ElasticsearchException.class, () -> run(errorQuery)); - assertThat(error.getMessage(), containsString("[MATCH_PHRASE] function is only supported in WHERE and STATS commands")); + assertThat(error.getMessage(), containsString("[MatchPhrase] function is only supported in WHERE and STATS commands")); var query = """ FROM test @@ -265,7 +265,7 @@ public void testMatchPhraseithStats() { try (var resp = run(query)) { assertColumnNames(resp.columns(), List.of("c", "d")); assertColumnTypes(resp.columns(), List.of("long", "long")); - assertValues(resp.values(), List.of(List.of(2L, 4L))); + assertValues(resp.values(), List.of(List.of(2L, 1L))); } query = """ @@ -291,7 +291,7 @@ public void testMatchPhraseWithinEval() { """; var error = expectThrows(VerificationException.class, () -> run(query)); - assertThat(error.getMessage(), containsString("[MATCH_PHRASE] function is only supported in WHERE and STATS commands")); + assertThat(error.getMessage(), containsString("[MatchPhrase] function is only supported in WHERE and STATS commands")); } private void createAndPopulateIndex() { From 4e7a699e3e4cf3b13d4085151083ef0ff42c3d2d Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Thu, 22 May 2025 11:31:05 -0400 Subject: [PATCH 18/42] PR feedback - add tests, support boost and numeric data --- .../functionNamedParams/match_phrase.md | 3 + .../_snippets/functions/types/match_phrase.md | 20 +- .../definition/functions/match_phrase.json | 426 +++++++++++++++++- .../resources/match-phrase-function.csv-spec | 170 +++++++ .../src/main/resources/scoring.csv-spec | 26 ++ .../esql/plugin/MatchPhraseFunctionIT.java | 31 ++ .../xpack/esql/plugin/ScoringIT.java | 3 + .../function/fulltext/MatchPhrase.java | 57 ++- .../esql/querydsl/query/MatchPhraseQuery.java | 4 +- .../function/fulltext/MatchPhraseTests.java | 178 +++++++- 10 files changed, 879 insertions(+), 39 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md index c0af0cf068e82..9bb0a9c96b0b2 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md @@ -5,6 +5,9 @@ `zero_terms_query` : (keyword) Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none. +`boost` +: (float) Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0. + `analyzer` : (keyword) Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used. diff --git a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md index b49384a4e211f..3e7ce148b1587 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md @@ -4,13 +4,29 @@ | field | query | options | result | | --- | --- | --- | --- | +| boolean | boolean | named parameters | boolean | | boolean | keyword | named parameters | boolean | +| date | date | named parameters | boolean | | date | keyword | named parameters | boolean | +| date_nanos | date_nanos | named parameters | boolean | | date_nanos | keyword | named parameters | boolean | +| double | double | named parameters | boolean | +| double | integer | named parameters | boolean | +| double | long | named parameters | boolean | +| integer | double | named parameters | boolean | +| integer | integer | named parameters | boolean | +| integer | long | named parameters | boolean | +| ip | ip | named parameters | boolean | | ip | keyword | named parameters | boolean | | keyword | keyword | named parameters | boolean | -| keyword | text | named parameters | boolean | +| long | double | named parameters | boolean | +| long | integer | named parameters | boolean | +| long | long | named parameters | boolean | | text | keyword | named parameters | boolean | -| text | text | named parameters | boolean | +| unsigned_long | double | named parameters | boolean | +| unsigned_long | integer | named parameters | boolean | +| unsigned_long | long | named parameters | boolean | +| unsigned_long | unsigned_long | named parameters | boolean | | version | keyword | named parameters | boolean | +| version | version | named parameters | boolean | diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index 4e569d7b5813c..d859ff69aa3c6 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -4,6 +4,31 @@ "name" : "match_phrase", "description" : "Use `MATCH_PHRASE` to perform a match_phrase query on the specified field.\nUsing `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL.\n\nMatchPhrase can be used on text fields, as well as other field types like keyword, boolean, or date types.\nMatchPhrase is not supported for semantic_text or numeric types.\n\nMatchPhrase can use function named parameters to specify additional options for the\nmatch_phrase query.\nAll match_phrase query parameters are supported.\n\n`MATCH_PHRASE` returns true if the provided query matches the row.", "signatures" : [ + { + "params" : [ + { + "name" : "field", + "type" : "boolean", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "boolean", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, { "params" : [ { @@ -21,7 +46,32 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "date", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "date", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -46,7 +96,32 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "date_nanos", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "date_nanos", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -71,7 +146,182 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "double", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "double", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "double", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "integer", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "double", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "long", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "integer", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "double", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "integer", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "integer", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "integer", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "long", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "ip", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "ip", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -96,7 +346,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -121,7 +371,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -133,20 +383,70 @@ "params" : [ { "name" : "field", - "type" : "keyword", + "type" : "long", "optional" : false, "description" : "Field that the query will target." }, { "name" : "query", - "type" : "text", + "type" : "double", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "long", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "integer", "optional" : false, "description" : "Value to find in the provided field." }, { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "long", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "long", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -171,7 +471,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -183,20 +483,95 @@ "params" : [ { "name" : "field", - "type" : "text", + "type" : "unsigned_long", "optional" : false, "description" : "Field that the query will target." }, { "name" : "query", - "type" : "text", + "type" : "double", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "unsigned_long", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "integer", "optional" : false, "description" : "Value to find in the provided field." }, { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "unsigned_long", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "long", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "unsigned_long", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "unsigned_long", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -221,7 +596,32 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "optional" : true, + "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." + } + ], + "variadic" : false, + "returnType" : "boolean" + }, + { + "params" : [ + { + "name" : "field", + "type" : "version", + "optional" : false, + "description" : "Field that the query will target." + }, + { + "name" : "query", + "type" : "version", + "optional" : false, + "description" : "Value to find in the provided field." + }, + { + "name" : "options", + "type" : "function_named_parameters", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index eddc27540fdd2..123b436f2cedf 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -269,6 +269,70 @@ emp_no:integer | first_name:keyword | last_name:keyword 10043 | Yishay | Tzvieli ; +testMatchPhraseBooleanField +required_capability: match_phrase_function +required_capability: match_additional_types + +from employees +| where match_phrase(still_hired, true) and height > 2.08 +| keep first_name, still_hired, height; +ignoreOrder:true + +first_name:keyword | still_hired:boolean | height:double +Saniya | true | 2.1 +Yongqiao | true | 2.1 +Kwee | true | 2.1 +Amabile | true | 2.09 +; + +testMatchPhraseIntegerField +required_capability: match_phrase_function +required_capability: match_additional_types + +from employees +| where match(emp_no, 10004) +| keep emp_no, first_name; + +emp_no:integer | first_name:keyword +10004 | Chirstian +; + +testMatchPhraseDoubleField +required_capability: match_phrase_function +required_capability: match_additional_types + +from employees +| where match_phrase(salary_change, 9.07) +| keep emp_no, salary_change; + +emp_no:integer | salary_change:double +10014 | [-1.89, 9.07] +; + +testMatchPhraseLongField +required_capability: match_phrase_function +required_capability: match_additional_types + +from date_nanos +| where match_phrase(num, 1698069301543123456) +| keep num; + +num:long +1698069301543123456 +; + +testMatchPhraseUnsignedLongField +required_capability: match_phrase_function +required_capability: match_additional_types + +from ul_logs +| where match_phrase(bytes_out, 12749081495402663265) +| keep bytes_out; + +bytes_out:unsigned_long +12749081495402663265 +; + testMatchPhraseVersionField required_capability: match_phrase_function @@ -327,6 +391,21 @@ book_no:keyword 7140 ; +testMatchPhraseWithOptionsBoost +required_capability: match_phrase_function +from books +| where match_phrase(title, "Lord of the Rings", {"boost": 5}) +| keep book_no; +ignoreOrder:true + +book_no:keyword +2714 +2675 +4023 +7140 +; + + testMatchPhraseWithOptionsZeroTermsNone required_capability: match_phrase_function from books @@ -382,3 +461,94 @@ book_no:keyword 4023 7140 ; + +testMatchPhraseInStatsNonPushable +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +from books +| where length(title) > 40 +| stats c = count(*) where match_phrase(title, "Lord of the Rings") +; + +c:long +3 +; + +testMatchPhraseInStatsPushableAndNonPushable +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +from books +| stats c = count(*) where (match_phrase(title, "lord of the rings") and ratings > 4.5) or (match(author, "fyodor dostoevsky") and length(title) > 50) +; + +c:long +6 +; + +testMatchPhraseInStatsPushable +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +from books +| stats c = count(*) where match_phrase(author, "j. r. r. tolkien") +; + +c:long +9 +; + +testMatchPhraseInStatsWithOptions +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +FROM books +| STATS c = count(*) where match_phrase(title, "There and Back Again", {"slop": "5"}) +; + +c:long +1 +; + +testMatchPhraseInStatsWithNonPushableDisjunctions +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +FROM books +| STATS c = count(*) where match_phrase(title, "lord of the rings") or length(title) > 130 +; + +c:long +5 +; + +testMatchPhraseInStatsWithMultipleAggs +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where +FROM books +| STATS c = count(*) where match_phrase(title, "lord of the rings"), m = max(book_no::integer) where match_phrase(author, "j. r. r. tolkien"), n = min(book_no::integer) where match_phrase(author, "fyodor dostoevsky") +; + +c:long | m:integer | n:integer +4 | 7670 | 1211 +; + + +testMatchPhraseInStatsWithGrouping +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where +FROM books +| STATS r = AVG(ratings) where match_phrase(title, "Lord of the Rings") by author | WHERE r is not null +; +ignoreOrder: true + +r:double | author: text +4.75 | Alan Lee +4.674999952316284 | J. R. R. Tolkien +4.670000076293945 | John Ronald Reuel Tolkien +4.670000076293945 | Agnes Perkins +4.670000076293945 | Charles Adolph Huttar +4.670000076293945 | Walter Scheps +4.559999942779541 | J.R.R. Tolkien +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index 401d946334c45..908dd154f5e85 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -558,3 +558,29 @@ from books metadata _score avg_score:double | max_score:double | min_score:double 3.869828939437866 | 5.123856544494629 | 3.0124807357788086 ; + +testMatchPhraseWithScore +required_capability: match_phrase +required_capability: metadata_score + +from books metadata _score +| where match_phrase(title, "J. R. R. Tolkien") +| keep book_no, title, author, _score; +ignoreOrder:true + +book_no:keyword | title:text | author:text | _score:double +2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 +; + +testMatchPhraseWithScoreBoost +required_capability: match_phrase +required_capability: metadata_score + +from books metadata _score +| where match_phrase(title, "J. R. R. Tolkien", {"boost": 5}) +| keep book_no, title, author, _score; +ignoreOrder:true + +book_no:keyword | title:text | author:text | _score:double +2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 +; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java index 80876d06ca6ac..44f28e0c9ea93 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java @@ -16,6 +16,7 @@ import org.hamcrest.Matchers; import org.junit.Before; +import java.util.Collections; import java.util.List; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -45,6 +46,36 @@ public void testSimpleWhereMatchPhrase() { } } + public void testSimpleWhereMatchPhraseNoResults() { + var query = """ + FROM test + | WHERE match_phrase(content, "fox brown") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), Collections.emptyList()); + } + } + + public void testSimpleWhereMatchPhraseAndSlop() { + var query = """ + FROM test + | WHERE match_phrase(content, "fox brown", {"slop": 5}) + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(6))); + } + } + public void testCombinedWhereMatchPhrase() { var query = """ FROM test diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java index a9dada8e2fe31..c03908e3287f2 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java @@ -53,6 +53,9 @@ public static List params() { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { params.add(new Object[] { "term(content, \"fox\")" }); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + params.add(new Object[] { "match_phrase(content, \"fox\")" }); + } return params; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index 56fc29695cc41..e44189e681c53 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -25,6 +25,7 @@ import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.util.Check; +import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.Example; import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesTo; import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesToLifecycle; @@ -60,11 +61,14 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; +import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; import static org.elasticsearch.xpack.esql.core.type.DataType.FLOAT; import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; import static org.elasticsearch.xpack.esql.core.type.DataType.IP; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; +import static org.elasticsearch.xpack.esql.core.type.DataType.LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; +import static org.elasticsearch.xpack.esql.core.type.DataType.UNSIGNED_LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.VERSION; import static org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison.formatIncompatibleTypesMessage; @@ -78,8 +82,31 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P "MatchPhrase", MatchPhrase::readFrom ); - public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); - public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD, TEXT); + public static final Set FIELD_DATA_TYPES = Set.of( + KEYWORD, + TEXT, + BOOLEAN, + DATETIME, + DATE_NANOS, + DOUBLE, + INTEGER, + IP, + LONG, + UNSIGNED_LONG, + VERSION + ); + public static final Set QUERY_DATA_TYPES = Set.of( + KEYWORD, + BOOLEAN, + DATETIME, + DATE_NANOS, + DOUBLE, + INTEGER, + IP, + LONG, + UNSIGNED_LONG, + VERSION + ); protected final Expression field; @@ -121,12 +148,12 @@ public MatchPhrase( Source source, @Param( name = "field", - type = { "keyword", "text", "boolean", "date", "date_nanos", "ip", "version" }, + type = { "keyword", "text", "boolean", "date", "date_nanos", "double", "integer", "ip", "long", "unsigned_long", "version" }, description = "Field that the query will target." ) Expression field, @Param( name = "query", - type = { "keyword", "text" }, + type = { "keyword", "boolean", "date", "date_nanos", "double", "integer", "ip", "long", "unsigned_long", "version" }, description = "Value to find in the provided field." ) Expression matchPhraseQuery, @MapParam( @@ -152,6 +179,12 @@ public MatchPhrase( valueHint = { "none", "all" }, description = "Indicates whether all documents or none are returned if the analyzer removes all tokens, such as " + "when using a stop filter. Defaults to none." + ), + @MapParam.MapParamEntry( + name = "boost", + type = "float", + valueHint = { "2.5" }, + description = "Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0." ) }, description = "(Optional) MatchPhrase additional options as <>." + " See <> for more information.", @@ -306,8 +339,10 @@ public Object queryAsObject() { } // Converts specific types to the correct type for the query - if (query().dataType() == DataType.DATETIME && queryAsObject instanceof Long) { - // When casting to date and datetime, we get a long back. But MatchPhrase query needs a date string + if (query().dataType() == DataType.UNSIGNED_LONG) { + return NumericUtils.unsignedLongAsBigInteger((Long) queryAsObject); + } else if (query().dataType() == DataType.DATETIME && queryAsObject instanceof Long) { + // When casting to date and datetime, we get a long back. But Match query needs a date string return EsqlDataTypeConverter.dateTimeToString((Long) queryAsObject); } else if (query().dataType() == DATE_NANOS && queryAsObject instanceof Long) { return EsqlDataTypeConverter.nanoTimeToString((Long) queryAsObject); @@ -331,12 +366,12 @@ private FieldAttribute fieldAsFieldAttribute() { @Override public boolean equals(Object o) { // MatchPhrase does not serialize options, as they get included in the query builder. We need to override equals and hashcode to - // ignore options when comparing two Match functions + // ignore options when comparing two MatchPhrase functions if (o == null || getClass() != o.getClass()) return false; - MatchPhrase match = (MatchPhrase) o; - return Objects.equals(field(), match.field()) - && Objects.equals(query(), match.query()) - && Objects.equals(queryBuilder(), match.queryBuilder()); + MatchPhrase matchPhrase = (MatchPhrase) o; + return Objects.equals(field(), matchPhrase.field()) + && Objects.equals(query(), matchPhrase.query()) + && Objects.equals(queryBuilder(), matchPhrase.queryBuilder()); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java index 6f6735efa7482..be6f244ac4acf 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -6,6 +6,7 @@ */ package org.elasticsearch.xpack.esql.querydsl.query; +import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.MatchPhraseQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -29,7 +30,8 @@ public class MatchPhraseQuery extends Query { BUILDER_APPLIERS = Map.ofEntries( entry(ANALYZER_FIELD.getPreferredName(), (qb, s) -> qb.analyzer(s.toString())), entry(SLOP_FIELD.getPreferredName(), (qb, s) -> qb.slop(Integer.parseInt(s.toString()))), - entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)) + entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)), + entry(AbstractQueryBuilder.BOOST_FIELD.getPreferredName(), (qb, s) -> qb.boost((Float) s)) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java index adc36af5a9328..0e3891b5a8543 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -17,12 +17,14 @@ import org.elasticsearch.xpack.esql.core.expression.MapExpression; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase; import org.elasticsearch.xpack.esql.expression.function.FunctionName; import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier; import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; +import java.math.BigInteger; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; @@ -52,10 +54,13 @@ private static List testCaseSuppliers() { List suppliers = new ArrayList<>(); addQueryAsStringTestCases(suppliers); addStringTestCases(suppliers); + addNonNumericCases(suppliers); + addNumericCases(suppliers); + addUnsignedLongCases(suppliers); return suppliers; } - public static void addQueryAsStringTestCases(List suppliers) { + private static void addQueryAsStringTestCases(List suppliers) { suppliers.addAll( TestCaseSupplier.forBinaryNotCasting( @@ -126,7 +131,7 @@ public static void addQueryAsStringTestCases(List suppliers) { ); } - public static void addStringTestCases(List suppliers) { + private static void addStringTestCases(List suppliers) { for (DataType fieldType : DataType.stringTypes()) { if (DataType.UNDER_CONSTRUCTION.containsKey(fieldType)) { continue; @@ -141,20 +146,169 @@ public static void addStringTestCases(List suppliers) { (o1, o2) -> true ) ); - - suppliers.add( - TestCaseSupplier.testCaseSupplier( - queryDataSupplier, - new TestCaseSupplier.TypedDataSupplier(fieldType.typeName(), () -> randomAlphaOfLength(10), DataType.TEXT), - (d1, d2) -> equalTo("string"), - DataType.BOOLEAN, - (o1, o2) -> true - ) - ); } } } + private static void addNonNumericCases(List suppliers) { + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.booleanCases(), + TestCaseSupplier.booleanCases(), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.ipCases(), + TestCaseSupplier.ipCases(), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.versionCases(""), + TestCaseSupplier.versionCases(""), + List.of(), + false + ) + ); + // Datetime + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.dateCases(), + TestCaseSupplier.dateCases(), + List.of(), + false + ) + ); + + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.dateNanosCases(), + TestCaseSupplier.dateNanosCases(), + List.of(), + false + ) + ); + } + + private static void addNumericCases(List suppliers) { + suppliers.addAll( + TestCaseSupplier.forBinaryComparisonWithWidening( + new TestCaseSupplier.NumericTypeTestConfigs<>( + new TestCaseSupplier.NumericTypeTestConfig<>( + (Integer.MIN_VALUE >> 1) - 1, + (Integer.MAX_VALUE >> 1) - 1, + (l, r) -> true, + "EqualsIntsEvaluator" + ), + new TestCaseSupplier.NumericTypeTestConfig<>( + (Long.MIN_VALUE >> 1) - 1, + (Long.MAX_VALUE >> 1) - 1, + (l, r) -> true, + "EqualsLongsEvaluator" + ), + new TestCaseSupplier.NumericTypeTestConfig<>( + Double.NEGATIVE_INFINITY, + Double.POSITIVE_INFINITY, + // NB: this has different behavior than Double::equals + (l, r) -> true, + "EqualsDoublesEvaluator" + ) + ), + "field", + "query", + (lhs, rhs) -> List.of(), + false + ) + ); + } + + private static void addUnsignedLongCases(List suppliers) { + // TODO: These should be integrated into the type cross product above, but are currently broken + // see https://github.com/elastic/elasticsearch/issues/102935 + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), + TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), + TestCaseSupplier.intCases(Integer.MIN_VALUE, Integer.MAX_VALUE, true), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), + TestCaseSupplier.longCases(Long.MIN_VALUE, Long.MAX_VALUE, true), + List.of(), + false + ) + ); + suppliers.addAll( + TestCaseSupplier.forBinaryNotCasting( + null, + "field", + "query", + Object::equals, + DataType.BOOLEAN, + TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), + TestCaseSupplier.doubleCases(Double.MIN_VALUE, Double.MAX_VALUE, true), + List.of(), + false + ) + ); + } + /** * Adds function named parameters to all the test case suppliers provided */ From 81b2eba507a92594c8f4d4bb2cec9a58a154c470 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 08:40:31 -0400 Subject: [PATCH 19/42] Revert "PR feedback - add tests, support boost and numeric data" This reverts commit 4e7a699e3e4cf3b13d4085151083ef0ff42c3d2d. --- .../functionNamedParams/match_phrase.md | 3 - .../_snippets/functions/types/match_phrase.md | 20 +- .../definition/functions/match_phrase.json | 426 +----------------- .../resources/match-phrase-function.csv-spec | 170 ------- .../src/main/resources/scoring.csv-spec | 26 -- .../esql/plugin/MatchPhraseFunctionIT.java | 31 -- .../xpack/esql/plugin/ScoringIT.java | 3 - .../function/fulltext/MatchPhrase.java | 57 +-- .../esql/querydsl/query/MatchPhraseQuery.java | 4 +- .../function/fulltext/MatchPhraseTests.java | 178 +------- 10 files changed, 39 insertions(+), 879 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md index 9bb0a9c96b0b2..c0af0cf068e82 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md @@ -5,9 +5,6 @@ `zero_terms_query` : (keyword) Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none. -`boost` -: (float) Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0. - `analyzer` : (keyword) Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used. diff --git a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md index 3e7ce148b1587..b49384a4e211f 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md @@ -4,29 +4,13 @@ | field | query | options | result | | --- | --- | --- | --- | -| boolean | boolean | named parameters | boolean | | boolean | keyword | named parameters | boolean | -| date | date | named parameters | boolean | | date | keyword | named parameters | boolean | -| date_nanos | date_nanos | named parameters | boolean | | date_nanos | keyword | named parameters | boolean | -| double | double | named parameters | boolean | -| double | integer | named parameters | boolean | -| double | long | named parameters | boolean | -| integer | double | named parameters | boolean | -| integer | integer | named parameters | boolean | -| integer | long | named parameters | boolean | -| ip | ip | named parameters | boolean | | ip | keyword | named parameters | boolean | | keyword | keyword | named parameters | boolean | -| long | double | named parameters | boolean | -| long | integer | named parameters | boolean | -| long | long | named parameters | boolean | +| keyword | text | named parameters | boolean | | text | keyword | named parameters | boolean | -| unsigned_long | double | named parameters | boolean | -| unsigned_long | integer | named parameters | boolean | -| unsigned_long | long | named parameters | boolean | -| unsigned_long | unsigned_long | named parameters | boolean | +| text | text | named parameters | boolean | | version | keyword | named parameters | boolean | -| version | version | named parameters | boolean | diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index d859ff69aa3c6..4e569d7b5813c 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -4,31 +4,6 @@ "name" : "match_phrase", "description" : "Use `MATCH_PHRASE` to perform a match_phrase query on the specified field.\nUsing `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL.\n\nMatchPhrase can be used on text fields, as well as other field types like keyword, boolean, or date types.\nMatchPhrase is not supported for semantic_text or numeric types.\n\nMatchPhrase can use function named parameters to specify additional options for the\nmatch_phrase query.\nAll match_phrase query parameters are supported.\n\n`MATCH_PHRASE` returns true if the provided query matches the row.", "signatures" : [ - { - "params" : [ - { - "name" : "field", - "type" : "boolean", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "boolean", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, { "params" : [ { @@ -46,32 +21,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "date", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "date", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -96,32 +46,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "date_nanos", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "date_nanos", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -146,182 +71,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "double", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "double", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "double", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "integer", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "double", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "long", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "integer", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "double", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "integer", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "integer", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "integer", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "long", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "ip", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "ip", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -346,7 +96,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -371,57 +121,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "long", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "double", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "long", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "integer", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -433,20 +133,20 @@ "params" : [ { "name" : "field", - "type" : "long", + "type" : "keyword", "optional" : false, "description" : "Field that the query will target." }, { "name" : "query", - "type" : "long", + "type" : "text", "optional" : false, "description" : "Value to find in the provided field." }, { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -471,7 +171,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -483,95 +183,20 @@ "params" : [ { "name" : "field", - "type" : "unsigned_long", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "double", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "unsigned_long", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "integer", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "unsigned_long", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "long", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "unsigned_long", + "type" : "text", "optional" : false, "description" : "Field that the query will target." }, { "name" : "query", - "type" : "unsigned_long", + "type" : "text", "optional" : false, "description" : "Value to find in the provided field." }, { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -596,32 +221,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "version", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "version", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index 123b436f2cedf..eddc27540fdd2 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -269,70 +269,6 @@ emp_no:integer | first_name:keyword | last_name:keyword 10043 | Yishay | Tzvieli ; -testMatchPhraseBooleanField -required_capability: match_phrase_function -required_capability: match_additional_types - -from employees -| where match_phrase(still_hired, true) and height > 2.08 -| keep first_name, still_hired, height; -ignoreOrder:true - -first_name:keyword | still_hired:boolean | height:double -Saniya | true | 2.1 -Yongqiao | true | 2.1 -Kwee | true | 2.1 -Amabile | true | 2.09 -; - -testMatchPhraseIntegerField -required_capability: match_phrase_function -required_capability: match_additional_types - -from employees -| where match(emp_no, 10004) -| keep emp_no, first_name; - -emp_no:integer | first_name:keyword -10004 | Chirstian -; - -testMatchPhraseDoubleField -required_capability: match_phrase_function -required_capability: match_additional_types - -from employees -| where match_phrase(salary_change, 9.07) -| keep emp_no, salary_change; - -emp_no:integer | salary_change:double -10014 | [-1.89, 9.07] -; - -testMatchPhraseLongField -required_capability: match_phrase_function -required_capability: match_additional_types - -from date_nanos -| where match_phrase(num, 1698069301543123456) -| keep num; - -num:long -1698069301543123456 -; - -testMatchPhraseUnsignedLongField -required_capability: match_phrase_function -required_capability: match_additional_types - -from ul_logs -| where match_phrase(bytes_out, 12749081495402663265) -| keep bytes_out; - -bytes_out:unsigned_long -12749081495402663265 -; - testMatchPhraseVersionField required_capability: match_phrase_function @@ -391,21 +327,6 @@ book_no:keyword 7140 ; -testMatchPhraseWithOptionsBoost -required_capability: match_phrase_function -from books -| where match_phrase(title, "Lord of the Rings", {"boost": 5}) -| keep book_no; -ignoreOrder:true - -book_no:keyword -2714 -2675 -4023 -7140 -; - - testMatchPhraseWithOptionsZeroTermsNone required_capability: match_phrase_function from books @@ -461,94 +382,3 @@ book_no:keyword 4023 7140 ; - -testMatchPhraseInStatsNonPushable -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where - -from books -| where length(title) > 40 -| stats c = count(*) where match_phrase(title, "Lord of the Rings") -; - -c:long -3 -; - -testMatchPhraseInStatsPushableAndNonPushable -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where - -from books -| stats c = count(*) where (match_phrase(title, "lord of the rings") and ratings > 4.5) or (match(author, "fyodor dostoevsky") and length(title) > 50) -; - -c:long -6 -; - -testMatchPhraseInStatsPushable -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where - -from books -| stats c = count(*) where match_phrase(author, "j. r. r. tolkien") -; - -c:long -9 -; - -testMatchPhraseInStatsWithOptions -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where - -FROM books -| STATS c = count(*) where match_phrase(title, "There and Back Again", {"slop": "5"}) -; - -c:long -1 -; - -testMatchPhraseInStatsWithNonPushableDisjunctions -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where - -FROM books -| STATS c = count(*) where match_phrase(title, "lord of the rings") or length(title) > 130 -; - -c:long -5 -; - -testMatchPhraseInStatsWithMultipleAggs -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where -FROM books -| STATS c = count(*) where match_phrase(title, "lord of the rings"), m = max(book_no::integer) where match_phrase(author, "j. r. r. tolkien"), n = min(book_no::integer) where match_phrase(author, "fyodor dostoevsky") -; - -c:long | m:integer | n:integer -4 | 7670 | 1211 -; - - -testMatchPhraseInStatsWithGrouping -required_capability: match_phrase_function -required_capability: full_text_functions_in_stats_where -FROM books -| STATS r = AVG(ratings) where match_phrase(title, "Lord of the Rings") by author | WHERE r is not null -; -ignoreOrder: true - -r:double | author: text -4.75 | Alan Lee -4.674999952316284 | J. R. R. Tolkien -4.670000076293945 | John Ronald Reuel Tolkien -4.670000076293945 | Agnes Perkins -4.670000076293945 | Charles Adolph Huttar -4.670000076293945 | Walter Scheps -4.559999942779541 | J.R.R. Tolkien -; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index 908dd154f5e85..401d946334c45 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -558,29 +558,3 @@ from books metadata _score avg_score:double | max_score:double | min_score:double 3.869828939437866 | 5.123856544494629 | 3.0124807357788086 ; - -testMatchPhraseWithScore -required_capability: match_phrase -required_capability: metadata_score - -from books metadata _score -| where match_phrase(title, "J. R. R. Tolkien") -| keep book_no, title, author, _score; -ignoreOrder:true - -book_no:keyword | title:text | author:text | _score:double -2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 -; - -testMatchPhraseWithScoreBoost -required_capability: match_phrase -required_capability: metadata_score - -from books metadata _score -| where match_phrase(title, "J. R. R. Tolkien", {"boost": 5}) -| keep book_no, title, author, _score; -ignoreOrder:true - -book_no:keyword | title:text | author:text | _score:double -2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 -; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java index 44f28e0c9ea93..80876d06ca6ac 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java @@ -16,7 +16,6 @@ import org.hamcrest.Matchers; import org.junit.Before; -import java.util.Collections; import java.util.List; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -46,36 +45,6 @@ public void testSimpleWhereMatchPhrase() { } } - public void testSimpleWhereMatchPhraseNoResults() { - var query = """ - FROM test - | WHERE match_phrase(content, "fox brown") - | KEEP id - | SORT id - """; - - try (var resp = run(query)) { - assertColumnNames(resp.columns(), List.of("id")); - assertColumnTypes(resp.columns(), List.of("integer")); - assertValues(resp.values(), Collections.emptyList()); - } - } - - public void testSimpleWhereMatchPhraseAndSlop() { - var query = """ - FROM test - | WHERE match_phrase(content, "fox brown", {"slop": 5}) - | KEEP id - | SORT id - """; - - try (var resp = run(query)) { - assertColumnNames(resp.columns(), List.of("id")); - assertColumnTypes(resp.columns(), List.of("integer")); - assertValues(resp.values(), List.of(List.of(1), List.of(6))); - } - } - public void testCombinedWhereMatchPhrase() { var query = """ FROM test diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java index c03908e3287f2..a9dada8e2fe31 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java @@ -53,9 +53,6 @@ public static List params() { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { params.add(new Object[] { "term(content, \"fox\")" }); } - if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { - params.add(new Object[] { "match_phrase(content, \"fox\")" }); - } return params; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index e44189e681c53..56fc29695cc41 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -25,7 +25,6 @@ import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.util.Check; -import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.Example; import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesTo; import org.elasticsearch.xpack.esql.expression.function.FunctionAppliesToLifecycle; @@ -61,14 +60,11 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; -import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; import static org.elasticsearch.xpack.esql.core.type.DataType.FLOAT; import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; import static org.elasticsearch.xpack.esql.core.type.DataType.IP; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; -import static org.elasticsearch.xpack.esql.core.type.DataType.LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; -import static org.elasticsearch.xpack.esql.core.type.DataType.UNSIGNED_LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.VERSION; import static org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison.formatIncompatibleTypesMessage; @@ -82,31 +78,8 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P "MatchPhrase", MatchPhrase::readFrom ); - public static final Set FIELD_DATA_TYPES = Set.of( - KEYWORD, - TEXT, - BOOLEAN, - DATETIME, - DATE_NANOS, - DOUBLE, - INTEGER, - IP, - LONG, - UNSIGNED_LONG, - VERSION - ); - public static final Set QUERY_DATA_TYPES = Set.of( - KEYWORD, - BOOLEAN, - DATETIME, - DATE_NANOS, - DOUBLE, - INTEGER, - IP, - LONG, - UNSIGNED_LONG, - VERSION - ); + public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); + public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD, TEXT); protected final Expression field; @@ -148,12 +121,12 @@ public MatchPhrase( Source source, @Param( name = "field", - type = { "keyword", "text", "boolean", "date", "date_nanos", "double", "integer", "ip", "long", "unsigned_long", "version" }, + type = { "keyword", "text", "boolean", "date", "date_nanos", "ip", "version" }, description = "Field that the query will target." ) Expression field, @Param( name = "query", - type = { "keyword", "boolean", "date", "date_nanos", "double", "integer", "ip", "long", "unsigned_long", "version" }, + type = { "keyword", "text" }, description = "Value to find in the provided field." ) Expression matchPhraseQuery, @MapParam( @@ -179,12 +152,6 @@ public MatchPhrase( valueHint = { "none", "all" }, description = "Indicates whether all documents or none are returned if the analyzer removes all tokens, such as " + "when using a stop filter. Defaults to none." - ), - @MapParam.MapParamEntry( - name = "boost", - type = "float", - valueHint = { "2.5" }, - description = "Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0." ) }, description = "(Optional) MatchPhrase additional options as <>." + " See <> for more information.", @@ -339,10 +306,8 @@ public Object queryAsObject() { } // Converts specific types to the correct type for the query - if (query().dataType() == DataType.UNSIGNED_LONG) { - return NumericUtils.unsignedLongAsBigInteger((Long) queryAsObject); - } else if (query().dataType() == DataType.DATETIME && queryAsObject instanceof Long) { - // When casting to date and datetime, we get a long back. But Match query needs a date string + if (query().dataType() == DataType.DATETIME && queryAsObject instanceof Long) { + // When casting to date and datetime, we get a long back. But MatchPhrase query needs a date string return EsqlDataTypeConverter.dateTimeToString((Long) queryAsObject); } else if (query().dataType() == DATE_NANOS && queryAsObject instanceof Long) { return EsqlDataTypeConverter.nanoTimeToString((Long) queryAsObject); @@ -366,12 +331,12 @@ private FieldAttribute fieldAsFieldAttribute() { @Override public boolean equals(Object o) { // MatchPhrase does not serialize options, as they get included in the query builder. We need to override equals and hashcode to - // ignore options when comparing two MatchPhrase functions + // ignore options when comparing two Match functions if (o == null || getClass() != o.getClass()) return false; - MatchPhrase matchPhrase = (MatchPhrase) o; - return Objects.equals(field(), matchPhrase.field()) - && Objects.equals(query(), matchPhrase.query()) - && Objects.equals(queryBuilder(), matchPhrase.queryBuilder()); + MatchPhrase match = (MatchPhrase) o; + return Objects.equals(field(), match.field()) + && Objects.equals(query(), match.query()) + && Objects.equals(queryBuilder(), match.queryBuilder()); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java index be6f244ac4acf..6f6735efa7482 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -6,7 +6,6 @@ */ package org.elasticsearch.xpack.esql.querydsl.query; -import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.MatchPhraseQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -30,8 +29,7 @@ public class MatchPhraseQuery extends Query { BUILDER_APPLIERS = Map.ofEntries( entry(ANALYZER_FIELD.getPreferredName(), (qb, s) -> qb.analyzer(s.toString())), entry(SLOP_FIELD.getPreferredName(), (qb, s) -> qb.slop(Integer.parseInt(s.toString()))), - entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)), - entry(AbstractQueryBuilder.BOOST_FIELD.getPreferredName(), (qb, s) -> qb.boost((Float) s)) + entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java index 0e3891b5a8543..adc36af5a9328 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -17,14 +17,12 @@ import org.elasticsearch.xpack.esql.core.expression.MapExpression; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; -import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.AbstractFunctionTestCase; import org.elasticsearch.xpack.esql.expression.function.FunctionName; import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier; import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; -import java.math.BigInteger; import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; @@ -54,13 +52,10 @@ private static List testCaseSuppliers() { List suppliers = new ArrayList<>(); addQueryAsStringTestCases(suppliers); addStringTestCases(suppliers); - addNonNumericCases(suppliers); - addNumericCases(suppliers); - addUnsignedLongCases(suppliers); return suppliers; } - private static void addQueryAsStringTestCases(List suppliers) { + public static void addQueryAsStringTestCases(List suppliers) { suppliers.addAll( TestCaseSupplier.forBinaryNotCasting( @@ -131,7 +126,7 @@ private static void addQueryAsStringTestCases(List suppliers) ); } - private static void addStringTestCases(List suppliers) { + public static void addStringTestCases(List suppliers) { for (DataType fieldType : DataType.stringTypes()) { if (DataType.UNDER_CONSTRUCTION.containsKey(fieldType)) { continue; @@ -146,167 +141,18 @@ private static void addStringTestCases(List suppliers) { (o1, o2) -> true ) ); - } - } - } - private static void addNonNumericCases(List suppliers) { - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.booleanCases(), - TestCaseSupplier.booleanCases(), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.ipCases(), - TestCaseSupplier.ipCases(), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.versionCases(""), - TestCaseSupplier.versionCases(""), - List.of(), - false - ) - ); - // Datetime - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.dateCases(), - TestCaseSupplier.dateCases(), - List.of(), - false - ) - ); - - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.dateNanosCases(), - TestCaseSupplier.dateNanosCases(), - List.of(), - false - ) - ); - } - - private static void addNumericCases(List suppliers) { - suppliers.addAll( - TestCaseSupplier.forBinaryComparisonWithWidening( - new TestCaseSupplier.NumericTypeTestConfigs<>( - new TestCaseSupplier.NumericTypeTestConfig<>( - (Integer.MIN_VALUE >> 1) - 1, - (Integer.MAX_VALUE >> 1) - 1, - (l, r) -> true, - "EqualsIntsEvaluator" - ), - new TestCaseSupplier.NumericTypeTestConfig<>( - (Long.MIN_VALUE >> 1) - 1, - (Long.MAX_VALUE >> 1) - 1, - (l, r) -> true, - "EqualsLongsEvaluator" - ), - new TestCaseSupplier.NumericTypeTestConfig<>( - Double.NEGATIVE_INFINITY, - Double.POSITIVE_INFINITY, - // NB: this has different behavior than Double::equals - (l, r) -> true, - "EqualsDoublesEvaluator" + suppliers.add( + TestCaseSupplier.testCaseSupplier( + queryDataSupplier, + new TestCaseSupplier.TypedDataSupplier(fieldType.typeName(), () -> randomAlphaOfLength(10), DataType.TEXT), + (d1, d2) -> equalTo("string"), + DataType.BOOLEAN, + (o1, o2) -> true ) - ), - "field", - "query", - (lhs, rhs) -> List.of(), - false - ) - ); - } - - private static void addUnsignedLongCases(List suppliers) { - // TODO: These should be integrated into the type cross product above, but are currently broken - // see https://github.com/elastic/elasticsearch/issues/102935 - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), - TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), - TestCaseSupplier.intCases(Integer.MIN_VALUE, Integer.MAX_VALUE, true), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), - TestCaseSupplier.longCases(Long.MIN_VALUE, Long.MAX_VALUE, true), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.ulongCases(BigInteger.ZERO, NumericUtils.UNSIGNED_LONG_MAX, true), - TestCaseSupplier.doubleCases(Double.MIN_VALUE, Double.MAX_VALUE, true), - List.of(), - false - ) - ); + ); + } + } } /** From 00b21ac9db3f93f6c65769d877b008a7662488d9 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 08:45:04 -0400 Subject: [PATCH 20/42] Apply testing/PR feedback outside numeric support only --- .../resources/match-phrase-function.csv-spec | 105 ++++++++++++++++++ .../src/main/resources/scoring.csv-spec | 26 +++++ .../esql/plugin/MatchPhraseFunctionIT.java | 31 ++++++ .../xpack/esql/plugin/ScoringIT.java | 3 + .../function/fulltext/MatchPhrase.java | 16 ++- .../esql/querydsl/query/MatchPhraseQuery.java | 4 +- 6 files changed, 179 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index eddc27540fdd2..c129b2a010dc4 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -382,3 +382,108 @@ book_no:keyword 4023 7140 ; + +testMatchPhraseWithOptionsBoost +required_capability: match_phrase_function +from books +| where match_phrase(title, "Lord of the Rings", {"boost": 5}) +| keep book_no; +ignoreOrder:true + +book_no:keyword +2714 +2675 +4023 +7140 +; + +testMatchPhraseInStatsNonPushable +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +from books +| where length(title) > 40 +| stats c = count(*) where match_phrase(title, "Lord of the Rings") +; + +c:long +3 +; + +testMatchPhraseInStatsPushableAndNonPushable +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +from books +| stats c = count(*) where (match_phrase(title, "lord of the rings") and ratings > 4.5) or (match(author, "fyodor dostoevsky") and length(title) > 50) +; + +c:long +6 +; + +testMatchPhraseInStatsPushable +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +from books +| stats c = count(*) where match_phrase(author, "j. r. r. tolkien") +; + +c:long +9 +; + +testMatchPhraseInStatsWithOptions +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +FROM books +| STATS c = count(*) where match_phrase(title, "There and Back Again", {"slop": "5"}) +; + +c:long +1 +; + +testMatchPhraseInStatsWithNonPushableDisjunctions +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where + +FROM books +| STATS c = count(*) where match_phrase(title, "lord of the rings") or length(title) > 130 +; + +c:long +5 +; + +testMatchPhraseInStatsWithMultipleAggs +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where +FROM books +| STATS c = count(*) where match_phrase(title, "lord of the rings"), m = max(book_no::integer) where match_phrase(author, "j. r. r. tolkien"), n = min(book_no::integer) where match_phrase(author, "fyodor dostoevsky") +; + +c:long | m:integer | n:integer +4 | 7670 | 1211 +; + + +testMatchPhraseInStatsWithGrouping +required_capability: match_phrase_function +required_capability: full_text_functions_in_stats_where +FROM books +| STATS r = AVG(ratings) where match_phrase(title, "Lord of the Rings") by author | WHERE r is not null +; +ignoreOrder: true + +r:double | author: text +4.75 | Alan Lee +4.674999952316284 | J. R. R. Tolkien +4.670000076293945 | John Ronald Reuel Tolkien +4.670000076293945 | Agnes Perkins +4.670000076293945 | Charles Adolph Huttar +4.670000076293945 | Walter Scheps +4.559999942779541 | J.R.R. Tolkien +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index 401d946334c45..908dd154f5e85 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -558,3 +558,29 @@ from books metadata _score avg_score:double | max_score:double | min_score:double 3.869828939437866 | 5.123856544494629 | 3.0124807357788086 ; + +testMatchPhraseWithScore +required_capability: match_phrase +required_capability: metadata_score + +from books metadata _score +| where match_phrase(title, "J. R. R. Tolkien") +| keep book_no, title, author, _score; +ignoreOrder:true + +book_no:keyword | title:text | author:text | _score:double +2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 +; + +testMatchPhraseWithScoreBoost +required_capability: match_phrase +required_capability: metadata_score + +from books metadata _score +| where match_phrase(title, "J. R. R. Tolkien", {"boost": 5}) +| keep book_no, title, author, _score; +ignoreOrder:true + +book_no:keyword | title:text | author:text | _score:double +2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 +; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java index 80876d06ca6ac..44f28e0c9ea93 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java @@ -16,6 +16,7 @@ import org.hamcrest.Matchers; import org.junit.Before; +import java.util.Collections; import java.util.List; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -45,6 +46,36 @@ public void testSimpleWhereMatchPhrase() { } } + public void testSimpleWhereMatchPhraseNoResults() { + var query = """ + FROM test + | WHERE match_phrase(content, "fox brown") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), Collections.emptyList()); + } + } + + public void testSimpleWhereMatchPhraseAndSlop() { + var query = """ + FROM test + | WHERE match_phrase(content, "fox brown", {"slop": 5}) + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(6))); + } + } + public void testCombinedWhereMatchPhrase() { var query = """ FROM test diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java index a9dada8e2fe31..c03908e3287f2 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/ScoringIT.java @@ -53,6 +53,9 @@ public static List params() { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { params.add(new Object[] { "term(content, \"fox\")" }); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + params.add(new Object[] { "match_phrase(content, \"fox\")" }); + } return params; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index 56fc29695cc41..17714900e8644 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -152,6 +152,12 @@ public MatchPhrase( valueHint = { "none", "all" }, description = "Indicates whether all documents or none are returned if the analyzer removes all tokens, such as " + "when using a stop filter. Defaults to none." + ), + @MapParam.MapParamEntry( + name = "boost", + type = "float", + valueHint = { "2.5" }, + description = "Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0." ) }, description = "(Optional) MatchPhrase additional options as <>." + " See <> for more information.", @@ -331,12 +337,12 @@ private FieldAttribute fieldAsFieldAttribute() { @Override public boolean equals(Object o) { // MatchPhrase does not serialize options, as they get included in the query builder. We need to override equals and hashcode to - // ignore options when comparing two Match functions + // ignore options when comparing two MatchPhrase functions if (o == null || getClass() != o.getClass()) return false; - MatchPhrase match = (MatchPhrase) o; - return Objects.equals(field(), match.field()) - && Objects.equals(query(), match.query()) - && Objects.equals(queryBuilder(), match.queryBuilder()); + MatchPhrase matchPhrase = (MatchPhrase) o; + return Objects.equals(field(), matchPhrase.field()) + && Objects.equals(query(), matchPhrase.query()) + && Objects.equals(queryBuilder(), matchPhrase.queryBuilder()); } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java index 6f6735efa7482..be6f244ac4acf 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchPhraseQuery.java @@ -6,6 +6,7 @@ */ package org.elasticsearch.xpack.esql.querydsl.query; +import org.elasticsearch.index.query.AbstractQueryBuilder; import org.elasticsearch.index.query.MatchPhraseQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -29,7 +30,8 @@ public class MatchPhraseQuery extends Query { BUILDER_APPLIERS = Map.ofEntries( entry(ANALYZER_FIELD.getPreferredName(), (qb, s) -> qb.analyzer(s.toString())), entry(SLOP_FIELD.getPreferredName(), (qb, s) -> qb.slop(Integer.parseInt(s.toString()))), - entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)) + entry(ZERO_TERMS_QUERY_FIELD.getPreferredName(), (qb, s) -> qb.zeroTermsQuery((String) s)), + entry(AbstractQueryBuilder.BOOST_FIELD.getPreferredName(), (qb, s) -> qb.boost((Float) s)) ); } From 2151b87b35ed9c60d40235a541bb90c5543542df Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 09:49:12 -0400 Subject: [PATCH 21/42] Regenerate docs --- .../functionNamedParams/match_phrase.md | 3 +++ .../definition/functions/match_phrase.json | 18 +++++++++--------- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md index c0af0cf068e82..9bb0a9c96b0b2 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/functionNamedParams/match_phrase.md @@ -5,6 +5,9 @@ `zero_terms_query` : (keyword) Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none. +`boost` +: (float) Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0. + `analyzer` : (keyword) Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used. diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index 4e569d7b5813c..102f53dcf944b 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -21,7 +21,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -46,7 +46,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -71,7 +71,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -96,7 +96,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -121,7 +121,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -146,7 +146,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -171,7 +171,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -196,7 +196,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } @@ -221,7 +221,7 @@ { "name" : "options", "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", + "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", "optional" : true, "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." } From 286ce04afb9554fc63a02845c74b7ee282b7f75a Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 09:50:49 -0400 Subject: [PATCH 22/42] Add negative test --- .../src/main/resources/match-phrase-function.csv-spec | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index c129b2a010dc4..18b7865465458 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -487,3 +487,11 @@ r:double | author: text 4.670000076293945 | Walter Scheps 4.559999942779541 | J.R.R. Tolkien ; + +from books +| where match_phrase(title, "Lord Rings") +| keep book_no +; + +book_no:keyword +; From ee5fdc58b8f0c4bca91dfb58ed91ddc2ffa840d9 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 14:33:32 -0400 Subject: [PATCH 23/42] Update x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec Co-authored-by: Carlos Delgado <6339205+carlosdelest@users.noreply.github.com> --- .../src/main/resources/match-phrase-function.csv-spec | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index 18b7865465458..f3ad1947aa88d 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -488,6 +488,7 @@ r:double | author: text 4.559999942779541 | J.R.R. Tolkien ; +testMatchPhraseRequiresExactPhraseMatch from books | where match_phrase(title, "Lord Rings") | keep book_no From 2eb24b4b2ac60a66d9f22ab0d0eb42a72f8620ab Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 14:36:26 -0400 Subject: [PATCH 24/42] Update x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec Co-authored-by: Carlos Delgado <6339205+carlosdelest@users.noreply.github.com> --- .../src/main/resources/match-phrase-function.csv-spec | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index f3ad1947aa88d..8896a29323543 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -306,7 +306,7 @@ testMatchPhraseDateNanosFieldAsString required_capability: match_phrase_function from date_nanos -| where match(nanos, "2023-10-23T13:55:01.543123456Z") +| where match_phrase(nanos, "2023-10-23T13:55:01.543123456Z") | keep nanos; nanos:date_nanos From b68c3a077046665080ceb0382a4c789516400092 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 14:36:40 -0400 Subject: [PATCH 25/42] Update x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec Co-authored-by: Carlos Delgado <6339205+carlosdelest@users.noreply.github.com> --- .../src/main/resources/match-phrase-function.csv-spec | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index 8896a29323543..565a643f5aa11 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -284,7 +284,7 @@ testMatchPhraseIpField required_capability: match_phrase_function from sample_data -| where match(client_ip, "172.21.0.5") +| where match_phrase(client_ip, "172.21.0.5") | keep client_ip, message; client_ip:ip | message:keyword From ace470d67730027866832bdff00b930c4aa9c89d Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 14:36:55 -0400 Subject: [PATCH 26/42] PR feedback --- .../xpack/esql/plugin/MatchPhraseFunctionIT.java | 9 +++++++++ .../expression/function/fulltext/FullTextWritables.java | 5 ++++- .../elasticsearch/xpack/esql/analysis/VerifierTests.java | 3 +++ 3 files changed, 16 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java index 44f28e0c9ea93..58ba2a5313063 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchPhraseFunctionIT.java @@ -13,6 +13,9 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.hamcrest.Matchers; import org.junit.Before; @@ -31,6 +34,12 @@ public void setupIndex() { createAndPopulateIndex(); } + @Override + protected EsqlQueryResponse run(EsqlQueryRequest request) { + assumeTrue("match_phrase function capability not available", EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()); + return super.run(request); + } + public void testSimpleWhereMatchPhrase() { var query = """ FROM test diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java index 313b42088b962..b47594cedb08f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java @@ -21,7 +21,6 @@ public static List getNamedWriteables() { entries.add(QueryString.ENTRY); entries.add(Match.ENTRY); - entries.add(MatchPhrase.ENTRY); entries.add(MultiMatch.ENTRY); entries.add(Kql.ENTRY); @@ -29,6 +28,10 @@ public static List getNamedWriteables() { entries.add(Term.ENTRY); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + entries.add(MatchPhrase.ENTRY); + } + return Collections.unmodifiableList(entries); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index b13a72b7c1545..1a93250e9ebb1 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1216,6 +1216,7 @@ public void testMatchInsideEval() throws Exception { } public void testMatchPhraseInsideEval() throws Exception { + assumeTrue("match_phrase function capability not available", EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()); assertEquals( "1:36: [MatchPhrase] function is only supported in WHERE and STATS commands\n" + "line 1:49: [MatchPhrase] function cannot operate on [title], which is not a field from an index mapping", @@ -1260,6 +1261,7 @@ public void testMatchWithNonIndexedColumnCurrentlyUnsupported() { } public void testMatchPhraseWithNonIndexedColumnCurrentlyUnsupported() { + assumeTrue("match_phrase function capability not available", EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()); assertEquals( "1:74: [MatchPhrase] function cannot operate on [initial], which is not a field from an index mapping", error("from test | eval initial = substring(first_name, 1) | where match_phrase(initial, \"A\")") @@ -1278,6 +1280,7 @@ public void testMatchFunctionIsNotNullable() { } public void testMatchPhraseFunctionIsNotNullable() { + assumeTrue("match_phrase function capability not available", EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()); assertEquals( "1:55: [MatchPhrase] function cannot operate on [text::keyword], which is not a field from an index mapping", error("row n = null | eval text = n + 5 | where match_phrase(text::keyword, \"Anna\")") From 344c8c2bdb918f8e6f1c5f7a2ad1d36b49db90c9 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 14:52:14 -0400 Subject: [PATCH 27/42] Fix auto-commit error --- .../src/main/resources/match-phrase-function.csv-spec | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index 565a643f5aa11..e8604245bc06c 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -489,6 +489,7 @@ r:double | author: text ; testMatchPhraseRequiresExactPhraseMatch +required_capability: match_phrase_function from books | where match_phrase(title, "Lord Rings") | keep book_no From 42579ec9b07e2688f3d83be8b3c2a53c7a52776a Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 28 May 2025 14:58:28 -0400 Subject: [PATCH 28/42] Regenerate docs --- .../esql/_snippets/functions/appendix/values.md | 5 +++-- .../esql/kibana/definition/functions/values.json | 2 +- .../query-languages/esql/kibana/docs/functions/values.md | 3 ++- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/appendix/values.md b/docs/reference/query-languages/esql/_snippets/functions/appendix/values.md index 4a21c08c79756..f8928db80b968 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/appendix/values.md +++ b/docs/reference/query-languages/esql/_snippets/functions/appendix/values.md @@ -1,7 +1,8 @@ % This is generated by ESQL's AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it. -::::{note} -Use [`TOP`](/reference/query-languages/esql/functions-operators/aggregation-functions.md#esql-top) if you need to keep repeated values. +::::{tip} +Use [`TOP`](/reference/query-languages/esql/functions-operators/aggregation-functions.md#esql-top) +if you need to keep repeated values. :::: ::::{warning} This can use a significant amount of memory and ES|QL doesn’t yet diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/values.json b/docs/reference/query-languages/esql/kibana/definition/functions/values.json index 7983f9d2b66ab..6ee883d6fd280 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/values.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/values.json @@ -2,7 +2,7 @@ "comment" : "This is generated by ESQL’s AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it.", "type" : "agg", "name" : "values", - "description" : "Returns unique values as a multivalued field. The order of the returned values isn’t guaranteed.\nIf you need the values returned in order use `MV_SORT`.", + "description" : "Returns unique values as a multivalued field. The order of the returned values isn’t guaranteed.\nIf you need the values returned in order use\n`MV_SORT`.", "signatures" : [ { "params" : [ diff --git a/docs/reference/query-languages/esql/kibana/docs/functions/values.md b/docs/reference/query-languages/esql/kibana/docs/functions/values.md index 130c7f3d60e20..3a95a12d03cd3 100644 --- a/docs/reference/query-languages/esql/kibana/docs/functions/values.md +++ b/docs/reference/query-languages/esql/kibana/docs/functions/values.md @@ -2,7 +2,8 @@ ### VALUES Returns unique values as a multivalued field. The order of the returned values isn’t guaranteed. -If you need the values returned in order use [`MV_SORT`](https://www.elastic.co/docs/reference/query-languages/esql/functions-operators/mv-functions#esql-mv_sort). +If you need the values returned in order use +[`MV_SORT`](https://www.elastic.co/docs/reference/query-languages/esql/functions-operators/mv-functions#esql-mv_sort). ```esql FROM employees From e3e42d6db3ecc4a0f147664155e88bda4d5b01e8 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Thu, 29 May 2025 15:51:57 -0400 Subject: [PATCH 29/42] Update x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java Co-authored-by: Liam Thompson <32779855+leemthompo@users.noreply.github.com> --- .../xpack/esql/expression/function/fulltext/MatchPhrase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index 17714900e8644..ed5a7e8915ef3 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -105,7 +105,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P MatchPhrase can use <> to specify additional options for the match_phrase query. - All <> query parameters are supported. + All [`match_phrase`](/reference/query-languages/query-dsl/query-dsl-match-query-phrase.md) query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row.""", examples = { From a3fc9b61eea6d57d053e8e8ccd10ba61f2865259 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Thu, 29 May 2025 15:51:49 -0400 Subject: [PATCH 30/42] Remove non text field types --- .../_snippets/functions/types/match_phrase.md | 5 - .../definition/functions/match_phrase.json | 125 ------------------ .../src/main/resources/scoring.csv-spec | 2 +- .../function/fulltext/MatchPhrase.java | 14 +- .../function/fulltext/MatchPhraseTests.java | 72 ---------- 5 files changed, 4 insertions(+), 214 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md index b49384a4e211f..480771afb6984 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md @@ -4,13 +4,8 @@ | field | query | options | result | | --- | --- | --- | --- | -| boolean | keyword | named parameters | boolean | -| date | keyword | named parameters | boolean | -| date_nanos | keyword | named parameters | boolean | -| ip | keyword | named parameters | boolean | | keyword | keyword | named parameters | boolean | | keyword | text | named parameters | boolean | | text | keyword | named parameters | boolean | | text | text | named parameters | boolean | -| version | keyword | named parameters | boolean | diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index 102f53dcf944b..80328c618d243 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -4,106 +4,6 @@ "name" : "match_phrase", "description" : "Use `MATCH_PHRASE` to perform a match_phrase query on the specified field.\nUsing `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL.\n\nMatchPhrase can be used on text fields, as well as other field types like keyword, boolean, or date types.\nMatchPhrase is not supported for semantic_text or numeric types.\n\nMatchPhrase can use function named parameters to specify additional options for the\nmatch_phrase query.\nAll match_phrase query parameters are supported.\n\n`MATCH_PHRASE` returns true if the provided query matches the row.", "signatures" : [ - { - "params" : [ - { - "name" : "field", - "type" : "boolean", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "keyword", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "date", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "keyword", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "date_nanos", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "keyword", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "ip", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "keyword", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, { "params" : [ { @@ -203,31 +103,6 @@ ], "variadic" : false, "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "version", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "keyword", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" } ], "examples" : [ diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index 908dd154f5e85..d794642dc9ac0 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -582,5 +582,5 @@ from books metadata _score ignoreOrder:true book_no:keyword | title:text | author:text | _score:double -2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 +2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 42 ; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index ed5a7e8915ef3..fb97d8302d401 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -57,8 +57,6 @@ import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; -import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; -import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; import static org.elasticsearch.xpack.esql.core.type.DataType.FLOAT; import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; @@ -78,7 +76,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P "MatchPhrase", MatchPhrase::readFrom ); - public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT, BOOLEAN, DATETIME, DATE_NANOS, IP, VERSION); + public static final Set FIELD_DATA_TYPES = Set.of(KEYWORD, TEXT); public static final Set QUERY_DATA_TYPES = Set.of(KEYWORD, TEXT); protected final Expression field; @@ -119,11 +117,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P ) public MatchPhrase( Source source, - @Param( - name = "field", - type = { "keyword", "text", "boolean", "date", "date_nanos", "ip", "version" }, - description = "Field that the query will target." - ) Expression field, + @Param(name = "field", type = { "keyword", "text" }, description = "Field that the query will target.") Expression field, @Param( name = "query", type = { "keyword", "text" }, @@ -211,9 +205,7 @@ protected TypeResolution resolveParams() { } private TypeResolution resolveField() { - return isNotNull(field, sourceText(), FIRST).and( - isType(field, FIELD_DATA_TYPES::contains, sourceText(), FIRST, "keyword, text, boolean, date, date_nanos, ip, version") - ); + return isNotNull(field, sourceText(), FIRST).and(isType(field, FIELD_DATA_TYPES::contains, sourceText(), FIRST, "keyword, text")); } private TypeResolution resolveQuery() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java index adc36af5a9328..6d74195b821e4 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -50,82 +50,10 @@ public static Iterable parameters() { private static List testCaseSuppliers() { List suppliers = new ArrayList<>(); - addQueryAsStringTestCases(suppliers); addStringTestCases(suppliers); return suppliers; } - public static void addQueryAsStringTestCases(List suppliers) { - - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.booleanCases(), - TestCaseSupplier.stringCases(DataType.KEYWORD), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.ipCases(), - TestCaseSupplier.stringCases(DataType.KEYWORD), - List.of(), - false - ) - ); - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.versionCases(""), - TestCaseSupplier.stringCases(DataType.KEYWORD), - List.of(), - false - ) - ); - // Datetime - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.dateCases(), - TestCaseSupplier.stringCases(DataType.KEYWORD), - List.of(), - false - ) - ); - - suppliers.addAll( - TestCaseSupplier.forBinaryNotCasting( - null, - "field", - "query", - Object::equals, - DataType.BOOLEAN, - TestCaseSupplier.dateNanosCases(), - TestCaseSupplier.stringCases(DataType.KEYWORD), - List.of(), - false - ) - ); - } - public static void addStringTestCases(List suppliers) { for (DataType fieldType : DataType.stringTypes()) { if (DataType.UNDER_CONSTRUCTION.containsKey(fieldType)) { From aa018d4586dadcfa005e26aefbe75bb4fb69fa9c Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Thu, 29 May 2025 16:13:20 -0400 Subject: [PATCH 31/42] Fake test data --- .../src/main/resources/scoring.csv-spec | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index d794642dc9ac0..9a1a08507a592 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -560,27 +560,27 @@ avg_score:double | max_score:double | min_score:double ; testMatchPhraseWithScore -required_capability: match_phrase + +required_capability: match_phrase_function required_capability: metadata_score from books metadata _score -| where match_phrase(title, "J. R. R. Tolkien") -| keep book_no, title, author, _score; -ignoreOrder:true +| where match_phrase(author, "J. R. R. Tolkien") +| keep book_no, title, author, _score +; book_no:keyword | title:text | author:text | _score:double -2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 1.9662091732025146 +123 | Invalid Data | Invalid Data | 42 ; testMatchPhraseWithScoreBoost -required_capability: match_phrase -required_capability: metadata_score +required_capability: match_phrase_function from books metadata _score | where match_phrase(title, "J. R. R. Tolkien", {"boost": 5}) -| keep book_no, title, author, _score; -ignoreOrder:true +| keep book_no, title, author, _score +; book_no:keyword | title:text | author:text | _score:double -2847 | To Love A Dark Stranger (Lovegram Historical Romance) | Colleen Faulkner | 42 +123 | Invalid Data | Invalid Data | 42 ; From 1b16c8d7923704cfbf8aba1fb454f75cd0c83ded Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 30 May 2025 08:46:18 -0400 Subject: [PATCH 32/42] Remove tests that no longer should pass without ip/date/version support --- .../resources/match-phrase-function.csv-spec | 44 ------------------- 1 file changed, 44 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec index e8604245bc06c..5f2a9c85117c2 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/match-phrase-function.csv-spec @@ -269,50 +269,6 @@ emp_no:integer | first_name:keyword | last_name:keyword 10043 | Yishay | Tzvieli ; -testMatchPhraseVersionField -required_capability: match_phrase_function - -from apps -| where match_phrase(version, "2.1") -| keep name, version; - -name:keyword | version:version -bbbbb | 2.1 -; - -testMatchPhraseIpField -required_capability: match_phrase_function - -from sample_data -| where match_phrase(client_ip, "172.21.0.5") -| keep client_ip, message; - -client_ip:ip | message:keyword -172.21.0.5 | Disconnected -; - -testMatchPhraseDateFieldAsString -required_capability: match_phrase_function - -from date_nanos -| where match_phrase(millis, "2023-10-23T13:55:01.543Z") -| keep millis; - -millis:date -2023-10-23T13:55:01.543Z -; - -testMatchPhraseDateNanosFieldAsString -required_capability: match_phrase_function - -from date_nanos -| where match_phrase(nanos, "2023-10-23T13:55:01.543123456Z") -| keep nanos; - -nanos:date_nanos -2023-10-23T13:55:01.543123456Z -; - testMatchPhraseWithOptionsSlop required_capability: match_phrase_function from books From 50fe44f3b6cd5faa7871e3f694c702b70296177d Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 30 May 2025 10:25:14 -0400 Subject: [PATCH 33/42] Put real data in score tests now that I was able to engineer a failure --- .../src/main/resources/scoring.csv-spec | 26 +++++++++++++++---- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index 9a1a08507a592..df00f22073671 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -569,18 +569,34 @@ from books metadata _score | keep book_no, title, author, _score ; -book_no:keyword | title:text | author:text | _score:double -123 | Invalid Data | Invalid Data | 42 +book_no:keyword | title:text | author:text | _score:double + 7480 | The Hobbit | J. R. R. Tolkien | 7.587270736694336 + 6405 | The Hobbit or There and Back Again | J. R. R. Tolkien | 7.587270736694336 + 6760 | Roverandom | J. R. R. Tolkien | 7.503391265869141 + 7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) | [Alan Lee, J. R. R. Tolkien] | 6.170376777648926 + 7670 | A Middle English Reader and Vocabulary | [J. R. R. Tolkien, Kenneth Sisam] | 4.533677101135254 + 7635 | Oliphaunt (Beastly Verse) | J. R. R. Tolkien | 5.634923934936523 + 4289 | Poems from the Hobbit | J R R Tolkien | 5.634923934936523 + 1463 | Realms of Tolkien: Images of Middle-earth | J. R. R. Tolkien | 5.634923934936523 + 2714 | Return of the King Being the Third Part of The Lord of the Rings | J. R. R. Tolkien | 5.634923934936523 ; testMatchPhraseWithScoreBoost required_capability: match_phrase_function from books metadata _score -| where match_phrase(title, "J. R. R. Tolkien", {"boost": 5}) +| where match_phrase(author, "J. R. R. Tolkien", {"boost": 5}) | keep book_no, title, author, _score ; -book_no:keyword | title:text | author:text | _score:double -123 | Invalid Data | Invalid Data | 42 +book_no:keyword | title:text | author:text | _score:double + 7480 | The Hobbit | J. R. R. Tolkien | 37.93635177612305 + 6405 | The Hobbit or There and Back Again | J. R. R. Tolkien | 37.93635177612305 + 6760 | Roverandom | J. R. R. Tolkien | 37.51696014404297 + 7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) | [Alan Lee, J. R. R. Tolkien] | 30.851882934570312 + 7670 | A Middle English Reader and Vocabulary | [J. R. R. Tolkien, Kenneth Sisam] | 22.668384552001953 + 7635 | Oliphaunt (Beastly Verse) | J. R. R. Tolkien | 28.174617767333984 + 4289 | Poems from the Hobbit | J R R Tolkien | 28.174617767333984 + 1463 | Realms of Tolkien: Images of Middle-earth | J. R. R. Tolkien | 28.174617767333984 + 2714 | Return of the King Being the Third Part of The Lord of the Rings | J. R. R. Tolkien | 28.174617767333984 ; From 9591c18da21473413b51e8977044a85c040ecb46 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 30 May 2025 13:31:56 -0400 Subject: [PATCH 34/42] Realized the scoring test might be flakey because how it was written, updated --- .../functions/description/match_phrase.md | 2 +- .../definition/functions/match_phrase.json | 2 +- .../kibana/docs/functions/match_phrase.md | 2 +- .../src/main/resources/scoring.csv-spec | 30 +++++-------------- 4 files changed, 11 insertions(+), 25 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md index 168c1fed49a8b..6970bbfb1a3e2 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/description/match_phrase.md @@ -2,5 +2,5 @@ **Description** -Use `MATCH_PHRASE` to perform a [match_phrase query](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. MatchPhrase can be used on [text](/reference/elasticsearch/mapping-reference/text.md) fields, as well as other field types like keyword, boolean, or date types. MatchPhrase is not supported for [semantic_text](/reference/elasticsearch/mapping-reference/semantic-text.md) or numeric types. MatchPhrase can use [function named parameters](/reference/query-languages/esql/esql-syntax.md#esql-function-named-params) to specify additional options for the match_phrase query. All [match_phrase](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. +Use `MATCH_PHRASE` to perform a [match_phrase query](/reference/query-languages/query-dsl/query-dsl-match-query.md#query-dsl-match-query-phrase) on the specified field. Using `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL. MatchPhrase can be used on [text](/reference/elasticsearch/mapping-reference/text.md) fields, as well as other field types like keyword, boolean, or date types. MatchPhrase is not supported for [semantic_text](/reference/elasticsearch/mapping-reference/semantic-text.md) or numeric types. MatchPhrase can use [function named parameters](/reference/query-languages/esql/esql-syntax.md#esql-function-named-params) to specify additional options for the match_phrase query. All [`match_phrase`](/reference/query-languages/query-dsl/query-dsl-match-query-phrase.md) query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index 80328c618d243..060d70a1b6ac6 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -2,7 +2,7 @@ "comment" : "This is generated by ESQL’s AbstractFunctionTestCase. Do no edit it. See ../README.md for how to regenerate it.", "type" : "scalar", "name" : "match_phrase", - "description" : "Use `MATCH_PHRASE` to perform a match_phrase query on the specified field.\nUsing `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL.\n\nMatchPhrase can be used on text fields, as well as other field types like keyword, boolean, or date types.\nMatchPhrase is not supported for semantic_text or numeric types.\n\nMatchPhrase can use function named parameters to specify additional options for the\nmatch_phrase query.\nAll match_phrase query parameters are supported.\n\n`MATCH_PHRASE` returns true if the provided query matches the row.", + "description" : "Use `MATCH_PHRASE` to perform a match_phrase query on the specified field.\nUsing `MATCH_PHRASE` is equivalent to using the `match_phrase` query in the Elasticsearch Query DSL.\n\nMatchPhrase can be used on text fields, as well as other field types like keyword, boolean, or date types.\nMatchPhrase is not supported for semantic_text or numeric types.\n\nMatchPhrase can use function named parameters to specify additional options for the\nmatch_phrase query.\nAll `match_phrase` query parameters are supported.\n\n`MATCH_PHRASE` returns true if the provided query matches the row.", "signatures" : [ { "params" : [ diff --git a/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md b/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md index ac27ba2da06f9..ee8552563277c 100644 --- a/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md +++ b/docs/reference/query-languages/esql/kibana/docs/functions/match_phrase.md @@ -9,7 +9,7 @@ MatchPhrase is not supported for [semantic_text](https://www.elastic.co/docs/ref MatchPhrase can use [function named parameters](https://www.elastic.co/docs/reference/query-languages/esql/esql-syntax#esql-function-named-params) to specify additional options for the match_phrase query. -All [match_phrase](https://www.elastic.co/docs/reference/query-languages/query-dsl/query-dsl-match-query#query-dsl-match-query-phrase) query parameters are supported. +All [`match_phrase`](https://www.elastic.co/docs/reference/query-languages/query-dsl/query-dsl-match-query-phrase) query parameters are supported. `MATCH_PHRASE` returns true if the provided query matches the row. diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec index df00f22073671..90a92a658a497 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -565,38 +565,24 @@ required_capability: match_phrase_function required_capability: metadata_score from books metadata _score -| where match_phrase(author, "J. R. R. Tolkien") +| where match_phrase(title, "J. R. R. Tolkien") | keep book_no, title, author, _score ; -book_no:keyword | title:text | author:text | _score:double - 7480 | The Hobbit | J. R. R. Tolkien | 7.587270736694336 - 6405 | The Hobbit or There and Back Again | J. R. R. Tolkien | 7.587270736694336 - 6760 | Roverandom | J. R. R. Tolkien | 7.503391265869141 - 7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) | [Alan Lee, J. R. R. Tolkien] | 6.170376777648926 - 7670 | A Middle English Reader and Vocabulary | [J. R. R. Tolkien, Kenneth Sisam] | 4.533677101135254 - 7635 | Oliphaunt (Beastly Verse) | J. R. R. Tolkien | 5.634923934936523 - 4289 | Poems from the Hobbit | J R R Tolkien | 5.634923934936523 - 1463 | Realms of Tolkien: Images of Middle-earth | J. R. R. Tolkien | 5.634923934936523 - 2714 | Return of the King Being the Third Part of The Lord of the Rings | J. R. R. Tolkien | 5.634923934936523 +book_no:keyword | title:text | author:text | _score:double + 5335 | Letters of J R R Tolkien | J.R.R. Tolkien | 9.017186164855957 + 2130 | The J. R. R. Tolkien Audio Collection | [Christopher Tolkien, John Ronald Reuel Tolkien] | 8.412636756896973 ; testMatchPhraseWithScoreBoost required_capability: match_phrase_function from books metadata _score -| where match_phrase(author, "J. R. R. Tolkien", {"boost": 5}) +| where match_phrase(title, "J. R. R. Tolkien", {"boost": 5}) | keep book_no, title, author, _score ; -book_no:keyword | title:text | author:text | _score:double - 7480 | The Hobbit | J. R. R. Tolkien | 37.93635177612305 - 6405 | The Hobbit or There and Back Again | J. R. R. Tolkien | 37.93635177612305 - 6760 | Roverandom | J. R. R. Tolkien | 37.51696014404297 - 7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) | [Alan Lee, J. R. R. Tolkien] | 30.851882934570312 - 7670 | A Middle English Reader and Vocabulary | [J. R. R. Tolkien, Kenneth Sisam] | 22.668384552001953 - 7635 | Oliphaunt (Beastly Verse) | J. R. R. Tolkien | 28.174617767333984 - 4289 | Poems from the Hobbit | J R R Tolkien | 28.174617767333984 - 1463 | Realms of Tolkien: Images of Middle-earth | J. R. R. Tolkien | 28.174617767333984 - 2714 | Return of the King Being the Third Part of The Lord of the Rings | J. R. R. Tolkien | 28.174617767333984 +book_no:keyword | title:text | author:text | _score:double + 5335 | Letters of J R R Tolkien | J.R.R. Tolkien | 45.0859260559082 + 2130 | The J. R. R. Tolkien Audio Collection | [Christopher Tolkien, John Ronald Reuel Tolkien] | 42.06318283081055 ; From 1cc6eba7b0470c373bb47604b2d5162131ac910e Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Mon, 2 Jun 2025 09:47:56 -0400 Subject: [PATCH 35/42] PR feedback --- .../xpack/esql/expression/function/EsqlFunctionRegistry.java | 4 ++-- .../xpack/esql/expression/function/fulltext/MatchPhrase.java | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java index e4c3a7694e8c4..16f433d9ce28e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java @@ -441,7 +441,6 @@ private static FunctionDefinition[][] functions() { new FunctionDefinition[] { def(Kql.class, uni(Kql::new), "kql"), def(Match.class, tri(Match::new), "match"), - def(MatchPhrase.class, tri(MatchPhrase::new), "match_phrase"), def(MultiMatch.class, MultiMatch::new, "multi_match"), def(QueryString.class, bi(QueryString::new), "qstr") } }; @@ -460,7 +459,8 @@ private static FunctionDefinition[][] snapshotFunctions() { def(AvgOverTime.class, uni(AvgOverTime::new), "avg_over_time"), def(LastOverTime.class, LastOverTime::withUnresolvedTimestamp, "last_over_time"), def(FirstOverTime.class, FirstOverTime::withUnresolvedTimestamp, "first_over_time"), - def(Term.class, bi(Term::new), "term") } }; + def(Term.class, bi(Term::new), "term"), + def(MatchPhrase.class, tri(MatchPhrase::new), "match_phrase") } }; } public EsqlFunctionRegistry snapshotRegistry() { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index fb97d8302d401..b8824b2e65c7f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -186,8 +186,7 @@ private static MatchPhrase readFrom(StreamInput in) throws IOException { Source source = Source.readFrom((PlanStreamInput) in); Expression field = in.readNamedWriteable(Expression.class); Expression query = in.readNamedWriteable(Expression.class); - QueryBuilder queryBuilder = null; - queryBuilder = in.readOptionalNamedWriteable(QueryBuilder.class); + QueryBuilder queryBuilder = in.readOptionalNamedWriteable(QueryBuilder.class); return new MatchPhrase(source, field, query, null, queryBuilder); } From 77c9fbf8fe5732c5bf8b6c9e13b0fb4c92706f9f Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Mon, 2 Jun 2025 13:12:08 -0400 Subject: [PATCH 36/42] PR feedback --- .../definition/functions/match_phrase.json | 2 +- .../function/fulltext/MatchPhrase.java | 22 +++---------------- .../function/fulltext/MatchPhraseTests.java | 10 --------- 3 files changed, 4 insertions(+), 30 deletions(-) diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index 060d70a1b6ac6..37528f336ac4f 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -110,5 +110,5 @@ null ], "preview" : true, - "snapshot_only" : false + "snapshot_only" : true } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index b8824b2e65c7f..eb2a381f2b589 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -118,11 +118,7 @@ public class MatchPhrase extends FullTextFunction implements OptionalArgument, P public MatchPhrase( Source source, @Param(name = "field", type = { "keyword", "text" }, description = "Field that the query will target.") Expression field, - @Param( - name = "query", - type = { "keyword", "text" }, - description = "Value to find in the provided field." - ) Expression matchPhraseQuery, + @Param(name = "query", type = { "keyword" }, description = "Value to find in the provided field.") Expression matchPhraseQuery, @MapParam( name = "options", params = { @@ -200,7 +196,7 @@ public final void writeTo(StreamOutput out) throws IOException { @Override protected TypeResolution resolveParams() { - return resolveField().and(resolveQuery()).and(resolveOptions(options(), THIRD)).and(checkParamCompatibility()); + return resolveField().and(resolveQuery()).and(resolveOptions(options(), THIRD)); } private TypeResolution resolveField() { @@ -208,23 +204,11 @@ private TypeResolution resolveField() { } private TypeResolution resolveQuery() { - return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword, text").and( + return isType(query(), QUERY_DATA_TYPES::contains, sourceText(), SECOND, "keyword").and( isNotNullAndFoldable(query(), sourceText(), SECOND) ); } - private TypeResolution checkParamCompatibility() { - DataType fieldType = field().dataType(); - DataType queryType = query().dataType(); - - // Field and query types should match. If the query is a string, then it can match any field type. - if ((fieldType == queryType) || (queryType == KEYWORD) || (queryType == TEXT)) { - return TypeResolution.TYPE_RESOLVED; - } - - return new TypeResolution(formatIncompatibleTypesMessage(fieldType, queryType, sourceText())); - } - @Override protected Map resolvedOptions() throws InvalidArgumentException { return matchPhraseQueryOptions(); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java index 6d74195b821e4..ce996962398bf 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -69,16 +69,6 @@ public static void addStringTestCases(List suppliers) { (o1, o2) -> true ) ); - - suppliers.add( - TestCaseSupplier.testCaseSupplier( - queryDataSupplier, - new TestCaseSupplier.TypedDataSupplier(fieldType.typeName(), () -> randomAlphaOfLength(10), DataType.TEXT), - (d1, d2) -> equalTo("string"), - DataType.BOOLEAN, - (o1, o2) -> true - ) - ); } } } From 6983392a4fb2ce6343e013a023501d2abeda4e22 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Mon, 2 Jun 2025 17:37:50 +0000 Subject: [PATCH 37/42] [CI] Auto commit changes from spotless --- .../xpack/esql/expression/function/fulltext/MatchPhrase.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java index eb2a381f2b589..471619dfc771e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhrase.java @@ -64,7 +64,6 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; import static org.elasticsearch.xpack.esql.core.type.DataType.VERSION; -import static org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison.formatIncompatibleTypesMessage; /** * Full text function that performs a {@link org.elasticsearch.xpack.esql.querydsl.query.MatchPhraseQuery} . From f94faebb9ac749b8072032f2ad71441a167833c0 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 3 Jun 2025 11:34:57 -0400 Subject: [PATCH 38/42] Add check to MatchPhrase tests --- .../expression/function/fulltext/MatchPhraseTests.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java index ce996962398bf..496f269da9dc0 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/fulltext/MatchPhraseTests.java @@ -11,6 +11,7 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.Literal; @@ -22,6 +23,7 @@ import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier; import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; +import org.junit.Before; import java.util.ArrayList; import java.util.List; @@ -39,6 +41,11 @@ @FunctionName("match_phrase") public class MatchPhraseTests extends AbstractFunctionTestCase { + @Before + public void checkCapability() { + assumeTrue("MatchPhrase is not supported in this version of ESQL", EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()); + } + public MatchPhraseTests(@Name("TestCase") Supplier testCaseSupplier) { this.testCase = testCaseSupplier.get(); } From 373c8eb917b11a1900b2294815ebce10d2e17b28 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 3 Jun 2025 13:18:49 -0400 Subject: [PATCH 39/42] Fix merge errors --- .../function/fulltext/FullTextFunction.java | 48 +++++++++++++++++++ .../expression/function/fulltext/Match.java | 1 + 2 files changed, 49 insertions(+) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 19402be5fe733..bf7564ee40db4 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -21,6 +21,7 @@ import org.elasticsearch.xpack.esql.core.InvalidArgumentException; import org.elasticsearch.xpack.esql.core.expression.EntryExpression; import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.FoldContext; import org.elasticsearch.xpack.esql.core.expression.Literal; import org.elasticsearch.xpack.esql.core.expression.MapExpression; @@ -31,7 +32,9 @@ import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.DataTypeConverter; +import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.evaluator.mapper.EvaluatorMapper; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; import org.elasticsearch.xpack.esql.expression.predicate.logical.BinaryLogic; import org.elasticsearch.xpack.esql.expression.predicate.logical.Not; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; @@ -57,6 +60,8 @@ import static org.elasticsearch.xpack.esql.common.Failure.fail; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.DEFAULT; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isFoldable; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isMapExpression; +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNull; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isNotNullAndFoldable; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; @@ -365,4 +370,47 @@ protected static void populateOptionsMap( } } } + + protected TypeResolution resolveOptions(Expression options, TypeResolutions.ParamOrdinal paramOrdinal) { + if (options != null) { + TypeResolution resolution = isNotNull(options, sourceText(), paramOrdinal); + if (resolution.unresolved()) { + return resolution; + } + // MapExpression does not have a DataType associated with it + resolution = isMapExpression(options, sourceText(), paramOrdinal); + if (resolution.unresolved()) { + return resolution; + } + + try { + resolvedOptions(); + } catch (InvalidArgumentException e) { + return new TypeResolution(e.getMessage()); + } + } + return TypeResolution.TYPE_RESOLVED; + } + + protected Map resolvedOptions() throws InvalidArgumentException { + return Map.of(); + } + + public static String getNameFromFieldAttribute(FieldAttribute fieldAttribute) { + String fieldName = fieldAttribute.name(); + if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { + // If we have multiple field types, we allow the query to be done, but getting the underlying field name + fieldName = multiTypeEsField.getName(); + } + return fieldName; + } + + public static FieldAttribute fieldAsFieldAttribute(Expression field) { + Expression fieldExpression = field; + // Field may be converted to other data type (field_name :: data_type), so we need to check the original field + if (fieldExpression instanceof AbstractConvertFunction convertFunction) { + fieldExpression = convertFunction.field(); + } + return fieldExpression instanceof FieldAttribute fieldAttribute ? fieldAttribute : null; + } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java index da42e7e60af94..575932f325dac 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java @@ -26,6 +26,7 @@ import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.core.util.Check; import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.Example; From b9410f41c391f2112eec8839e1e7def99073d8bc Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Tue, 3 Jun 2025 17:29:13 +0000 Subject: [PATCH 40/42] [CI] Auto commit changes from spotless --- .../xpack/esql/expression/function/fulltext/Match.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java index 575932f325dac..da42e7e60af94 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java @@ -26,7 +26,6 @@ import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; -import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.core.util.Check; import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.Example; From 7db150f09dbc49641a69ffaf76688d44122cb19c Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Tue, 3 Jun 2025 16:15:59 -0400 Subject: [PATCH 41/42] Test generated docs --- .../_snippets/functions/types/match_phrase.md | 2 - .../definition/functions/match_phrase.json | 50 ------------------- 2 files changed, 52 deletions(-) diff --git a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md index 480771afb6984..8564939f598f0 100644 --- a/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md +++ b/docs/reference/query-languages/esql/_snippets/functions/types/match_phrase.md @@ -5,7 +5,5 @@ | field | query | options | result | | --- | --- | --- | --- | | keyword | keyword | named parameters | boolean | -| keyword | text | named parameters | boolean | | text | keyword | named parameters | boolean | -| text | text | named parameters | boolean | diff --git a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json index 37528f336ac4f..fee736b505793 100644 --- a/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json +++ b/docs/reference/query-languages/esql/kibana/definition/functions/match_phrase.json @@ -29,31 +29,6 @@ "variadic" : false, "returnType" : "boolean" }, - { - "params" : [ - { - "name" : "field", - "type" : "keyword", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "text", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" - }, { "params" : [ { @@ -78,31 +53,6 @@ ], "variadic" : false, "returnType" : "boolean" - }, - { - "params" : [ - { - "name" : "field", - "type" : "text", - "optional" : false, - "description" : "Field that the query will target." - }, - { - "name" : "query", - "type" : "text", - "optional" : false, - "description" : "Value to find in the provided field." - }, - { - "name" : "options", - "type" : "function_named_parameters", - "mapParams" : "{name='zero_terms_query', values=[none, all], description='Indicates whether all documents or none are returned if the analyzer removes all tokens, such as when using a stop filter. Defaults to none.'}, {name='boost', values=[2.5], description='Floating point number used to decrease or increase the relevance scores of the query. Defaults to 1.0.'}, {name='analyzer', values=[standard], description='Analyzer used to convert the text in the query value into token. Defaults to the index-time analyzer mapped for the field. If no analyzer is mapped, the index’s default analyzer is used.'}, {name='slop', values=[1], description='Maximum number of positions allowed between matching tokens. Defaults to 0. Transposed terms have a slop of 2.'}", - "optional" : true, - "description" : "(Optional) MatchPhrase additional options as <>. See <> for more information." - } - ], - "variadic" : false, - "returnType" : "boolean" } ], "examples" : [ From e68053839a82e325a93d4be42a810b9c4262b382 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Wed, 4 Jun 2025 11:00:02 -0400 Subject: [PATCH 42/42] Add additional verifier tests --- .../xpack/esql/analysis/VerifierTests.java | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 29d7f75e5e6bf..311d22c743248 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -19,6 +19,7 @@ import org.elasticsearch.xpack.esql.core.type.InvalidMappedField; import org.elasticsearch.xpack.esql.core.type.UnsupportedEsField; import org.elasticsearch.xpack.esql.expression.function.fulltext.Match; +import org.elasticsearch.xpack.esql.expression.function.fulltext.MatchPhrase; import org.elasticsearch.xpack.esql.expression.function.fulltext.MultiMatch; import org.elasticsearch.xpack.esql.expression.function.fulltext.QueryString; import org.elasticsearch.xpack.esql.index.EsIndex; @@ -1363,6 +1364,9 @@ public void testFullTextFunctionsOnlyAllowedInWhere() throws Exception { if (EsqlCapabilities.Cap.MULTI_MATCH_FUNCTION.isEnabled()) { checkFullTextFunctionsOnlyAllowedInWhere("MultiMatch", "multi_match(\"Meditation\", title, body)", "function"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkFullTextFunctionsOnlyAllowedInWhere("MatchPhrase", "match_phrase(title, \"Meditation\")", "function"); + } } private void checkFullTextFunctionsOnlyAllowedInWhere(String functionName, String functionInvocation, String functionType) @@ -1398,6 +1402,9 @@ public void testFullTextFunctionsDisjunctions() { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { checkWithFullTextFunctionsDisjunctions("term(title, \"Meditation\")"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkWithFullTextFunctionsDisjunctions("match_phrase(title, \"Meditation\")"); + } } private void checkWithFullTextFunctionsDisjunctions(String functionInvocation) { @@ -1459,6 +1466,9 @@ public void testFullTextFunctionsWithNonBooleanFunctions() { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { checkFullTextFunctionsWithNonBooleanFunctions("Term", "term(title, \"Meditation\")", "function"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkFullTextFunctionsWithNonBooleanFunctions("MatchPhrase", "match_phrase(title, \"Meditation\")", "function"); + } } private void checkFullTextFunctionsWithNonBooleanFunctions(String functionName, String functionInvocation, String functionType) { @@ -1526,6 +1536,9 @@ public void testFullTextFunctionsTargetsExistingField() throws Exception { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { testFullTextFunctionTargetsExistingField("term(fist_name, \"Meditation\")"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + testFullTextFunctionTargetsExistingField("match_phrase(title, \"Meditation\")"); + } } private void testFullTextFunctionTargetsExistingField(String functionInvocation) throws Exception { @@ -2050,6 +2063,9 @@ public void testFullTextFunctionOptions() { if (EsqlCapabilities.Cap.MULTI_MATCH_FUNCTION.isEnabled()) { checkOptionDataTypes(MultiMatch.OPTIONS, "FROM test | WHERE MULTI_MATCH(\"Jean\", title, body, {\"%s\": %s})"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkOptionDataTypes(MatchPhrase.ALLOWED_OPTIONS, "FROM test | WHERE MATCH_PHRASE(title, \"Jean\", {\"%s\": %s})"); + } } /** @@ -2110,6 +2126,9 @@ public void testFullTextFunctionCurrentlyUnsupportedBehaviour() throws Exception if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { testFullTextFunctionsCurrentlyUnsupportedBehaviour("term(title, \"Meditation\")"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + testFullTextFunctionsCurrentlyUnsupportedBehaviour("match_phrase(title, \"Meditation\")"); + } } private void testFullTextFunctionsCurrentlyUnsupportedBehaviour(String functionInvocation) throws Exception { @@ -2132,6 +2151,10 @@ public void testFullTextFunctionsNullArgs() throws Exception { checkFullTextFunctionNullArgs("term(null, \"query\")", "first"); checkFullTextFunctionNullArgs("term(title, null)", "second"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkFullTextFunctionNullArgs("match_phrase(null, \"query\")", "first"); + checkFullTextFunctionNullArgs("match_phrase(title, null)", "second"); + } } private void checkFullTextFunctionNullArgs(String functionInvocation, String argOrdinal) throws Exception { @@ -2152,6 +2175,9 @@ public void testFullTextFunctionsConstantQuery() throws Exception { if (EsqlCapabilities.Cap.TERM_FUNCTION.isEnabled()) { checkFullTextFunctionsConstantQuery("term(title, tags)", "second"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkFullTextFunctionsConstantQuery("match_phrase(title, tags)", "second"); + } } private void checkFullTextFunctionsConstantQuery(String functionInvocation, String argOrdinal) throws Exception { @@ -2178,6 +2204,9 @@ public void testFullTextFunctionsInStats() { if (EsqlCapabilities.Cap.MULTI_MATCH_FUNCTION.isEnabled()) { checkFullTextFunctionsInStats("multi_match(\"Meditation\", title, body)"); } + if (EsqlCapabilities.Cap.MATCH_PHRASE_FUNCTION.isEnabled()) { + checkFullTextFunctionsInStats("match_phrase(title, \"Meditation\")"); + } } private void checkFullTextFunctionsInStats(String functionInvocation) {