From 3bcc6546d9b4f7b267758bbbb98a74f23cf131ef Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Fri, 2 May 2025 15:09:38 -0400 Subject: [PATCH 01/10] 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/10] 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/10] 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/10] 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/10] 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/10] 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/10] 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/10] 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/10] 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/10] 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 = {