extraCheckers,
+ Settings settings
) {
this.indexResolver = indexResolver;
this.preAnalyzer = new PreAnalyzer();
this.functionRegistry = new EsqlFunctionRegistry();
this.mapper = new Mapper();
this.metrics = new Metrics(functionRegistry);
- this.verifier = new Verifier(metrics, licenseState, extraCheckers);
+ this.verifier = new Verifier(metrics, licenseState, extraCheckers, settings);
this.planTelemetryManager = new PlanTelemetryManager(meterRegistry);
this.queryLog = queryLog;
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java
index 5f675adbcd50..dab2b35025aa 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizer.java
@@ -30,7 +30,8 @@ import static org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer.operat
* This class is part of the planner. Data node level logical optimizations. At this point we have access to
* {@link org.elasticsearch.xpack.esql.stats.SearchStats} which provides access to metadata about the index.
*
- * NB: This class also reapplies all the rules from {@link LogicalPlanOptimizer#operators()} and {@link LogicalPlanOptimizer#cleanup()}
+ *
NB: This class also reapplies all the rules from {@link LogicalPlanOptimizer#operators(boolean)}
+ * and {@link LogicalPlanOptimizer#cleanup()}
*/
public class LocalLogicalPlanOptimizer extends ParameterizedRuleExecutor {
@@ -58,8 +59,8 @@ public class LocalLogicalPlanOptimizer extends ParameterizedRuleExecutor localOperators() {
- var operators = operators();
- var rules = operators().rules();
+ var operators = operators(true);
+ var rules = operators.rules();
List> newRules = new ArrayList<>(rules.length);
// apply updates to existing rules that have different applicability locally
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java
index f64f4fe38ac0..14a858f85fd2 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizer.java
@@ -82,7 +82,7 @@ import java.util.List;
* The {@link LogicalPlanOptimizer#substitutions()} phase rewrites things to expand out shorthand in the syntax. For example,
* a nested expression embedded in a stats gets replaced with an eval followed by a stats, followed by another eval. This phase
* also applies surrogates, such as replacing an average with a sum divided by a count.
- * {@link LogicalPlanOptimizer#operators()} (NB: The word "operator" is extremely overloaded and referrers to many different
+ * {@link LogicalPlanOptimizer#operators(boolean)} (NB: The word "operator" is extremely overloaded and referrers to many different
* things.) transform the tree in various different ways. This includes folding (i.e. computing constant expressions at parse
* time), combining expressions, dropping redundant clauses, and some normalization such as putting literals on the right whenever
* possible. These rules are run in a loop until none of the rules make any changes to the plan (there is also a safety shut off
@@ -90,14 +90,14 @@ import java.util.List;
* {@link LogicalPlanOptimizer#cleanup()} Which can replace sorts+limit with a TopN
*
*
- * Note that the {@link LogicalPlanOptimizer#operators()} and {@link LogicalPlanOptimizer#cleanup()} steps are reapplied at the
+ *
Note that the {@link LogicalPlanOptimizer#operators(boolean)} and {@link LogicalPlanOptimizer#cleanup()} steps are reapplied at the
* {@link LocalLogicalPlanOptimizer} layer.
*/
public class LogicalPlanOptimizer extends ParameterizedRuleExecutor {
private static final List> RULES = List.of(
substitutions(),
- operators(),
+ operators(false),
new Batch<>("Skip Compute", new SkipQueryOnLimitZero()),
cleanup(),
new Batch<>("Set as Optimized", Limiter.ONCE, new SetAsOptimized())
@@ -160,10 +160,10 @@ public class LogicalPlanOptimizer extends ParameterizedRuleExecutor operators() {
+ protected static Batch operators(boolean local) {
return new Batch<>(
"Operator Optimization",
- new CombineProjections(),
+ new CombineProjections(local),
new CombineEvals(),
new PruneEmptyPlans(),
new PropagateEmptyRelation(),
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java
index e9ca958c5e97..0a3f0cca0d9e 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/CombineProjections.java
@@ -18,18 +18,24 @@ import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
import org.elasticsearch.xpack.esql.expression.function.grouping.GroupingFunction;
import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
+import org.elasticsearch.xpack.esql.plan.logical.Eval;
import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.esql.plan.logical.Project;
import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
+import java.util.Set;
public final class CombineProjections extends OptimizerRules.OptimizerRule {
+ // don't drop groupings from a local plan, as the layout has already been agreed upon
+ private final boolean local;
- public CombineProjections() {
+ public CombineProjections(boolean local) {
super(OptimizerRules.TransformDirection.UP);
+ this.local = local;
}
@Override
@@ -60,27 +66,91 @@ public final class CombineProjections extends OptimizerRules.OptimizerRule groupingAttrs = new ArrayList<>(a.groupings().size());
- for (Expression grouping : groupings) {
- if (grouping instanceof Attribute attribute) {
- groupingAttrs.add(attribute);
- } else if (grouping instanceof Alias as && as.child() instanceof GroupingFunction.NonEvaluatableGroupingFunction) {
- groupingAttrs.add(as);
+ if (plan instanceof Aggregate a && child instanceof Project p) {
+ var groupings = a.groupings();
+
+ // sanity checks
+ for (Expression grouping : groupings) {
+ if ((grouping instanceof Attribute
+ || grouping instanceof Alias as && as.child() instanceof GroupingFunction.NonEvaluatableGroupingFunction) == false) {
+ // After applying ReplaceAggregateNestedExpressionWithEval,
+ // evaluatable groupings can only contain attributes.
+ throw new EsqlIllegalArgumentException("Expected an attribute or grouping function, got {}", grouping);
+ }
+ }
+ assert groupings.size() <= 1
+ || groupings.stream()
+ .anyMatch(group -> group.anyMatch(expr -> expr instanceof GroupingFunction.NonEvaluatableGroupingFunction)) == false
+ : "CombineProjections only tested with a single CATEGORIZE with no additional groups";
+
+ // Collect the alias map for resolving the source (f1 = 1, f2 = f1, etc..)
+ AttributeMap.Builder aliasesBuilder = AttributeMap.builder();
+ for (NamedExpression ne : p.projections()) {
+ // Record the aliases.
+ // Projections are just aliases for attributes, so casting is safe.
+ aliasesBuilder.put(ne.toAttribute(), (Attribute) Alias.unwrap(ne));
+ }
+ var aliases = aliasesBuilder.build();
+
+ // Propagate any renames from the lower projection into the upper groupings.
+ List resolvedGroupings = new ArrayList<>();
+ for (Expression grouping : groupings) {
+ Expression transformed = grouping.transformUp(Attribute.class, as -> aliases.resolve(as, as));
+ resolvedGroupings.add(transformed);
+ }
+
+ // This can lead to duplicates in the groupings: e.g.
+ // | EVAL x = y | STATS ... BY x, y
+ if (local) {
+ // On the data node, the groupings must be preserved because they affect the physical output (see
+ // AbstractPhysicalOperationProviders#intermediateAttributes).
+ // In case that propagating the lower projection leads to duplicates in the resolved groupings, we'll leave an Eval in place
+ // of the original projection to create new attributes for the duplicate groups.
+ Set seenResolvedGroupings = new HashSet<>(resolvedGroupings.size());
+ List newGroupings = new ArrayList<>();
+ List aliasesAgainstDuplication = new ArrayList<>();
+
+ for (int i = 0; i < groupings.size(); i++) {
+ Expression resolvedGrouping = resolvedGroupings.get(i);
+ if (seenResolvedGroupings.add(resolvedGrouping)) {
+ newGroupings.add(resolvedGrouping);
} else {
- // After applying ReplaceAggregateNestedExpressionWithEval,
- // evaluatable groupings can only contain attributes.
- throw new EsqlIllegalArgumentException("Expected an Attribute, got {}", grouping);
+ // resolving the renames leads to a duplicate here - we need to alias the underlying attribute this refers to.
+ // should really only be 1 attribute, anyway, but going via .references() includes the case of a
+ // GroupingFunction.NonEvaluatableGroupingFunction.
+ Attribute coreAttribute = resolvedGrouping.references().iterator().next();
+
+ Alias renameAgainstDuplication = new Alias(
+ coreAttribute.source(),
+ TemporaryNameUtils.locallyUniqueTemporaryName(coreAttribute.name()),
+ coreAttribute
+ );
+ aliasesAgainstDuplication.add(renameAgainstDuplication);
+
+ // propagate the new alias into the new grouping
+ AttributeMap.Builder resolverBuilder = AttributeMap.builder();
+ resolverBuilder.put(coreAttribute, renameAgainstDuplication.toAttribute());
+ AttributeMap resolver = resolverBuilder.build();
+
+ newGroupings.add(resolvedGrouping.transformUp(Attribute.class, attr -> resolver.resolve(attr, attr)));
}
}
- plan = a.with(
- p.child(),
- combineUpperGroupingsAndLowerProjections(groupingAttrs, p.projections()),
- combineProjections(a.aggregates(), p.projections())
- );
+
+ LogicalPlan newChild = aliasesAgainstDuplication.isEmpty()
+ ? p.child()
+ : new Eval(p.source(), p.child(), aliasesAgainstDuplication);
+ plan = a.with(newChild, newGroupings, combineProjections(a.aggregates(), p.projections()));
+ } else {
+ // On the coordinator, we can just discard the duplicates.
+ // All substitutions happen before; groupings must be attributes at this point except for non-evaluatable groupings which
+ // will be an alias like `c = CATEGORIZE(attribute)`.
+ // Due to such aliases, we can't use an AttributeSet to deduplicate. But we can use a regular set to deduplicate based on
+ // regular equality (i.e. based on names) instead of name ids.
+ // TODO: The deduplication based on simple equality will be insufficient in case of multiple non-evaluatable groupings, e.g.
+ // for `| EVAL x = y | STATS ... BY CATEGORIZE(x), CATEGORIZE(y)`. That will require semantic equality instead. Also
+ // applies in the local case below.
+ List newGroupings = new ArrayList<>(new LinkedHashSet<>(resolvedGroupings));
+ plan = a.with(p.child(), newGroupings, combineProjections(a.aggregates(), p.projections()));
}
}
@@ -143,39 +213,6 @@ public final class CombineProjections extends OptimizerRules.OptimizerRule combineUpperGroupingsAndLowerProjections(
- List extends NamedExpression> upperGroupings,
- List extends NamedExpression> lowerProjections
- ) {
- assert upperGroupings.size() <= 1
- || upperGroupings.stream()
- .anyMatch(group -> group.anyMatch(expr -> expr instanceof GroupingFunction.NonEvaluatableGroupingFunction)) == false
- : "CombineProjections only tested with a single CATEGORIZE with no additional groups";
- // Collect the alias map for resolving the source (f1 = 1, f2 = f1, etc..)
- AttributeMap.Builder aliasesBuilder = AttributeMap.builder();
- for (NamedExpression ne : lowerProjections) {
- // Record the aliases.
- // Projections are just aliases for attributes, so casting is safe.
- aliasesBuilder.put(ne.toAttribute(), (Attribute) Alias.unwrap(ne));
- }
- var aliases = aliasesBuilder.build();
-
- // Propagate any renames from the lower projection into the upper groupings.
- // This can lead to duplicates: e.g.
- // | EVAL x = y | STATS ... BY x, y
- // All substitutions happen before; groupings must be attributes at this point except for non-evaluatable groupings which will be
- // an alias like `c = CATEGORIZE(attribute)`.
- // Therefore, it is correct to deduplicate based on simple equality (based on names) instead of name ids (Set vs. AttributeSet).
- // TODO: The deduplication based on simple equality will be insufficient in case of multiple non-evaluatable groupings, e.g. for
- // `| EVAL x = y | STATS ... BY CATEGORIZE(x), CATEGORIZE(y)`. That will require semantic equality instead.
- LinkedHashSet resolvedGroupings = new LinkedHashSet<>();
- for (NamedExpression ne : upperGroupings) {
- NamedExpression transformed = (NamedExpression) ne.transformUp(Attribute.class, a -> aliases.resolve(a, a));
- resolvedGroupings.add(transformed);
- }
- return new ArrayList<>(resolvedGroupings);
- }
-
/**
* Replace grouping alias previously contained in the aggregations that might have been projected away.
*/
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java
index 91b8606c4030..9268dd08bc7e 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/IdentifierBuilder.java
@@ -20,17 +20,22 @@ import org.elasticsearch.xpack.esql.parser.EsqlBaseParser.IdentifierContext;
import org.elasticsearch.xpack.esql.parser.EsqlBaseParser.IndexStringContext;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
import static org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.SelectorResolver.SELECTOR_SEPARATOR;
import static org.elasticsearch.transport.RemoteClusterAware.REMOTE_CLUSTER_INDEX_SEPARATOR;
-import static org.elasticsearch.transport.RemoteClusterAware.isRemoteIndexName;
+import static org.elasticsearch.transport.RemoteClusterAware.splitIndexName;
import static org.elasticsearch.xpack.esql.core.util.StringUtils.EXCLUSION;
import static org.elasticsearch.xpack.esql.core.util.StringUtils.WILDCARD;
import static org.elasticsearch.xpack.esql.parser.ParserUtils.source;
abstract class IdentifierBuilder extends AbstractBuilder {
+ private static final String BLANK_INDEX_ERROR_MESSAGE = "Blank index specified in index pattern";
+
+ private static final String INVALID_ESQL_CHARS = Strings.INVALID_FILENAME_CHARS.replace("'*',", "");
+
@Override
public String visitIdentifier(IdentifierContext ctx) {
return ctx == null ? null : unquoteIdentifier(ctx.QUOTED_IDENTIFIER(), ctx.UNQUOTED_IDENTIFIER());
@@ -88,39 +93,21 @@ abstract class IdentifierBuilder extends AbstractBuilder {
String indexPattern = c.unquotedIndexString() != null ? c.unquotedIndexString().getText() : visitIndexString(c.indexString());
String clusterString = visitClusterString(c.clusterString());
String selectorString = visitSelectorString(c.selectorString());
- // skip validating index on remote cluster, because the behavior of remote cluster is not consistent with local cluster
- // For example, invalid#index is an invalid index name, however FROM *:invalid#index does not return an error
- if (clusterString == null) {
- hasSeenStar.set(indexPattern.contains(WILDCARD) || hasSeenStar.get());
- validateIndexPattern(indexPattern, c, hasSeenStar.get());
- // Other instances of Elasticsearch may have differing selectors so only validate selector string if remote cluster
- // string is unset
- if (selectorString != null) {
- try {
- // Ensures that the selector provided is one of the valid kinds
- IndexNameExpressionResolver.SelectorResolver.validateIndexSelectorString(indexPattern, selectorString);
- } catch (InvalidIndexNameException e) {
- throw new ParsingException(e, source(c), e.getMessage());
- }
- }
- } else {
- validateClusterString(clusterString, c);
- // Do not allow selectors on remote cluster expressions until they are supported
- if (selectorString != null) {
- throwOnMixingSelectorWithCluster(reassembleIndexName(clusterString, indexPattern, selectorString), c);
- }
- }
+
+ hasSeenStar.set(hasSeenStar.get() || indexPattern.contains(WILDCARD));
+ validate(clusterString, indexPattern, selectorString, c, hasSeenStar.get());
patterns.add(reassembleIndexName(clusterString, indexPattern, selectorString));
});
return Strings.collectionToDelimitedString(patterns, ",");
}
+ private static void throwInvalidIndexNameException(String indexPattern, String message, EsqlBaseParser.IndexPatternContext ctx) {
+ var ie = new InvalidIndexNameException(indexPattern, message);
+ throw new ParsingException(ie, source(ctx), ie.getMessage());
+ }
+
private static void throwOnMixingSelectorWithCluster(String indexPattern, EsqlBaseParser.IndexPatternContext c) {
- InvalidIndexNameException ie = new InvalidIndexNameException(
- indexPattern,
- "Selectors are not yet supported on remote cluster patterns"
- );
- throw new ParsingException(ie, source(c), ie.getMessage());
+ throwInvalidIndexNameException(indexPattern, "Selectors are not yet supported on remote cluster patterns", c);
}
private static String reassembleIndexName(String clusterString, String indexPattern, String selectorString) {
@@ -144,56 +131,192 @@ abstract class IdentifierBuilder extends AbstractBuilder {
}
}
- private static void validateIndexPattern(String indexPattern, EsqlBaseParser.IndexPatternContext ctx, boolean hasSeenStar) {
- // multiple index names can be in the same double quote, e.g. indexPattern = "idx1, *, -idx2"
- String[] indices = indexPattern.split(",");
- boolean hasExclusion = false;
- for (String index : indices) {
- // Strip spaces off first because validation checks are not written to handle them
- index = index.strip();
- if (isRemoteIndexName(index)) { // skip the validation if there is remote cluster
- // Ensure that there are no selectors as they are not yet supported
- if (index.contains(SELECTOR_SEPARATOR)) {
- throwOnMixingSelectorWithCluster(index, ctx);
- }
- continue;
+ /**
+ * Takes the parsed constituent strings and validates them.
+ * @param clusterString Name of the remote cluster. Can be null.
+ * @param indexPattern Name of the index or pattern; can also have multiple patterns in case of quoting,
+ * e.g. {@code FROM """index*,-index1"""}.
+ * @param selectorString Selector string, i.e. "::data" or "::failures". Can be null.
+ * @param ctx Index Pattern Context for generating error messages with offsets.
+ * @param hasSeenStar If we've seen an asterisk so far.
+ */
+ private static void validate(
+ String clusterString,
+ String indexPattern,
+ String selectorString,
+ EsqlBaseParser.IndexPatternContext ctx,
+ boolean hasSeenStar
+ ) {
+ /*
+ * At this point, only 3 formats are possible:
+ * "index_pattern(s)",
+ * remote:index_pattern, and,
+ * index_pattern::selector_string.
+ *
+ * The grammar prohibits remote:"index_pattern(s)" or "index_pattern(s)"::selector_string as they're
+ * partially quoted. So if either of cluster string or selector string are present, there's no need
+ * to split the pattern by comma since comma requires partial quoting.
+ */
+
+ String[] patterns;
+ if (clusterString == null && selectorString == null) {
+ // Pattern could be quoted or is singular like "index_name".
+ patterns = indexPattern.split(",", -1);
+ } else {
+ // Either of cluster string or selector string is present. Pattern is unquoted.
+ patterns = new String[] { indexPattern };
+ }
+
+ patterns = Arrays.stream(patterns).map(String::strip).toArray(String[]::new);
+ if (Arrays.stream(patterns).anyMatch(String::isBlank)) {
+ throwInvalidIndexNameException(indexPattern, BLANK_INDEX_ERROR_MESSAGE, ctx);
+ }
+
+ // Edge case: happens when all the index names in a pattern are empty like "FROM ",,,,,"".
+ if (patterns.length == 0) {
+ throwInvalidIndexNameException(indexPattern, BLANK_INDEX_ERROR_MESSAGE, ctx);
+ } else if (patterns.length == 1) {
+ // Pattern is either an unquoted string or a quoted string with a single index (no comma sep).
+ validateSingleIndexPattern(clusterString, patterns[0], selectorString, ctx, hasSeenStar);
+ } else {
+ /*
+ * Presence of multiple patterns requires a comma and comma requires quoting. If quoting is present,
+ * cluster string and selector string cannot be present; they need to be attached within the quoting.
+ * So we attempt to extract them later.
+ */
+ for (String pattern : patterns) {
+ validateSingleIndexPattern(null, pattern, null, ctx, hasSeenStar);
}
+ }
+ }
+
+ /**
+ * Validates the constituent strings. Will extract the cluster string and/or selector string from the index
+ * name if clubbed together inside a quoted string.
+ *
+ * @param clusterString Name of the remote cluster. Can be null.
+ * @param indexName Name of the index.
+ * @param selectorString Selector string, i.e. "::data" or "::failures". Can be null.
+ * @param ctx Index Pattern Context for generating error messages with offsets.
+ * @param hasSeenStar If we've seen an asterisk so far.
+ */
+ private static void validateSingleIndexPattern(
+ String clusterString,
+ String indexName,
+ String selectorString,
+ EsqlBaseParser.IndexPatternContext ctx,
+ boolean hasSeenStar
+ ) {
+ indexName = indexName.strip();
+
+ /*
+ * Precedence:
+ * 1. Cannot mix cluster and selector strings.
+ * 2. Cluster string must be valid.
+ * 3. Index name must be valid.
+ * 4. Selector string must be valid.
+ *
+ * Since cluster string and/or selector string can be clubbed with the index name, we must try to
+ * manually extract them before we attempt to do #2, #3, and #4.
+ */
+
+ // It is possible to specify a pattern like "remote_cluster:index_name". Try to extract such details from the index string.
+ if (clusterString == null && selectorString == null) {
try {
- Tuple splitPattern = IndexNameExpressionResolver.splitSelectorExpression(index);
- if (splitPattern.v2() != null) {
- index = splitPattern.v1();
+ var split = splitIndexName(indexName);
+ clusterString = split[0];
+ indexName = split[1];
+ } catch (IllegalArgumentException e) {
+ throw new ParsingException(e, source(ctx), e.getMessage());
+ }
+ }
+
+ // At the moment, selector strings for remote indices is not allowed.
+ if (clusterString != null && selectorString != null) {
+ throwOnMixingSelectorWithCluster(reassembleIndexName(clusterString, indexName, selectorString), ctx);
+ }
+
+ // Validation in the right precedence.
+ if (clusterString != null) {
+ clusterString = clusterString.strip();
+ validateClusterString(clusterString, ctx);
+ }
+
+ /*
+ * It is possible for selector string to be attached to the index: "index_name::selector_string".
+ * Try to extract the selector string.
+ */
+ try {
+ Tuple splitPattern = IndexNameExpressionResolver.splitSelectorExpression(indexName);
+ if (splitPattern.v2() != null) {
+ // Cluster string too was clubbed with the index name like selector string.
+ if (clusterString != null) {
+ throwOnMixingSelectorWithCluster(reassembleIndexName(clusterString, splitPattern.v1(), splitPattern.v2()), ctx);
+ } else {
+ // We've seen a selectorString. Use it.
+ selectorString = splitPattern.v2();
}
+ }
+
+ indexName = splitPattern.v1();
+ } catch (InvalidIndexNameException e) {
+ throw new ParsingException(e, source(ctx), e.getMessage());
+ }
+
+ resolveAndValidateIndex(indexName, ctx, hasSeenStar);
+ if (selectorString != null) {
+ selectorString = selectorString.strip();
+ try {
+ // Ensures that the selector provided is one of the valid kinds.
+ IndexNameExpressionResolver.SelectorResolver.validateIndexSelectorString(indexName, selectorString);
} catch (InvalidIndexNameException e) {
- // throws exception if the selector expression is invalid. Selector resolution does not complain about exclusions
throw new ParsingException(e, source(ctx), e.getMessage());
}
- hasSeenStar = index.contains(WILDCARD) || hasSeenStar;
- index = index.replace(WILDCARD, "").strip();
- if (index.isBlank()) {
- continue;
+ }
+ }
+
+ private static void resolveAndValidateIndex(String index, EsqlBaseParser.IndexPatternContext ctx, boolean hasSeenStar) {
+ // If index name is blank without any replacements, it was likely blank right from the beginning and is invalid.
+ if (index.isBlank()) {
+ throwInvalidIndexNameException(index, BLANK_INDEX_ERROR_MESSAGE, ctx);
+ }
+
+ hasSeenStar = hasSeenStar || index.contains(WILDCARD);
+ index = index.replace(WILDCARD, "").strip();
+ if (index.isBlank()) {
+ return;
+ }
+ var hasExclusion = index.startsWith(EXCLUSION);
+ index = removeExclusion(index);
+ String tempName;
+ try {
+ // remove the exclusion outside of <>, from index names with DateMath expression,
+ // e.g. -<-logstash-{now/d}> becomes <-logstash-{now/d}> before calling resolveDateMathExpression
+ tempName = IndexNameExpressionResolver.resolveDateMathExpression(index);
+ } catch (ElasticsearchParseException e) {
+ // throws exception if the DateMath expression is invalid, resolveDateMathExpression does not complain about exclusions
+ throw new ParsingException(e, source(ctx), e.getMessage());
+ }
+ hasExclusion = tempName.startsWith(EXCLUSION) || hasExclusion;
+ index = tempName.equals(index) ? index : removeExclusion(tempName);
+ try {
+ MetadataCreateIndexService.validateIndexOrAliasName(index, InvalidIndexNameException::new);
+ } catch (InvalidIndexNameException e) {
+ // ignore invalid index name if it has exclusions and there is an index with wildcard before it
+ if (hasSeenStar && hasExclusion) {
+ return;
}
- hasExclusion = index.startsWith(EXCLUSION);
- index = removeExclusion(index);
- String tempName;
- try {
- // remove the exclusion outside of <>, from index names with DateMath expression,
- // e.g. -<-logstash-{now/d}> becomes <-logstash-{now/d}> before calling resolveDateMathExpression
- tempName = IndexNameExpressionResolver.resolveDateMathExpression(index);
- } catch (ElasticsearchParseException e) {
- // throws exception if the DateMath expression is invalid, resolveDateMathExpression does not complain about exclusions
- throw new ParsingException(e, source(ctx), e.getMessage());
- }
- hasExclusion = tempName.startsWith(EXCLUSION) || hasExclusion;
- index = tempName.equals(index) ? index : removeExclusion(tempName);
- try {
- MetadataCreateIndexService.validateIndexOrAliasName(index, InvalidIndexNameException::new);
- } catch (InvalidIndexNameException e) {
- // ignore invalid index name if it has exclusions and there is an index with wildcard before it
- if (hasSeenStar && hasExclusion) {
- continue;
- }
- throw new ParsingException(e, source(ctx), e.getMessage());
+
+ /*
+ * We only modify this particular message because it mentions '*' as an invalid char.
+ * However, we do allow asterisk in the index patterns: wildcarded patterns. Let's not
+ * mislead the user by mentioning this char in the error message.
+ */
+ if (e.getMessage().contains("must not contain the following characters")) {
+ throwInvalidIndexNameException(index, "must not contain the following characters " + INVALID_ESQL_CHARS, ctx);
}
+
+ throw new ParsingException(e, source(ctx), e.getMessage());
}
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java
index 0b2fafcf2df2..acf685f3dcd9 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java
@@ -7,6 +7,9 @@
package org.elasticsearch.xpack.esql.planner;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.BooleanClause;
@@ -14,6 +17,7 @@ import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.logging.HeaderWarning;
+import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.compute.aggregation.AggregatorMode;
import org.elasticsearch.compute.aggregation.GroupingAggregator;
import org.elasticsearch.compute.aggregation.blockhash.BlockHash;
@@ -31,7 +35,9 @@ import org.elasticsearch.compute.operator.Operator;
import org.elasticsearch.compute.operator.OrdinalsGroupingOperator;
import org.elasticsearch.compute.operator.SourceOperator;
import org.elasticsearch.compute.operator.TimeSeriesAggregationOperator;
+import org.elasticsearch.core.AbstractRefCounted;
import org.elasticsearch.core.Nullable;
+import org.elasticsearch.core.Releasable;
import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisRegistry;
@@ -76,7 +82,6 @@ import org.elasticsearch.xpack.esql.plugin.EsqlPlugin;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
@@ -90,14 +95,41 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi
private static final Logger logger = LogManager.getLogger(EsPhysicalOperationProviders.class);
/**
- * Context of each shard we're operating against.
+ * Context of each shard we're operating against. Note these objects are shared across multiple operators as
+ * {@link org.elasticsearch.core.RefCounted}.
*/
- public interface ShardContext extends org.elasticsearch.compute.lucene.ShardContext {
+ public abstract static class ShardContext implements org.elasticsearch.compute.lucene.ShardContext, Releasable {
+ private final AbstractRefCounted refCounted = new AbstractRefCounted() {
+ @Override
+ protected void closeInternal() {
+ ShardContext.this.close();
+ }
+ };
+
+ @Override
+ public void incRef() {
+ refCounted.incRef();
+ }
+
+ @Override
+ public boolean tryIncRef() {
+ return refCounted.tryIncRef();
+ }
+
+ @Override
+ public boolean decRef() {
+ return refCounted.decRef();
+ }
+
+ @Override
+ public boolean hasReferences() {
+ return refCounted.hasReferences();
+ }
/**
* Convert a {@link QueryBuilder} into a real {@link Query lucene query}.
*/
- Query toQuery(QueryBuilder queryBuilder);
+ public abstract Query toQuery(QueryBuilder queryBuilder);
/**
* Tuning parameter for deciding when to use the "merge" stored field loader.
@@ -107,7 +139,7 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi
* A value of {@code .2} means we'll use the sequential reader even if we only
* need one in ten documents.
*/
- double storedFieldsSequentialProportion();
+ public abstract double storedFieldsSequentialProportion();
}
private final List shardContexts;
@@ -177,19 +209,39 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi
/** A hack to pretend an unmapped field still exists. */
private static class DefaultShardContextForUnmappedField extends DefaultShardContext {
+ private static final FieldType UNMAPPED_FIELD_TYPE = new FieldType(KeywordFieldMapper.Defaults.FIELD_TYPE);
+ static {
+ UNMAPPED_FIELD_TYPE.setDocValuesType(DocValuesType.NONE);
+ UNMAPPED_FIELD_TYPE.setIndexOptions(IndexOptions.NONE);
+ UNMAPPED_FIELD_TYPE.setStored(false);
+ UNMAPPED_FIELD_TYPE.freeze();
+ }
private final KeywordEsField unmappedEsField;
DefaultShardContextForUnmappedField(DefaultShardContext ctx, PotentiallyUnmappedKeywordEsField unmappedEsField) {
- super(ctx.index, ctx.ctx, ctx.aliasFilter);
+ super(ctx.index, ctx.releasable, ctx.ctx, ctx.aliasFilter);
this.unmappedEsField = unmappedEsField;
}
@Override
public @Nullable MappedFieldType fieldType(String name) {
var superResult = super.fieldType(name);
- return superResult == null && name.equals(unmappedEsField.getName())
- ? new KeywordFieldMapper.KeywordFieldType(name, false /* isIndexed */, false /* hasDocValues */, Map.of() /* meta */)
- : superResult;
+ return superResult == null && name.equals(unmappedEsField.getName()) ? createUnmappedFieldType(name, this) : superResult;
+ }
+
+ static MappedFieldType createUnmappedFieldType(String name, DefaultShardContext context) {
+ var builder = new KeywordFieldMapper.Builder(name, context.ctx.indexVersionCreated());
+ builder.docValues(false);
+ builder.indexed(false);
+ return new KeywordFieldMapper.KeywordFieldType(
+ name,
+ UNMAPPED_FIELD_TYPE,
+ Lucene.KEYWORD_ANALYZER,
+ Lucene.KEYWORD_ANALYZER,
+ Lucene.KEYWORD_ANALYZER,
+ builder,
+ context.ctx.isSourceSynthetic()
+ );
}
}
@@ -349,18 +401,24 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi
);
}
- public static class DefaultShardContext implements ShardContext {
+ public static class DefaultShardContext extends ShardContext {
private final int index;
+ /**
+ * In production, this will be a {@link org.elasticsearch.search.internal.SearchContext}, but we don't want to drag that huge
+ * dependency here.
+ */
+ private final Releasable releasable;
private final SearchExecutionContext ctx;
private final AliasFilter aliasFilter;
private final String shardIdentifier;
- public DefaultShardContext(int index, SearchExecutionContext ctx, AliasFilter aliasFilter) {
+ public DefaultShardContext(int index, Releasable releasable, SearchExecutionContext ctx, AliasFilter aliasFilter) {
this.index = index;
+ this.releasable = releasable;
this.ctx = ctx;
this.aliasFilter = aliasFilter;
// Build the shardIdentifier once up front so we can reuse references to it in many places.
- this.shardIdentifier = ctx.getFullyQualifiedIndex().getName() + ":" + ctx.getShardId();
+ this.shardIdentifier = this.ctx.getFullyQualifiedIndex().getName() + ":" + this.ctx.getShardId();
}
@Override
@@ -473,6 +531,11 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi
public double storedFieldsSequentialProportion() {
return EsqlPlugin.STORED_FIELDS_SEQUENTIAL_PROPORTION.get(ctx.getIndexSettings().getSettings());
}
+
+ @Override
+ public void close() {
+ releasable.close();
+ }
}
private static class TypeConvertingBlockLoader implements BlockLoader {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java
index 4f55a2a6e8ce..ac49213d6151 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeResponse.java
@@ -7,6 +7,7 @@
package org.elasticsearch.xpack.esql.plugin;
+import org.elasticsearch.TransportVersion;
import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.io.stream.StreamInput;
@@ -20,6 +21,7 @@ import java.io.IOException;
import java.util.List;
import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED;
+import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED_8_19;
/**
* The compute result of {@link DataNodeRequest} or {@link ClusterComputeRequest}
@@ -58,7 +60,7 @@ final class ComputeResponse extends TransportResponse {
}
ComputeResponse(StreamInput in) throws IOException {
- if (in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) {
+ if (supportsCompletionInfo(in.getTransportVersion())) {
completionInfo = DriverCompletionInfo.readFrom(in);
} else if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) {
if (in.readBoolean()) {
@@ -92,7 +94,7 @@ final class ComputeResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
- if (out.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) {
+ if (supportsCompletionInfo(out.getTransportVersion())) {
completionInfo.writeTo(out);
} else if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_12_0)) {
out.writeBoolean(true);
@@ -111,6 +113,11 @@ final class ComputeResponse extends TransportResponse {
}
}
+ private static boolean supportsCompletionInfo(TransportVersion version) {
+ return version.onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)
+ || version.isPatchFrom(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED_8_19);
+ }
+
public DriverCompletionInfo getCompletionInfo() {
return completionInfo;
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java
index 6d15f88a26f1..4adc97d28fee 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java
@@ -26,6 +26,7 @@ import org.elasticsearch.compute.operator.exchange.ExchangeService;
import org.elasticsearch.compute.operator.exchange.ExchangeSink;
import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler;
import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler;
+import org.elasticsearch.core.RefCounted;
import org.elasticsearch.core.Releasable;
import org.elasticsearch.core.Releasables;
import org.elasticsearch.core.Tuple;
@@ -541,7 +542,12 @@ public class ComputeService {
}
};
contexts.add(
- new EsPhysicalOperationProviders.DefaultShardContext(i, searchExecutionContext, searchContext.request().getAliasFilter())
+ new EsPhysicalOperationProviders.DefaultShardContext(
+ i,
+ searchContext,
+ searchExecutionContext,
+ searchContext.request().getAliasFilter()
+ )
);
}
EsPhysicalOperationProviders physicalOperationProviders = new EsPhysicalOperationProviders(
@@ -579,6 +585,9 @@ public class ComputeService {
LOGGER.debug("Local execution plan:\n{}", localExecutionPlan.describe());
}
var drivers = localExecutionPlan.createDrivers(context.sessionId());
+ // After creating the drivers (and therefore, the operators), we can safely decrement the reference count since the operators
+ // will hold a reference to the contexts where relevant.
+ contexts.forEach(RefCounted::decRef);
if (drivers.isEmpty()) {
throw new IllegalStateException("no drivers created");
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java
index cdf418df1526..e21f6d7e44b5 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/DataNodeComputeResponse.java
@@ -7,6 +7,7 @@
package org.elasticsearch.xpack.esql.plugin;
+import org.elasticsearch.TransportVersion;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.compute.operator.DriverCompletionInfo;
@@ -19,6 +20,7 @@ import java.util.List;
import java.util.Map;
import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED;
+import static org.elasticsearch.TransportVersions.ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED_8_19;
/**
* The compute result of {@link DataNodeRequest}
@@ -33,7 +35,7 @@ final class DataNodeComputeResponse extends TransportResponse {
}
DataNodeComputeResponse(StreamInput in) throws IOException {
- if (in.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) {
+ if (supportsCompletionInfo(in.getTransportVersion())) {
this.completionInfo = DriverCompletionInfo.readFrom(in);
this.shardLevelFailures = in.readMap(ShardId::new, StreamInput::readException);
return;
@@ -49,7 +51,7 @@ final class DataNodeComputeResponse extends TransportResponse {
@Override
public void writeTo(StreamOutput out) throws IOException {
- if (out.getTransportVersion().onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)) {
+ if (supportsCompletionInfo(out.getTransportVersion())) {
completionInfo.writeTo(out);
out.writeMap(shardLevelFailures, (o, v) -> v.writeTo(o), StreamOutput::writeException);
return;
@@ -65,6 +67,11 @@ final class DataNodeComputeResponse extends TransportResponse {
new ComputeResponse(completionInfo).writeTo(out);
}
+ private static boolean supportsCompletionInfo(TransportVersion version) {
+ return version.onOrAfter(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED)
+ || version.isPatchFrom(ESQL_DOCUMENTS_FOUND_AND_VALUES_LOADED_8_19);
+ }
+
public DriverCompletionInfo completionInfo() {
return completionInfo;
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java
index 19f926b36ef2..293a7be6be04 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java
@@ -190,6 +190,11 @@ public class EsqlPlugin extends Plugin implements ActionPlugin, ExtensiblePlugin
);
private final List extraCheckers = new ArrayList<>();
+ private final Settings settings;
+
+ public EsqlPlugin(Settings settings) {
+ this.settings = settings;
+ }
@Override
public Collection> createComponents(PluginServices services) {
@@ -209,7 +214,8 @@ public class EsqlPlugin extends Plugin implements ActionPlugin, ExtensiblePlugin
services.telemetryProvider().getMeterRegistry(),
getLicenseState(),
new EsqlQueryLog(services.clusterService().getClusterSettings(), services.slowLogFieldProvider()),
- extraCheckers
+ extraCheckers,
+ settings
),
new ExchangeService(
services.clusterService().getSettings(),
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java
index 29951070a96c..345bf3b8767e 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java
@@ -35,7 +35,7 @@ public final class QueryPragmas implements Writeable {
public static final Setting EXCHANGE_CONCURRENT_CLIENTS = Setting.intSetting("exchange_concurrent_clients", 2);
public static final Setting ENRICH_MAX_WORKERS = Setting.intSetting("enrich_max_workers", 1);
- private static final Setting TASK_CONCURRENCY = Setting.intSetting(
+ public static final Setting TASK_CONCURRENCY = Setting.intSetting(
"task_concurrency",
ThreadPool.searchOrGetThreadPoolSize(EsExecutors.allocatedProcessors(Settings.EMPTY))
);
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java
index 4cc928fe07cb..6629b0b09d08 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java
@@ -349,7 +349,7 @@ public class TransportEsqlQueryAction extends HandledTransportAction throw new IllegalArgumentException();
}
- ;
return new EsqlQueryResponse(columns, pages, documentsFound, valuesLoaded, profile, columnar, isAsync, executionInfo);
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/NamedWriteablesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/NamedWriteablesTests.java
index 1fba8f66d0f1..186120a59d3f 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/NamedWriteablesTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/NamedWriteablesTests.java
@@ -8,6 +8,7 @@
package org.elasticsearch.xpack.esql.action;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.compute.operator.Operator;
import org.elasticsearch.compute.operator.topn.TopNOperatorStatus;
import org.elasticsearch.test.ESTestCase;
@@ -18,7 +19,7 @@ import static org.hamcrest.Matchers.equalTo;
public class NamedWriteablesTests extends ESTestCase {
public void testTopNStatus() throws Exception {
- try (EsqlPlugin plugin = new EsqlPlugin()) {
+ try (EsqlPlugin plugin = new EsqlPlugin(Settings.EMPTY)) {
NamedWriteableRegistry registry = new NamedWriteableRegistry(plugin.getNamedWriteables());
TopNOperatorStatus origin = new TopNOperatorStatus(
randomNonNegativeInt(),
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java
index dabcc6cbce89..f3bdf29688b9 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java
@@ -35,6 +35,7 @@ import org.elasticsearch.compute.data.Page;
import org.elasticsearch.compute.operator.DriverContext;
import org.elasticsearch.compute.operator.Operator;
import org.elasticsearch.compute.operator.SourceOperator;
+import org.elasticsearch.compute.test.NoOpReleasable;
import org.elasticsearch.compute.test.OperatorTestCase;
import org.elasticsearch.compute.test.SequenceLongBlockSourceOperator;
import org.elasticsearch.core.IOUtils;
@@ -246,11 +247,7 @@ public class LookupFromIndexOperatorTests extends OperatorTestCase {
}""");
DirectoryReader reader = DirectoryReader.open(lookupIndexDirectory);
SearchExecutionContext executionCtx = mapperHelper.createSearchExecutionContext(mapperService, newSearcher(reader));
- EsPhysicalOperationProviders.DefaultShardContext ctx = new EsPhysicalOperationProviders.DefaultShardContext(
- 0,
- executionCtx,
- AliasFilter.EMPTY
- );
+ var ctx = new EsPhysicalOperationProviders.DefaultShardContext(0, new NoOpReleasable(), executionCtx, AliasFilter.EMPTY);
return new AbstractLookupService.LookupShardContext(ctx, executionCtx, () -> {
try {
IOUtils.close(reader, mapperService);
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java
index 8251611dcfe4..f2b70c99253b 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalLogicalPlanOptimizerTests.java
@@ -744,6 +744,36 @@ public class LocalLogicalPlanOptimizerTests extends ESTestCase {
assertEquals("integer_long_field", unionTypeField.fieldName().string());
}
+ /**
+ * \_Aggregate[[first_name{r}#7, $$first_name$temp_name$17{r}#18],[SUM(salary{f}#11,true[BOOLEAN]) AS SUM(salary)#5, first_nam
+ * e{r}#7, first_name{r}#7 AS last_name#10]]
+ * \_Eval[[null[KEYWORD] AS first_name#7, null[KEYWORD] AS $$first_name$temp_name$17#18]]
+ * \_EsRelation[test][_meta_field{f}#12, emp_no{f}#6, first_name{f}#7, ge..]
+ */
+ public void testGroupingByMissingFields() {
+ var plan = plan("FROM test | STATS SUM(salary) BY first_name, last_name");
+ var testStats = statsForMissingField("first_name", "last_name");
+ var localPlan = localPlan(plan, testStats);
+ Limit limit = as(localPlan, Limit.class);
+ Aggregate aggregate = as(limit.child(), Aggregate.class);
+ assertThat(aggregate.groupings(), hasSize(2));
+ ReferenceAttribute grouping1 = as(aggregate.groupings().get(0), ReferenceAttribute.class);
+ ReferenceAttribute grouping2 = as(aggregate.groupings().get(1), ReferenceAttribute.class);
+ Eval eval = as(aggregate.child(), Eval.class);
+ assertThat(eval.fields(), hasSize(2));
+ Alias eval1 = eval.fields().get(0);
+ Literal literal1 = as(eval1.child(), Literal.class);
+ assertNull(literal1.value());
+ assertThat(literal1.dataType(), is(DataType.KEYWORD));
+ Alias eval2 = eval.fields().get(1);
+ Literal literal2 = as(eval2.child(), Literal.class);
+ assertNull(literal2.value());
+ assertThat(literal2.dataType(), is(DataType.KEYWORD));
+ assertThat(grouping1.id(), equalTo(eval1.id()));
+ assertThat(grouping2.id(), equalTo(eval2.id()));
+ as(eval.child(), EsRelation.class);
+ }
+
private IsNotNull isNotNull(Expression field) {
return new IsNotNull(EMPTY, field);
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java
index cb5c53193949..dc7256e3c452 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java
@@ -472,10 +472,12 @@ public class StatementParserTests extends AbstractStatementParserTests {
"foo, test-*, abc, xyz", test123
""");
assertStringAsIndexPattern("foo,test,xyz", command + " foo, test,xyz");
+ assertStringAsIndexPattern("", command + " ");
assertStringAsIndexPattern(
",",
command + " , \"\""
);
+ assertStringAsIndexPattern("", command + " \"\"");
assertStringAsIndexPattern(
"-,-<-logstash-{now/M{yyyy.MM}}>,"
+ "-,-<-logstash-{now/d{yyyy.MM.dd|+12:00}}>",
@@ -494,18 +496,30 @@ public class StatementParserTests extends AbstractStatementParserTests {
lineNumber,
"mismatched input '\"index|pattern\"' expecting UNQUOTED_SOURCE"
);
- assertStringAsIndexPattern("*:index|pattern", command + " \"*:index|pattern\"");
+ // Entire index pattern is quoted. So it's not a parse error but a semantic error where the index name
+ // is invalid.
+ expectError(command + " \"*:index|pattern\"", "Invalid index name [index|pattern], must not contain the following characters");
clusterAndIndexAsIndexPattern(command, "cluster:index");
clusterAndIndexAsIndexPattern(command, "cluster:.index");
clusterAndIndexAsIndexPattern(command, "cluster*:index*");
- clusterAndIndexAsIndexPattern(command, "cluster*:*");// this is not a valid pattern, * should be inside <>
- clusterAndIndexAsIndexPattern(command, "cluster*:");
+ clusterAndIndexAsIndexPattern(command, "cluster*:*");
clusterAndIndexAsIndexPattern(command, "cluster*:*");
clusterAndIndexAsIndexPattern(command, "*:index*");
clusterAndIndexAsIndexPattern(command, "*:*");
+ expectError(
+ command + " \"cluster:index|pattern\"",
+ "Invalid index name [index|pattern], must not contain the following characters"
+ );
+ expectError(command + " *:\"index|pattern\"", "expecting UNQUOTED_SOURCE");
if (EsqlCapabilities.Cap.INDEX_COMPONENT_SELECTORS.isEnabled()) {
assertStringAsIndexPattern("foo::data", command + " foo::data");
assertStringAsIndexPattern("foo::failures", command + " foo::failures");
+ expectErrorWithLineNumber(
+ command + " *,\"-foo\"::data",
+ "*,-foo::data",
+ lineNumber,
+ "mismatched input '::' expecting {, '|', ',', 'metadata'}"
+ );
expectErrorWithLineNumber(
command + " cluster:\"foo::data\"",
" cluster:\"foo::data\"",
@@ -585,6 +599,7 @@ public class StatementParserTests extends AbstractStatementParserTests {
Map commands = new HashMap<>();
commands.put("FROM {}", "line 1:6: ");
if (Build.current().isSnapshot()) {
+ commands.put("TS {}", "line 1:4: ");
commands.put("ROW x = 1 | LOOKUP_🐔 {} ON j", "line 1:22: ");
}
String lineNumber;
@@ -625,7 +640,11 @@ public class StatementParserTests extends AbstractStatementParserTests {
expectInvalidIndexNameErrorWithLineNumber(command, "index::failure", lineNumber);
// Cluster name cannot be combined with selector yet.
- var parseLineNumber = command.contains("FROM") ? 6 : 9;
+ int parseLineNumber = 6;
+ if (command.startsWith("TS")) {
+ parseLineNumber = 4;
+ }
+
expectDoubleColonErrorWithLineNumber(command, "cluster:foo::data", parseLineNumber + 11);
expectDoubleColonErrorWithLineNumber(command, "cluster:foo::failures", parseLineNumber + 11);
@@ -633,19 +652,18 @@ public class StatementParserTests extends AbstractStatementParserTests {
expectErrorWithLineNumber(
command,
"cluster:\"foo\"::data",
- "line 1:14: ",
+ command.startsWith("FROM") ? "line 1:14: " : "line 1:12: ",
"mismatched input '\"foo\"' expecting UNQUOTED_SOURCE"
);
expectErrorWithLineNumber(
command,
"cluster:\"foo\"::failures",
- "line 1:14: ",
+ command.startsWith("FROM") ? "line 1:14: " : "line 1:12: ",
"mismatched input '\"foo\"' expecting UNQUOTED_SOURCE"
);
- // TODO: Edge case that will be invalidated in follow up (https://github.com/elastic/elasticsearch/issues/122651)
- // expectDoubleColonErrorWithLineNumber(command, "\"cluster:foo\"::data", parseLineNumber + 13);
- // expectDoubleColonErrorWithLineNumber(command, "\"cluster:foo\"::failures", parseLineNumber + 13);
+ expectDoubleColonErrorWithLineNumber(command, "\"cluster:foo\"::data", parseLineNumber + 13);
+ expectDoubleColonErrorWithLineNumber(command, "\"cluster:foo\"::failures", parseLineNumber + 13);
expectErrorWithLineNumber(
command,
@@ -689,7 +707,7 @@ public class StatementParserTests extends AbstractStatementParserTests {
expectErrorWithLineNumber(
command,
"cluster:\"index,index2\"::failures",
- "line 1:14: ",
+ command.startsWith("FROM") ? "line 1:14: " : "line 1:12: ",
"mismatched input '\"index,index2\"' expecting UNQUOTED_SOURCE"
);
}
@@ -746,16 +764,33 @@ public class StatementParserTests extends AbstractStatementParserTests {
clustersAndIndices(command, "index*", "-index#pattern");
clustersAndIndices(command, "*", "-<--logstash-{now/M{yyyy.MM}}>");
clustersAndIndices(command, "index*", "-<--logstash#-{now/M{yyyy.MM}}>");
+ expectInvalidIndexNameErrorWithLineNumber(command, "*, index#pattern", lineNumber, "index#pattern", "must not contain '#'");
+ expectInvalidIndexNameErrorWithLineNumber(
+ command,
+ "index*, index#pattern",
+ indexStarLineNumber,
+ "index#pattern",
+ "must not contain '#'"
+ );
+ expectDateMathErrorWithLineNumber(command, "cluster*:", commands.get(command), dateMathError);
expectDateMathErrorWithLineNumber(command, "*, \"-<-logstash-{now/D}>\"", lineNumber, dateMathError);
expectDateMathErrorWithLineNumber(command, "*, -<-logstash-{now/D}>", lineNumber, dateMathError);
expectDateMathErrorWithLineNumber(command, "\"*, -<-logstash-{now/D}>\"", commands.get(command), dateMathError);
expectDateMathErrorWithLineNumber(command, "\"*, -<-logst:ash-{now/D}>\"", commands.get(command), dateMathError);
if (EsqlCapabilities.Cap.INDEX_COMPONENT_SELECTORS.isEnabled()) {
- clustersAndIndices(command, "*", "-index#pattern::data");
- clustersAndIndices(command, "*", "-index#pattern::data");
+ clustersAndIndices(command, "*", "-index::data");
+ clustersAndIndices(command, "*", "-index::failures");
+ clustersAndIndices(command, "*", "-index*pattern::data");
+ clustersAndIndices(command, "*", "-index*pattern::failures");
+
+ // This is by existing design: refer to the comment in IdentifierBuilder#resolveAndValidateIndex() in the last
+ // catch clause. If there's an index with a wildcard before an invalid index, we don't error out.
clustersAndIndices(command, "index*", "-index#pattern::data");
clustersAndIndices(command, "*", "-<--logstash-{now/M{yyyy.MM}}>::data");
clustersAndIndices(command, "index*", "-<--logstash#-{now/M{yyyy.MM}}>::data");
+
+ expectError(command + "index1,", "unit [-] not supported for date math [+-/d]");
+
// Throw on invalid date math
expectDateMathErrorWithLineNumber(
command,
@@ -3140,6 +3175,128 @@ public class StatementParserTests extends AbstractStatementParserTests {
assertThat(joinType.coreJoin().joinName(), equalTo("LEFT OUTER"));
}
+ public void testInvalidFromPatterns() {
+ var sourceCommands = Build.current().isSnapshot() ? new String[] { "FROM", "TS" } : new String[] { "FROM" };
+ var indexIsBlank = "Blank index specified in index pattern";
+ var remoteIsEmpty = "remote part is empty";
+ var invalidDoubleColonUsage = "invalid usage of :: separator";
+
+ expectError(randomFrom(sourceCommands) + " \"\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \" \"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \",,,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \",,, \"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \", , ,,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \",,,\",*", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*,,,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"index1,,,,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"index1,index2,,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"index1,<-+^,index2\",*", "must not contain the following characters");
+ expectError(randomFrom(sourceCommands) + " \"\",*", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*: ,*,\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*: ,*,\",validIndexName", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"\", \" \", \" \",validIndexName", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"index1\", \"index2\", \" ,index3,index4\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"index1,index2,,index3\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"index1,index2, ,index3\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*, \"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*\", \"\"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*\", \" \"", indexIsBlank);
+ expectError(randomFrom(sourceCommands) + " \"*\", \":index1\"", remoteIsEmpty);
+ expectError(randomFrom(sourceCommands) + " \"index1,*,:index2\"", remoteIsEmpty);
+ expectError(randomFrom(sourceCommands) + " \"*\", \"::data\"", remoteIsEmpty);
+ expectError(randomFrom(sourceCommands) + " \"*\", \"::failures\"", remoteIsEmpty);
+ expectError(randomFrom(sourceCommands) + " \"*,index1::\"", invalidDoubleColonUsage);
+ expectError(randomFrom(sourceCommands) + " \"*\", index1, index2, \"index3:: \"", invalidDoubleColonUsage);
+ expectError(randomFrom(sourceCommands) + " \"*,index1::*\"", invalidDoubleColonUsage);
+ }
+
+ public void testInvalidPatternsWithIntermittentQuotes() {
+ // There are 3 ways of crafting invalid index patterns that conforms to the grammar defined through ANTLR.
+ // 1. Not quoting the pattern,
+ // 2. Quoting individual patterns ("index1", "index2", ...), and,
+ // 3. Clubbing all the patterns into a single quoted string ("index1,index2,...).
+ //
+ // Note that in these tests, we unquote a pattern and then quote it immediately.
+ // This is because when randomly generating an index pattern, it may look like: "foo"::data.
+ // To convert it into a quoted string like "foo::data", we need to unquote and then re-quote it.
+
+ // Prohibited char in a quoted cross cluster index pattern should result in an error.
+ {
+ var randomIndex = randomIndexPattern();
+ // Select an invalid char to sneak in.
+ // Note: some chars like '|' and '"' are excluded to generate a proper invalid name.
+ Character[] invalidChars = { ' ', '/', '<', '>', '?' };
+ var randomInvalidChar = randomFrom(invalidChars);
+
+ // Construct the new invalid index pattern.
+ var invalidIndexName = "foo" + randomInvalidChar + "bar";
+ var remoteIndexWithInvalidChar = quote(randomIdentifier() + ":" + invalidIndexName);
+ var query = "FROM " + randomIndex + "," + remoteIndexWithInvalidChar;
+ expectError(
+ query,
+ "Invalid index name ["
+ + invalidIndexName
+ + "], must not contain the following characters [' ','\"',',','/','<','>','?','\\','|']"
+ );
+ }
+
+ // Colon outside a quoted string should result in an ANTLR error: a comma is expected.
+ {
+ var randomIndex = randomIndexPattern();
+
+ // In the form of: "*|cluster alias:random string".
+ var malformedClusterAlias = quote((randomBoolean() ? "*" : randomIdentifier()) + ":" + randomIdentifier());
+
+ // We do not generate a cross cluster pattern or else we'd be getting a different error (which is tested in
+ // the next test).
+ var remoteIndex = quote(unquoteIndexPattern(randomIndexPattern(without(CROSS_CLUSTER))));
+ // Format: FROM , "":
+ var query = "FROM " + randomIndex + "," + malformedClusterAlias + ":" + remoteIndex;
+ expectError(query, " mismatched input ':'");
+ }
+
+ // If an explicit cluster string is present, then we expect an unquoted string next.
+ {
+ var randomIndex = randomIndexPattern();
+ var remoteClusterAlias = randomBoolean() ? "*" : randomIdentifier();
+ // In the form of: random string:random string.
+ var malformedRemoteIndex = quote(unquoteIndexPattern(randomIndexPattern(CROSS_CLUSTER)));
+ // Format: FROM , :"random string:random string"
+ var query = "FROM " + randomIndex + "," + remoteClusterAlias + ":" + malformedRemoteIndex;
+ // Since "random string:random string" is partially quoted, expect a ANTLR's parse error.
+ expectError(query, "expecting UNQUOTED_SOURCE");
+ }
+
+ if (EsqlCapabilities.Cap.INDEX_COMPONENT_SELECTORS.isEnabled()) {
+ // If a stream in on a remote and the pattern is entirely quoted, we should be able to validate it.
+ // Note: invalid selector syntax is covered in a different test.
+ {
+ var fromPattern = randomIndexPattern();
+ var malformedIndexSelectorPattern = quote(
+ (randomIdentifier()) + ":" + unquoteIndexPattern(randomIndexPattern(INDEX_SELECTOR, without(CROSS_CLUSTER)))
+ );
+ // Format: FROM , ":::"
+ var query = "FROM " + fromPattern + "," + malformedIndexSelectorPattern;
+ expectError(query, "Selectors are not yet supported on remote cluster patterns");
+ }
+
+ // If a stream in on a remote and the cluster alias and index pattern are separately quoted, we should
+ // still be able to validate it.
+ // Note: invalid selector syntax is covered in a different test.
+ {
+ var fromPattern = randomIndexPattern();
+ var malformedIndexSelectorPattern = quote(randomIdentifier())
+ + ":"
+ + quote(unquoteIndexPattern(randomIndexPattern(INDEX_SELECTOR, without(CROSS_CLUSTER))));
+ // Format: FROM , "":"::"
+ var query = "FROM " + fromPattern + "," + malformedIndexSelectorPattern;
+ // Everything after "" is extraneous input and hence ANTLR's error.
+ expectError(query, "mismatched input ':'");
+ }
+ }
+ }
+
public void testInvalidJoinPatterns() {
assumeTrue("LOOKUP JOIN requires corresponding capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled());
@@ -3172,6 +3329,18 @@ public class StatementParserTests extends AbstractStatementParserTests {
// If one or more patterns participating in LOOKUP JOINs are partially quoted, we expect the partial quoting
// error messages to take precedence over any LOOKUP JOIN error messages.
+
+ {
+ // Generate a syntactically invalid (partial quoted) pattern.
+ var fromPatterns = quote(randomIdentifier()) + ":" + unquoteIndexPattern(randomIndexPattern(without(CROSS_CLUSTER)));
+ var joinPattern = randomIndexPattern();
+ expectError(
+ "FROM " + fromPatterns + " | LOOKUP JOIN " + joinPattern + " ON " + randomIdentifier(),
+ // Since the from pattern is partially quoted, we get an error at the end of the partially quoted string.
+ " mismatched input ':'"
+ );
+ }
+
{
// Generate a syntactically invalid (partial quoted) pattern.
var fromPatterns = randomIdentifier() + ":" + quote(randomIndexPatterns(without(CROSS_CLUSTER)));
@@ -3184,6 +3353,17 @@ public class StatementParserTests extends AbstractStatementParserTests {
);
}
+ {
+ var fromPatterns = randomIndexPattern();
+ // Generate a syntactically invalid (partial quoted) pattern.
+ var joinPattern = quote(randomIdentifier()) + ":" + unquoteIndexPattern(randomIndexPattern(without(CROSS_CLUSTER)));
+ expectError(
+ "FROM " + fromPatterns + " | LOOKUP JOIN " + joinPattern + " ON " + randomIdentifier(),
+ // Since the join pattern is partially quoted, we get an error at the end of the partially quoted string.
+ "mismatched input ':'"
+ );
+ }
+
{
var fromPatterns = randomIndexPattern();
// Generate a syntactically invalid (partial quoted) pattern.
@@ -3251,6 +3431,31 @@ public class StatementParserTests extends AbstractStatementParserTests {
+ "], index pattern selectors are not supported in LOOKUP JOIN"
);
}
+
+ {
+ // Although we don't support selector strings for remote indices, it's alright.
+ // The parser error message takes precedence.
+ var fromPatterns = randomIndexPatterns();
+ var joinPattern = quote(randomIdentifier()) + "::" + randomFrom("data", "failures");
+ // After the end of the partially quoted string, i.e. the index name, parser now expects "ON..." and not a selector string.
+ expectError(
+ "FROM " + fromPatterns + " | LOOKUP JOIN " + joinPattern + " ON " + randomIdentifier(),
+ "mismatched input ':' expecting 'on'"
+ );
+ }
+
+ {
+ // Although we don't support selector strings for remote indices, it's alright.
+ // The parser error message takes precedence.
+ var fromPatterns = randomIndexPatterns();
+ var joinPattern = randomIdentifier() + "::" + quote(randomFrom("data", "failures"));
+ // After the index name and "::", parser expects an unquoted string, i.e. the selector string should not be
+ // partially quoted.
+ expectError(
+ "FROM " + fromPatterns + " | LOOKUP JOIN " + joinPattern + " ON " + randomIdentifier(),
+ " mismatched input ':' expecting UNQUOTED_SOURCE"
+ );
+ }
}
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java
index 0fee0c13178d..9bc2118c0451 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java
@@ -23,14 +23,21 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.compute.lucene.DataPartitioning;
import org.elasticsearch.compute.lucene.LuceneSourceOperator;
import org.elasticsearch.compute.lucene.LuceneTopNSourceOperator;
+import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator;
import org.elasticsearch.compute.operator.SourceOperator;
+import org.elasticsearch.compute.test.NoOpReleasable;
import org.elasticsearch.compute.test.TestBlockFactory;
import org.elasticsearch.core.IOUtils;
import org.elasticsearch.core.Releasable;
import org.elasticsearch.core.Releasables;
import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.cache.query.TrivialQueryCachingPolicy;
+import org.elasticsearch.index.mapper.BlockLoader;
+import org.elasticsearch.index.mapper.BlockSourceReader;
+import org.elasticsearch.index.mapper.FallbackSyntheticSourceBlockLoader;
+import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperServiceTestCase;
+import org.elasticsearch.index.query.SearchExecutionContext;
import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.ExtensiblePlugin;
import org.elasticsearch.plugins.Plugin;
@@ -42,10 +49,12 @@ import org.elasticsearch.xpack.esql.core.expression.Literal;
import org.elasticsearch.xpack.esql.core.tree.Source;
import org.elasticsearch.xpack.esql.core.type.DataType;
import org.elasticsearch.xpack.esql.core.type.EsField;
+import org.elasticsearch.xpack.esql.core.type.PotentiallyUnmappedKeywordEsField;
import org.elasticsearch.xpack.esql.core.util.StringUtils;
import org.elasticsearch.xpack.esql.expression.Order;
import org.elasticsearch.xpack.esql.index.EsIndex;
import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
import org.elasticsearch.xpack.esql.plan.physical.ParallelExec;
import org.elasticsearch.xpack.esql.plugin.EsqlPlugin;
@@ -64,6 +73,7 @@ import java.util.Map;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
public class LocalExecutionPlannerTests extends MapperServiceTestCase {
@@ -84,10 +94,17 @@ public class LocalExecutionPlannerTests extends MapperServiceTestCase {
private final ArrayList releasables = new ArrayList<>();
+ private Settings settings = SETTINGS;
+
public LocalExecutionPlannerTests(@Name("estimatedRowSizeIsHuge") boolean estimatedRowSizeIsHuge) {
this.estimatedRowSizeIsHuge = estimatedRowSizeIsHuge;
}
+ @Override
+ protected Settings getIndexSettings() {
+ return settings;
+ }
+
@Override
protected Collection getPlugins() {
var plugin = new SpatialPlugin();
@@ -229,6 +246,47 @@ public class LocalExecutionPlannerTests extends MapperServiceTestCase {
assertThat(plan.driverFactories, hasSize(2));
}
+ public void testPlanUnmappedFieldExtractStoredSource() throws Exception {
+ var blockLoader = constructBlockLoader();
+ // In case of stored source we expect bytes based block source loader (this loads source from _source)
+ assertThat(blockLoader, instanceOf(BlockSourceReader.BytesRefsBlockLoader.class));
+ }
+
+ public void testPlanUnmappedFieldExtractSyntheticSource() throws Exception {
+ // Enables synthetic source, so that fallback synthetic source blocker loader is used:
+ settings = Settings.builder().put(settings).put("index.mapping.source.mode", "synthetic").build();
+
+ var blockLoader = constructBlockLoader();
+ // In case of synthetic source we expect bytes based block source loader (this loads source from _ignored_source)
+ assertThat(blockLoader, instanceOf(FallbackSyntheticSourceBlockLoader.class));
+ }
+
+ private BlockLoader constructBlockLoader() throws IOException {
+ EsQueryExec queryExec = new EsQueryExec(
+ Source.EMPTY,
+ index().name(),
+ IndexMode.STANDARD,
+ index().indexNameWithModes(),
+ List.of(new FieldAttribute(Source.EMPTY, EsQueryExec.DOC_ID_FIELD.getName(), EsQueryExec.DOC_ID_FIELD)),
+ null,
+ null,
+ null,
+ between(1, 1000)
+ );
+ FieldExtractExec fieldExtractExec = new FieldExtractExec(
+ Source.EMPTY,
+ queryExec,
+ List.of(
+ new FieldAttribute(Source.EMPTY, "potentially_unmapped", new PotentiallyUnmappedKeywordEsField("potentially_unmapped"))
+ ),
+ MappedFieldType.FieldExtractPreference.NONE
+ );
+ LocalExecutionPlanner.LocalExecutionPlan plan = planner().plan("test", FoldContext.small(), fieldExtractExec);
+ var p = plan.driverFactories.get(0).driverSupplier().physicalOperation();
+ var fieldInfo = ((ValuesSourceReaderOperator.Factory) p.intermediateOperatorFactories.get(0)).fields().get(0);
+ return fieldInfo.blockLoader().apply(0);
+ }
+
private int randomEstimatedRowSize(boolean huge) {
int hugeBoundary = SourceOperator.MIN_TARGET_PAGE_SIZE * 10;
return huge ? between(hugeBoundary, Integer.MAX_VALUE) : between(1, hugeBoundary);
@@ -296,10 +354,11 @@ public class LocalExecutionPlannerTests extends MapperServiceTestCase {
true
);
for (int i = 0; i < numShards; i++) {
+ SearchExecutionContext searchExecutionContext = createSearchExecutionContext(createMapperService(mapping(b -> {
+ b.startObject("point").field("type", "geo_point").endObject();
+ })), searcher);
shardContexts.add(
- new EsPhysicalOperationProviders.DefaultShardContext(i, createSearchExecutionContext(createMapperService(mapping(b -> {
- b.startObject("point").field("type", "geo_point").endObject();
- })), searcher), AliasFilter.EMPTY)
+ new EsPhysicalOperationProviders.DefaultShardContext(i, new NoOpReleasable(), searchExecutionContext, AliasFilter.EMPTY)
);
}
releasables.add(searcher);
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java
index d5aa1af7feec..a8916f140ea1 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java
@@ -26,6 +26,7 @@ import org.elasticsearch.compute.data.IntBlock;
import org.elasticsearch.compute.data.IntVector;
import org.elasticsearch.compute.data.LongBlock;
import org.elasticsearch.compute.data.Page;
+import org.elasticsearch.compute.lucene.ShardRefCounted;
import org.elasticsearch.compute.operator.DriverContext;
import org.elasticsearch.compute.operator.HashAggregationOperator;
import org.elasticsearch.compute.operator.Operator;
@@ -188,6 +189,7 @@ public class TestPhysicalOperationProviders extends AbstractPhysicalOperationPro
var page = pageIndex.page;
BlockFactory blockFactory = driverContext.blockFactory();
DocVector docVector = new DocVector(
+ ShardRefCounted.ALWAYS_REFERENCED,
// The shard ID is used to encode the index ID.
blockFactory.newConstantIntVector(index, page.getPositionCount()),
blockFactory.newConstantIntVector(0, page.getPositionCount()),
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java
index 9b7615d0cc37..ababc8ed3765 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/ClusterRequestTests.java
@@ -57,7 +57,7 @@ public class ClusterRequestTests extends AbstractWireSerializingTestCase writeables = new ArrayList<>();
writeables.addAll(new SearchModule(Settings.EMPTY, List.of()).getNamedWriteables());
- writeables.addAll(new EsqlPlugin().getNamedWriteables());
+ writeables.addAll(new EsqlPlugin(Settings.EMPTY).getNamedWriteables());
return new NamedWriteableRegistry(writeables);
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java
index abf9b527f008..1fc481711df9 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plugin/DataNodeRequestSerializationTests.java
@@ -60,7 +60,7 @@ public class DataNodeRequestSerializationTests extends AbstractWireSerializingTe
protected NamedWriteableRegistry getNamedWriteableRegistry() {
List writeables = new ArrayList<>();
writeables.addAll(new SearchModule(Settings.EMPTY, List.of()).getNamedWriteables());
- writeables.addAll(new EsqlPlugin().getNamedWriteables());
+ writeables.addAll(new EsqlPlugin(Settings.EMPTY).getNamedWriteables());
return new NamedWriteableRegistry(writeables);
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java
index 752e61c240cd..2d8151d8fc2a 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java
@@ -150,7 +150,14 @@ public class PlanExecutorMetricsTests extends ESTestCase {
return null;
}).when(esqlClient).execute(eq(EsqlResolveFieldsAction.TYPE), any(), any());
- var planExecutor = new PlanExecutor(indexResolver, MeterRegistry.NOOP, new XPackLicenseState(() -> 0L), mockQueryLog(), List.of());
+ var planExecutor = new PlanExecutor(
+ indexResolver,
+ MeterRegistry.NOOP,
+ new XPackLicenseState(() -> 0L),
+ mockQueryLog(),
+ List.of(),
+ Settings.EMPTY
+ );
var enrichResolver = mockEnrichResolver();
var request = new EsqlQueryRequest();
diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleService.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleService.java
index 389bac89aaed..1dae42956dec 100644
--- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleService.java
+++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/IndexLifecycleService.java
@@ -18,7 +18,7 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.ProjectState;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
-import org.elasticsearch.cluster.metadata.Metadata;
+import org.elasticsearch.cluster.metadata.ProjectId;
import org.elasticsearch.cluster.metadata.ProjectMetadata;
import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata;
import org.elasticsearch.cluster.service.ClusterService;
@@ -28,7 +28,7 @@ import org.elasticsearch.common.scheduler.SchedulerEngine;
import org.elasticsearch.common.scheduler.TimeValueSchedule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
-import org.elasticsearch.core.FixForMultiProject;
+import org.elasticsearch.core.NotMultiProjectCapable;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.core.TimeValue;
@@ -183,9 +183,12 @@ public class IndexLifecycleService
void onMaster(ClusterState clusterState) {
maybeScheduleJob();
- // TODO multi-project: this probably needs a per-project iteration
- @FixForMultiProject
- final ProjectState state = clusterState.projectState(Metadata.DEFAULT_PROJECT_ID);
+ for (var projectId : clusterState.metadata().projects().keySet()) {
+ onMaster(clusterState.projectState(projectId));
+ }
+ }
+
+ void onMaster(ProjectState state) {
final ProjectMetadata projectMetadata = state.metadata();
final IndexLifecycleMetadata currentMetadata = projectMetadata.custom(IndexLifecycleMetadata.TYPE);
if (currentMetadata != null) {
@@ -260,13 +263,13 @@ public class IndexLifecycleService
}
if (safeToStop && OperationMode.STOPPING == currentMode) {
- stopILM();
+ stopILM(state.projectId());
}
}
}
- private void stopILM() {
- submitUnbatchedTask("ilm_operation_mode_update[stopped]", OperationModeUpdateTask.ilmMode(OperationMode.STOPPED));
+ private void stopILM(ProjectId projectId) {
+ submitUnbatchedTask("ilm_operation_mode_update[stopped]", OperationModeUpdateTask.ilmMode(projectId, OperationMode.STOPPED));
}
@Override
@@ -409,6 +412,7 @@ public class IndexLifecycleService
});
}
+ @NotMultiProjectCapable(description = "See comment inside the method")
@Override
public void applyClusterState(ClusterChangedEvent event) {
// only act if we are master, otherwise keep idle until elected
@@ -416,20 +420,21 @@ public class IndexLifecycleService
return;
}
- @FixForMultiProject
- final IndexLifecycleMetadata ilmMetadata = event.state()
- .metadata()
- .getProject(Metadata.DEFAULT_PROJECT_ID)
- .custom(IndexLifecycleMetadata.TYPE);
- if (ilmMetadata == null) {
- return;
- }
- final IndexLifecycleMetadata previousIlmMetadata = event.previousState()
- .metadata()
- .getProject(Metadata.DEFAULT_PROJECT_ID)
- .custom(IndexLifecycleMetadata.TYPE);
- if (event.previousState().nodes().isLocalNodeElectedMaster() == false || ilmMetadata != previousIlmMetadata) {
- policyRegistry.update(ilmMetadata);
+ // We're updating the policy registry cache here, which doesn't actually work with multiple projects because the policies from one
+ // project would overwrite the polices from another project. However, since we're not planning on running ILM in a multi-project
+ // cluster, we can ignore this.
+ for (var project : event.state().metadata().projects().values()) {
+ final IndexLifecycleMetadata ilmMetadata = project.custom(IndexLifecycleMetadata.TYPE);
+ if (ilmMetadata == null) {
+ continue;
+ }
+ final var previousProject = event.previousState().metadata().projects().get(project.id());
+ final IndexLifecycleMetadata previousIlmMetadata = previousProject == null
+ ? null
+ : previousProject.custom(IndexLifecycleMetadata.TYPE);
+ if (event.previousState().nodes().isLocalNodeElectedMaster() == false || ilmMetadata != previousIlmMetadata) {
+ policyRegistry.update(ilmMetadata);
+ }
}
}
@@ -461,10 +466,13 @@ public class IndexLifecycleService
* @param clusterState the current cluster state
* @param fromClusterStateChange whether things are triggered from the cluster-state-listener or the scheduler
*/
- @FixForMultiProject
void triggerPolicies(ClusterState clusterState, boolean fromClusterStateChange) {
- @FixForMultiProject
- final var state = clusterState.projectState(Metadata.DEFAULT_PROJECT_ID);
+ for (var projectId : clusterState.metadata().projects().keySet()) {
+ triggerPolicies(clusterState.projectState(projectId), fromClusterStateChange);
+ }
+ }
+
+ void triggerPolicies(ProjectState state, boolean fromClusterStateChange) {
final var projectMetadata = state.metadata();
IndexLifecycleMetadata currentMetadata = projectMetadata.custom(IndexLifecycleMetadata.TYPE);
@@ -472,7 +480,7 @@ public class IndexLifecycleService
if (currentMetadata == null) {
if (currentMode == OperationMode.STOPPING) {
// There are no policies and ILM is in stopping mode, so stop ILM and get out of here
- stopILM();
+ stopILM(state.projectId());
}
return;
}
@@ -555,7 +563,7 @@ public class IndexLifecycleService
}
if (safeToStop && OperationMode.STOPPING == currentMode) {
- stopILM();
+ stopILM(state.projectId());
}
}
@@ -585,7 +593,7 @@ public class IndexLifecycleService
return policyRegistry;
}
- static Set indicesOnShuttingDownNodesInDangerousStep(ClusterState state, String nodeId) {
+ static boolean hasIndicesInDangerousStepForNodeShutdown(ClusterState state, String nodeId) {
final Set shutdownNodes = PluginShutdownService.shutdownTypeNodes(
state,
SingleNodeShutdownMetadata.Type.REMOVE,
@@ -593,43 +601,46 @@ public class IndexLifecycleService
SingleNodeShutdownMetadata.Type.REPLACE
);
if (shutdownNodes.isEmpty()) {
- return Set.of();
+ return true;
}
- // Returning a set of strings will cause weird behavior with multiple projects
- @FixForMultiProject
- Set indicesPreventingShutdown = state.metadata()
- .projects()
- .values()
- .stream()
- .flatMap(project -> project.indices().entrySet().stream())
- // Filter out to only consider managed indices
- .filter(indexToMetadata -> Strings.hasText(indexToMetadata.getValue().getLifecyclePolicyName()))
- // Only look at indices in the shrink action
- .filter(indexToMetadata -> ShrinkAction.NAME.equals(indexToMetadata.getValue().getLifecycleExecutionState().action()))
- // Only look at indices on a step that may potentially be dangerous if we removed the node
- .filter(indexToMetadata -> {
- String step = indexToMetadata.getValue().getLifecycleExecutionState().step();
- return SetSingleNodeAllocateStep.NAME.equals(step)
- || CheckShrinkReadyStep.NAME.equals(step)
- || ShrinkStep.NAME.equals(step)
- || ShrunkShardsAllocatedStep.NAME.equals(step);
- })
- // Only look at indices where the node picked for the shrink is the node marked as shutting down
- .filter(indexToMetadata -> {
- String nodePicked = indexToMetadata.getValue()
- .getSettings()
- .get(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_id");
- return nodeId.equals(nodePicked);
- })
- .map(Map.Entry::getKey)
- .collect(Collectors.toSet());
- logger.trace(
- "with nodes marked as shutdown for removal {}, indices {} are preventing shutdown",
- shutdownNodes,
- indicesPreventingShutdown
- );
- return indicesPreventingShutdown;
+ boolean result = true;
+ for (var project : state.metadata().projects().values()) {
+ Set indicesPreventingShutdown = project.indices()
+ .entrySet()
+ .stream()
+ // Filter out to only consider managed indices
+ .filter(indexToMetadata -> Strings.hasText(indexToMetadata.getValue().getLifecyclePolicyName()))
+ // Only look at indices in the shrink action
+ .filter(indexToMetadata -> ShrinkAction.NAME.equals(indexToMetadata.getValue().getLifecycleExecutionState().action()))
+ // Only look at indices on a step that may potentially be dangerous if we removed the node
+ .filter(indexToMetadata -> {
+ String step = indexToMetadata.getValue().getLifecycleExecutionState().step();
+ return SetSingleNodeAllocateStep.NAME.equals(step)
+ || CheckShrinkReadyStep.NAME.equals(step)
+ || ShrinkStep.NAME.equals(step)
+ || ShrunkShardsAllocatedStep.NAME.equals(step);
+ })
+ // Only look at indices where the node picked for the shrink is the node marked as shutting down
+ .filter(indexToMetadata -> {
+ String nodePicked = indexToMetadata.getValue()
+ .getSettings()
+ .get(IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_SETTING.getKey() + "_id");
+ return nodeId.equals(nodePicked);
+ })
+ .map(Map.Entry::getKey)
+ .collect(Collectors.toSet());
+ logger.trace(
+ "with nodes marked as shutdown for removal {}, indices {} in project {} are preventing shutdown",
+ shutdownNodes,
+ indicesPreventingShutdown,
+ project.id()
+ );
+ if (indicesPreventingShutdown.isEmpty() == false) {
+ result = false;
+ }
+ }
+ return result;
}
@Override
@@ -641,8 +652,7 @@ public class IndexLifecycleService
case REPLACE:
case REMOVE:
case SIGTERM:
- Set indices = indicesOnShuttingDownNodesInDangerousStep(clusterService.state(), nodeId);
- return indices.isEmpty();
+ return hasIndicesInDangerousStepForNodeShutdown(clusterService.state(), nodeId);
default:
throw new IllegalArgumentException("unknown shutdown type: " + shutdownType);
}
diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStartILMAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStartILMAction.java
index 5a2ff6d58bfa..d6964fd7c791 100644
--- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStartILMAction.java
+++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStartILMAction.java
@@ -15,6 +15,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.cluster.project.ProjectResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.core.SuppressForbidden;
@@ -29,12 +30,15 @@ import org.elasticsearch.xpack.core.ilm.action.ILMActions;
public class TransportStartILMAction extends AcknowledgedTransportMasterNodeAction {
+ private final ProjectResolver projectResolver;
+
@Inject
public TransportStartILMAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
- ActionFilters actionFilters
+ ActionFilters actionFilters,
+ ProjectResolver projectResolver
) {
super(
ILMActions.START.name(),
@@ -45,13 +49,15 @@ public class TransportStartILMAction extends AcknowledgedTransportMasterNodeActi
StartILMRequest::new,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
+ this.projectResolver = projectResolver;
}
@Override
protected void masterOperation(Task task, StartILMRequest request, ClusterState state, ActionListener listener) {
+ final var projectId = projectResolver.getProjectId();
submitUnbatchedTask(
"ilm_operation_mode_update[running]",
- OperationModeUpdateTask.wrap(OperationModeUpdateTask.ilmMode(OperationMode.RUNNING), request, listener)
+ OperationModeUpdateTask.wrap(OperationModeUpdateTask.ilmMode(projectId, OperationMode.RUNNING), request, listener)
);
}
diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStopILMAction.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStopILMAction.java
index 1c231da4ec13..fd8736eb2a28 100644
--- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStopILMAction.java
+++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/ilm/action/TransportStopILMAction.java
@@ -15,6 +15,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
+import org.elasticsearch.cluster.project.ProjectResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.core.SuppressForbidden;
@@ -29,12 +30,15 @@ import org.elasticsearch.xpack.core.ilm.action.ILMActions;
public class TransportStopILMAction extends AcknowledgedTransportMasterNodeAction {
+ private final ProjectResolver projectResolver;
+
@Inject
public TransportStopILMAction(
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
- ActionFilters actionFilters
+ ActionFilters actionFilters,
+ ProjectResolver projectResolver
) {
super(
ILMActions.STOP.name(),
@@ -45,13 +49,15 @@ public class TransportStopILMAction extends AcknowledgedTransportMasterNodeActio
StopILMRequest::new,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
+ this.projectResolver = projectResolver;
}
@Override
protected void masterOperation(Task task, StopILMRequest request, ClusterState state, ActionListener listener) {
+ final var projectId = projectResolver.getProjectId();
submitUnbatchedTask(
"ilm_operation_mode_update[stopping]",
- OperationModeUpdateTask.wrap(OperationModeUpdateTask.ilmMode(OperationMode.STOPPING), request, listener)
+ OperationModeUpdateTask.wrap(OperationModeUpdateTask.ilmMode(projectId, OperationMode.STOPPING), request, listener)
);
}
diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java
index 68ab9613e9de..61eb44ee5497 100644
--- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java
+++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/IndexLifecycleServiceTests.java
@@ -18,6 +18,7 @@ import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.LifecycleExecutionState;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.NodesShutdownMetadata;
+import org.elasticsearch.cluster.metadata.ProjectMetadata;
import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeUtils;
@@ -169,13 +170,11 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.numberOfReplicas(randomIntBetween(0, 5))
.build();
Map indices = Map.of(index.getName(), indexMetadata);
- Metadata metadata = Metadata.builder()
+ var project = ProjectMetadata.builder(randomProjectIdOrDefault())
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap, OperationMode.STOPPED))
- .indices(indices)
- .persistentSettings(settings(IndexVersion.current()).build())
- .build();
+ .indices(indices);
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
- .metadata(metadata)
+ .putProjectMetadata(project)
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
.build();
ClusterChangedEvent event = new ClusterChangedEvent("_source", currentState, ClusterState.EMPTY_STATE);
@@ -208,13 +207,11 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.numberOfReplicas(randomIntBetween(0, 5))
.build();
Map indices = Map.of(index.getName(), indexMetadata);
- Metadata metadata = Metadata.builder()
+ var project = ProjectMetadata.builder(randomProjectIdOrDefault())
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap, OperationMode.STOPPING))
- .indices(indices)
- .persistentSettings(settings(IndexVersion.current()).build())
- .build();
+ .indices(indices);
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
- .metadata(metadata)
+ .putProjectMetadata(project)
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
.build();
@@ -264,13 +261,11 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.numberOfReplicas(randomIntBetween(0, 5))
.build();
Map indices = Map.of(index.getName(), indexMetadata);
- Metadata metadata = Metadata.builder()
+ var project = ProjectMetadata.builder(randomProjectIdOrDefault())
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap, OperationMode.STOPPING))
- .indices(indices)
- .persistentSettings(settings(IndexVersion.current()).build())
- .build();
+ .indices(indices);
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
- .metadata(metadata)
+ .putProjectMetadata(project)
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
.build();
@@ -312,13 +307,11 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.numberOfReplicas(randomIntBetween(0, 5))
.build();
Map indices = Map.of(index.getName(), indexMetadata);
- Metadata metadata = Metadata.builder()
+ var project = ProjectMetadata.builder(randomProjectIdOrDefault())
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap, OperationMode.STOPPING))
- .indices(indices)
- .persistentSettings(settings(IndexVersion.current()).build())
- .build();
+ .indices(indices);
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
- .metadata(metadata)
+ .putProjectMetadata(project)
.nodes(DiscoveryNodes.builder().localNodeId(nodeId).masterNodeId(nodeId).add(masterNode).build())
.build();
@@ -429,11 +422,9 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.build();
Map indices = Map.of(index1.getName(), i1indexMetadata, index2.getName(), i2indexMetadata);
- Metadata metadata = Metadata.builder()
+ var project = ProjectMetadata.builder(randomProjectIdOrDefault())
.putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(policyMap, OperationMode.RUNNING))
- .indices(indices)
- .persistentSettings(settings(IndexVersion.current()).build())
- .build();
+ .indices(indices);
Settings settings = Settings.builder().put(LifecycleSettings.LIFECYCLE_POLL_INTERVAL, "1s").build();
var clusterSettings = new ClusterSettings(
@@ -443,7 +434,7 @@ public class IndexLifecycleServiceTests extends ESTestCase {
ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool, clusterSettings);
DiscoveryNode node = clusterService.localNode();
ClusterState currentState = ClusterState.builder(ClusterName.DEFAULT)
- .metadata(metadata)
+ .putProjectMetadata(project)
.nodes(DiscoveryNodes.builder().add(node).masterNodeId(node.getId()).localNodeId(node.getId()))
.build();
ClusterServiceUtils.setState(clusterService, currentState);
@@ -533,15 +524,16 @@ public class IndexLifecycleServiceTests extends ESTestCase {
}
}
- public void testIndicesOnShuttingDownNodesInDangerousStep() {
+ public void testHasIndicesInDangerousStepForNodeShutdown() {
for (SingleNodeShutdownMetadata.Type type : List.of(
SingleNodeShutdownMetadata.Type.REMOVE,
SingleNodeShutdownMetadata.Type.SIGTERM,
SingleNodeShutdownMetadata.Type.REPLACE
)) {
- ClusterState state = ClusterState.builder(ClusterName.DEFAULT).build();
- assertThat(IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "regular_node"), equalTo(Set.of()));
- assertThat(IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "shutdown_node"), equalTo(Set.of()));
+ final var project = ProjectMetadata.builder(randomProjectIdOrDefault()).build();
+ ClusterState state = ClusterState.builder(ClusterName.DEFAULT).putProjectMetadata(project).build();
+ assertThat(IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "regular_node"), equalTo(true));
+ assertThat(IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "shutdown_node"), equalTo(true));
IndexMetadata nonDangerousIndex = IndexMetadata.builder("no_danger")
.settings(settings(IndexVersion.current()).put(LifecycleSettings.LIFECYCLE_NAME, "mypolicy"))
@@ -583,14 +575,12 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.build();
Map indices = Map.of("no_danger", nonDangerousIndex, "danger", dangerousIndex);
- Metadata metadata = Metadata.builder()
- .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(Map.of(), OperationMode.RUNNING))
- .indices(indices)
- .persistentSettings(settings(IndexVersion.current()).build())
- .build();
-
state = ClusterState.builder(ClusterName.DEFAULT)
- .metadata(metadata)
+ .putProjectMetadata(
+ ProjectMetadata.builder(project)
+ .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata(Map.of(), OperationMode.RUNNING))
+ .indices(indices)
+ )
.nodes(
DiscoveryNodes.builder()
.localNodeId(nodeId)
@@ -613,8 +603,8 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.build();
// No danger yet, because no node is shutting down
- assertThat(IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "regular_node"), equalTo(Set.of()));
- assertThat(IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "shutdown_node"), equalTo(Set.of()));
+ assertThat(IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "regular_node"), equalTo(true));
+ assertThat(IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "shutdown_node"), equalTo(true));
state = ClusterState.builder(state)
.metadata(
@@ -638,12 +628,12 @@ public class IndexLifecycleServiceTests extends ESTestCase {
)
.build();
- assertThat(IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "regular_node"), equalTo(Set.of()));
+ assertThat(IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "regular_node"), equalTo(true));
// No danger, because this is a "RESTART" type shutdown
assertThat(
"restart type shutdowns are not considered dangerous",
- IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "shutdown_node"),
- equalTo(Set.of())
+ IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "shutdown_node"),
+ equalTo(true)
);
final String targetNodeName = type == SingleNodeShutdownMetadata.Type.REPLACE ? randomAlphaOfLengthBetween(10, 20) : null;
@@ -673,7 +663,7 @@ public class IndexLifecycleServiceTests extends ESTestCase {
.build();
// The dangerous index should be calculated as being in danger now
- assertThat(IndexLifecycleService.indicesOnShuttingDownNodesInDangerousStep(state, "shutdown_node"), equalTo(Set.of("danger")));
+ assertThat(IndexLifecycleService.hasIndicesInDangerousStepForNodeShutdown(state, "shutdown_node"), equalTo(false));
}
}
}
diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportStopILMActionTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportStopILMActionTests.java
index 073cb5554443..7569a70155c9 100644
--- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportStopILMActionTests.java
+++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/ilm/action/TransportStopILMActionTests.java
@@ -11,6 +11,7 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
import org.elasticsearch.cluster.ClusterState;
+import org.elasticsearch.cluster.project.TestProjectResolvers;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.tasks.Task;
@@ -41,7 +42,8 @@ public class TransportStopILMActionTests extends ESTestCase {
transportService,
clusterService,
threadPool,
- mock(ActionFilters.class)
+ mock(ActionFilters.class),
+ TestProjectResolvers.singleProject(randomProjectIdOrDefault())
);
Task task = new Task(
randomLong(),
diff --git a/x-pack/plugin/inference/build.gradle b/x-pack/plugin/inference/build.gradle
index b58e1e941b16..9486d239e5de 100644
--- a/x-pack/plugin/inference/build.gradle
+++ b/x-pack/plugin/inference/build.gradle
@@ -405,3 +405,7 @@ tasks.named("thirdPartyAudit").configure {
tasks.named('yamlRestTest') {
usesDefaultDistribution("Uses the inference API")
}
+
+artifacts {
+ restXpackTests(new File(projectDir, "src/yamlRestTest/resources/rest-api-spec/test"))
+}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java
index 2bc481cc484d..3d05600709b2 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java
@@ -17,6 +17,7 @@ import java.util.Set;
import static org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper.SEMANTIC_TEXT_EXCLUDE_SUB_FIELDS_FROM_FIELD_CAPS;
import static org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper.SEMANTIC_TEXT_INDEX_OPTIONS;
+import static org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper.SEMANTIC_TEXT_INDEX_OPTIONS_WITH_DEFAULTS;
import static org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper.SEMANTIC_TEXT_SUPPORT_CHUNKING_CONFIG;
import static org.elasticsearch.xpack.inference.queries.SemanticKnnVectorQueryRewriteInterceptor.SEMANTIC_KNN_FILTER_FIX;
import static org.elasticsearch.xpack.inference.queries.SemanticKnnVectorQueryRewriteInterceptor.SEMANTIC_KNN_VECTOR_QUERY_REWRITE_INTERCEPTION_SUPPORTED;
@@ -66,7 +67,8 @@ public class InferenceFeatures implements FeatureSpecification {
SEMANTIC_TEXT_MATCH_ALL_HIGHLIGHTER,
SEMANTIC_TEXT_EXCLUDE_SUB_FIELDS_FROM_FIELD_CAPS,
SEMANTIC_TEXT_INDEX_OPTIONS,
- COHERE_V2_API
+ COHERE_V2_API,
+ SEMANTIC_TEXT_INDEX_OPTIONS_WITH_DEFAULTS
);
}
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java
index 5400bf6acc67..fd5f1ce2735a 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java
@@ -69,6 +69,7 @@ import org.elasticsearch.inference.ChunkingSettings;
import org.elasticsearch.inference.InferenceResults;
import org.elasticsearch.inference.MinimalServiceSettings;
import org.elasticsearch.inference.SimilarityMeasure;
+import org.elasticsearch.inference.TaskType;
import org.elasticsearch.search.fetch.StoredFieldsSpec;
import org.elasticsearch.search.lookup.Source;
import org.elasticsearch.search.vectors.KnnVectorQueryBuilder;
@@ -139,6 +140,9 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
"semantic_text.exclude_sub_fields_from_field_caps"
);
public static final NodeFeature SEMANTIC_TEXT_INDEX_OPTIONS = new NodeFeature("semantic_text.index_options");
+ public static final NodeFeature SEMANTIC_TEXT_INDEX_OPTIONS_WITH_DEFAULTS = new NodeFeature(
+ "semantic_text.index_options_with_defaults"
+ );
public static final String CONTENT_TYPE = "semantic_text";
public static final String DEFAULT_ELSER_2_INFERENCE_ID = DEFAULT_ELSER_ID;
@@ -166,19 +170,9 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
public static class Builder extends FieldMapper.Builder {
private final ModelRegistry modelRegistry;
private final boolean useLegacyFormat;
+ private final IndexVersion indexVersionCreated;
- private final Parameter inferenceId = Parameter.stringParam(
- INFERENCE_ID_FIELD,
- false,
- mapper -> ((SemanticTextFieldType) mapper.fieldType()).inferenceId,
- DEFAULT_ELSER_2_INFERENCE_ID
- ).addValidator(v -> {
- if (Strings.isEmpty(v)) {
- throw new IllegalArgumentException(
- "[" + INFERENCE_ID_FIELD + "] on mapper [" + leafName() + "] of type [" + CONTENT_TYPE + "] must not be empty"
- );
- }
- }).alwaysSerialize();
+ private final Parameter inferenceId;
private final Parameter searchInferenceId = Parameter.stringParam(
SEARCH_INFERENCE_ID_FIELD,
@@ -193,25 +187,9 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
}
});
- private final Parameter modelSettings = new Parameter<>(
- MODEL_SETTINGS_FIELD,
- true,
- () -> null,
- (n, c, o) -> SemanticTextField.parseModelSettingsFromMap(o),
- mapper -> ((SemanticTextFieldType) mapper.fieldType()).modelSettings,
- XContentBuilder::field,
- Objects::toString
- ).acceptsNull().setMergeValidator(SemanticTextFieldMapper::canMergeModelSettings);
+ private final Parameter modelSettings;
- private final Parameter indexOptions = new Parameter<>(
- INDEX_OPTIONS_FIELD,
- true,
- () -> null,
- (n, c, o) -> parseIndexOptionsFromMap(n, o, c.indexVersionCreated()),
- mapper -> ((SemanticTextFieldType) mapper.fieldType()).indexOptions,
- XContentBuilder::field,
- Objects::toString
- ).acceptsNull();
+ private final Parameter indexOptions;
@SuppressWarnings("unchecked")
private final Parameter chunkingSettings = new Parameter<>(
@@ -248,6 +226,50 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
super(name);
this.modelRegistry = modelRegistry;
this.useLegacyFormat = InferenceMetadataFieldsMapper.isEnabled(indexSettings.getSettings()) == false;
+ this.indexVersionCreated = indexSettings.getIndexVersionCreated();
+
+ this.inferenceId = Parameter.stringParam(
+ INFERENCE_ID_FIELD,
+ false,
+ mapper -> ((SemanticTextFieldType) mapper.fieldType()).inferenceId,
+ DEFAULT_ELSER_2_INFERENCE_ID
+ ).addValidator(v -> {
+ if (Strings.isEmpty(v)) {
+ throw new IllegalArgumentException(
+ "[" + INFERENCE_ID_FIELD + "] on mapper [" + leafName() + "] of type [" + CONTENT_TYPE + "] must not be empty"
+ );
+ }
+ }).alwaysSerialize();
+
+ this.modelSettings = new Parameter<>(
+ MODEL_SETTINGS_FIELD,
+ true,
+ () -> null,
+ (n, c, o) -> SemanticTextField.parseModelSettingsFromMap(o),
+ mapper -> ((SemanticTextFieldType) mapper.fieldType()).modelSettings,
+ XContentBuilder::field,
+ Objects::toString
+ ).acceptsNull().setMergeValidator(SemanticTextFieldMapper::canMergeModelSettings);
+
+ this.indexOptions = new Parameter<>(
+ INDEX_OPTIONS_FIELD,
+ true,
+ () -> null,
+ (n, c, o) -> parseIndexOptionsFromMap(n, o, c.indexVersionCreated()),
+ mapper -> ((SemanticTextFieldType) mapper.fieldType()).indexOptions,
+ (b, n, v) -> {
+ if (v == null) {
+ MinimalServiceSettings resolvedModelSettings = modelSettings.get() != null
+ ? modelSettings.get()
+ : modelRegistry.getMinimalServiceSettings(inferenceId.get());
+ b.field(INDEX_OPTIONS_FIELD, defaultIndexOptions(indexVersionCreated, resolvedModelSettings));
+ } else {
+ b.field(INDEX_OPTIONS_FIELD, v);
+ }
+ },
+ Objects::toString
+ ).acceptsNull();
+
this.inferenceFieldBuilder = c -> {
// Resolve the model setting from the registry if it has not been set yet.
var resolvedModelSettings = modelSettings.get() != null ? modelSettings.get() : getResolvedModelSettings(c, false);
@@ -365,8 +387,11 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
validateServiceSettings(modelSettings.get(), resolvedModelSettings);
}
- if (context.getMergeReason() != MapperService.MergeReason.MAPPING_RECOVERY && indexOptions.get() != null) {
- validateIndexOptions(indexOptions.get(), inferenceId.getValue(), resolvedModelSettings);
+ // If index_options are specified by the user, we will validate them against the model settings to ensure compatibility.
+ // We do not serialize or otherwise store model settings at this time, this happens when the underlying vector field is created.
+ SemanticTextIndexOptions builderIndexOptions = indexOptions.get();
+ if (context.getMergeReason() != MapperService.MergeReason.MAPPING_RECOVERY && builderIndexOptions != null) {
+ validateIndexOptions(builderIndexOptions, inferenceId.getValue(), resolvedModelSettings);
}
final String fullName = context.buildFullName(leafName());
@@ -1166,6 +1191,9 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
}
denseVectorMapperBuilder.dimensions(modelSettings.dimensions());
denseVectorMapperBuilder.elementType(modelSettings.elementType());
+ // Here is where we persist index_options. If they are specified by the user, we will use those index_options,
+ // otherwise we will determine if we can set default index options. If we can't, we won't persist any index_options
+ // and the field will use the defaults for the dense_vector field.
if (indexOptions != null) {
DenseVectorFieldMapper.DenseVectorIndexOptions denseVectorIndexOptions =
(DenseVectorFieldMapper.DenseVectorIndexOptions) indexOptions.indexOptions();
@@ -1208,7 +1236,6 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
// As embedding models for text perform better with BBQ, we aggressively default semantic_text fields to use optimized index
// options
if (indexVersionDefaultsToBbqHnsw(indexVersionCreated)) {
-
DenseVectorFieldMapper.DenseVectorIndexOptions defaultBbqHnswIndexOptions = defaultBbqHnswDenseVectorIndexOptions();
return defaultBbqHnswIndexOptions.validate(modelSettings.elementType(), modelSettings.dimensions(), false)
? defaultBbqHnswIndexOptions
@@ -1230,11 +1257,24 @@ public class SemanticTextFieldMapper extends FieldMapper implements InferenceFie
return new DenseVectorFieldMapper.BBQHnswIndexOptions(m, efConstruction, rescoreVector);
}
- static SemanticTextIndexOptions defaultBbqHnswSemanticTextIndexOptions() {
- return new SemanticTextIndexOptions(
- SemanticTextIndexOptions.SupportedIndexOptions.DENSE_VECTOR,
- defaultBbqHnswDenseVectorIndexOptions()
- );
+ static SemanticTextIndexOptions defaultIndexOptions(IndexVersion indexVersionCreated, MinimalServiceSettings modelSettings) {
+
+ if (modelSettings == null) {
+ return null;
+ }
+
+ SemanticTextIndexOptions defaultIndexOptions = null;
+ if (modelSettings.taskType() == TaskType.TEXT_EMBEDDING) {
+ DenseVectorFieldMapper.DenseVectorIndexOptions denseVectorIndexOptions = defaultDenseVectorIndexOptions(
+ indexVersionCreated,
+ modelSettings
+ );
+ defaultIndexOptions = denseVectorIndexOptions == null
+ ? null
+ : new SemanticTextIndexOptions(SemanticTextIndexOptions.SupportedIndexOptions.DENSE_VECTOR, denseVectorIndexOptions);
+ }
+
+ return defaultIndexOptions;
}
private static boolean canMergeModelSettings(MinimalServiceSettings previous, MinimalServiceSettings current, Conflicts conflicts) {
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextIndexOptions.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextIndexOptions.java
index c062adad2f55..db647499f446 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextIndexOptions.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextIndexOptions.java
@@ -20,6 +20,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Locale;
import java.util.Map;
+import java.util.Objects;
/**
* Represents index options for a semantic_text field.
@@ -50,6 +51,25 @@ public class SemanticTextIndexOptions implements ToXContent {
return indexOptions;
}
+ @Override
+ public boolean equals(Object other) {
+ if (other == this) {
+ return true;
+ }
+
+ if (other == null || getClass() != other.getClass()) {
+ return false;
+ }
+
+ SemanticTextIndexOptions otherSemanticTextIndexOptions = (SemanticTextIndexOptions) other;
+ return type == otherSemanticTextIndexOptions.type && Objects.equals(indexOptions, otherSemanticTextIndexOptions.indexOptions);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(type, indexOptions);
+ }
+
public enum SupportedIndexOptions {
DENSE_VECTOR("dense_vector") {
@Override
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/embeddings/AmazonBedrockEmbeddingsTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/embeddings/AmazonBedrockEmbeddingsTaskSettings.java
index bb0a8a3348ad..ad06e669ff56 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/embeddings/AmazonBedrockEmbeddingsTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/embeddings/AmazonBedrockEmbeddingsTaskSettings.java
@@ -79,9 +79,16 @@ public record AmazonBedrockEmbeddingsTaskSettings(@Nullable CohereTruncation coh
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.AMAZON_BEDROCK_TASK_SETTINGS;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.AMAZON_BEDROCK_TASK_SETTINGS)
+ || version.isPatchFrom(TransportVersions.AMAZON_BEDROCK_TASK_SETTINGS_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalEnum(cohereTruncation());
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettings.java
index 1767653fd1a5..07b81c22c808 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettings.java
@@ -183,7 +183,7 @@ public class CohereServiceSettings extends FilteredXContentObject implements Ser
rateLimitSettings = DEFAULT_RATE_LIMIT_SETTINGS;
}
if (in.getTransportVersion().onOrAfter(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
- || in.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
+ || in.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19)) {
this.apiVersion = in.readEnum(CohereServiceSettings.CohereApiVersion.class);
} else {
this.apiVersion = CohereServiceSettings.CohereApiVersion.V1;
@@ -286,7 +286,7 @@ public class CohereServiceSettings extends FilteredXContentObject implements Ser
rateLimitSettings.writeTo(out);
}
if (out.getTransportVersion().onOrAfter(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
- || out.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
+ || out.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19)) {
out.writeEnum(apiVersion);
}
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettings.java
index efe58ed19a00..7f8ef305e5db 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettings.java
@@ -103,7 +103,7 @@ public class CohereCompletionServiceSettings extends FilteredXContentObject impl
modelId = in.readOptionalString();
rateLimitSettings = new RateLimitSettings(in);
if (in.getTransportVersion().onOrAfter(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
- || in.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
+ || in.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19)) {
this.apiVersion = in.readEnum(CohereServiceSettings.CohereApiVersion.class);
} else {
this.apiVersion = CohereServiceSettings.CohereApiVersion.V1;
@@ -156,7 +156,7 @@ public class CohereCompletionServiceSettings extends FilteredXContentObject impl
out.writeOptionalString(modelId);
rateLimitSettings.writeTo(out);
if (out.getTransportVersion().onOrAfter(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
- || out.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
+ || out.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19)) {
out.writeEnum(apiVersion);
}
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettings.java
index a17fff7f165c..651b8758c37f 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettings.java
@@ -125,7 +125,7 @@ public class CohereRerankServiceSettings extends FilteredXContentObject implemen
}
if (in.getTransportVersion().onOrAfter(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
- || in.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
+ || in.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19)) {
this.apiVersion = in.readEnum(CohereServiceSettings.CohereApiVersion.class);
} else {
this.apiVersion = CohereServiceSettings.CohereApiVersion.V1;
@@ -207,7 +207,7 @@ public class CohereRerankServiceSettings extends FilteredXContentObject implemen
rateLimitSettings.writeTo(out);
}
if (out.getTransportVersion().onOrAfter(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
- || out.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
+ || out.getTransportVersion().isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19)) {
out.writeEnum(apiVersion);
}
}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomSecretSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomSecretSettings.java
index ac6b7ab10c8b..4c2ff22a5829 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomSecretSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomSecretSettings.java
@@ -90,9 +90,16 @@ public class CustomSecretSettings implements SecretSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED)
+ || version.isPatchFrom(TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeMap(secretParameters, StreamOutput::writeSecureString);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomServiceSettings.java
index 83048120bc54..931eb3b79855 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomServiceSettings.java
@@ -394,9 +394,16 @@ public class CustomServiceSettings extends FilteredXContentObject implements Ser
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED)
+ || version.isPatchFrom(TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
textEmbeddingSettings.writeTo(out);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomTaskSettings.java
index bb665cc196bd..2d43e4278100 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/custom/CustomTaskSettings.java
@@ -100,9 +100,16 @@ public class CustomTaskSettings implements TaskSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED)
+ || version.isPatchFrom(TransportVersions.INFERENCE_CUSTOM_SERVICE_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeGenericMap(parameters);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/deepseek/DeepSeekChatCompletionModel.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/deepseek/DeepSeekChatCompletionModel.java
index 5e9a7e5f93a0..06f21e19a640 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/deepseek/DeepSeekChatCompletionModel.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/deepseek/DeepSeekChatCompletionModel.java
@@ -176,9 +176,16 @@ public class DeepSeekChatCompletionModel extends Model {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_DEEPSEEK;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_DEEPSEEK)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_DEEPSEEK_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(modelId);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/densetextembeddings/ElasticInferenceServiceDenseTextEmbeddingsServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/densetextembeddings/ElasticInferenceServiceDenseTextEmbeddingsServiceSettings.java
index 5047f34a1b2e..e8eeee5a34dd 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/densetextembeddings/ElasticInferenceServiceDenseTextEmbeddingsServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/densetextembeddings/ElasticInferenceServiceDenseTextEmbeddingsServiceSettings.java
@@ -205,9 +205,16 @@ public class ElasticInferenceServiceDenseTextEmbeddingsServiceSettings extends F
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_ELASTIC_DENSE_TEXT_EMBEDDINGS_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_ELASTIC_DENSE_TEXT_EMBEDDINGS_ADDED)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_ELASTIC_DENSE_TEXT_EMBEDDINGS_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(modelId);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/rerank/ElasticInferenceServiceRerankServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/rerank/ElasticInferenceServiceRerankServiceSettings.java
index c20846c7fdfc..eff22c277193 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/rerank/ElasticInferenceServiceRerankServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/rerank/ElasticInferenceServiceRerankServiceSettings.java
@@ -83,9 +83,16 @@ public class ElasticInferenceServiceRerankServiceSettings extends FilteredXConte
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_ELASTIC_RERANK;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_ELASTIC_RERANK)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_ELASTIC_RERANK_ADDED_8_19);
+ }
+
@Override
protected XContentBuilder toXContentFragmentOfExposedFields(XContentBuilder builder, Params params) throws IOException {
builder.field(MODEL_ID, modelId);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/completion/GoogleVertexAiChatCompletionServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/completion/GoogleVertexAiChatCompletionServiceSettings.java
index 105d76a9f8cc..a753fc5dc66f 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/completion/GoogleVertexAiChatCompletionServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/completion/GoogleVertexAiChatCompletionServiceSettings.java
@@ -118,9 +118,16 @@ public class GoogleVertexAiChatCompletionServiceSettings extends FilteredXConten
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_VERTEXAI_CHATCOMPLETION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_VERTEXAI_CHATCOMPLETION_ADDED)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_VERTEXAI_CHATCOMPLETION_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(projectId);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/completion/HuggingFaceChatCompletionServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/completion/HuggingFaceChatCompletionServiceSettings.java
index af88316ef516..cdc2529428be 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/completion/HuggingFaceChatCompletionServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/completion/HuggingFaceChatCompletionServiceSettings.java
@@ -144,9 +144,16 @@ public class HuggingFaceChatCompletionServiceSettings extends FilteredXContentOb
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_HUGGING_FACE_CHAT_COMPLETION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_HUGGING_FACE_CHAT_COMPLETION_ADDED)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_HUGGING_FACE_CHAT_COMPLETION_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalString(modelId);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankServiceSettings.java
index 3d4c6aef71e9..b0b21b26395a 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankServiceSettings.java
@@ -115,9 +115,16 @@ public class HuggingFaceRerankServiceSettings extends FilteredXContentObject
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_HUGGING_FACE_RERANK_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_HUGGING_FACE_RERANK_ADDED)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_HUGGING_FACE_RERANK_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(uri.toString());
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankTaskSettings.java
index 9f90386edff9..8b9e9113bce1 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/rerank/HuggingFaceRerankTaskSettings.java
@@ -118,9 +118,16 @@ public class HuggingFaceRerankTaskSettings implements TaskSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_HUGGING_FACE_RERANK_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_HUGGING_FACE_RERANK_ADDED)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_HUGGING_FACE_RERANK_ADDED_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalVInt(topNDocumentsOnly);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/completion/MistralChatCompletionServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/completion/MistralChatCompletionServiceSettings.java
index 676653d54a56..89b9475ad65d 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/completion/MistralChatCompletionServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/completion/MistralChatCompletionServiceSettings.java
@@ -78,9 +78,16 @@ public class MistralChatCompletionServiceSettings extends FilteredXContentObject
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_MISTRAL_CHAT_COMPLETION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_MISTRAL_CHAT_COMPLETION_ADDED)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_MISTRAL_CHAT_COMPLETION_ADDED_8_19);
+ }
+
@Override
public String modelId() {
return this.modelId;
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerServiceSettings.java
index 2caf97bdd05b..b7a554d387c8 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerServiceSettings.java
@@ -111,9 +111,16 @@ record SageMakerServiceSettings(
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(endpointName());
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerTaskSettings.java
index c1c244cc3705..fd9eb2d20c5d 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/model/SageMakerTaskSettings.java
@@ -101,9 +101,16 @@ record SageMakerTaskSettings(
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalString(customAttributes);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredServiceSchema.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredServiceSchema.java
index 9fb320a2d364..b3d948a85de9 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredServiceSchema.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredServiceSchema.java
@@ -29,9 +29,16 @@ public interface SageMakerStoredServiceSchema extends ServiceSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) {
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredTaskSchema.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredTaskSchema.java
index 2aa2f9556d41..09a73f0f42ea 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredTaskSchema.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/SageMakerStoredTaskSchema.java
@@ -39,9 +39,16 @@ public interface SageMakerStoredTaskSchema extends TaskSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) {}
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/ElasticTextEmbeddingPayload.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/ElasticTextEmbeddingPayload.java
index cf9d24a86dcc..6e1407beab1d 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/ElasticTextEmbeddingPayload.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/ElasticTextEmbeddingPayload.java
@@ -250,9 +250,16 @@ public class ElasticTextEmbeddingPayload implements ElasticPayload {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER_ELASTIC;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER_ELASTIC)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_ELASTIC_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalVInt(dimensions);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/SageMakerElasticTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/SageMakerElasticTaskSettings.java
index 3cdcbb35ffdc..088de2068741 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/SageMakerElasticTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/elastic/SageMakerElasticTaskSettings.java
@@ -50,9 +50,16 @@ record SageMakerElasticTaskSettings(@Nullable Map passthroughSet
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER_ELASTIC;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER_ELASTIC)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_ELASTIC_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeGenericMap(passthroughSettings);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/OpenAiTextEmbeddingPayload.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/OpenAiTextEmbeddingPayload.java
index 276c407d694d..6fcbd309551e 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/OpenAiTextEmbeddingPayload.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/OpenAiTextEmbeddingPayload.java
@@ -138,9 +138,16 @@ public class OpenAiTextEmbeddingPayload implements SageMakerSchemaPayload {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalInt(dimensions);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/SageMakerOpenAiTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/SageMakerOpenAiTaskSettings.java
index 4eeba9f69022..b8ce19ba712b 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/SageMakerOpenAiTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/sagemaker/schema/openai/SageMakerOpenAiTaskSettings.java
@@ -37,9 +37,16 @@ record SageMakerOpenAiTaskSettings(@Nullable String user) implements SageMakerSt
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.ML_INFERENCE_SAGEMAKER_CHAT_COMPLETION;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.ML_INFERENCE_SAGEMAKER_CHAT_COMPLETION)
+ || version.isPatchFrom(TransportVersions.ML_INFERENCE_SAGEMAKER_CHAT_COMPLETION_8_19);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalString(user);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/VoyageAIServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/VoyageAIServiceSettings.java
index 75497d1a4b4f..ba7db5bc16f4 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/VoyageAIServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/VoyageAIServiceSettings.java
@@ -108,9 +108,16 @@ public class VoyageAIServiceSettings extends FilteredXContentObject implements S
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.VOYAGE_AI_INTEGRATION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED)
+ || version.isPatchFrom(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED_BACKPORT_8_X);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(modelId);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsServiceSettings.java
index cc4db278d0e2..a0960fb6f74a 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsServiceSettings.java
@@ -226,9 +226,16 @@ public class VoyageAIEmbeddingsServiceSettings extends FilteredXContentObject im
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.VOYAGE_AI_INTEGRATION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED)
+ || version.isPatchFrom(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED_BACKPORT_8_X);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
commonSettings.writeTo(out);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsTaskSettings.java
index 2c6bf3a59c61..11728075fe2b 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/embeddings/VoyageAIEmbeddingsTaskSettings.java
@@ -162,9 +162,16 @@ public class VoyageAIEmbeddingsTaskSettings implements TaskSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.VOYAGE_AI_INTEGRATION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED)
+ || version.isPatchFrom(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED_BACKPORT_8_X);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalEnum(inputType);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankServiceSettings.java
index 1d3607922c5c..4e23efac2701 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankServiceSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankServiceSettings.java
@@ -90,9 +90,16 @@ public class VoyageAIRerankServiceSettings extends FilteredXContentObject implem
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.VOYAGE_AI_INTEGRATION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED)
+ || version.isPatchFrom(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED_BACKPORT_8_X);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
commonSettings.writeTo(out);
diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankTaskSettings.java
index a5004fde1e17..9e57b5848767 100644
--- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankTaskSettings.java
+++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/voyageai/rerank/VoyageAIRerankTaskSettings.java
@@ -135,9 +135,16 @@ public class VoyageAIRerankTaskSettings implements TaskSettings {
@Override
public TransportVersion getMinimalSupportedVersion() {
+ assert false : "should never be called when supportsVersion is used";
return TransportVersions.VOYAGE_AI_INTEGRATION_ADDED;
}
+ @Override
+ public boolean supportsVersion(TransportVersion version) {
+ return version.onOrAfter(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED)
+ || version.isPatchFrom(TransportVersions.VOYAGE_AI_INTEGRATION_ADDED_BACKPORT_8_X);
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalInt(topKDocumentsOnly);
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettingsTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettingsTests.java
index 0ce016956cda..cac416fd454a 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettingsTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/CohereServiceSettingsTests.java
@@ -7,15 +7,17 @@
package org.elasticsearch.xpack.inference.services.cohere;
+import org.elasticsearch.TransportVersion;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.ValidationException;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.inference.SimilarityMeasure;
-import org.elasticsearch.test.AbstractWireSerializingTestCase;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.XContentFactory;
import org.elasticsearch.xcontent.XContentType;
+import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase;
import org.elasticsearch.xpack.inference.services.ConfigurationParseContext;
import org.elasticsearch.xpack.inference.services.ServiceFields;
import org.elasticsearch.xpack.inference.services.ServiceUtils;
@@ -30,7 +32,7 @@ import java.util.Map;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.is;
-public class CohereServiceSettingsTests extends AbstractWireSerializingTestCase {
+public class CohereServiceSettingsTests extends AbstractBWCWireSerializationTestCase {
public static CohereServiceSettings createRandomWithNonNullUrl() {
return createRandom(randomAlphaOfLength(15));
@@ -359,4 +361,22 @@ public class CohereServiceSettingsTests extends AbstractWireSerializingTestCase<
return map;
}
+
+ @Override
+ protected CohereServiceSettings mutateInstanceForVersion(CohereServiceSettings instance, TransportVersion version) {
+ if (version.before(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
+ && (version.isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19) == false)) {
+ return new CohereServiceSettings(
+ instance.uri(),
+ instance.similarity(),
+ instance.dimensions(),
+ instance.maxInputTokens(),
+ instance.modelId(),
+ instance.rateLimitSettings(),
+ CohereServiceSettings.CohereApiVersion.V1
+ );
+ }
+
+ return instance;
+ }
}
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettingsTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettingsTests.java
index 06ebdd158b92..92ebb3fdc0a0 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettingsTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/completion/CohereCompletionServiceSettingsTests.java
@@ -7,12 +7,14 @@
package org.elasticsearch.xpack.inference.services.cohere.completion;
+import org.elasticsearch.TransportVersion;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.Writeable;
-import org.elasticsearch.test.AbstractWireSerializingTestCase;
import org.elasticsearch.xcontent.XContentBuilder;
import org.elasticsearch.xcontent.XContentFactory;
import org.elasticsearch.xcontent.XContentType;
+import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase;
import org.elasticsearch.xpack.inference.services.ConfigurationParseContext;
import org.elasticsearch.xpack.inference.services.ServiceFields;
import org.elasticsearch.xpack.inference.services.cohere.CohereServiceSettings;
@@ -25,7 +27,7 @@ import java.util.Map;
import static org.hamcrest.Matchers.is;
-public class CohereCompletionServiceSettingsTests extends AbstractWireSerializingTestCase {
+public class CohereCompletionServiceSettingsTests extends AbstractBWCWireSerializationTestCase {
public static CohereCompletionServiceSettings createRandom() {
return new CohereCompletionServiceSettings(
@@ -110,4 +112,19 @@ public class CohereCompletionServiceSettingsTests extends AbstractWireSerializin
protected CohereCompletionServiceSettings mutateInstance(CohereCompletionServiceSettings instance) throws IOException {
return randomValueOtherThan(instance, this::createTestInstance);
}
+
+ @Override
+ protected CohereCompletionServiceSettings mutateInstanceForVersion(CohereCompletionServiceSettings instance, TransportVersion version) {
+ if (version.before(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
+ && (version.isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19) == false)) {
+ return new CohereCompletionServiceSettings(
+ instance.uri(),
+ instance.modelId(),
+ instance.rateLimitSettings(),
+ CohereServiceSettings.CohereApiVersion.V1
+ );
+ }
+
+ return instance;
+ }
}
diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettingsTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettingsTests.java
index 27a9fc38f392..773ccc5933aa 100644
--- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettingsTests.java
+++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/cohere/rerank/CohereRerankServiceSettingsTests.java
@@ -88,14 +88,15 @@ public class CohereRerankServiceSettingsTests extends AbstractBWCWireSerializati
CohereServiceSettings.DEFAULT_RATE_LIMIT_SETTINGS,
CohereServiceSettings.CohereApiVersion.V1
);
- } else if (version.before(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)) {
- return new CohereRerankServiceSettings(
- instance.uri(),
- instance.modelId(),
- instance.rateLimitSettings(),
- CohereServiceSettings.CohereApiVersion.V1
- );
- }
+ } else if (version.before(TransportVersions.ML_INFERENCE_COHERE_API_VERSION)
+ && version.isPatchFrom(TransportVersions.ML_INFERENCE_COHERE_API_VERSION_8_19) == false) {
+ return new CohereRerankServiceSettings(
+ instance.uri(),
+ instance.modelId(),
+ instance.rateLimitSettings(),
+ CohereServiceSettings.CohereApiVersion.V1
+ );
+ }
return instance;
}
diff --git a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping.yml b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping.yml
index 5cc0d8368516..637087071b8c 100644
--- a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping.yml
+++ b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping.yml
@@ -833,3 +833,147 @@ setup:
type: int8_flat
- match: { status: 400 }
+
+
+---
+"Displaying default index_options with and without include_defaults":
+ - requires:
+ cluster_features: "semantic_text.index_options_with_defaults"
+ reason: Index options defaults support introduced in 9.2.0
+
+ # Semantic text defaults to BBQ HNSW starting in 8.19.0/9.1.0
+ - do:
+ indices.create:
+ index: test-index-options-dense
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: false
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: dense-inference-id-compatible-with-bbq
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-dense
+
+ - not_exists: test-index-options-dense.mappings.properties.semantic_field.index_options
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-dense
+ fields: semantic_field
+ include_defaults: true
+
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.type": "bbq_hnsw" }
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.m": 16 }
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.ef_construction": 100 }
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.rescore_vector.oversample": 3 }
+
+ # Validate that actually specifying the same values as our defaults will still serialize the user provided index_options
+ - do:
+ indices.create:
+ index: test-index-options-dense2
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: false
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: dense-inference-id-compatible-with-bbq
+ index_options:
+ dense_vector:
+ type: bbq_hnsw
+ m: 16
+ ef_construction: 100
+ rescore_vector:
+ oversample: 3
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-dense2
+
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.type": "bbq_hnsw" }
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.m": 16 }
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.ef_construction": 100 }
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.rescore_vector.oversample": 3 }
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-dense2
+ fields: semantic_field
+ include_defaults: true
+
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.type": "bbq_hnsw" }
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.m": 16 }
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.ef_construction": 100 }
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.rescore_vector.oversample": 3 }
+
+ # Indices not compatible with BBQ for whatever reason will fall back to whatever `dense_vector` defaults are.
+ - do:
+ indices.create:
+ index: test-index-options-dense-no-bbq
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: false
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: dense-inference-id
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-dense-no-bbq
+
+ - not_exists: test-index-options-dense-no-bbq.mappings.properties.semantic_field.index_options
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-dense-no-bbq
+ fields: semantic_field
+ include_defaults: true
+
+ - not_exists: test-index-options-dense-no-bbq.mappings.properties.semantic_field.index_options
+
+ # Sparse embeddings models do not have index options for semantic_text in 8.19/9.1.
+ - do:
+ indices.create:
+ index: test-index-options-sparse
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: false
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: sparse-inference-id
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-sparse
+
+ - not_exists: test-index-options-sparse.mappings.properties.semantic_field.index_options
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-sparse
+ fields: semantic_field
+ include_defaults: true
+
+ - not_exists: test-index-options-sparse.mappings.properties.semantic_field.index_options
+
diff --git a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping_bwc.yml b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping_bwc.yml
index b089d8c43933..1121958b39ed 100644
--- a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping_bwc.yml
+++ b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/10_semantic_text_field_mapping_bwc.yml
@@ -736,3 +736,146 @@ setup:
type: int8_flat
- match: { status: 400 }
+
+---
+"Displaying default index_options with and without include_defaults":
+ - requires:
+ cluster_features: "semantic_text.index_options_with_defaults"
+ reason: Index options defaults support introduced in 9.2.0
+
+ # Semantic text defaults to BBQ HNSW starting in 8.19.0/9.1.0
+ - do:
+ indices.create:
+ index: test-index-options-dense
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: true
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: dense-inference-id-compatible-with-bbq
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-dense
+
+ - not_exists: test-index-options-dense.mappings.properties.semantic_field.index_options
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-dense
+ fields: semantic_field
+ include_defaults: true
+
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.type": "bbq_hnsw" }
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.m": 16 }
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.ef_construction": 100 }
+ - match: { "test-index-options-dense.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.rescore_vector.oversample": 3 }
+
+ # Validate that actually specifying the same values as our defaults will still serialize the user provided index_options
+ - do:
+ indices.create:
+ index: test-index-options-dense2
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: true
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: dense-inference-id-compatible-with-bbq
+ index_options:
+ dense_vector:
+ type: bbq_hnsw
+ m: 16
+ ef_construction: 100
+ rescore_vector:
+ oversample: 3
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-dense2
+
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.type": "bbq_hnsw" }
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.m": 16 }
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.ef_construction": 100 }
+ - match: { "test-index-options-dense2.mappings.properties.semantic_field.index_options.dense_vector.rescore_vector.oversample": 3 }
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-dense2
+ fields: semantic_field
+ include_defaults: true
+
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.type": "bbq_hnsw" }
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.m": 16 }
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.ef_construction": 100 }
+ - match: { "test-index-options-dense2.mappings.semantic_field.mapping.semantic_field.index_options.dense_vector.rescore_vector.oversample": 3 }
+
+ # Indices not compatible with BBQ for whatever reason will fall back to whatever `dense_vector` defaults are.
+ - do:
+ indices.create:
+ index: test-index-options-dense-no-bbq
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: true
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: dense-inference-id
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-dense-no-bbq
+
+ - not_exists: test-index-options-dense-no-bbq.mappings.properties.semantic_field.index_options
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-dense-no-bbq
+ fields: semantic_field
+ include_defaults: true
+
+ - not_exists: test-index-options-dense-no-bbq.mappings.properties.semantic_field.index_options
+
+ # Sparse embeddings models do not have index options for semantic_text in 8.19/9.1.
+ - do:
+ indices.create:
+ index: test-index-options-sparse
+ body:
+ settings:
+ index:
+ mapping:
+ semantic_text:
+ use_legacy_format: true
+ mappings:
+ properties:
+ semantic_field:
+ type: semantic_text
+ inference_id: sparse-inference-id
+
+ - do:
+ indices.get_mapping:
+ index: test-index-options-sparse
+
+ - not_exists: test-index-options-sparse.mappings.properties.semantic_field.index_options
+
+ - do:
+ indices.get_field_mapping:
+ index: test-index-options-sparse
+ fields: semantic_field
+ include_defaults: true
+
+ - not_exists: test-index-options-sparse.mappings.properties.semantic_field.index_options
+
diff --git a/x-pack/plugin/logsdb/build.gradle b/x-pack/plugin/logsdb/build.gradle
index 4496d5843afc..aebb860f9d5c 100644
--- a/x-pack/plugin/logsdb/build.gradle
+++ b/x-pack/plugin/logsdb/build.gradle
@@ -24,12 +24,13 @@ base {
restResources {
restApi {
- include 'bulk', 'search', '_common', 'indices', 'index', 'cluster', 'data_stream', 'ingest', 'cat', 'capabilities', 'esql.query'
+ include 'bulk', 'search', '_common', 'indices', 'index', 'cluster', 'data_stream', 'ingest', 'cat', 'capabilities', 'esql.query', 'field_caps'
}
}
dependencies {
compileOnly project(path: xpackModule('core'))
+ implementation project(':modules:mapper-extras')
testImplementation project(':modules:data-streams')
testImplementation(testArtifact(project(xpackModule('core'))))
javaRestTestImplementation(testArtifact(project(xpackModule('spatial'))))
diff --git a/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java
new file mode 100644
index 000000000000..80a77d76ea16
--- /dev/null
+++ b/x-pack/plugin/logsdb/qa/rolling-upgrade/src/javaRestTest/java/org/elasticsearch/upgrades/MatchOnlyTextRollingUpgradeIT.java
@@ -0,0 +1,252 @@
+/*
+ * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side
+ * Public License v 1"; you may not use this file except in compliance with, at
+ * your election, the "Elastic License 2.0", the "GNU Affero General Public
+ * License v3.0 only", or the "Server Side Public License, v 1".
+ */
+
+package org.elasticsearch.upgrades;
+
+import com.carrotsearch.randomizedtesting.annotations.Name;
+
+import org.elasticsearch.client.Request;
+import org.elasticsearch.client.Response;
+import org.elasticsearch.client.ResponseException;
+import org.elasticsearch.common.network.NetworkAddress;
+import org.elasticsearch.common.time.DateFormatter;
+import org.elasticsearch.common.time.FormatNames;
+import org.elasticsearch.common.xcontent.XContentHelper;
+import org.elasticsearch.test.rest.ObjectPath;
+import org.elasticsearch.xcontent.XContentType;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.time.Instant;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+
+import static org.elasticsearch.upgrades.LogsIndexModeRollingUpgradeIT.enableLogsdbByDefault;
+import static org.elasticsearch.upgrades.LogsIndexModeRollingUpgradeIT.getWriteBackingIndex;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.notNullValue;
+
+public class MatchOnlyTextRollingUpgradeIT extends AbstractRollingUpgradeWithSecurityTestCase {
+
+ static String BULK_ITEM_TEMPLATE =
+ """
+ {"@timestamp": "$now", "host.name": "$host", "method": "$method", "ip": "$ip", "message": "$message", "length": $length, "factor": $factor}
+ """;
+
+ private static final String TEMPLATE = """
+ {
+ "mappings": {
+ "properties": {
+ "@timestamp" : {
+ "type": "date"
+ },
+ "method": {
+ "type": "keyword"
+ },
+ "message": {
+ "type": "match_only_text"
+ },
+ "ip": {
+ "type": "ip"
+ },
+ "length": {
+ "type": "long"
+ },
+ "factor": {
+ "type": "double"
+ }
+ }
+ }
+ }""";
+
+ public MatchOnlyTextRollingUpgradeIT(@Name("upgradedNodes") int upgradedNodes) {
+ super(upgradedNodes);
+ }
+
+ public void testIndexing() throws Exception {
+ String dataStreamName = "logs-bwc-test";
+ if (isOldCluster()) {
+ startTrial();
+ enableLogsdbByDefault();
+ createTemplate(dataStreamName, getClass().getSimpleName().toLowerCase(Locale.ROOT), TEMPLATE);
+
+ Instant startTime = Instant.now().minusSeconds(60 * 60);
+ bulkIndex(dataStreamName, 4, 1024, startTime);
+
+ String firstBackingIndex = getWriteBackingIndex(client(), dataStreamName, 0);
+ var settings = (Map, ?>) getIndexSettingsWithDefaults(firstBackingIndex).get(firstBackingIndex);
+ assertThat(((Map, ?>) settings.get("settings")).get("index.mode"), equalTo("logsdb"));
+ assertThat(((Map, ?>) settings.get("defaults")).get("index.mapping.source.mode"), equalTo("SYNTHETIC"));
+
+ ensureGreen(dataStreamName);
+ search(dataStreamName);
+ query(dataStreamName);
+ } else if (isMixedCluster()) {
+ Instant startTime = Instant.now().minusSeconds(60 * 30);
+ bulkIndex(dataStreamName, 4, 1024, startTime);
+
+ ensureGreen(dataStreamName);
+ search(dataStreamName);
+ query(dataStreamName);
+ } else if (isUpgradedCluster()) {
+ ensureGreen(dataStreamName);
+ Instant startTime = Instant.now();
+ bulkIndex(dataStreamName, 4, 1024, startTime);
+ search(dataStreamName);
+ query(dataStreamName);
+
+ var forceMergeRequest = new Request("POST", "/" + dataStreamName + "/_forcemerge");
+ forceMergeRequest.addParameter("max_num_segments", "1");
+ assertOK(client().performRequest(forceMergeRequest));
+
+ ensureGreen(dataStreamName);
+ search(dataStreamName);
+ query(dataStreamName);
+ }
+ }
+
+ static void createTemplate(String dataStreamName, String id, String template) throws IOException {
+ final String INDEX_TEMPLATE = """
+ {
+ "index_patterns": ["$DATASTREAM"],
+ "template": $TEMPLATE,
+ "data_stream": {
+ }
+ }""";
+ var putIndexTemplateRequest = new Request("POST", "/_index_template/" + id);
+ putIndexTemplateRequest.setJsonEntity(INDEX_TEMPLATE.replace("$TEMPLATE", template).replace("$DATASTREAM", dataStreamName));
+ assertOK(client().performRequest(putIndexTemplateRequest));
+ }
+
+ static String bulkIndex(String dataStreamName, int numRequest, int numDocs, Instant startTime) throws Exception {
+ String firstIndex = null;
+ for (int i = 0; i < numRequest; i++) {
+ var bulkRequest = new Request("POST", "/" + dataStreamName + "/_bulk");
+ StringBuilder requestBody = new StringBuilder();
+ for (int j = 0; j < numDocs; j++) {
+ String hostName = "host" + j % 50; // Not realistic, but makes asserting search / query response easier.
+ String methodName = "method" + j % 5;
+ String ip = NetworkAddress.format(randomIp(true));
+ String param = "chicken" + randomInt(5);
+ String message = "the quick brown fox jumps over the " + param;
+ long length = randomLong();
+ double factor = randomDouble();
+
+ requestBody.append("{\"create\": {}}");
+ requestBody.append('\n');
+ requestBody.append(
+ BULK_ITEM_TEMPLATE.replace("$now", formatInstant(startTime))
+ .replace("$host", hostName)
+ .replace("$method", methodName)
+ .replace("$ip", ip)
+ .replace("$message", message)
+ .replace("$length", Long.toString(length))
+ .replace("$factor", Double.toString(factor))
+ );
+ requestBody.append('\n');
+
+ startTime = startTime.plusMillis(1);
+ }
+ bulkRequest.setJsonEntity(requestBody.toString());
+ bulkRequest.addParameter("refresh", "true");
+ var response = client().performRequest(bulkRequest);
+ assertOK(response);
+ var responseBody = entityAsMap(response);
+ assertThat("errors in response:\n " + responseBody, responseBody.get("errors"), equalTo(false));
+ if (firstIndex == null) {
+ firstIndex = (String) ((Map, ?>) ((Map, ?>) ((List>) responseBody.get("items")).get(0)).get("create")).get("_index");
+ }
+ }
+ return firstIndex;
+ }
+
+ void search(String dataStreamName) throws Exception {
+ var searchRequest = new Request("POST", "/" + dataStreamName + "/_search");
+ searchRequest.addParameter("pretty", "true");
+ searchRequest.setJsonEntity("""
+ {
+ "size": 500,
+ "query": {
+ "match_phrase": {
+ "message": "chicken"
+ }
+ }
+ }
+ """.replace("chicken", "chicken" + randomInt(5)));
+ var response = client().performRequest(searchRequest);
+ assertOK(response);
+ var responseBody = entityAsMap(response);
+ logger.info("{}", responseBody);
+
+ Integer totalCount = ObjectPath.evaluate(responseBody, "hits.total.value");
+ assertThat(totalCount, greaterThanOrEqualTo(512));
+ }
+
+ void query(String dataStreamName) throws Exception {
+ var queryRequest = new Request("POST", "/_query");
+ queryRequest.addParameter("pretty", "true");
+ queryRequest.setJsonEntity("""
+ {
+ "query": "FROM $ds | STATS max(length), max(factor) BY message | SORT message | LIMIT 5"
+ }
+ """.replace("$ds", dataStreamName));
+ var response = client().performRequest(queryRequest);
+ assertOK(response);
+ var responseBody = entityAsMap(response);
+ logger.info("{}", responseBody);
+
+ String column1 = ObjectPath.evaluate(responseBody, "columns.0.name");
+ String column2 = ObjectPath.evaluate(responseBody, "columns.1.name");
+ String column3 = ObjectPath.evaluate(responseBody, "columns.2.name");
+ assertThat(column1, equalTo("max(length)"));
+ assertThat(column2, equalTo("max(factor)"));
+ assertThat(column3, equalTo("message"));
+
+ String key = ObjectPath.evaluate(responseBody, "values.0.2");
+ assertThat(key, equalTo("the quick brown fox jumps over the chicken0"));
+ Long maxRx = ObjectPath.evaluate(responseBody, "values.0.0");
+ assertThat(maxRx, notNullValue());
+ Double maxTx = ObjectPath.evaluate(responseBody, "values.0.1");
+ assertThat(maxTx, notNullValue());
+ }
+
+ protected static void startTrial() throws IOException {
+ Request startTrial = new Request("POST", "/_license/start_trial");
+ startTrial.addParameter("acknowledge", "true");
+ try {
+ assertOK(client().performRequest(startTrial));
+ } catch (ResponseException e) {
+ var responseBody = entityAsMap(e.getResponse());
+ String error = ObjectPath.evaluate(responseBody, "error_message");
+ assertThat(error, containsString("Trial was already activated."));
+ }
+ }
+
+ static Map getIndexSettingsWithDefaults(String index) throws IOException {
+ Request request = new Request("GET", "/" + index + "/_settings");
+ request.addParameter("flat_settings", "true");
+ request.addParameter("include_defaults", "true");
+ Response response = client().performRequest(request);
+ try (InputStream is = response.getEntity().getContent()) {
+ return XContentHelper.convertToMap(
+ XContentType.fromMediaType(response.getEntity().getContentType().getValue()).xContent(),
+ is,
+ true
+ );
+ }
+ }
+
+ static String formatInstant(Instant instant) {
+ return DateFormatter.forPattern(FormatNames.STRICT_DATE_OPTIONAL_TIME.getName()).format(instant);
+ }
+
+}
diff --git a/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/LogsDBPlugin.java b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/LogsDBPlugin.java
index 695406fb9bb3..70236c8e085c 100644
--- a/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/LogsDBPlugin.java
+++ b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/LogsDBPlugin.java
@@ -12,21 +12,27 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettingProvider;
import org.elasticsearch.index.IndexVersion;
+import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.license.LicenseService;
import org.elasticsearch.license.XPackLicenseState;
import org.elasticsearch.plugins.ActionPlugin;
+import org.elasticsearch.plugins.MapperPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.xpack.core.XPackPlugin;
import org.elasticsearch.xpack.core.action.XPackInfoFeatureAction;
import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
+import org.elasticsearch.xpack.logsdb.patternedtext.PatternedTextFieldMapper;
+import org.elasticsearch.xpack.logsdb.patternedtext.PatternedTextFieldType;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
+import java.util.Map;
+import static java.util.Collections.singletonMap;
import static org.elasticsearch.xpack.logsdb.LogsdbLicenseService.FALLBACK_SETTING;
-public class LogsDBPlugin extends Plugin implements ActionPlugin {
+public class LogsDBPlugin extends Plugin implements ActionPlugin, MapperPlugin {
private final Settings settings;
private final LogsdbLicenseService licenseService;
@@ -98,6 +104,15 @@ public class LogsDBPlugin extends Plugin implements ActionPlugin {
return actions;
}
+ @Override
+ public Map getMappers() {
+ if (PatternedTextFieldMapper.PATTERNED_TEXT_MAPPER.isEnabled()) {
+ return singletonMap(PatternedTextFieldType.CONTENT_TYPE, PatternedTextFieldMapper.PARSER);
+ } else {
+ return Map.of();
+ }
+ }
+
protected XPackLicenseState getLicenseState() {
return XPackPlugin.getSharedLicenseState();
}
diff --git a/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/patternedtext/PatternedTextDocValues.java b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/patternedtext/PatternedTextDocValues.java
new file mode 100644
index 000000000000..b7dfdc95683e
--- /dev/null
+++ b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/patternedtext/PatternedTextDocValues.java
@@ -0,0 +1,88 @@
+/*
+ * 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.logsdb.patternedtext;
+
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+
+public class PatternedTextDocValues extends BinaryDocValues {
+ private final SortedSetDocValues templateDocValues;
+ private final SortedSetDocValues argsDocValues;
+
+ PatternedTextDocValues(SortedSetDocValues templateDocValues, SortedSetDocValues argsDocValues) {
+ this.templateDocValues = templateDocValues;
+ this.argsDocValues = argsDocValues;
+ }
+
+ static PatternedTextDocValues from(LeafReader leafReader, String templateFieldName, String argsFieldName) throws IOException {
+ SortedSetDocValues templateDocValues = DocValues.getSortedSet(leafReader, templateFieldName);
+ if (templateDocValues.getValueCount() == 0) {
+ return null;
+ }
+
+ SortedSetDocValues argsDocValues = DocValues.getSortedSet(leafReader, argsFieldName);
+ return new PatternedTextDocValues(templateDocValues, argsDocValues);
+ }
+
+ private String getNextStringValue() throws IOException {
+ assert templateDocValues.docValueCount() == 1;
+ String template = templateDocValues.lookupOrd(templateDocValues.nextOrd()).utf8ToString();
+ int argsCount = PatternedTextValueProcessor.countArgs(template);
+ if (argsCount > 0) {
+ assert argsDocValues.docValueCount() == 1;
+ var mergedArgs = argsDocValues.lookupOrd(argsDocValues.nextOrd());
+ var args = PatternedTextValueProcessor.decodeRemainingArgs(mergedArgs.utf8ToString());
+ return PatternedTextValueProcessor.merge(new PatternedTextValueProcessor.Parts(template, args));
+ } else {
+ return template;
+ }
+ }
+
+ @Override
+ public BytesRef binaryValue() throws IOException {
+ return new BytesRef(getNextStringValue());
+ }
+
+ @Override
+ public boolean advanceExact(int i) throws IOException {
+ argsDocValues.advanceExact(i);
+ // If template has a value, then message has a value. We don't have to check args here, since there may not be args for the doc
+ return templateDocValues.advanceExact(i);
+ }
+
+ @Override
+ public int docID() {
+ return templateDocValues.docID();
+ }
+
+ @Override
+ public int nextDoc() throws IOException {
+ int templateNext = templateDocValues.nextDoc();
+ var argsAdvance = argsDocValues.advance(templateNext);
+ assert argsAdvance >= templateNext;
+ return templateNext;
+ }
+
+ @Override
+ public int advance(int i) throws IOException {
+ int templateAdvance = templateDocValues.advance(i);
+ var argsAdvance = argsDocValues.advance(templateAdvance);
+ assert argsAdvance >= templateAdvance;
+ return templateAdvance;
+ }
+
+ @Override
+ public long cost() {
+ return templateDocValues.cost() + argsDocValues.cost();
+ }
+}
diff --git a/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/patternedtext/PatternedTextFieldMapper.java b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/patternedtext/PatternedTextFieldMapper.java
new file mode 100644
index 000000000000..55f5616f4ac7
--- /dev/null
+++ b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/patternedtext/PatternedTextFieldMapper.java
@@ -0,0 +1,176 @@
+/*
+ * 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.logsdb.patternedtext;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.util.FeatureFlag;
+import org.elasticsearch.index.IndexVersion;
+import org.elasticsearch.index.analysis.IndexAnalyzers;
+import org.elasticsearch.index.analysis.NamedAnalyzer;
+import org.elasticsearch.index.mapper.CompositeSyntheticFieldLoader;
+import org.elasticsearch.index.mapper.DocumentParserContext;
+import org.elasticsearch.index.mapper.FieldMapper;
+import org.elasticsearch.index.mapper.MapperBuilderContext;
+import org.elasticsearch.index.mapper.TextParams;
+import org.elasticsearch.index.mapper.TextSearchInfo;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * A {@link FieldMapper} that assigns every document the same value.
+ */
+public class PatternedTextFieldMapper extends FieldMapper {
+
+ public static final FeatureFlag PATTERNED_TEXT_MAPPER = new FeatureFlag("patterned_text");
+
+ public static class Defaults {
+ public static final FieldType FIELD_TYPE;
+
+ static {
+ final FieldType ft = new FieldType();
+ ft.setTokenized(true);
+ ft.setStored(false);
+ ft.setStoreTermVectors(false);
+ ft.setOmitNorms(true);
+ ft.setIndexOptions(IndexOptions.DOCS);
+ FIELD_TYPE = freezeAndDeduplicateFieldType(ft);
+ }
+ }
+
+ public static class Builder extends FieldMapper.Builder {
+
+ private final IndexVersion indexCreatedVersion;
+
+ private final Parameter