From b09553aa15aaa7f80c13c0df79d6b20f89faed1c Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Fri, 17 Jan 2025 12:42:39 +0100 Subject: [PATCH 01/24] Add a pre-mapping logical plan processing step This adds a processing step that runs rules which can modify an optimized LogicalPlan by executing async calls to the services avaialble to ESQL's transport query action. The rules need to implement a new interface (`MappingPreProcessor`). The resulting plan won't be further optimized logically, but mapped and further processed as a physical plan. --- .../xpack/esql/EsqlTestUtils.java | 17 ++- .../xpack/esql/MockQueryBuilderResolver.java | 30 ---- .../xpack/esql/execution/PlanExecutor.java | 6 +- .../function/fulltext/FullTextFunction.java | 8 +- .../FullTextFunctionMapperPreprocessor.java | 140 ++++++++++++++++++ .../MapperPreprocessorExecutor.java | 59 ++++++++ .../preprocessor/MappingPreProcessor.java | 30 ++++ .../esql/plugin/TransportActionServices.java | 24 +++ .../esql/plugin/TransportEsqlQueryAction.java | 15 +- .../xpack/esql/session/EsqlSession.java | 42 +++--- .../elasticsearch/xpack/esql/CsvTests.java | 2 +- .../esql/stats/PlanExecutorMetricsTests.java | 4 +- 12 files changed, 311 insertions(+), 66 deletions(-) delete mode 100644 x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/MockQueryBuilderResolver.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportActionServices.java diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java index 7e25fb29fdb78..011072942f81b 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -11,6 +11,8 @@ import org.apache.lucene.sandbox.document.HalfFloatPoint; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker; @@ -31,9 +33,12 @@ import org.elasticsearch.geo.ShapeTestUtils; import org.elasticsearch.index.IndexMode; import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.search.SearchService; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.RemoteTransportException; +import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.json.JsonXContent; import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.analysis.EnrichResolution; @@ -71,8 +76,8 @@ import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier; import org.elasticsearch.xpack.esql.plugin.EsqlPlugin; import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.session.Configuration; -import org.elasticsearch.xpack.esql.session.QueryBuilderResolver; import org.elasticsearch.xpack.esql.stats.Metrics; import org.elasticsearch.xpack.esql.stats.SearchStats; import org.elasticsearch.xpack.versionfield.Version; @@ -138,6 +143,7 @@ import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.mockito.Mockito.mock; public final class EsqlTestUtils { @@ -358,7 +364,14 @@ public static LogicalOptimizerContext unboundLogicalOptimizerContext() { public static final Verifier TEST_VERIFIER = new Verifier(new Metrics(new EsqlFunctionRegistry()), new XPackLicenseState(() -> 0L)); - public static final QueryBuilderResolver MOCK_QUERY_BUILDER_RESOLVER = new MockQueryBuilderResolver(); + public static final TransportActionServices MOCK_TRANSPORT_ACTION_SERVICES = new TransportActionServices( + mock(TransportService.class), + mock(SearchService.class), + null, + mock(ClusterService.class), + mock(IndexNameExpressionResolver.class), + null + ); private EsqlTestUtils() {} diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/MockQueryBuilderResolver.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/MockQueryBuilderResolver.java deleted file mode 100644 index 7af3a89108fc0..0000000000000 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/MockQueryBuilderResolver.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.session.QueryBuilderResolver; -import org.elasticsearch.xpack.esql.session.Result; - -import java.util.function.BiConsumer; - -public class MockQueryBuilderResolver extends QueryBuilderResolver { - public MockQueryBuilderResolver() { - super(null, null, null, null); - } - - @Override - public void resolveQueryBuilders( - LogicalPlan plan, - ActionListener listener, - BiConsumer> callback - ) { - callback.accept(plan, listener); - } -} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java index 94913581f696d..c9aea94e1d36c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/execution/PlanExecutor.java @@ -21,10 +21,10 @@ import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext; import org.elasticsearch.xpack.esql.optimizer.LogicalPlanOptimizer; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.session.Configuration; import org.elasticsearch.xpack.esql.session.EsqlSession; import org.elasticsearch.xpack.esql.session.IndexResolver; -import org.elasticsearch.xpack.esql.session.QueryBuilderResolver; import org.elasticsearch.xpack.esql.session.Result; import org.elasticsearch.xpack.esql.stats.Metrics; import org.elasticsearch.xpack.esql.stats.PlanningMetrics; @@ -62,7 +62,7 @@ public void esql( EsqlExecutionInfo executionInfo, IndicesExpressionGrouper indicesExpressionGrouper, EsqlSession.PlanRunner planRunner, - QueryBuilderResolver queryBuilderResolver, + TransportActionServices services, ActionListener listener ) { final PlanningMetrics planningMetrics = new PlanningMetrics(); @@ -78,7 +78,7 @@ public void esql( verifier, planningMetrics, indicesExpressionGrouper, - queryBuilderResolver + services ); QueryMetric clientId = QueryMetric.fromString("rest"); metrics.total(clientId); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 4da7c01139c24..a8e8420ca6bc6 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -7,7 +7,7 @@ package org.elasticsearch.xpack.esql.expression.function.fulltext; -import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.xpack.esql.capabilities.PostAnalysisPlanVerificationAware; import org.elasticsearch.xpack.esql.common.Failures; @@ -113,11 +113,7 @@ public Expression query() { */ public Object queryAsObject() { Object queryAsObject = query().fold(FoldContext.small() /* TODO remove me */); - if (queryAsObject instanceof BytesRef bytesRef) { - return bytesRef.utf8ToString(); - } - - return queryAsObject; + return BytesRefs.toString(queryAsObject); } /** diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java new file mode 100644 index 0000000000000..1d44ce7ab71af --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java @@ -0,0 +1,140 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ResolvedIndices; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryRewriteContext; +import org.elasticsearch.index.query.Rewriteable; +import org.elasticsearch.xpack.esql.expression.function.fulltext.FullTextFunction; +import org.elasticsearch.xpack.esql.plan.logical.EsRelation; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.PlannerUtils; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; +import org.elasticsearch.xpack.esql.session.IndexResolver; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} + * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. + * {@link FullTextFunctionMapperPreprocessor#preprocess(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by + * replacing {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. + */ +public class FullTextFunctionMapperPreprocessor implements MappingPreProcessor { + + @Override + public void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener) { + Set unresolved = fullTextFunctions(plan); + Set indexNames = indexNames(plan); + + if (indexNames == null || indexNames.isEmpty() || unresolved.isEmpty()) { + listener.onResponse(plan); + return; + } + QueryRewriteContext ctx = queryRewriteContext(services, indexNames); + FullTextFunctionsRewritable rewritable = new FullTextFunctionsRewritable(unresolved); + Rewriteable.rewriteAndFetch(rewritable, ctx, new ActionListener<>() { + @Override + public void onResponse(FullTextFunctionsRewritable fullTextFunctionsRewritable) { + try { + LogicalPlan newPlan = planWithResolvedQueryBuilders(plan, fullTextFunctionsRewritable.results()); + listener.onResponse(newPlan); + } catch (Exception e) { + onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + } + + public LogicalPlan planWithResolvedQueryBuilders(LogicalPlan plan, Map newQueryBuilders) { + LogicalPlan newPlan = plan.transformExpressionsDown(FullTextFunction.class, m -> { + if (newQueryBuilders.containsKey(m)) { + return m.replaceQueryBuilder(newQueryBuilders.get(m)); + } + return m; + }); + // The given plan was already analyzed and optimized, so we set the resulted plan to optimized as well. + newPlan.setOptimized(); + return newPlan; + } + + private static QueryRewriteContext queryRewriteContext(TransportActionServices services, Set indexNames) { + ResolvedIndices resolvedIndices = ResolvedIndices.resolveWithIndexNamesAndOptions( + indexNames.toArray(String[]::new), + IndexResolver.FIELD_CAPS_INDICES_OPTIONS, + services.clusterService().state(), + services.indexNameExpressionResolver(), + services.transportService().getRemoteClusterService(), + System.currentTimeMillis() + ); + + return services.searchService().getRewriteContext(System::currentTimeMillis, resolvedIndices, null); + } + + private static Set fullTextFunctions(LogicalPlan plan) { + Set functions = new HashSet<>(); + plan.forEachExpressionDown(FullTextFunction.class, functions::add); + return functions; + } + + public Set indexNames(LogicalPlan plan) { + Set indexNames = new HashSet<>(); + plan.forEachDown(EsRelation.class, esRelation -> indexNames.addAll(esRelation.index().concreteIndices())); + return indexNames; + } + + private static class FullTextFunctionsRewritable + implements + Rewriteable { + + private final Map queryBuilderMap; + + FullTextFunctionsRewritable(Map queryBuilderMap) { + this.queryBuilderMap = queryBuilderMap; + } + + FullTextFunctionsRewritable(Set functions) { + this.queryBuilderMap = new HashMap<>(); + + for (FullTextFunction func : functions) { + queryBuilderMap.put(func, func.asQuery(PlannerUtils.TRANSLATOR_HANDLER).asBuilder()); + } + } + + @Override + public FullTextFunctionMapperPreprocessor.FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOException { + Map results = new HashMap<>(); + + boolean hasChanged = false; + for (var entry : queryBuilderMap.entrySet()) { + var initial = entry.getValue(); + var rewritten = initial.rewrite(ctx); + hasChanged |= rewritten != initial; + + results.put(entry.getKey(), rewritten); + } + + return hasChanged ? new FullTextFunctionMapperPreprocessor.FullTextFunctionsRewritable(results) : this; + } + + public Map results() { + return queryBuilderMap; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java new file mode 100644 index 0000000000000..8d3a889e4bd07 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java @@ -0,0 +1,59 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +public class MapperPreprocessorExecutor { + + private final TransportActionServices services; + private final List proprocessors = new ArrayList<>(); + + public MapperPreprocessorExecutor(TransportActionServices services) { + this.services = services; + } + + public MapperPreprocessorExecutor addPreprocessor(MappingPreProcessor preProcessor) { + proprocessors.add(preProcessor); + return this; + } + + public MapperPreprocessorExecutor addPreprocessors(Collection preProcessors) { + proprocessors.addAll(preProcessors); + return this; + } + + public void execute(LogicalPlan plan, ActionListener listener) { + execute(plan, 0, listener); + } + + private void execute(LogicalPlan plan, int index, ActionListener listener) { + if (index == proprocessors.size()) { + listener.onResponse(plan); + return; + } + + proprocessors.get(index).preprocess(plan, services, new ActionListener<>() { + @Override + public void onResponse(LogicalPlan p) { + execute(p, index + 1, listener); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java new file mode 100644 index 0000000000000..cc918a836ec81 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java @@ -0,0 +1,30 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; + +/** + * Interface for a LogicalPlan processing rule occurring after the optimization, but before mapping to a physical plan. + * This step occurs on the coordinator. The rule may use services provided to the transport action and thus can resolve indices, rewrite + * queries, perform substitutions, etc. + * Note that the LogicalPlan following the rules' changes will not undergo another logical optimization round. The changes these rules + * should apply are only those that require access to services that need to be performed asynchronously. + */ +public interface MappingPreProcessor { + + /** + * Process a logical plan making use of the available services and provide the updated plan to the provided listener. + * @param plan the logical plan to process + * @param services the services available from the transport action + * @param listener the listener to notify when processing is complete + */ + void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener); +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportActionServices.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportActionServices.java new file mode 100644 index 0000000000000..ad112542e000a --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportActionServices.java @@ -0,0 +1,24 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plugin; + +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.compute.operator.exchange.ExchangeService; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.usage.UsageService; + +public record TransportActionServices( + TransportService transportService, + SearchService searchService, + ExchangeService exchangeService, + ClusterService clusterService, + IndexNameExpressionResolver indexNameExpressionResolver, + UsageService usageService +) {} 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 84173eeecc060..3d6dc0dc83e3e 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 @@ -50,7 +50,6 @@ import org.elasticsearch.xpack.esql.execution.PlanExecutor; import org.elasticsearch.xpack.esql.session.Configuration; import org.elasticsearch.xpack.esql.session.EsqlSession.PlanRunner; -import org.elasticsearch.xpack.esql.session.QueryBuilderResolver; import org.elasticsearch.xpack.esql.session.Result; import java.io.IOException; @@ -78,8 +77,8 @@ public class TransportEsqlQueryAction extends HandledTransportAction asyncTaskManagementService; private final RemoteClusterService remoteClusterService; - private final QueryBuilderResolver queryBuilderResolver; private final UsageService usageService; + private final TransportActionServices services; @Inject @SuppressWarnings("this-escape") @@ -134,8 +133,16 @@ public TransportEsqlQueryAction( bigArrays ); this.remoteClusterService = transportService.getRemoteClusterService(); - this.queryBuilderResolver = new QueryBuilderResolver(searchService, clusterService, transportService, indexNameExpressionResolver); this.usageService = usageService; + + this.services = new TransportActionServices( + transportService, + searchService, + exchangeService, + clusterService, + indexNameExpressionResolver, + usageService + ); } @Override @@ -209,7 +216,7 @@ private void innerExecute(Task task, EsqlQueryRequest request, ActionListener { recordCCSTelemetry(task, executionInfo, request, null); listener.onResponse(toResponse(task, request, configuration, result)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index b10f766babb36..f61df90dbc5b0 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -73,6 +73,9 @@ import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; +import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.FullTextFunctionMapperPreprocessor; +import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MapperPreprocessorExecutor; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.stats.PlanningMetrics; import java.util.ArrayList; @@ -114,7 +117,7 @@ public interface PlanRunner { private final PhysicalPlanOptimizer physicalPlanOptimizer; private final PlanningMetrics planningMetrics; private final IndicesExpressionGrouper indicesExpressionGrouper; - private final QueryBuilderResolver queryBuilderResolver; + private final MapperPreprocessorExecutor mapperPreprocessorExecutor; public EsqlSession( String sessionId, @@ -128,7 +131,7 @@ public EsqlSession( Verifier verifier, PlanningMetrics planningMetrics, IndicesExpressionGrouper indicesExpressionGrouper, - QueryBuilderResolver queryBuilderResolver + TransportActionServices services ) { this.sessionId = sessionId; this.configuration = configuration; @@ -142,7 +145,9 @@ public EsqlSession( this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); this.planningMetrics = planningMetrics; this.indicesExpressionGrouper = indicesExpressionGrouper; - this.queryBuilderResolver = queryBuilderResolver; + this.mapperPreprocessorExecutor = new MapperPreprocessorExecutor(services); + + mapperPreprocessorExecutor.addPreprocessor(new FullTextFunctionMapperPreprocessor()); } public String sessionId() { @@ -162,16 +167,7 @@ public void execute(EsqlQueryRequest request, EsqlExecutionInfo executionInfo, P new EsqlSessionCCSUtils.CssPartialErrorsActionListener(executionInfo, listener) { @Override public void onResponse(LogicalPlan analyzedPlan) { - try { - var optimizedPlan = optimizedPlan(analyzedPlan); - queryBuilderResolver.resolveQueryBuilders( - optimizedPlan, - listener, - (newPlan, next) -> executeOptimizedPlan(request, executionInfo, planRunner, newPlan, next) - ); - } catch (Exception e) { - listener.onFailure(e); - } + executeOptimizedPlan(request, executionInfo, planRunner, optimizedPlan(analyzedPlan), listener); } } ); @@ -188,11 +184,21 @@ public void executeOptimizedPlan( LogicalPlan optimizedPlan, ActionListener listener ) { - PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(optimizedPlan, request); - // TODO: this could be snuck into the underlying listener - EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); - // execute any potential subplans - executeSubPlans(physicalPlan, planRunner, executionInfo, request, listener); + mapperPreprocessorExecutor.execute(optimizedPlan, new ActionListener<>() { + @Override + public void onResponse(LogicalPlan preprocessedPlan) { + PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(preprocessedPlan, request); + // TODO: this could be snuck into the underlying listener + EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); + // execute any potential subplans + executeSubPlans(physicalPlan, planRunner, executionInfo, request, listener); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); } private record PlanTuple(PhysicalPlan physical, LogicalPlan logical) {} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index ed1ee71ff1968..8afe8bb00ba6b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -516,7 +516,7 @@ private ActualResults executePlan(BigArrays bigArrays) throws Exception { TEST_VERIFIER, new PlanningMetrics(), null, - EsqlTestUtils.MOCK_QUERY_BUILDER_RESOLVER + EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES ); TestPhysicalOperationProviders physicalOperationProviders = testOperationProviders(foldCtx, testDatasets); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java index a3c5cd9168b4f..12db9ad96e2d9 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/stats/PlanExecutorMetricsTests.java @@ -125,7 +125,7 @@ public void testFailedMetric() { new EsqlExecutionInfo(randomBoolean()), groupIndicesByCluster, runPhase, - EsqlTestUtils.MOCK_QUERY_BUILDER_RESOLVER, + EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES, new ActionListener<>() { @Override public void onResponse(Result result) { @@ -156,7 +156,7 @@ public void onFailure(Exception e) { new EsqlExecutionInfo(randomBoolean()), groupIndicesByCluster, runPhase, - EsqlTestUtils.MOCK_QUERY_BUILDER_RESOLVER, + EsqlTestUtils.MOCK_TRANSPORT_ACTION_SERVICES, new ActionListener<>() { @Override public void onResponse(Result result) {} From cbddbc6b5e7701f992435ddebaef46b8c751e332 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Fri, 17 Jan 2025 12:49:19 +0100 Subject: [PATCH 02/24] Update docs/changelog/120368.yaml --- docs/changelog/120368.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/120368.yaml diff --git a/docs/changelog/120368.yaml b/docs/changelog/120368.yaml new file mode 100644 index 0000000000000..f51b42e3f241a --- /dev/null +++ b/docs/changelog/120368.yaml @@ -0,0 +1,5 @@ +pr: 120368 +summary: Add a pre-mapping logical plan processing step +area: ES|QL +type: enhancement +issues: [] From baec4eec67c0b1e699ea1a65d54144e764f2ba0f Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 17 Jan 2025 11:55:13 +0000 Subject: [PATCH 03/24] [CI] Auto commit changes from spotless --- .../main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java index 011072942f81b..49d5c397d06fc 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -36,7 +36,6 @@ import org.elasticsearch.search.SearchService; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.json.JsonXContent; From b17f6e60aafa450005ca234c5d6e7b26e5e97664 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Mon, 20 Jan 2025 14:08:56 +0100 Subject: [PATCH 04/24] Minor listerns refactoring --- .../FullTextFunctionMapperPreprocessor.java | 23 ++++----------- .../MapperPreprocessorExecutor.java | 15 ++-------- .../xpack/esql/session/EsqlSession.java | 28 +++++++------------ 3 files changed, 18 insertions(+), 48 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java index 1d44ce7ab71af..9dc1c1d0f154d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java @@ -44,25 +44,14 @@ public void preprocess(LogicalPlan plan, TransportActionServices services, Actio } QueryRewriteContext ctx = queryRewriteContext(services, indexNames); FullTextFunctionsRewritable rewritable = new FullTextFunctionsRewritable(unresolved); - Rewriteable.rewriteAndFetch(rewritable, ctx, new ActionListener<>() { - @Override - public void onResponse(FullTextFunctionsRewritable fullTextFunctionsRewritable) { - try { - LogicalPlan newPlan = planWithResolvedQueryBuilders(plan, fullTextFunctionsRewritable.results()); - listener.onResponse(newPlan); - } catch (Exception e) { - onFailure(e); - } - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); + Rewriteable.rewriteAndFetch( + rewritable, + ctx, + listener.delegateFailureAndWrap((l, r) -> l.onResponse(updateQueryBuilders(plan, r.results()))) + ); } - public LogicalPlan planWithResolvedQueryBuilders(LogicalPlan plan, Map newQueryBuilders) { + public LogicalPlan updateQueryBuilders(LogicalPlan plan, Map newQueryBuilders) { LogicalPlan newPlan = plan.transformExpressionsDown(FullTextFunction.class, m -> { if (newQueryBuilders.containsKey(m)) { return m.replaceQueryBuilder(newQueryBuilders.get(m)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java index 8d3a889e4bd07..b7a942373e06b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java @@ -41,19 +41,8 @@ public void execute(LogicalPlan plan, ActionListener listener) { private void execute(LogicalPlan plan, int index, ActionListener listener) { if (index == proprocessors.size()) { listener.onResponse(plan); - return; + } else { + proprocessors.get(index).preprocess(plan, services, listener.delegateFailureAndWrap((l, p) -> execute(p, index + 1, l))); } - - proprocessors.get(index).preprocess(plan, services, new ActionListener<>() { - @Override - public void onResponse(LogicalPlan p) { - execute(p, index + 1, listener); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index f61df90dbc5b0..d652c43b98eb8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -145,9 +145,9 @@ public EsqlSession( this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); this.planningMetrics = planningMetrics; this.indicesExpressionGrouper = indicesExpressionGrouper; - this.mapperPreprocessorExecutor = new MapperPreprocessorExecutor(services); - - mapperPreprocessorExecutor.addPreprocessor(new FullTextFunctionMapperPreprocessor()); + this.mapperPreprocessorExecutor = new MapperPreprocessorExecutor(services).addPreprocessor( + new FullTextFunctionMapperPreprocessor() + ); } public String sessionId() { @@ -184,21 +184,13 @@ public void executeOptimizedPlan( LogicalPlan optimizedPlan, ActionListener listener ) { - mapperPreprocessorExecutor.execute(optimizedPlan, new ActionListener<>() { - @Override - public void onResponse(LogicalPlan preprocessedPlan) { - PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(preprocessedPlan, request); - // TODO: this could be snuck into the underlying listener - EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); - // execute any potential subplans - executeSubPlans(physicalPlan, planRunner, executionInfo, request, listener); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); + mapperPreprocessorExecutor.execute(optimizedPlan, listener.delegateFailureAndWrap((l, p) -> { + PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(p, request); + // TODO: this could be snuck into the underlying listener + EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); + // execute any potential subplans + executeSubPlans(physicalPlan, planRunner, executionInfo, request, l); + })); } private record PlanTuple(PhysicalPlan physical, LogicalPlan logical) {} From 8fbd0fcc54f788919acbc4b6c6cabecf3d6e39dd Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Mon, 20 Jan 2025 14:28:00 +0100 Subject: [PATCH 05/24] Fix wrong auto-merge --- .../preprocessor/FullTextFunctionMapperPreprocessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java index 9dc1c1d0f154d..a4a93f39981c9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java @@ -15,7 +15,7 @@ import org.elasticsearch.xpack.esql.expression.function.fulltext.FullTextFunction; import org.elasticsearch.xpack.esql.plan.logical.EsRelation; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.planner.PlannerUtils; +import org.elasticsearch.xpack.esql.planner.TranslatorHandler; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.session.IndexResolver; @@ -102,7 +102,7 @@ private static class FullTextFunctionsRewritable this.queryBuilderMap = new HashMap<>(); for (FullTextFunction func : functions) { - queryBuilderMap.put(func, func.asQuery(PlannerUtils.TRANSLATOR_HANDLER).asBuilder()); + queryBuilderMap.put(func, func.asQuery(TranslatorHandler.TRANSLATOR_HANDLER).asBuilder()); } } From 62a586276a16cbd4bbb89cefda50b0841d980339 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Mon, 27 Jan 2025 17:53:59 +0100 Subject: [PATCH 06/24] Have the FTFMapperPreprocessor update the plan in-place --- .../xpack/esql/core/util/Holder.java | 13 ++ .../esql/capabilities/TranslationAware.java | 5 + .../function/fulltext/FullTextFunction.java | 32 ++-- .../FullTextFunctionMapperPreprocessor.java | 90 ++++++++++ .../FullTextFunctionMapperPreprocessor.java | 129 ------------- .../MapperPreprocessorExecutor.java | 33 ++-- .../xpack/esql/session/EsqlSession.java | 32 ++-- .../esql/session/QueryBuilderResolver.java | 169 ------------------ .../queries/SemanticQueryBuilder.java | 5 +- 9 files changed, 159 insertions(+), 349 deletions(-) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java delete mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java delete mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java index 1290bbca59ee7..a6f5aaae1731f 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java @@ -26,6 +26,19 @@ public void set(T value) { this.value = value; } + /** + * Sets a value in the holder, but only if none has already been set. + * @param value the new value to set. + * @return the previously held value, if any was set. + */ + public T trySet(T value) { + T old = this.value; + if (old == null) { + this.value = value; + } + return old; + } + public T get() { return value; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java index 8ef528b6668ab..5228bfe5170ed 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java @@ -11,6 +11,7 @@ import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; +import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MappingPreProcessor; /** * Expressions implementing this interface can get called on data nodes to provide an Elasticsearch/Lucene query. @@ -42,4 +43,8 @@ interface SingleValueTranslationAware extends TranslationAware { */ Expression singleValueField(); } + + interface QueryRewriter extends TranslationAware { + MappingPreProcessor queryRewriter(); + } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index ed59191e092b6..2d690550d4bf1 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -32,6 +32,7 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.OrderBy; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; +import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MappingPreProcessor; import org.elasticsearch.xpack.esql.querydsl.query.TranslationAwareExpressionQuery; import java.util.List; @@ -50,7 +51,11 @@ * These functions needs to be pushed down to Lucene queries to be executed - there's no Evaluator for them, but depend on * {@link org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer} to rewrite them into Lucene queries. */ -public abstract class FullTextFunction extends Function implements TranslationAware, PostAnalysisPlanVerificationAware { +public abstract class FullTextFunction extends Function + implements + TranslationAware, + TranslationAware.QueryRewriter, + PostAnalysisPlanVerificationAware { private final Expression query; private final QueryBuilder queryBuilder; @@ -116,6 +121,11 @@ public Object queryAsObject() { return BytesRefs.toString(queryAsObject); } + @Override + public MappingPreProcessor queryRewriter() { + return FullTextFunctionMapperPreprocessor.INSTANCE; + } + /** * Returns the param ordinal for the query parameter so it can be used in error messages * @@ -279,26 +289,6 @@ private static boolean onlyFullTextFunctionsInExpression(Expression expression) return false; } - /** - * Checks whether an expression contains a full text function as part of it - * - * @param expression expression to check - * @return true if the expression or any of its children is a full text function, false otherwise - */ - private static boolean anyFullTextFunctionsInExpression(Expression expression) { - if (expression instanceof FullTextFunction) { - return true; - } - - for (Expression child : expression.children()) { - if (anyFullTextFunctionsInExpression(child)) { - return true; - } - } - - return false; - } - /** * Checks all commands that exist before a specific type satisfy conditions. * diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java new file mode 100644 index 0000000000000..9c1f6cb33aa19 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java @@ -0,0 +1,90 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.expression.function.fulltext; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ResolvedIndices; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryRewriteContext; +import org.elasticsearch.index.query.Rewriteable; +import org.elasticsearch.xpack.esql.core.util.Holder; +import org.elasticsearch.xpack.esql.plan.logical.EsRelation; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.planner.TranslatorHandler; +import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MappingPreProcessor; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; +import org.elasticsearch.xpack.esql.session.IndexResolver; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; + +/** + * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} + * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. + * {@link FullTextFunctionMapperPreprocessor#preprocess(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by + * replacing {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. + */ +public final class FullTextFunctionMapperPreprocessor implements MappingPreProcessor { + + public static final FullTextFunctionMapperPreprocessor INSTANCE = new FullTextFunctionMapperPreprocessor(); + + @Override + public void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener) { + Rewriteable.rewriteAndFetch( + new FullTextFunctionsRewritable(plan), + queryRewriteContext(services, indexNames(plan)), + listener.delegateFailureAndWrap((l, r) -> l.onResponse(r.plan)) + ); + } + + private static QueryRewriteContext queryRewriteContext(TransportActionServices services, Set indexNames) { + ResolvedIndices resolvedIndices = ResolvedIndices.resolveWithIndexNamesAndOptions( + indexNames.toArray(String[]::new), + IndexResolver.FIELD_CAPS_INDICES_OPTIONS, + services.clusterService().state(), + services.indexNameExpressionResolver(), + services.transportService().getRemoteClusterService(), + System.currentTimeMillis() + ); + + return services.searchService().getRewriteContext(System::currentTimeMillis, resolvedIndices, null); + } + + public Set indexNames(LogicalPlan plan) { + Set indexNames = new HashSet<>(); + plan.forEachDown(EsRelation.class, esRelation -> indexNames.addAll(esRelation.index().concreteIndices())); + return indexNames; + } + + private record FullTextFunctionsRewritable(LogicalPlan plan) + implements + Rewriteable { + @Override + public FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOException { + Holder exceptionHolder = new Holder<>(); + Holder updated = new Holder<>(false); + LogicalPlan newPlan = plan.transformExpressionsDown(FullTextFunction.class, f -> { + QueryBuilder builder = f.queryBuilder(), initial = builder; + builder = builder == null ? f.asQuery(TranslatorHandler.TRANSLATOR_HANDLER).asBuilder() : builder; + try { + builder = builder.rewrite(ctx); + } catch (IOException e) { + exceptionHolder.trySet(e); + } + var rewrite = builder != initial; + updated.set(updated.get() || rewrite); + return rewrite ? f.replaceQueryBuilder(builder) : f; + }); + if (exceptionHolder.get() != null) { + throw exceptionHolder.get(); + } + return updated.get() ? new FullTextFunctionsRewritable(newPlan) : this; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java deleted file mode 100644 index a4a93f39981c9..0000000000000 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/FullTextFunctionMapperPreprocessor.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ResolvedIndices; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryRewriteContext; -import org.elasticsearch.index.query.Rewriteable; -import org.elasticsearch.xpack.esql.expression.function.fulltext.FullTextFunction; -import org.elasticsearch.xpack.esql.plan.logical.EsRelation; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.planner.TranslatorHandler; -import org.elasticsearch.xpack.esql.plugin.TransportActionServices; -import org.elasticsearch.xpack.esql.session.IndexResolver; - -import java.io.IOException; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - -/** - * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} - * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. - * {@link FullTextFunctionMapperPreprocessor#preprocess(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by - * replacing {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. - */ -public class FullTextFunctionMapperPreprocessor implements MappingPreProcessor { - - @Override - public void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener) { - Set unresolved = fullTextFunctions(plan); - Set indexNames = indexNames(plan); - - if (indexNames == null || indexNames.isEmpty() || unresolved.isEmpty()) { - listener.onResponse(plan); - return; - } - QueryRewriteContext ctx = queryRewriteContext(services, indexNames); - FullTextFunctionsRewritable rewritable = new FullTextFunctionsRewritable(unresolved); - Rewriteable.rewriteAndFetch( - rewritable, - ctx, - listener.delegateFailureAndWrap((l, r) -> l.onResponse(updateQueryBuilders(plan, r.results()))) - ); - } - - public LogicalPlan updateQueryBuilders(LogicalPlan plan, Map newQueryBuilders) { - LogicalPlan newPlan = plan.transformExpressionsDown(FullTextFunction.class, m -> { - if (newQueryBuilders.containsKey(m)) { - return m.replaceQueryBuilder(newQueryBuilders.get(m)); - } - return m; - }); - // The given plan was already analyzed and optimized, so we set the resulted plan to optimized as well. - newPlan.setOptimized(); - return newPlan; - } - - private static QueryRewriteContext queryRewriteContext(TransportActionServices services, Set indexNames) { - ResolvedIndices resolvedIndices = ResolvedIndices.resolveWithIndexNamesAndOptions( - indexNames.toArray(String[]::new), - IndexResolver.FIELD_CAPS_INDICES_OPTIONS, - services.clusterService().state(), - services.indexNameExpressionResolver(), - services.transportService().getRemoteClusterService(), - System.currentTimeMillis() - ); - - return services.searchService().getRewriteContext(System::currentTimeMillis, resolvedIndices, null); - } - - private static Set fullTextFunctions(LogicalPlan plan) { - Set functions = new HashSet<>(); - plan.forEachExpressionDown(FullTextFunction.class, functions::add); - return functions; - } - - public Set indexNames(LogicalPlan plan) { - Set indexNames = new HashSet<>(); - plan.forEachDown(EsRelation.class, esRelation -> indexNames.addAll(esRelation.index().concreteIndices())); - return indexNames; - } - - private static class FullTextFunctionsRewritable - implements - Rewriteable { - - private final Map queryBuilderMap; - - FullTextFunctionsRewritable(Map queryBuilderMap) { - this.queryBuilderMap = queryBuilderMap; - } - - FullTextFunctionsRewritable(Set functions) { - this.queryBuilderMap = new HashMap<>(); - - for (FullTextFunction func : functions) { - queryBuilderMap.put(func, func.asQuery(TranslatorHandler.TRANSLATOR_HANDLER).asBuilder()); - } - } - - @Override - public FullTextFunctionMapperPreprocessor.FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOException { - Map results = new HashMap<>(); - - boolean hasChanged = false; - for (var entry : queryBuilderMap.entrySet()) { - var initial = entry.getValue(); - var rewritten = initial.rewrite(ctx); - hasChanged |= rewritten != initial; - - results.put(entry.getKey(), rewritten); - } - - return hasChanged ? new FullTextFunctionMapperPreprocessor.FullTextFunctionsRewritable(results) : this; - } - - public Map results() { - return queryBuilderMap; - } - } -} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java index b7a942373e06b..ea38b3d4e4ebf 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java @@ -8,41 +8,42 @@ package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.xpack.esql.capabilities.TranslationAware; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; -import java.util.ArrayList; -import java.util.Collection; +import java.util.HashSet; import java.util.List; +import java.util.Set; public class MapperPreprocessorExecutor { private final TransportActionServices services; - private final List proprocessors = new ArrayList<>(); public MapperPreprocessorExecutor(TransportActionServices services) { this.services = services; } - public MapperPreprocessorExecutor addPreprocessor(MappingPreProcessor preProcessor) { - proprocessors.add(preProcessor); - return this; - } - - public MapperPreprocessorExecutor addPreprocessors(Collection preProcessors) { - proprocessors.addAll(preProcessors); - return this; + public void execute(LogicalPlan plan, ActionListener listener) { + execute(plan, queryRewriters(plan), 0, listener); } - public void execute(LogicalPlan plan, ActionListener listener) { - execute(plan, 0, listener); + private static List queryRewriters(LogicalPlan plan) { + Set queryRewriters = new HashSet<>(); + plan.forEachExpressionDown(e -> { + if (e instanceof TranslationAware.QueryRewriter qr) { + queryRewriters.add(qr.queryRewriter()); + } + }); + return List.copyOf(queryRewriters); } - private void execute(LogicalPlan plan, int index, ActionListener listener) { - if (index == proprocessors.size()) { + private void execute(LogicalPlan plan, List preprocessors, int index, ActionListener listener) { + if (index == preprocessors.size()) { listener.onResponse(plan); } else { - proprocessors.get(index).preprocess(plan, services, listener.delegateFailureAndWrap((l, p) -> execute(p, index + 1, l))); + preprocessors.get(index) + .preprocess(plan, services, listener.delegateFailureAndWrap((l, p) -> execute(p, preprocessors, index + 1, l))); } } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index d652c43b98eb8..bd891f1a94fce 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -73,7 +73,6 @@ import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; -import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.FullTextFunctionMapperPreprocessor; import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MapperPreprocessorExecutor; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.stats.PlanningMetrics; @@ -145,9 +144,7 @@ public EsqlSession( this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); this.planningMetrics = planningMetrics; this.indicesExpressionGrouper = indicesExpressionGrouper; - this.mapperPreprocessorExecutor = new MapperPreprocessorExecutor(services).addPreprocessor( - new FullTextFunctionMapperPreprocessor() - ); + this.mapperPreprocessorExecutor = new MapperPreprocessorExecutor(services); } public String sessionId() { @@ -167,12 +164,25 @@ public void execute(EsqlQueryRequest request, EsqlExecutionInfo executionInfo, P new EsqlSessionCCSUtils.CssPartialErrorsActionListener(executionInfo, listener) { @Override public void onResponse(LogicalPlan analyzedPlan) { - executeOptimizedPlan(request, executionInfo, planRunner, optimizedPlan(analyzedPlan), listener); + preMapping(request, executionInfo, planRunner, optimizedPlan(analyzedPlan), listener); } } ); } + public void preMapping( + EsqlQueryRequest request, + EsqlExecutionInfo executionInfo, + PlanRunner planRunner, + LogicalPlan optimizedPlan, + ActionListener listener + ) { + mapperPreprocessorExecutor.execute(optimizedPlan, listener.delegateFailureAndWrap((l, p) -> { + p.setOptimized(); // might have been updated by the preprocessor + executeOptimizedPlan(request, executionInfo, planRunner, p, listener); + })); + } + /** * Execute an analyzed plan. Most code should prefer calling {@link #execute} but * this is public for testing. @@ -184,13 +194,11 @@ public void executeOptimizedPlan( LogicalPlan optimizedPlan, ActionListener listener ) { - mapperPreprocessorExecutor.execute(optimizedPlan, listener.delegateFailureAndWrap((l, p) -> { - PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(p, request); - // TODO: this could be snuck into the underlying listener - EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); - // execute any potential subplans - executeSubPlans(physicalPlan, planRunner, executionInfo, request, l); - })); + PhysicalPlan physicalPlan = logicalPlanToPhysicalPlan(optimizedPlan, request); + // TODO: this could be snuck into the underlying listener + EsqlSessionCCSUtils.updateExecutionInfoAtEndOfPlanning(executionInfo); + // execute any potential subplans + executeSubPlans(physicalPlan, planRunner, executionInfo, request, listener); } private record PlanTuple(PhysicalPlan physical, LogicalPlan logical) {} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java deleted file mode 100644 index 91103ef286f72..0000000000000 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java +++ /dev/null @@ -1,169 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql.session; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ResolvedIndices; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryRewriteContext; -import org.elasticsearch.index.query.Rewriteable; -import org.elasticsearch.search.SearchService; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.esql.core.util.Holder; -import org.elasticsearch.xpack.esql.expression.function.fulltext.FullTextFunction; -import org.elasticsearch.xpack.esql.plan.logical.EsRelation; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; - -import java.io.IOException; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.function.BiConsumer; - -import static org.elasticsearch.xpack.esql.planner.TranslatorHandler.TRANSLATOR_HANDLER; - -/** - * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} - * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. - * {@link QueryBuilderResolver#resolveQueryBuilders(LogicalPlan, ActionListener, BiConsumer)} will rewrite the plan by replacing - * {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. - */ -public class QueryBuilderResolver { - private final SearchService searchService; - private final ClusterService clusterService; - private final TransportService transportService; - private final IndexNameExpressionResolver indexNameExpressionResolver; - - public QueryBuilderResolver( - SearchService searchService, - ClusterService clusterService, - TransportService transportService, - IndexNameExpressionResolver indexNameExpressionResolver - ) { - this.searchService = searchService; - this.clusterService = clusterService; - this.transportService = transportService; - this.indexNameExpressionResolver = indexNameExpressionResolver; - } - - public void resolveQueryBuilders( - LogicalPlan plan, - ActionListener listener, - BiConsumer> callback - ) { - if (plan.optimized() == false) { - listener.onFailure(new IllegalStateException("Expected optimized plan before query builder rewrite.")); - return; - } - - Set unresolved = fullTextFunctions(plan); - Set indexNames = indexNames(plan); - - if (indexNames == null || indexNames.isEmpty() || unresolved.isEmpty()) { - callback.accept(plan, listener); - return; - } - QueryRewriteContext ctx = queryRewriteContext(indexNames); - FullTextFunctionsRewritable rewritable = new FullTextFunctionsRewritable(unresolved); - Rewriteable.rewriteAndFetch(rewritable, ctx, new ActionListener() { - @Override - public void onResponse(FullTextFunctionsRewritable fullTextFunctionsRewritable) { - try { - LogicalPlan newPlan = planWithResolvedQueryBuilders(plan, fullTextFunctionsRewritable.results()); - callback.accept(newPlan, listener); - } catch (Exception e) { - onFailure(e); - } - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); - } - - private Set fullTextFunctions(LogicalPlan plan) { - Set functions = new HashSet<>(); - plan.forEachExpressionDown(FullTextFunction.class, func -> functions.add(func)); - return functions; - } - - public Set indexNames(LogicalPlan plan) { - Holder> indexNames = new Holder<>(); - - plan.forEachDown(EsRelation.class, esRelation -> { indexNames.set(esRelation.index().concreteIndices()); }); - - return indexNames.get(); - } - - public LogicalPlan planWithResolvedQueryBuilders(LogicalPlan plan, Map newQueryBuilders) { - LogicalPlan newPlan = plan.transformExpressionsDown(FullTextFunction.class, m -> { - if (newQueryBuilders.keySet().contains(m)) { - return m.replaceQueryBuilder(newQueryBuilders.get(m)); - } - return m; - }); - // The given plan was already analyzed and optimized, so we set the resulted plan to optimized as well. - newPlan.setOptimized(); - return newPlan; - } - - private QueryRewriteContext queryRewriteContext(Set indexNames) { - ResolvedIndices resolvedIndices = ResolvedIndices.resolveWithIndexNamesAndOptions( - indexNames.toArray(String[]::new), - IndexResolver.FIELD_CAPS_INDICES_OPTIONS, - clusterService.state(), - indexNameExpressionResolver, - transportService.getRemoteClusterService(), - System.currentTimeMillis() - ); - - return searchService.getRewriteContext(() -> System.currentTimeMillis(), resolvedIndices, null); - } - - private class FullTextFunctionsRewritable implements Rewriteable { - - private final Map queryBuilderMap; - - FullTextFunctionsRewritable(Map queryBuilderMap) { - this.queryBuilderMap = queryBuilderMap; - } - - FullTextFunctionsRewritable(Set functions) { - this.queryBuilderMap = new HashMap<>(); - - for (FullTextFunction func : functions) { - queryBuilderMap.put(func, TRANSLATOR_HANDLER.asQuery(func).asBuilder()); - } - } - - @Override - public FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOException { - Map results = new HashMap<>(); - - boolean hasChanged = false; - for (var entry : queryBuilderMap.entrySet()) { - var initial = entry.getValue(); - var rewritten = initial.rewrite(ctx); - hasChanged |= rewritten != initial; - - results.put(entry.getKey(), rewritten); - } - - return hasChanged ? new FullTextFunctionsRewritable(results) : this; - } - - public Map results() { - return queryBuilderMap; - } - } -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilder.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilder.java index 285739fe0936f..eafdb6366afd4 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilder.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilder.java @@ -332,11 +332,12 @@ private static String getInferenceIdForForField(Collection indexM protected boolean doEquals(SemanticQueryBuilder other) { return Objects.equals(fieldName, other.fieldName) && Objects.equals(query, other.query) - && Objects.equals(inferenceResults, other.inferenceResults); + && Objects.equals(inferenceResults, other.inferenceResults) + && Objects.equals(inferenceResultsSupplier, other.inferenceResultsSupplier); } @Override protected int doHashCode() { - return Objects.hash(fieldName, query, inferenceResults); + return Objects.hash(fieldName, query, inferenceResults, inferenceResultsSupplier); } } From 6040f3a1f0d0bab30133ba581757141efa0ea660 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Tue, 28 Jan 2025 08:29:54 +0100 Subject: [PATCH 07/24] rename if'aces back --- .../function/fulltext/FullTextFunction.java | 6 +++--- .../preprocessor/MapperPreprocessorExecutor.java | 13 ++++++------- .../mapper/preprocessor/MappingPreProcessor.java | 4 ++++ 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index 2d690550d4bf1..f38d4967a76c5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -54,8 +54,8 @@ public abstract class FullTextFunction extends Function implements TranslationAware, - TranslationAware.QueryRewriter, - PostAnalysisPlanVerificationAware { + PostAnalysisPlanVerificationAware, + MappingPreProcessor.MappingPreProcessorSupplier { private final Expression query; private final QueryBuilder queryBuilder; @@ -122,7 +122,7 @@ public Object queryAsObject() { } @Override - public MappingPreProcessor queryRewriter() { + public MappingPreProcessor mappingPreProcessor() { return FullTextFunctionMapperPreprocessor.INSTANCE; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java index ea38b3d4e4ebf..10eca26107008 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.xpack.esql.capabilities.TranslationAware; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; @@ -25,17 +24,17 @@ public MapperPreprocessorExecutor(TransportActionServices services) { } public void execute(LogicalPlan plan, ActionListener listener) { - execute(plan, queryRewriters(plan), 0, listener); + execute(plan, preprocessors(plan), 0, listener); } - private static List queryRewriters(LogicalPlan plan) { - Set queryRewriters = new HashSet<>(); + private static List preprocessors(LogicalPlan plan) { + Set preprocessors = new HashSet<>(); plan.forEachExpressionDown(e -> { - if (e instanceof TranslationAware.QueryRewriter qr) { - queryRewriters.add(qr.queryRewriter()); + if (e instanceof MappingPreProcessor.MappingPreProcessorSupplier supplier) { + preprocessors.add(supplier.mappingPreProcessor()); } }); - return List.copyOf(queryRewriters); + return List.copyOf(preprocessors); } private void execute(LogicalPlan plan, List preprocessors, int index, ActionListener listener) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java index cc918a836ec81..4542df645bf60 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java @@ -27,4 +27,8 @@ public interface MappingPreProcessor { * @param listener the listener to notify when processing is complete */ void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener); + + interface MappingPreProcessorSupplier { + MappingPreProcessor mappingPreProcessor(); + } } From aebbccfda9537e6eef7831aa9551ee26b5a42dd6 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Tue, 28 Jan 2025 07:55:04 +0000 Subject: [PATCH 08/24] [CI] Auto commit changes from spotless --- .../main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java | 1 - 1 file changed, 1 deletion(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java index bbafaa13be063..4cebadc22f65a 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -77,7 +77,6 @@ import org.elasticsearch.xpack.esql.plugin.QueryPragmas; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.session.Configuration; -import org.elasticsearch.xpack.esql.session.QueryBuilderResolver; import org.elasticsearch.xpack.esql.stats.SearchStats; import org.elasticsearch.xpack.esql.telemetry.Metrics; import org.elasticsearch.xpack.versionfield.Version; From 39954c64a3ed15cc0d0b2179b4e5a38bab3c8cd2 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Thu, 30 Jan 2025 08:37:08 +0100 Subject: [PATCH 09/24] Drop the pre-mapping executor --- .../esql/capabilities/TranslationAware.java | 5 - .../function/fulltext/FullTextFunction.java | 12 +- .../expression/function/fulltext/Match.java | 2 - ...ocessor.java => QueryBuilderResolver.java} | 39 ++-- .../MapperPreprocessorExecutor.java | 48 ----- .../preprocessor/MappingPreProcessor.java | 34 ---- .../xpack/esql/session/EsqlSession.java | 8 +- .../esql/session/QueryBuilderResolver.java | 167 ------------------ 8 files changed, 28 insertions(+), 287 deletions(-) rename x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/{FullTextFunctionMapperPreprocessor.java => QueryBuilderResolver.java} (74%) delete mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java delete mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java delete mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java index 5228bfe5170ed..8ef528b6668ab 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/capabilities/TranslationAware.java @@ -11,7 +11,6 @@ import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; -import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MappingPreProcessor; /** * Expressions implementing this interface can get called on data nodes to provide an Elasticsearch/Lucene query. @@ -43,8 +42,4 @@ interface SingleValueTranslationAware extends TranslationAware { */ Expression singleValueField(); } - - interface QueryRewriter extends TranslationAware { - MappingPreProcessor queryRewriter(); - } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java index f38d4967a76c5..a40723a3a645e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunction.java @@ -32,7 +32,6 @@ import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.OrderBy; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; -import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MappingPreProcessor; import org.elasticsearch.xpack.esql.querydsl.query.TranslationAwareExpressionQuery; import java.util.List; @@ -51,11 +50,7 @@ * These functions needs to be pushed down to Lucene queries to be executed - there's no Evaluator for them, but depend on * {@link org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizer} to rewrite them into Lucene queries. */ -public abstract class FullTextFunction extends Function - implements - TranslationAware, - PostAnalysisPlanVerificationAware, - MappingPreProcessor.MappingPreProcessorSupplier { +public abstract class FullTextFunction extends Function implements TranslationAware, PostAnalysisPlanVerificationAware { private final Expression query; private final QueryBuilder queryBuilder; @@ -121,11 +116,6 @@ public Object queryAsObject() { return BytesRefs.toString(queryAsObject); } - @Override - public MappingPreProcessor mappingPreProcessor() { - return FullTextFunctionMapperPreprocessor.INSTANCE; - } - /** * Returns the param ordinal for the query parameter so it can be used in error messages * diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java index ea5f3d9b83543..c87349e6b1288 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java @@ -30,8 +30,6 @@ public class Match extends AbstractMatchFullTextFunction { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Expression.class, "Match", Match::readFrom); - private transient Boolean isOperator; - @FunctionInfo( returnType = "boolean", preview = true, diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java similarity index 74% rename from x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java index 53ccc38e3364a..19f1c75b183bb 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextFunctionMapperPreprocessor.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java @@ -16,7 +16,6 @@ import org.elasticsearch.xpack.esql.plan.logical.EsRelation; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; -import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MappingPreProcessor; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.session.IndexResolver; @@ -27,20 +26,30 @@ /** * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. - * {@link FullTextFunctionMapperPreprocessor#preprocess(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by + * {@link QueryBuilderResolver#preprocess(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by * replacing {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. */ -public final class FullTextFunctionMapperPreprocessor implements MappingPreProcessor { +public final class QueryBuilderResolver { - public static final FullTextFunctionMapperPreprocessor INSTANCE = new FullTextFunctionMapperPreprocessor(); + public static final QueryBuilderResolver INSTANCE = new QueryBuilderResolver(); + + private QueryBuilderResolver() {} - @Override public void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener) { - Rewriteable.rewriteAndFetch( - new FullTextFunctionsRewritable(plan), - queryRewriteContext(services, indexNames(plan)), - listener.delegateFailureAndWrap((l, r) -> l.onResponse(r.plan)) - ); + var hasFullTextFunctions = plan.anyMatch(p -> { + Holder hasFullTextFunction = new Holder<>(false); + p.forEachExpression(FullTextFunction.class, unused -> hasFullTextFunction.set(true)); + return hasFullTextFunction.get(); + }); + if (hasFullTextFunctions) { + Rewriteable.rewriteAndFetch( + new FullTextFunctionsRewritable(plan), + queryRewriteContext(services, indexNames(plan)), + listener.delegateFailureAndWrap((l, r) -> l.onResponse(r.plan)) + ); + } else { + listener.onResponse(plan); + } } private static QueryRewriteContext queryRewriteContext(TransportActionServices services, Set indexNames) { @@ -62,9 +71,7 @@ public Set indexNames(LogicalPlan plan) { return indexNames; } - private record FullTextFunctionsRewritable(LogicalPlan plan) - implements - Rewriteable { + private record FullTextFunctionsRewritable(LogicalPlan plan) implements Rewriteable { @Override public FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOException { Holder exceptionHolder = new Holder<>(); @@ -77,9 +84,9 @@ public FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOExc } catch (IOException e) { exceptionHolder.trySet(e); } - var rewrite = builder != initial; - updated.set(updated.get() || rewrite); - return rewrite ? f.replaceQueryBuilder(builder) : f; + var rewritten = builder != initial; + updated.set(updated.get() || rewritten); + return rewritten ? f.replaceQueryBuilder(builder) : f; }); if (exceptionHolder.get() != null) { throw exceptionHolder.get(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java deleted file mode 100644 index 10eca26107008..0000000000000 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MapperPreprocessorExecutor.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.plugin.TransportActionServices; - -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -public class MapperPreprocessorExecutor { - - private final TransportActionServices services; - - public MapperPreprocessorExecutor(TransportActionServices services) { - this.services = services; - } - - public void execute(LogicalPlan plan, ActionListener listener) { - execute(plan, preprocessors(plan), 0, listener); - } - - private static List preprocessors(LogicalPlan plan) { - Set preprocessors = new HashSet<>(); - plan.forEachExpressionDown(e -> { - if (e instanceof MappingPreProcessor.MappingPreProcessorSupplier supplier) { - preprocessors.add(supplier.mappingPreProcessor()); - } - }); - return List.copyOf(preprocessors); - } - - private void execute(LogicalPlan plan, List preprocessors, int index, ActionListener listener) { - if (index == preprocessors.size()) { - listener.onResponse(plan); - } else { - preprocessors.get(index) - .preprocess(plan, services, listener.delegateFailureAndWrap((l, p) -> execute(p, preprocessors, index + 1, l))); - } - } -} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java deleted file mode 100644 index 4542df645bf60..0000000000000 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/preprocessor/MappingPreProcessor.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql.planner.mapper.preprocessor; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; -import org.elasticsearch.xpack.esql.plugin.TransportActionServices; - -/** - * Interface for a LogicalPlan processing rule occurring after the optimization, but before mapping to a physical plan. - * This step occurs on the coordinator. The rule may use services provided to the transport action and thus can resolve indices, rewrite - * queries, perform substitutions, etc. - * Note that the LogicalPlan following the rules' changes will not undergo another logical optimization round. The changes these rules - * should apply are only those that require access to services that need to be performed asynchronously. - */ -public interface MappingPreProcessor { - - /** - * Process a logical plan making use of the available services and provide the updated plan to the provided listener. - * @param plan the logical plan to process - * @param services the services available from the transport action - * @param listener the listener to notify when processing is complete - */ - void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener); - - interface MappingPreProcessorSupplier { - MappingPreProcessor mappingPreProcessor(); - } -} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index b5bfe4b820b70..561119b09107e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -48,6 +48,7 @@ import org.elasticsearch.xpack.esql.enrich.ResolvedEnrichPolicy; import org.elasticsearch.xpack.esql.expression.UnresolvedNamePattern; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; +import org.elasticsearch.xpack.esql.expression.function.fulltext.QueryBuilderResolver; import org.elasticsearch.xpack.esql.index.EsIndex; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.index.MappingException; @@ -73,7 +74,6 @@ import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; -import org.elasticsearch.xpack.esql.planner.mapper.preprocessor.MapperPreprocessorExecutor; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.telemetry.PlanTelemetry; @@ -116,7 +116,7 @@ public interface PlanRunner { private final PhysicalPlanOptimizer physicalPlanOptimizer; private final PlanTelemetry planTelemetry; private final IndicesExpressionGrouper indicesExpressionGrouper; - private final MapperPreprocessorExecutor mapperPreprocessorExecutor; + private final TransportActionServices services; public EsqlSession( String sessionId, @@ -144,7 +144,7 @@ public EsqlSession( this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); this.planTelemetry = planTelemetry; this.indicesExpressionGrouper = indicesExpressionGrouper; - this.mapperPreprocessorExecutor = new MapperPreprocessorExecutor(services); + this.services = services; } public String sessionId() { @@ -177,7 +177,7 @@ public void preMapping( LogicalPlan optimizedPlan, ActionListener listener ) { - mapperPreprocessorExecutor.execute(optimizedPlan, listener.delegateFailureAndWrap((l, p) -> { + QueryBuilderResolver.INSTANCE.preprocess(optimizedPlan, services, listener.delegateFailureAndWrap((l, p) -> { p.setOptimized(); // might have been updated by the preprocessor executeOptimizedPlan(request, executionInfo, planRunner, p, listener); })); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java deleted file mode 100644 index 7db81069f9d3c..0000000000000 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/QueryBuilderResolver.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.esql.session; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ResolvedIndices; -import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryRewriteContext; -import org.elasticsearch.index.query.Rewriteable; -import org.elasticsearch.search.SearchService; -import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xpack.esql.core.util.Holder; -import org.elasticsearch.xpack.esql.expression.function.fulltext.FullTextFunction; -import org.elasticsearch.xpack.esql.plan.logical.EsRelation; -import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; - -import java.io.IOException; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.function.BiConsumer; - -import static org.elasticsearch.xpack.esql.planner.TranslatorHandler.TRANSLATOR_HANDLER; - -/** - * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} - * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. - * {@link QueryBuilderResolver#resolveQueryBuilders(LogicalPlan, ActionListener, BiConsumer)} will rewrite the plan by replacing - * {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. - */ -public class QueryBuilderResolver { - private final SearchService searchService; - private final ClusterService clusterService; - private final TransportService transportService; - private final IndexNameExpressionResolver indexNameExpressionResolver; - - public QueryBuilderResolver( - SearchService searchService, - ClusterService clusterService, - TransportService transportService, - IndexNameExpressionResolver indexNameExpressionResolver - ) { - this.searchService = searchService; - this.clusterService = clusterService; - this.transportService = transportService; - this.indexNameExpressionResolver = indexNameExpressionResolver; - } - - public void resolveQueryBuilders( - LogicalPlan plan, - ActionListener listener, - BiConsumer> callback - ) { - if (plan.optimized() == false) { - listener.onFailure(new IllegalStateException("Expected optimized plan before query builder rewrite.")); - return; - } - - Set unresolved = fullTextFunctions(plan); - Set indexNames = indexNames(plan); - - if (indexNames == null || indexNames.isEmpty() || unresolved.isEmpty()) { - callback.accept(plan, listener); - return; - } - QueryRewriteContext ctx = queryRewriteContext(indexNames); - FullTextFunctionsRewritable rewritable = new FullTextFunctionsRewritable(unresolved); - Rewriteable.rewriteAndFetch(rewritable, ctx, new ActionListener() { - @Override - public void onResponse(FullTextFunctionsRewritable fullTextFunctionsRewritable) { - try { - LogicalPlan newPlan = planWithResolvedQueryBuilders(plan, fullTextFunctionsRewritable.results()); - callback.accept(newPlan, listener); - } catch (Exception e) { - onFailure(e); - } - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); - } - - private Set fullTextFunctions(LogicalPlan plan) { - Set functions = new HashSet<>(); - plan.forEachExpressionDown(FullTextFunction.class, func -> functions.add(func)); - return functions; - } - - public Set indexNames(LogicalPlan plan) { - Holder> indexNames = new Holder<>(); - plan.forEachDown(EsRelation.class, esRelation -> indexNames.set(esRelation.concreteIndices())); - return indexNames.get(); - } - - public LogicalPlan planWithResolvedQueryBuilders(LogicalPlan plan, Map newQueryBuilders) { - LogicalPlan newPlan = plan.transformExpressionsDown(FullTextFunction.class, m -> { - if (newQueryBuilders.keySet().contains(m)) { - return m.replaceQueryBuilder(newQueryBuilders.get(m)); - } - return m; - }); - // The given plan was already analyzed and optimized, so we set the resulted plan to optimized as well. - newPlan.setOptimized(); - return newPlan; - } - - private QueryRewriteContext queryRewriteContext(Set indexNames) { - ResolvedIndices resolvedIndices = ResolvedIndices.resolveWithIndexNamesAndOptions( - indexNames.toArray(String[]::new), - IndexResolver.FIELD_CAPS_INDICES_OPTIONS, - clusterService.state(), - indexNameExpressionResolver, - transportService.getRemoteClusterService(), - System.currentTimeMillis() - ); - - return searchService.getRewriteContext(() -> System.currentTimeMillis(), resolvedIndices, null); - } - - private class FullTextFunctionsRewritable implements Rewriteable { - - private final Map queryBuilderMap; - - FullTextFunctionsRewritable(Map queryBuilderMap) { - this.queryBuilderMap = queryBuilderMap; - } - - FullTextFunctionsRewritable(Set functions) { - this.queryBuilderMap = new HashMap<>(); - - for (FullTextFunction func : functions) { - queryBuilderMap.put(func, TRANSLATOR_HANDLER.asQuery(func).asBuilder()); - } - } - - @Override - public FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOException { - Map results = new HashMap<>(); - - boolean hasChanged = false; - for (var entry : queryBuilderMap.entrySet()) { - var initial = entry.getValue(); - var rewritten = initial.rewrite(ctx); - hasChanged |= rewritten != initial; - - results.put(entry.getKey(), rewritten); - } - - return hasChanged ? new FullTextFunctionsRewritable(results) : this; - } - - public Map results() { - return queryBuilderMap; - } - } -} From e08fd4d1c38693382b01606c2e5e1d40ea1443a3 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Thu, 30 Jan 2025 09:11:26 +0100 Subject: [PATCH 10/24] drop wrong changelog --- docs/changelog/120368.yaml | 5 ----- 1 file changed, 5 deletions(-) delete mode 100644 docs/changelog/120368.yaml diff --git a/docs/changelog/120368.yaml b/docs/changelog/120368.yaml deleted file mode 100644 index f51b42e3f241a..0000000000000 --- a/docs/changelog/120368.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 120368 -summary: Add a pre-mapping logical plan processing step -area: ES|QL -type: enhancement -issues: [] From 155acc75666df1c209483701c9ce9c938a5aa758 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Thu, 30 Jan 2025 11:19:19 +0100 Subject: [PATCH 11/24] Reintroduce a light premapping layer --- .../fulltext/QueryBuilderResolver.java | 4 +- .../esql/planner/premapper/PreMapper.java | 37 +++++++++++++++++++ .../xpack/esql/session/EsqlSession.java | 24 ++++-------- 3 files changed, 46 insertions(+), 19 deletions(-) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java index 19f1c75b183bb..0c90137bed2ee 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java @@ -26,7 +26,7 @@ /** * Some {@link FullTextFunction} implementations such as {@link org.elasticsearch.xpack.esql.expression.function.fulltext.Match} * will be translated to a {@link QueryBuilder} that require a rewrite phase on the coordinator. - * {@link QueryBuilderResolver#preprocess(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by + * {@link QueryBuilderResolver#resolveQueryBuilders(LogicalPlan, TransportActionServices, ActionListener)} will rewrite the plan by * replacing {@link FullTextFunction} expression with new ones that hold rewritten {@link QueryBuilder}s. */ public final class QueryBuilderResolver { @@ -35,7 +35,7 @@ public final class QueryBuilderResolver { private QueryBuilderResolver() {} - public void preprocess(LogicalPlan plan, TransportActionServices services, ActionListener listener) { + public void resolveQueryBuilders(LogicalPlan plan, TransportActionServices services, ActionListener listener) { var hasFullTextFunctions = plan.anyMatch(p -> { Holder hasFullTextFunction = new Holder<>(false); p.forEachExpression(FullTextFunction.class, unused -> hasFullTextFunction.set(true)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java new file mode 100644 index 0000000000000..f756a0340a0db --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java @@ -0,0 +1,37 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.planner.premapper; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.xpack.esql.expression.function.fulltext.QueryBuilderResolver; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; +import org.elasticsearch.xpack.esql.plugin.TransportActionServices; + +/** + * The class is responsible for invoking any premapping steps that need to be applied to the logical plan, + * before this is being mapped to a physical one. + */ +public class PreMapper { + + private final TransportActionServices services; + + public PreMapper(TransportActionServices services) { + this.services = services; + } + + /** + * Invokes any premapping steps that need to be applied to the logical plan, before this is being mapped to a physical one. + */ + public void preMapper(LogicalPlan plan, ActionListener listener) { + queryRewrite(plan, listener); + } + + private void queryRewrite(LogicalPlan plan, ActionListener listener) { + QueryBuilderResolver.INSTANCE.resolveQueryBuilders(plan, services, listener); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index 77eb54f15349f..fc50d792438d0 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -48,7 +48,6 @@ import org.elasticsearch.xpack.esql.enrich.ResolvedEnrichPolicy; import org.elasticsearch.xpack.esql.expression.UnresolvedNamePattern; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; -import org.elasticsearch.xpack.esql.expression.function.fulltext.QueryBuilderResolver; import org.elasticsearch.xpack.esql.index.EsIndex; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.index.MappingException; @@ -74,6 +73,7 @@ import org.elasticsearch.xpack.esql.plan.physical.FragmentExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.planner.mapper.Mapper; +import org.elasticsearch.xpack.esql.planner.premapper.PreMapper; import org.elasticsearch.xpack.esql.plugin.TransportActionServices; import org.elasticsearch.xpack.esql.telemetry.PlanTelemetry; @@ -111,12 +111,12 @@ public interface PlanRunner { private final Verifier verifier; private final EsqlFunctionRegistry functionRegistry; private final LogicalPlanOptimizer logicalPlanOptimizer; + private final PreMapper preMapper; private final Mapper mapper; private final PhysicalPlanOptimizer physicalPlanOptimizer; private final PlanTelemetry planTelemetry; private final IndicesExpressionGrouper indicesExpressionGrouper; - private final TransportActionServices services; public EsqlSession( String sessionId, @@ -144,7 +144,7 @@ public EsqlSession( this.physicalPlanOptimizer = new PhysicalPlanOptimizer(new PhysicalOptimizerContext(configuration)); this.planTelemetry = planTelemetry; this.indicesExpressionGrouper = indicesExpressionGrouper; - this.services = services; + this.preMapper = new PreMapper(services); } public String sessionId() { @@ -164,25 +164,15 @@ public void execute(EsqlQueryRequest request, EsqlExecutionInfo executionInfo, P new EsqlSessionCCSUtils.CssPartialErrorsActionListener(executionInfo, listener) { @Override public void onResponse(LogicalPlan analyzedPlan) { - preMapping(request, executionInfo, planRunner, optimizedPlan(analyzedPlan), listener); + preMapper.preMapper(analyzedPlan, listener.delegateFailureAndWrap((l, p) -> { + p.setOptimized(); // might have been updated by the preprocessor + executeOptimizedPlan(request, executionInfo, planRunner, optimizedPlan(p), l); + })); } } ); } - public void preMapping( - EsqlQueryRequest request, - EsqlExecutionInfo executionInfo, - PlanRunner planRunner, - LogicalPlan optimizedPlan, - ActionListener listener - ) { - QueryBuilderResolver.INSTANCE.preprocess(optimizedPlan, services, listener.delegateFailureAndWrap((l, p) -> { - p.setOptimized(); // might have been updated by the preprocessor - executeOptimizedPlan(request, executionInfo, planRunner, p, listener); - })); - } - /** * Execute an analyzed plan. Most code should prefer calling {@link #execute} but * this is public for testing. From d9a6807110065ec191cab7efeae30b1e8bb51478 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Thu, 30 Jan 2025 22:44:15 +0100 Subject: [PATCH 12/24] Update some FTF verifications. Address review comments --- .../xpack/esql/core/util/Holder.java | 7 +- .../expression/function/fulltext/Match.java | 65 ++++++++++--------- .../fulltext/QueryBuilderResolver.java | 8 +-- .../expression/function/fulltext/Term.java | 35 ++++++---- .../esql/planner/premapper/PreMapper.java | 7 +- .../xpack/esql/session/EsqlSession.java | 10 +-- .../xpack/esql/analysis/VerifierTests.java | 23 ++++++- .../optimizer/LogicalPlanOptimizerTests.java | 32 --------- 8 files changed, 93 insertions(+), 94 deletions(-) diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java index a6f5aaae1731f..9aadcefb84e84 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/util/Holder.java @@ -29,14 +29,11 @@ public void set(T value) { /** * Sets a value in the holder, but only if none has already been set. * @param value the new value to set. - * @return the previously held value, if any was set. */ - public T trySet(T value) { - T old = this.value; - if (old == null) { + public void setIfAbsent(T value) { + if (this.value == null) { this.value = value; } - return old; } public T get() { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java index 3223e96da7136..ec599020e0b05 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Match.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.xpack.esql.capabilities.PostOptimizationVerificationAware; +import org.elasticsearch.xpack.esql.capabilities.PostAnalysisPlanVerificationAware; import org.elasticsearch.xpack.esql.common.Failure; import org.elasticsearch.xpack.esql.common.Failures; import org.elasticsearch.xpack.esql.core.InvalidArgumentException; @@ -30,6 +30,7 @@ import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.DataTypeConverter; import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; +import org.elasticsearch.xpack.esql.core.util.Check; import org.elasticsearch.xpack.esql.core.util.NumericUtils; import org.elasticsearch.xpack.esql.expression.function.Example; import org.elasticsearch.xpack.esql.expression.function.FunctionInfo; @@ -38,6 +39,7 @@ import org.elasticsearch.xpack.esql.expression.function.Param; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; import org.elasticsearch.xpack.esql.querydsl.query.MatchQuery; import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; @@ -48,6 +50,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.function.BiConsumer; import static java.util.Map.entry; import static org.elasticsearch.common.logging.LoggerMessageFormat.format; @@ -88,7 +91,7 @@ /** * Full text function that performs a {@link org.elasticsearch.xpack.esql.querydsl.query.MatchQuery} . */ -public class Match extends FullTextFunction implements OptionalArgument, PostOptimizationVerificationAware { +public class Match extends FullTextFunction implements OptionalArgument, PostAnalysisPlanVerificationAware { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Expression.class, "Match", Match::readFrom); public static final Set FIELD_DATA_TYPES = Set.of( @@ -429,23 +432,23 @@ public Expression replaceQueryBuilder(QueryBuilder queryBuilder) { } @Override - public void postOptimizationVerification(Failures failures) { - Expression fieldExpression = field(); - // Field may be converted to other data type (field_name :: data_type), so we need to check the original field - if (fieldExpression instanceof AbstractConvertFunction convertFunction) { - fieldExpression = convertFunction.field(); - } - if (fieldExpression instanceof FieldAttribute == false) { - failures.add( - Failure.fail( - field, - "[{}] {} cannot operate on [{}], which is not a field from an index mapping", - functionName(), - functionType(), - field.sourceText() - ) - ); - } + public BiConsumer postAnalysisPlanVerification() { + return (plan, failures) -> { + super.postAnalysisPlanVerification().accept(plan, failures); + plan.forEachExpression(Match.class, m -> { + if (m.fieldAsFieldAttribute() == null) { + failures.add( + Failure.fail( + m.field(), + "[{}] {} cannot operate on [{}], which is not a field from an index mapping", + functionName(), + functionType(), + m.field().sourceText() + ) + ); + } + }); + }; } @Override @@ -476,22 +479,24 @@ public Object queryAsObject() { @Override protected Query translate(TranslatorHandler handler) { + var fieldAttribute = fieldAsFieldAttribute(); + Check.notNull(fieldAttribute, "Match must have a field attribute as the first argument"); + String fieldName = fieldAttribute.name(); + if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { + // If we have multiple field types, we allow the query to be done, but getting the underlying field name + fieldName = multiTypeEsField.getName(); + } + // Make query lenient so mixed field types can be queried when a field type is incompatible with the value provided + return new MatchQuery(source(), fieldName, queryAsObject(), matchQueryOptions()); + } + + private FieldAttribute fieldAsFieldAttribute() { Expression fieldExpression = field; // Field may be converted to other data type (field_name :: data_type), so we need to check the original field if (fieldExpression instanceof AbstractConvertFunction convertFunction) { fieldExpression = convertFunction.field(); } - if (fieldExpression instanceof FieldAttribute fieldAttribute) { - String fieldName = fieldAttribute.name(); - if (fieldAttribute.field() instanceof MultiTypeEsField multiTypeEsField) { - // If we have multiple field types, we allow the query to be done, but getting the underlying field name - fieldName = multiTypeEsField.getName(); - } - // Make query lenient so mixed field types can be queried when a field type is incompatible with the value provided - return new MatchQuery(source(), fieldName, queryAsObject(), matchQueryOptions()); - } - - throw new IllegalArgumentException("Match must have a field attribute as the first argument"); + return fieldExpression instanceof FieldAttribute fieldAttribute ? fieldAttribute : null; } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java index 0c90137bed2ee..14607de433630 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/QueryBuilderResolver.java @@ -31,11 +31,9 @@ */ public final class QueryBuilderResolver { - public static final QueryBuilderResolver INSTANCE = new QueryBuilderResolver(); - private QueryBuilderResolver() {} - public void resolveQueryBuilders(LogicalPlan plan, TransportActionServices services, ActionListener listener) { + public static void resolveQueryBuilders(LogicalPlan plan, TransportActionServices services, ActionListener listener) { var hasFullTextFunctions = plan.anyMatch(p -> { Holder hasFullTextFunction = new Holder<>(false); p.forEachExpression(FullTextFunction.class, unused -> hasFullTextFunction.set(true)); @@ -65,7 +63,7 @@ private static QueryRewriteContext queryRewriteContext(TransportActionServices s return services.searchService().getRewriteContext(System::currentTimeMillis, resolvedIndices, null); } - public Set indexNames(LogicalPlan plan) { + private static Set indexNames(LogicalPlan plan) { Set indexNames = new HashSet<>(); plan.forEachDown(EsRelation.class, esRelation -> indexNames.addAll(esRelation.concreteIndices())); return indexNames; @@ -82,7 +80,7 @@ public FullTextFunctionsRewritable rewrite(QueryRewriteContext ctx) throws IOExc try { builder = builder.rewrite(ctx); } catch (IOException e) { - exceptionHolder.trySet(e); + exceptionHolder.setIfAbsent(e); } var rewritten = builder != initial; updated.set(updated.get() || rewritten); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Term.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Term.java index 4db1c38694757..1da28b3069675 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Term.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/Term.java @@ -12,7 +12,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.xpack.esql.capabilities.PostOptimizationVerificationAware; +import org.elasticsearch.xpack.esql.capabilities.PostAnalysisPlanVerificationAware; import org.elasticsearch.xpack.esql.common.Failure; import org.elasticsearch.xpack.esql.common.Failures; import org.elasticsearch.xpack.esql.core.expression.Expression; @@ -26,10 +26,12 @@ import org.elasticsearch.xpack.esql.expression.function.FunctionInfo; import org.elasticsearch.xpack.esql.expression.function.Param; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.planner.TranslatorHandler; import java.io.IOException; import java.util.List; +import java.util.function.BiConsumer; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; @@ -39,7 +41,7 @@ /** * Full text function that performs a {@link TermQuery} . */ -public class Term extends FullTextFunction implements PostOptimizationVerificationAware { +public class Term extends FullTextFunction implements PostAnalysisPlanVerificationAware { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Expression.class, "Term", Term::readFrom); @@ -104,18 +106,23 @@ private TypeResolution resolveField() { } @Override - public void postOptimizationVerification(Failures failures) { - if (field instanceof FieldAttribute == false) { - failures.add( - Failure.fail( - field, - "[{}] {} cannot operate on [{}], which is not a field from an index mapping", - functionName(), - functionType(), - field.sourceText() - ) - ); - } + public BiConsumer postAnalysisPlanVerification() { + return (plan, failures) -> { + super.postAnalysisPlanVerification().accept(plan, failures); + plan.forEachExpression(Term.class, t -> { + if (t.field() instanceof FieldAttribute == false) { // TODO: is a conversion possible, similar to Match's case? + failures.add( + Failure.fail( + t.field(), + "[{}] {} cannot operate on [{}], which is not a field from an index mapping", + t.functionName(), + t.functionType(), + t.field().sourceText() + ) + ); + } + }); + }; } @Override diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java index f756a0340a0db..f69754dc3ce89 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/premapper/PreMapper.java @@ -28,10 +28,13 @@ public PreMapper(TransportActionServices services) { * Invokes any premapping steps that need to be applied to the logical plan, before this is being mapped to a physical one. */ public void preMapper(LogicalPlan plan, ActionListener listener) { - queryRewrite(plan, listener); + queryRewrite(plan, listener.delegateFailureAndWrap((l, p) -> { + p.setOptimized(); + l.onResponse(p); + })); } private void queryRewrite(LogicalPlan plan, ActionListener listener) { - QueryBuilderResolver.INSTANCE.resolveQueryBuilders(plan, services, listener); + QueryBuilderResolver.resolveQueryBuilders(plan, services, listener); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index fc50d792438d0..ff1b599ef191e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -164,10 +164,12 @@ public void execute(EsqlQueryRequest request, EsqlExecutionInfo executionInfo, P new EsqlSessionCCSUtils.CssPartialErrorsActionListener(executionInfo, listener) { @Override public void onResponse(LogicalPlan analyzedPlan) { - preMapper.preMapper(analyzedPlan, listener.delegateFailureAndWrap((l, p) -> { - p.setOptimized(); // might have been updated by the preprocessor - executeOptimizedPlan(request, executionInfo, planRunner, optimizedPlan(p), l); - })); + preMapper.preMapper( + analyzedPlan, + listener.delegateFailureAndWrap( + (l, p) -> executeOptimizedPlan(request, executionInfo, planRunner, optimizedPlan(p), l) + ) + ); } } ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 4403477e51125..e7f2fa1b97fb7 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1186,9 +1186,9 @@ public void testWeightedAvg() { public void testMatchInsideEval() throws Exception { assumeTrue("Match operator is available just for snapshots", Build.current().isSnapshot()); - assertEquals( - "1:36: [:] operator is only supported in WHERE commands", + "1:36: [:] operator is only supported in WHERE commands\n" + + "line 1:36: [:] operator cannot operate on [title], which is not a field from an index mapping", error("row title = \"brown fox\" | eval x = title:\"fox\" ") ); } @@ -1217,6 +1217,25 @@ public void testMatchFunctionAndOperatorHaveCorrectErrorMessages() throws Except assertEquals("1:24: [:] operator cannot be used after LIMIT", error("from test | limit 10 | where first_name : \"Anna\"")); } + // These should pass eventually once we lift some restrictions on match function + public void testMatchWithNonIndexedColumnCurrentlyUnsupported() { + assertEquals( + "1:67: [MATCH] function cannot operate on [initial], which is not a field from an index mapping", + error("from test | eval initial = substring(first_name, 1) | where match(initial, \"A\")") + ); + assertEquals( + "1:67: [MATCH] function cannot operate on [text], which is not a field from an index mapping", + error("from test | eval text=concat(first_name, last_name) | where match(text, \"cat\")") + ); + } + + public void testMatchFunctionIsNotNullable() { + assertEquals( + "1:48: [MATCH] function cannot operate on [text::keyword], which is not a field from an index mapping", + error("row n = null | eval text = n + 5 | where match(text::keyword, \"Anna\")") + ); + } + public void testQueryStringFunctionsNotAllowedAfterCommands() throws Exception { // Source commands assertEquals("1:13: [QSTR] function cannot be used after SHOW", error("show info | where qstr(\"8.16.0\")")); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index c80e374540d09..ea88f4fd5ba06 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -7240,38 +7240,6 @@ public void testToDatePeriodToTimeDurationWithField() { assertEquals("1:60: argument of [to_timeduration(x)] must be a constant, received [x]", e.getMessage().substring(header.length())); } - // These should pass eventually once we lift some restrictions on match function - public void testMatchWithNonIndexedColumnCurrentlyUnsupported() { - final String header = "Found 1 problem\nline "; - VerificationException e = expectThrows(VerificationException.class, () -> plan(""" - from test | eval initial = substring(first_name, 1) | where match(initial, "A")""")); - assertTrue(e.getMessage().startsWith("Found ")); - assertEquals( - "1:67: [MATCH] function cannot operate on [initial], which is not a field from an index mapping", - e.getMessage().substring(header.length()) - ); - - e = expectThrows(VerificationException.class, () -> plan(""" - from test | eval text=concat(first_name, last_name) | where match(text, "cat")""")); - assertTrue(e.getMessage().startsWith("Found ")); - assertEquals( - "1:67: [MATCH] function cannot operate on [text], which is not a field from an index mapping", - e.getMessage().substring(header.length()) - ); - } - - public void testMatchFunctionIsNotNullable() { - String queryText = """ - row n = null | eval text = n + 5 | where match(text::keyword, "Anna") - """; - - VerificationException ve = expectThrows(VerificationException.class, () -> plan(queryText)); - assertThat( - ve.getMessage(), - containsString("[MATCH] function cannot operate on [text::keyword], which is not a field from an index mapping") - ); - } - public void testWhereNull() { var plan = plan(""" from test From adada00918962c4f248e3434707155f28974a704 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Thu, 30 Jan 2025 22:58:37 +0100 Subject: [PATCH 13/24] Update docs/changelog/121260.yaml --- docs/changelog/121260.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/121260.yaml diff --git a/docs/changelog/121260.yaml b/docs/changelog/121260.yaml new file mode 100644 index 0000000000000..40c7487f29b12 --- /dev/null +++ b/docs/changelog/121260.yaml @@ -0,0 +1,5 @@ +pr: 121260 +summary: Introduce a pre-mapping logical plan processing step +area: ES|QL +type: enhancement +issues: [] From da8849c11e0cd0fb57f2f496f5e571f9bf342134 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Fri, 31 Jan 2025 09:45:08 +0100 Subject: [PATCH 14/24] Tests fixes --- .../org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java | 2 +- .../org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java index b928b25929401..2da9bee3701d7 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java @@ -246,7 +246,7 @@ public void testWhereMatchWithRow() { var error = expectThrows(ElasticsearchException.class, () -> run(query)); assertThat( error.getMessage(), - containsString("[MATCH] function cannot operate on [\"a brown fox\"], which is not a field from an index mapping") + containsString("line 2:15: [MATCH] function cannot operate on [content], which is not a field from an index mapping") ); } diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java index bd7246518c958..dbe115cc66176 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java @@ -230,7 +230,7 @@ public void testWhereMatchWithRow() { var error = expectThrows(ElasticsearchException.class, () -> run(query)); assertThat( error.getMessage(), - containsString("[:] operator cannot operate on [\"a brown fox\"], which is not a field from an index mapping") + containsString("line 2:9: [:] operator cannot operate on [content], which is not a field from an index mapping") ); } From da5629997441e4183feac3f9de5e4c865b08fe45 Mon Sep 17 00:00:00 2001 From: Larisa Motova Date: Thu, 30 Jan 2025 11:53:56 -1000 Subject: [PATCH 15/24] [ES|QL] Add aggregate metric double feature flag to its capability (#121318) AggregateMetricDouble should be behind a feature flag but on release builds it was getting added when it should not have been. This commit addresses that bug. --- .../org/elasticsearch/xpack/esql/action/EsqlCapabilities.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 20de3e443107d..25518220e308b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -20,6 +20,8 @@ import java.util.Locale; import java.util.Set; +import static org.elasticsearch.xpack.esql.core.plugin.EsqlCorePlugin.AGGREGATE_METRIC_DOUBLE_FEATURE_FLAG; + /** * A {@link Set} of "capabilities" supported by the {@link RestEsqlQueryAction} * and {@link RestEsqlAsyncQueryAction} APIs. These are exposed over the @@ -784,7 +786,7 @@ public enum Cap { /** * Support for aggregate_metric_double type */ - AGGREGATE_METRIC_DOUBLE; + AGGREGATE_METRIC_DOUBLE(AGGREGATE_METRIC_DOUBLE_FEATURE_FLAG.isEnabled()); private final boolean enabled; From 9993e4a74fca39755e452a155bb813f792973b7d Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 31 Jan 2025 09:03:07 +1100 Subject: [PATCH 16/24] Mute org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT test {yaml=reference/snapshot-restore/apis/get-snapshot-api/line_751} #121345 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index e8ef5992d3e95..7e6b30aba38b8 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -379,6 +379,9 @@ tests: - class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT method: test {yaml=reference/rest-api/common-options/line_125} issue: https://github.com/elastic/elasticsearch/issues/121338 +- class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT + method: test {yaml=reference/snapshot-restore/apis/get-snapshot-api/line_751} + issue: https://github.com/elastic/elasticsearch/issues/121345 # Examples: # From 49071e0a39070e6a3353886dc86696425694bb1d Mon Sep 17 00:00:00 2001 From: Parker Timmins Date: Thu, 30 Jan 2025 16:09:04 -0600 Subject: [PATCH 17/24] ReindexDataStreamIndex bug in assertion caused by reference equality (#121325) Assertion was using reference equality on two boxed longs. So assertion could produce false positives. Change to Objects.equals to check value and avoid null check. --- docs/changelog/121325.yaml | 5 +++++ .../action/ReindexDataStreamIndexTransportAction.java | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/121325.yaml diff --git a/docs/changelog/121325.yaml b/docs/changelog/121325.yaml new file mode 100644 index 0000000000000..9a9edc67d19fa --- /dev/null +++ b/docs/changelog/121325.yaml @@ -0,0 +1,5 @@ +pr: 121325 +summary: '`ReindexDataStreamIndex` bug in assertion caused by reference equality' +area: Data streams +type: bug +issues: [] diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java index 8c12011ca4bb1..e8110886cbd5a 100644 --- a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java +++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java @@ -54,6 +54,7 @@ import java.util.Locale; import java.util.Map; +import java.util.Objects; import static org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock.WRITE; @@ -372,7 +373,7 @@ private void sanityCheck( listener.delegateFailureAndWrap((delegate, ignored) -> { getIndexDocCount(sourceIndexName, parentTaskId, delegate.delegateFailureAndWrap((delegate1, sourceCount) -> { getIndexDocCount(destIndexName, parentTaskId, delegate1.delegateFailureAndWrap((delegate2, destCount) -> { - assert sourceCount == destCount + assert Objects.equals(sourceCount, destCount) : String.format( Locale.ROOT, "source index [%s] has %d docs and dest [%s] has %d docs", From 7804f282ea820d8b74d2853f523bab070dbe0351 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 31 Jan 2025 09:14:26 +1100 Subject: [PATCH 18/24] Mute org.elasticsearch.xpack.security.profile.ProfileIntegTests testHasPrivileges #121346 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 7e6b30aba38b8..7ffd4e477b419 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -382,6 +382,9 @@ tests: - class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT method: test {yaml=reference/snapshot-restore/apis/get-snapshot-api/line_751} issue: https://github.com/elastic/elasticsearch/issues/121345 +- class: org.elasticsearch.xpack.security.profile.ProfileIntegTests + method: testHasPrivileges + issue: https://github.com/elastic/elasticsearch/issues/121346 # Examples: # From 26a16c7c1a53d4a2ae9e7945f8fc07b30e8f137d Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 31 Jan 2025 09:14:40 +1100 Subject: [PATCH 19/24] Mute org.elasticsearch.xpack.security.profile.ProfileIntegTests testActivateProfile #121151 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 7ffd4e477b419..1cc851e2b5855 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -385,6 +385,9 @@ tests: - class: org.elasticsearch.xpack.security.profile.ProfileIntegTests method: testHasPrivileges issue: https://github.com/elastic/elasticsearch/issues/121346 +- class: org.elasticsearch.xpack.security.profile.ProfileIntegTests + method: testActivateProfile + issue: https://github.com/elastic/elasticsearch/issues/121151 # Examples: # From 4d8fd097012dcb544bf8e420d8371715153844e9 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 30 Jan 2025 23:17:47 +0100 Subject: [PATCH 20/24] Update ESRestTestCase's ROLLUP_REQUESTS_OPTIONS (#121335) Sometimes there are multiple warning. --- .../org/elasticsearch/test/rest/ESRestTestCase.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 5738ab60f47eb..40b2bae2fc4b1 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -169,14 +169,16 @@ public abstract class ESRestTestCase extends ESTestCase { private static final String EXPECTED_ROLLUP_WARNING_MESSAGE = "The rollup functionality will be removed in Elasticsearch 10.0. See docs for more information."; public static final RequestOptions.Builder ROLLUP_REQUESTS_OPTIONS = RequestOptions.DEFAULT.toBuilder().setWarningsHandler(warnings -> { - // Either no warning, because of bwc integration test OR - // the expected warning, because on current version if (warnings.isEmpty()) { return false; - } else if (warnings.size() == 1 && EXPECTED_ROLLUP_WARNING_MESSAGE.equals(warnings.get(0))) { - return false; } else { - return true; + // Sometimes multiple rollup deprecation warnings. Transport actions can be invoked multiple time on different nodes. + for (String warning : warnings) { + if (EXPECTED_ROLLUP_WARNING_MESSAGE.equals(warning) == false) { + return true; + } + } + return false; } }); From d3ac9d8b5d99e52a71126d036de39d1a7c3446cd Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 31 Jan 2025 10:20:16 +1100 Subject: [PATCH 21/24] Mute org.elasticsearch.smoketest.SmokeTestMultiNodeClientYamlTestSuiteIT test {yaml=search.vectors/42_knn_search_int4_flat/Vector similarity with filter only} #121350 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 1cc851e2b5855..f6413da66de9d 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -388,6 +388,9 @@ tests: - class: org.elasticsearch.xpack.security.profile.ProfileIntegTests method: testActivateProfile issue: https://github.com/elastic/elasticsearch/issues/121151 +- class: org.elasticsearch.smoketest.SmokeTestMultiNodeClientYamlTestSuiteIT + method: test {yaml=search.vectors/42_knn_search_int4_flat/Vector similarity with filter only} + issue: https://github.com/elastic/elasticsearch/issues/121350 # Examples: # From 2919025da694e34ed164a913e2d7d975b38edff6 Mon Sep 17 00:00:00 2001 From: Oleksandr Kolomiiets Date: Thu, 30 Jan 2025 15:26:58 -0800 Subject: [PATCH 22/24] Fix propagation of dynamic mapping parameter when applying copy_to (#121109) --- docs/changelog/121109.yaml | 6 +++ rest-api-spec/build.gradle | 1 + .../rest-api-spec/test/mapping/10_copy_to.yml | 41 +++++++++++++++++++ .../index/mapper/DocumentParserContext.java | 8 ++++ .../index/mapper/MapperFeatures.java | 4 +- .../DefaultMappingParametersHandler.java | 7 +--- 6 files changed, 60 insertions(+), 7 deletions(-) create mode 100644 docs/changelog/121109.yaml create mode 100644 rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml diff --git a/docs/changelog/121109.yaml b/docs/changelog/121109.yaml new file mode 100644 index 0000000000000..6492eccbf975a --- /dev/null +++ b/docs/changelog/121109.yaml @@ -0,0 +1,6 @@ +pr: 121109 +summary: Fix propagation of dynamic mapping parameter when applying `copy_to` +area: Mapping +type: bug +issues: + - 113049 diff --git a/rest-api-spec/build.gradle b/rest-api-spec/build.gradle index 0069e34fe949e..67f0b5a0714a9 100644 --- a/rest-api-spec/build.gradle +++ b/rest-api-spec/build.gradle @@ -73,4 +73,5 @@ tasks.named("yamlRestCompatTestTransform").configure ({ task -> task.skipTest("index/92_metrics_auto_subobjects/Root without subobjects with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0") task.skipTest("index/91_metrics_no_subobjects/Metrics object indexing with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0") task.skipTest("index/91_metrics_no_subobjects/Root without subobjects with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0") + task.skipTest("indices.create/20_synthetic_source/synthetic_source with copy_to inside nested object", "temporary until backported") }) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml new file mode 100644 index 0000000000000..58f09ec71ad61 --- /dev/null +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml @@ -0,0 +1,41 @@ +--- +copy_to from object with dynamic strict to dynamic field: + - requires: + cluster_features: ["mapper.copy_to.dynamic_handling"] + reason: requires a fix + + - do: + indices.create: + index: test + body: + mappings: + properties: + one: + dynamic: strict + properties: + k: + type: keyword + copy_to: two.k + + - do: + index: + index: test + id: 1 + refresh: true + body: + one: + k: "hey" + + - do: + search: + index: test + body: + docvalue_fields: [ "two.k.keyword" ] + + - match: + hits.hits.0._source: + one: + k: "hey" + - match: + hits.hits.0.fields: + two.k.keyword: [ "hey" ] diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java index ba9e902fee5d9..127ec05b25e63 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java @@ -50,6 +50,13 @@ private Wrapper(ObjectMapper parent, DocumentParserContext in) { this.in = in; } + // Used to create a copy_to context. + // It is important to reset `dynamic` here since it is possible that we copy into a completely different object. + private Wrapper(RootObjectMapper root, DocumentParserContext in) { + super(root, ObjectMapper.Dynamic.getRootDynamic(in.mappingLookup()), in); + this.in = in; + } + @Override public Iterable nonRootDocuments() { return in.nonRootDocuments(); @@ -711,6 +718,7 @@ in synthetic _source (to be consistent with stored _source). ContentPath path = new ContentPath(); XContentParser parser = DotExpandingXContentParser.expandDots(new CopyToParser(copyToField, parser()), path); + return new Wrapper(root(), this) { @Override public ContentPath path() { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java index 0935e219fb5c0..7567fae7d73e6 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java @@ -30,6 +30,7 @@ public class MapperFeatures implements FeatureSpecification { public static final NodeFeature META_FETCH_FIELDS_ERROR_CODE_CHANGED = new NodeFeature("meta_fetch_fields_error_code_changed"); public static final NodeFeature SPARSE_VECTOR_STORE_SUPPORT = new NodeFeature("mapper.sparse_vector.store_support"); public static final NodeFeature SORT_FIELDS_CHECK_FOR_NESTED_OBJECT_FIX = new NodeFeature("mapper.nested.sorting_fields_check_fix"); + public static final NodeFeature DYNAMIC_HANDLING_IN_COPY_TO = new NodeFeature("mapper.copy_to.dynamic_handling"); @Override public Set getTestFeatures() { @@ -45,8 +46,9 @@ public Set getTestFeatures() { CONSTANT_KEYWORD_SYNTHETIC_SOURCE_WRITE_FIX, META_FETCH_FIELDS_ERROR_CODE_CHANGED, SPARSE_VECTOR_STORE_SUPPORT, - SORT_FIELDS_CHECK_FOR_NESTED_OBJECT_FIX, COUNTED_KEYWORD_SYNTHETIC_SOURCE_NATIVE_SUPPORT, + SORT_FIELDS_CHECK_FOR_NESTED_OBJECT_FIX, + DYNAMIC_HANDLING_IN_COPY_TO, SourceFieldMapper.SYNTHETIC_RECOVERY_SOURCE, ObjectMapper.SUBOBJECTS_FALSE_MAPPING_UPDATE_FIX ); diff --git a/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java b/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java index b639108ea6ad2..04cb9467270d4 100644 --- a/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java @@ -11,7 +11,6 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.ObjectMapper; -import org.elasticsearch.logsdb.datageneration.fields.DynamicMapping; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; @@ -50,11 +49,7 @@ private Supplier> keywordMapping( // We only add copy_to to keywords because we get into trouble with numeric fields that are copied to dynamic fields. // If first copied value is numeric, dynamic field is created with numeric field type and then copy of text values fail. // Actual value being copied does not influence the core logic of copy_to anyway. - // - // TODO - // We don't use copy_to on fields that are inside an object with dynamic: strict - // because we'll hit https://github.com/elastic/elasticsearch/issues/113049. - if (request.dynamicMapping() != DynamicMapping.FORBIDDEN && ESTestCase.randomDouble() <= 0.05) { + if (ESTestCase.randomDouble() <= 0.05) { var options = request.eligibleCopyToFields() .stream() .filter(f -> f.equals(request.fieldName()) == false) From 20080960c4a4073dbe82776b6f2e486fb4e8e317 Mon Sep 17 00:00:00 2001 From: Saikat Sarkar <132922331+saikatsarkar056@users.noreply.github.com> Date: Thu, 30 Jan 2025 21:40:04 -0700 Subject: [PATCH 23/24] Integrate watsonx for re-ranking task (#117176) * Integrate watsonx reranking to inference api * Add api_version to the watsonx api call * Fix the return_doc option * Add top_n parameter to task_settings * Add truncate_input_tokens parameter to task_settings * Add test for IbmWatonxRankedResponseEntity * Add test for IbmWatonxRankedRequestEntity * Add test for IbmWatonxRankedRequest * [CI] Auto commit changes from spotless * Add changelog * Fix transport version * Add test for IbmWatsonxService * Remove canHandleStreamingResponses * Add requireNonNull for modelId and projectId * Remove maxInputToken method * Convert all optionals to required * [CI] Auto commit changes from spotless * Set minimal_supported version to be ML_INFERENCE_IBM_WATSONX_RERANK_ADDED * Remove extraction of unused fields from IbmWatsonxRerankServiceSettings * Add space * Add space --------- Co-authored-by: elasticsearchmachine --- docs/changelog/117176.yaml | 5 + .../org/elasticsearch/TransportVersions.java | 1 + .../InferenceNamedWriteablesProvider.java | 13 ++ .../ibmwatsonx/IbmWatsonxActionCreator.java | 14 +- .../ibmwatsonx/IbmWatsonxActionVisitor.java | 3 + .../IbmWatsonxRerankRequestManager.java | 72 +++++++ .../ibmwatsonx/IbmWatsonxResponseHandler.java | 1 - .../ibmwatsonx/IbmWatsonxRerankRequest.java | 110 ++++++++++ .../IbmWatsonxRerankRequestEntity.java | 77 +++++++ .../request/ibmwatsonx/IbmWatsonxUtils.java | 1 + .../IbmWatsonxRankedResponseEntity.java | 157 ++++++++++++++ .../services/ibmwatsonx/IbmWatsonxModel.java | 7 + .../ibmwatsonx/IbmWatsonxService.java | 10 + .../rerank/IbmWatsonxRerankModel.java | 121 +++++++++++ .../IbmWatsonxRerankServiceSettings.java | 190 +++++++++++++++++ .../rerank/IbmWatsonxRerankTaskSettings.java | 192 ++++++++++++++++++ .../IbmWatsonxRerankRequestEntityTests.java | 60 ++++++ .../rerank/IbmWatsonxRerankRequestTests.java | 107 ++++++++++ .../IbmWatsonxRankedResponseEntityTests.java | 166 +++++++++++++++ .../ibmwatsonx/IbmWatsonxServiceTests.java | 37 ++++ .../rerank/IbmWatsonxRerankModelTests.java | 28 +++ 21 files changed, 1370 insertions(+), 2 deletions(-) create mode 100644 docs/changelog/117176.yaml create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java diff --git a/docs/changelog/117176.yaml b/docs/changelog/117176.yaml new file mode 100644 index 0000000000000..26e0d3635bc9e --- /dev/null +++ b/docs/changelog/117176.yaml @@ -0,0 +1,5 @@ +pr: 117176 +summary: Integrate IBM watsonx to Inference API for re-ranking task +area: Experiences +type: enhancement +issues: [] diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 8f747a59ae5e0..1144f94795713 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -171,6 +171,7 @@ static TransportVersion def(int id) { public static final TransportVersion LINEAR_RETRIEVER_SUPPORT = def(8_837_00_0); public static final TransportVersion TIMEOUT_GET_PARAM_FOR_RESOLVE_CLUSTER = def(8_838_00_0); public static final TransportVersion INFERENCE_REQUEST_ADAPTIVE_RATE_LIMITING = def(8_839_00_0); + public static final TransportVersion ML_INFERENCE_IBM_WATSONX_RERANK_ADDED = def(8_840_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java index 6fc9870034018..e8dc763116707 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java @@ -75,6 +75,8 @@ import org.elasticsearch.xpack.inference.services.huggingface.HuggingFaceServiceSettings; import org.elasticsearch.xpack.inference.services.huggingface.elser.HuggingFaceElserServiceSettings; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsServiceSettings; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankServiceSettings; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; import org.elasticsearch.xpack.inference.services.jinaai.JinaAIServiceSettings; import org.elasticsearch.xpack.inference.services.jinaai.embeddings.JinaAIEmbeddingsServiceSettings; import org.elasticsearch.xpack.inference.services.jinaai.embeddings.JinaAIEmbeddingsTaskSettings; @@ -364,6 +366,17 @@ private static void addIbmWatsonxNamedWritables(List namedWriteables) { diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java index 7cad7c42bdcf1..6b1097256e97f 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java @@ -12,9 +12,11 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.SenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.sender.IbmWatsonxEmbeddingsRequestManager; +import org.elasticsearch.xpack.inference.external.http.sender.IbmWatsonxRerankRequestManager; import org.elasticsearch.xpack.inference.external.http.sender.Sender; import org.elasticsearch.xpack.inference.services.ServiceComponents; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import java.util.Map; import java.util.Objects; @@ -22,7 +24,6 @@ import static org.elasticsearch.xpack.inference.external.action.ActionUtils.constructFailedToSendRequestMessage; public class IbmWatsonxActionCreator implements IbmWatsonxActionVisitor { - private final Sender sender; private final ServiceComponents serviceComponents; @@ -41,6 +42,17 @@ public ExecutableAction create(IbmWatsonxEmbeddingsModel model, Map taskSettings) { + var overriddenModel = IbmWatsonxRerankModel.of(model, taskSettings); + var requestCreator = IbmWatsonxRerankRequestManager.of(overriddenModel, serviceComponents.threadPool()); + var failedToSendRequestErrorMessage = constructFailedToSendRequestMessage( + overriddenModel.getServiceSettings().uri(), + "Ibm Watsonx rerank" + ); + return new SenderExecutableAction(sender, requestCreator, failedToSendRequestErrorMessage); + } + protected IbmWatsonxEmbeddingsRequestManager getEmbeddingsRequestManager( IbmWatsonxEmbeddingsModel model, Truncator truncator, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java index 0a13ec2fb4645..474533040e0c3 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java @@ -9,9 +9,12 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import java.util.Map; public interface IbmWatsonxActionVisitor { ExecutableAction create(IbmWatsonxEmbeddingsModel model, Map taskSettings); + + ExecutableAction create(IbmWatsonxRerankModel model, Map taskSettings); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java new file mode 100644 index 0000000000000..f503771510e72 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java @@ -0,0 +1,72 @@ +/* + * 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.inference.external.http.sender; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.inference.external.http.retry.RequestSender; +import org.elasticsearch.xpack.inference.external.http.retry.ResponseHandler; +import org.elasticsearch.xpack.inference.external.ibmwatsonx.IbmWatsonxResponseHandler; +import org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxRerankRequest; +import org.elasticsearch.xpack.inference.external.response.ibmwatsonx.IbmWatsonxRankedResponseEntity; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; + +import java.util.List; +import java.util.Objects; +import java.util.function.Supplier; + +public class IbmWatsonxRerankRequestManager extends IbmWatsonxRequestManager { + private static final Logger logger = LogManager.getLogger(IbmWatsonxRerankRequestManager.class); + private static final ResponseHandler HANDLER = createIbmWatsonxResponseHandler(); + + private static ResponseHandler createIbmWatsonxResponseHandler() { + return new IbmWatsonxResponseHandler( + "ibm watsonx rerank", + (request, response) -> IbmWatsonxRankedResponseEntity.fromResponse(response) + ); + } + + public static IbmWatsonxRerankRequestManager of(IbmWatsonxRerankModel model, ThreadPool threadPool) { + return new IbmWatsonxRerankRequestManager(Objects.requireNonNull(model), Objects.requireNonNull(threadPool)); + } + + private final IbmWatsonxRerankModel model; + + public IbmWatsonxRerankRequestManager(IbmWatsonxRerankModel model, ThreadPool threadPool) { + super(threadPool, model); + this.model = model; + } + + @Override + public void execute( + InferenceInputs inferenceInputs, + RequestSender requestSender, + Supplier hasRequestCompletedFunction, + ActionListener listener + ) { + var rerankInput = QueryAndDocsInputs.of(inferenceInputs); + + execute( + new ExecutableInferenceRequest( + requestSender, + logger, + getRerankRequest(rerankInput.getQuery(), rerankInput.getChunks(), model), + HANDLER, + hasRequestCompletedFunction, + listener + ) + ); + } + + protected IbmWatsonxRerankRequest getRerankRequest(String query, List chunks, IbmWatsonxRerankModel model) { + return new IbmWatsonxRerankRequest(query, chunks, model); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java index 6d1d3fb2a4f91..1f28a8cd61026 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java @@ -17,7 +17,6 @@ import static org.elasticsearch.core.Strings.format; public class IbmWatsonxResponseHandler extends BaseResponseHandler { - public IbmWatsonxResponseHandler(String requestType, ResponseParser parseFunction) { super(requestType, parseFunction, IbmWatsonxErrorResponseEntity::fromResponse); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java new file mode 100644 index 0000000000000..cfc1f367be45c --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java @@ -0,0 +1,110 @@ +/* + * 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.inference.external.request.ibmwatsonx; + +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.ByteArrayEntity; +import org.elasticsearch.common.Strings; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.inference.external.request.HttpRequest; +import org.elasticsearch.xpack.inference.external.request.Request; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; + +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Objects; + +public class IbmWatsonxRerankRequest implements IbmWatsonxRequest { + + private final String query; + private final List input; + private final IbmWatsonxRerankTaskSettings taskSettings; + private final IbmWatsonxRerankModel model; + + public IbmWatsonxRerankRequest(String query, List input, IbmWatsonxRerankModel model) { + Objects.requireNonNull(model); + + this.input = Objects.requireNonNull(input); + this.query = Objects.requireNonNull(query); + taskSettings = model.getTaskSettings(); + this.model = model; + } + + @Override + public HttpRequest createHttpRequest() { + URI uri; + + try { + uri = new URI(model.uri().toString()); + } catch (URISyntaxException ex) { + throw new IllegalArgumentException("cannot parse URI patter"); + } + + HttpPost httpPost = new HttpPost(uri); + + ByteArrayEntity byteEntity = new ByteArrayEntity( + Strings.toString( + new IbmWatsonxRerankRequestEntity( + query, + input, + taskSettings, + model.getServiceSettings().modelId(), + model.getServiceSettings().projectId() + ) + ).getBytes(StandardCharsets.UTF_8) + ); + + httpPost.setEntity(byteEntity); + httpPost.setHeader(HttpHeaders.CONTENT_TYPE, XContentType.JSON.mediaType()); + + decorateWithAuth(httpPost); + + return new HttpRequest(httpPost, getInferenceEntityId()); + } + + public void decorateWithAuth(HttpPost httpPost) { + IbmWatsonxRequest.decorateWithBearerToken(httpPost, model.getSecretSettings(), model.getInferenceEntityId()); + } + + @Override + public String getInferenceEntityId() { + return model.getInferenceEntityId(); + } + + @Override + public URI getURI() { + return model.uri(); + } + + @Override + public Request truncate() { + return this; + } + + public String getQuery() { + return query; + } + + public List getInput() { + return input; + } + + public IbmWatsonxRerankModel getModel() { + return model; + } + + @Override + public boolean[] getTruncationInfo() { + return null; + } + +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java new file mode 100644 index 0000000000000..36e5951ebdc15 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java @@ -0,0 +1,77 @@ +/* + * 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.inference.external.request.ibmwatsonx; + +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +public record IbmWatsonxRerankRequestEntity( + String query, + List inputs, + IbmWatsonxRerankTaskSettings taskSettings, + String modelId, + String projectId +) implements ToXContentObject { + + private static final String INPUTS_FIELD = "inputs"; + private static final String QUERY_FIELD = "query"; + private static final String MODEL_ID_FIELD = "model_id"; + private static final String PROJECT_ID_FIELD = "project_id"; + + public IbmWatsonxRerankRequestEntity { + Objects.requireNonNull(query); + Objects.requireNonNull(inputs); + Objects.requireNonNull(modelId); + Objects.requireNonNull(projectId); + Objects.requireNonNull(taskSettings); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + + builder.field(MODEL_ID_FIELD, modelId); + builder.field(QUERY_FIELD, query); + builder.startArray(INPUTS_FIELD); + for (String input : inputs) { + builder.startObject(); + builder.field("text", input); + builder.endObject(); + } + builder.endArray(); + builder.field(PROJECT_ID_FIELD, projectId); + + builder.startObject("parameters"); + { + if (taskSettings.getTruncateInputTokens() != null) { + builder.field("truncate_input_tokens", taskSettings.getTruncateInputTokens()); + } + + builder.startObject("return_options"); + { + if (taskSettings.getDoesReturnDocuments() != null) { + builder.field("inputs", taskSettings.getDoesReturnDocuments()); + } + if (taskSettings.getTopNDocumentsOnly() != null) { + builder.field("top_n", taskSettings.getTopNDocumentsOnly()); + } + } + builder.endObject(); + } + builder.endObject(); + + builder.endObject(); + + return builder; + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java index a506a33385dfb..91679288e5ae3 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java @@ -13,6 +13,7 @@ public class IbmWatsonxUtils { public static final String V1 = "v1"; public static final String TEXT = "text"; public static final String EMBEDDINGS = "embeddings"; + public static final String RERANKS = "reranks"; private IbmWatsonxUtils() {} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java new file mode 100644 index 0000000000000..05f369bd8961e --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java @@ -0,0 +1,157 @@ +/* + * 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. + * + * this file was contributed to by a generative AI + */ + +package org.elasticsearch.xpack.inference.external.response.ibmwatsonx; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.inference.results.RankedDocsResults; +import org.elasticsearch.xpack.inference.external.http.HttpResult; + +import java.io.IOException; + +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.common.xcontent.XContentParserUtils.parseList; +import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField; +import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownToken; +import static org.elasticsearch.xpack.inference.external.response.XContentUtils.moveToFirstToken; +import static org.elasticsearch.xpack.inference.external.response.XContentUtils.positionParserAtTokenAfterField; + +public class IbmWatsonxRankedResponseEntity { + + private static final Logger logger = LogManager.getLogger(IbmWatsonxRankedResponseEntity.class); + + /** + * Parses the Ibm Watsonx ranked response. + * + * For a request like: + * "model": "rerank-english-v2.0", + * "query": "database", + * "return_documents": true, + * "top_n": 3, + * "input": ["greenland", "google","john", "mysql","potter", "grammar"] + *

+ * The response will look like (without whitespace): + * { + * "rerank": [ + * { + * "index": 3, + * "relevance_score": 0.7989932 + * }, + * { + * "index": 5, + * "relevance_score": 0.61281824 + * }, + * { + * "index": 1, + * "relevance_score": 0.5762553 + * }, + * { + * "index": 4, + * "relevance_score": 0.47395563 + * }, + * { + * "index": 0, + * "relevance_score": 0.4338926 + * }, + * { + * "index": 2, + * "relevance_score": 0.42638257 + * } + * ], + * } + * + * @param response the http response from ibm watsonx + * @return the parsed response + * @throws IOException if there is an error parsing the response + */ + public static InferenceServiceResults fromResponse(HttpResult response) throws IOException { + var parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler(LoggingDeprecationHandler.INSTANCE); + + try (XContentParser jsonParser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, response.body())) { + moveToFirstToken(jsonParser); + + XContentParser.Token token = jsonParser.currentToken(); + ensureExpectedToken(XContentParser.Token.START_OBJECT, token, jsonParser); + + positionParserAtTokenAfterField(jsonParser, "results", FAILED_TO_FIND_FIELD_TEMPLATE); // TODO error message + + token = jsonParser.currentToken(); + if (token == XContentParser.Token.START_ARRAY) { + return new RankedDocsResults(parseList(jsonParser, IbmWatsonxRankedResponseEntity::parseRankedDocObject)); + } else { + throwUnknownToken(token, jsonParser); + } + + // This should never be reached. The above code should either return successfully or hit the throwUnknownToken + // or throw a parsing exception + throw new IllegalStateException("Reached an invalid state while parsing the Watsonx response"); + } + } + + private static RankedDocsResults.RankedDoc parseRankedDocObject(XContentParser parser) throws IOException { + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + int index = -1; + float score = -1; + String documentText = null; + parser.nextToken(); + while (parser.currentToken() != XContentParser.Token.END_OBJECT) { + if (parser.currentToken() == XContentParser.Token.FIELD_NAME) { + switch (parser.currentName()) { + case "index": + parser.nextToken(); // move to VALUE_NUMBER + index = parser.intValue(); + parser.nextToken(); // move to next FIELD_NAME or END_OBJECT + break; + case "score": + parser.nextToken(); // move to VALUE_NUMBER + score = parser.floatValue(); + parser.nextToken(); // move to next FIELD_NAME or END_OBJECT + break; + case "input": + parser.nextToken(); // move to START_OBJECT; document text is wrapped in an object + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + do { + if (parser.currentToken() == XContentParser.Token.FIELD_NAME && parser.currentName().equals("text")) { + parser.nextToken(); // move to VALUE_STRING + documentText = parser.text(); + } + } while (parser.nextToken() != XContentParser.Token.END_OBJECT); + parser.nextToken();// move past END_OBJECT + // parser should now be at the next FIELD_NAME or END_OBJECT + break; + default: + throwUnknownField(parser.currentName(), parser); + } + } else { + parser.nextToken(); + } + } + + if (index == -1) { + logger.warn("Failed to find required field [index] in Watsonx rerank response"); + } + if (score == -1) { + logger.warn("Failed to find required field [relevance_score] in Watsonx rerank response"); + } + // documentText may or may not be present depending on the request parameter + + return new RankedDocsResults.RankedDoc(index, score, documentText); + } + + private IbmWatsonxRankedResponseEntity() {} + + static String FAILED_TO_FIND_FIELD_TEMPLATE = "Failed to find required field [%s] in Watsonx rerank response"; +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java index 4f0b425cdaa51..09706f70e3684 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java @@ -12,6 +12,7 @@ import org.elasticsearch.inference.ModelConfigurations; import org.elasticsearch.inference.ModelSecrets; import org.elasticsearch.inference.ServiceSettings; +import org.elasticsearch.inference.TaskSettings; import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.ibmwatsonx.IbmWatsonxActionVisitor; @@ -38,6 +39,12 @@ public IbmWatsonxModel(IbmWatsonxModel model, ServiceSettings serviceSettings) { rateLimitServiceSettings = model.rateLimitServiceSettings(); } + public IbmWatsonxModel(IbmWatsonxModel model, TaskSettings taskSettings) { + super(model, taskSettings); + + rateLimitServiceSettings = model.rateLimitServiceSettings(); + } + public abstract ExecutableAction accept(IbmWatsonxActionVisitor creator, Map taskSettings, InputType inputType); public IbmWatsonxRateLimitServiceSettings rateLimitServiceSettings() { diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java index 477225f00d22b..3fa423c2dae19 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java @@ -41,6 +41,7 @@ import org.elasticsearch.xpack.inference.services.ServiceUtils; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsServiceSettings; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import org.elasticsearch.xpack.inference.services.validation.ModelValidatorBuilder; import java.util.EnumSet; @@ -138,6 +139,15 @@ private static IbmWatsonxModel createModel( secretSettings, context ); + case RERANK -> new IbmWatsonxRerankModel( + inferenceEntityId, + taskType, + NAME, + serviceSettings, + taskSettings, + secretSettings, + context + ); default -> throw new ElasticsearchStatusException(failureMessage, RestStatus.BAD_REQUEST); }; } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java new file mode 100644 index 0000000000000..cb4c509d88c2b --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java @@ -0,0 +1,121 @@ +/* + * 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.inference.services.ibmwatsonx.rerank; + +import org.apache.http.client.utils.URIBuilder; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.inference.InputType; +import org.elasticsearch.inference.ModelConfigurations; +import org.elasticsearch.inference.ModelSecrets; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.xpack.inference.external.action.ExecutableAction; +import org.elasticsearch.xpack.inference.external.action.ibmwatsonx.IbmWatsonxActionVisitor; +import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxModel; +import org.elasticsearch.xpack.inference.services.settings.DefaultSecretSettings; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Map; + +import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.ML; +import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.RERANKS; +import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.TEXT; +import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.V1; + +public class IbmWatsonxRerankModel extends IbmWatsonxModel { + public static IbmWatsonxRerankModel of(IbmWatsonxRerankModel model, Map taskSettings) { + var requestTaskSettings = IbmWatsonxRerankTaskSettings.fromMap(taskSettings); + return new IbmWatsonxRerankModel(model, IbmWatsonxRerankTaskSettings.of(model.getTaskSettings(), requestTaskSettings)); + } + + public IbmWatsonxRerankModel( + String modelId, + TaskType taskType, + String service, + Map serviceSettings, + Map taskSettings, + @Nullable Map secrets, + ConfigurationParseContext context + ) { + this( + modelId, + taskType, + service, + IbmWatsonxRerankServiceSettings.fromMap(serviceSettings, context), + IbmWatsonxRerankTaskSettings.fromMap(taskSettings), + DefaultSecretSettings.fromMap(secrets) + ); + } + + // should only be used for testing + IbmWatsonxRerankModel( + String modelId, + TaskType taskType, + String service, + IbmWatsonxRerankServiceSettings serviceSettings, + IbmWatsonxRerankTaskSettings taskSettings, + @Nullable DefaultSecretSettings secretSettings + ) { + super( + new ModelConfigurations(modelId, taskType, service, serviceSettings, taskSettings), + new ModelSecrets(secretSettings), + serviceSettings + ); + } + + private IbmWatsonxRerankModel(IbmWatsonxRerankModel model, IbmWatsonxRerankTaskSettings taskSettings) { + super(model, taskSettings); + } + + @Override + public IbmWatsonxRerankServiceSettings getServiceSettings() { + return (IbmWatsonxRerankServiceSettings) super.getServiceSettings(); + } + + @Override + public IbmWatsonxRerankTaskSettings getTaskSettings() { + return (IbmWatsonxRerankTaskSettings) super.getTaskSettings(); + } + + @Override + public DefaultSecretSettings getSecretSettings() { + return (DefaultSecretSettings) super.getSecretSettings(); + } + + public URI uri() { + URI uri; + try { + uri = buildUri(this.getServiceSettings().uri().toString(), this.getServiceSettings().apiVersion()); + } catch (URISyntaxException e) { + throw new RuntimeException(e); + } + + return uri; + } + + /** + * Accepts a visitor to create an executable action. The returned action will not return documents in the response. + * @param visitor _ + * @param taskSettings _ + * @param inputType ignored for rerank task + * @return the rerank action + */ + @Override + public ExecutableAction accept(IbmWatsonxActionVisitor visitor, Map taskSettings, InputType inputType) { + return visitor.create(this, taskSettings); + } + + public static URI buildUri(String uri, String apiVersion) throws URISyntaxException { + return new URIBuilder().setScheme("https") + .setHost(uri) + .setPathSegments(ML, V1, TEXT, RERANKS) + .setParameter("version", apiVersion) + .build(); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java new file mode 100644 index 0000000000000..969622f9ba54f --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java @@ -0,0 +1,190 @@ +/* + * 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.inference.services.ibmwatsonx.rerank; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; +import org.elasticsearch.common.ValidationException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.inference.ModelConfigurations; +import org.elasticsearch.inference.ServiceSettings; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxRateLimitServiceSettings; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxService; +import org.elasticsearch.xpack.inference.services.settings.FilteredXContentObject; +import org.elasticsearch.xpack.inference.services.settings.RateLimitSettings; + +import java.io.IOException; +import java.net.URI; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.xpack.inference.services.ServiceFields.MODEL_ID; +import static org.elasticsearch.xpack.inference.services.ServiceFields.URL; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.convertToUri; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.createUri; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.extractRequiredString; +import static org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxServiceFields.API_VERSION; +import static org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxServiceFields.PROJECT_ID; + +public class IbmWatsonxRerankServiceSettings extends FilteredXContentObject implements ServiceSettings, IbmWatsonxRateLimitServiceSettings { + public static final String NAME = "ibm_watsonx_rerank_service_settings"; + + /** + * Rate limits are defined at + * Watson Machine Learning plans. + * For Lite plan, you've 120 requests per minute. + */ + private static final RateLimitSettings DEFAULT_RATE_LIMIT_SETTINGS = new RateLimitSettings(120); + + public static IbmWatsonxRerankServiceSettings fromMap(Map map, ConfigurationParseContext context) { + ValidationException validationException = new ValidationException(); + + String url = extractRequiredString(map, URL, ModelConfigurations.SERVICE_SETTINGS, validationException); + URI uri = convertToUri(url, URL, ModelConfigurations.SERVICE_SETTINGS, validationException); + String apiVersion = extractRequiredString(map, API_VERSION, ModelConfigurations.SERVICE_SETTINGS, validationException); + + String modelId = extractRequiredString(map, MODEL_ID, ModelConfigurations.SERVICE_SETTINGS, validationException); + String projectId = extractRequiredString(map, PROJECT_ID, ModelConfigurations.SERVICE_SETTINGS, validationException); + + RateLimitSettings rateLimitSettings = RateLimitSettings.of( + map, + DEFAULT_RATE_LIMIT_SETTINGS, + validationException, + IbmWatsonxService.NAME, + context + ); + + if (validationException.validationErrors().isEmpty() == false) { + throw validationException; + } + + return new IbmWatsonxRerankServiceSettings(uri, apiVersion, modelId, projectId, rateLimitSettings); + } + + private final URI uri; + + private final String apiVersion; + + private final String modelId; + + private final String projectId; + + private final RateLimitSettings rateLimitSettings; + + public IbmWatsonxRerankServiceSettings( + URI uri, + String apiVersion, + String modelId, + String projectId, + @Nullable RateLimitSettings rateLimitSettings + ) { + this.uri = uri; + this.apiVersion = apiVersion; + this.projectId = projectId; + this.modelId = modelId; + this.rateLimitSettings = Objects.requireNonNullElse(rateLimitSettings, DEFAULT_RATE_LIMIT_SETTINGS); + } + + public IbmWatsonxRerankServiceSettings(StreamInput in) throws IOException { + this.uri = createUri(in.readString()); + this.apiVersion = in.readString(); + this.modelId = in.readString(); + this.projectId = in.readString(); + this.rateLimitSettings = new RateLimitSettings(in); + + } + + public URI uri() { + return uri; + } + + public String apiVersion() { + return apiVersion; + } + + @Override + public String modelId() { + return modelId; + } + + public String projectId() { + return projectId; + } + + @Override + public RateLimitSettings rateLimitSettings() { + return rateLimitSettings; + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + + toXContentFragmentOfExposedFields(builder, params); + + builder.endObject(); + return builder; + } + + @Override + protected XContentBuilder toXContentFragmentOfExposedFields(XContentBuilder builder, Params params) throws IOException { + builder.field(URL, uri.toString()); + + builder.field(API_VERSION, apiVersion); + + builder.field(MODEL_ID, modelId); + + builder.field(PROJECT_ID, projectId); + + rateLimitSettings.toXContent(builder, params); + + return builder; + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return TransportVersions.ML_INFERENCE_IBM_WATSONX_RERANK_ADDED; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(uri.toString()); + out.writeString(apiVersion); + + out.writeString(modelId); + out.writeString(projectId); + + rateLimitSettings.writeTo(out); + } + + @Override + public boolean equals(Object object) { + if (this == object) return true; + if (object == null || getClass() != object.getClass()) return false; + IbmWatsonxRerankServiceSettings that = (IbmWatsonxRerankServiceSettings) object; + return Objects.equals(uri, that.uri) + && Objects.equals(apiVersion, that.apiVersion) + && Objects.equals(modelId, that.modelId) + && Objects.equals(projectId, that.projectId) + && Objects.equals(rateLimitSettings, that.rateLimitSettings); + } + + @Override + public int hashCode() { + return Objects.hash(uri, apiVersion, modelId, projectId, rateLimitSettings); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java new file mode 100644 index 0000000000000..12f4b8f6fa33e --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java @@ -0,0 +1,192 @@ +/* + * 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.inference.services.ibmwatsonx.rerank; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.ValidationException; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.inference.InputType; +import org.elasticsearch.inference.ModelConfigurations; +import org.elasticsearch.inference.TaskSettings; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.xpack.inference.services.ServiceUtils.extractOptionalBoolean; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.extractOptionalPositiveInteger; + +public class IbmWatsonxRerankTaskSettings implements TaskSettings { + + public static final String NAME = "ibm_watsonx_rerank_task_settings"; + public static final String RETURN_DOCUMENTS = "return_documents"; + public static final String TOP_N_DOCS_ONLY = "top_n"; + public static final String TRUNCATE_INPUT_TOKENS = "truncate_input_tokens"; + + static final IbmWatsonxRerankTaskSettings EMPTY_SETTINGS = new IbmWatsonxRerankTaskSettings(null, null, null); + + public static IbmWatsonxRerankTaskSettings fromMap(Map map) { + ValidationException validationException = new ValidationException(); + + if (map == null || map.isEmpty()) { + return EMPTY_SETTINGS; + } + + Boolean returnDocuments = extractOptionalBoolean(map, RETURN_DOCUMENTS, validationException); + Integer topNDocumentsOnly = extractOptionalPositiveInteger( + map, + TOP_N_DOCS_ONLY, + ModelConfigurations.TASK_SETTINGS, + validationException + ); + Integer truncateInputTokens = extractOptionalPositiveInteger( + map, + TRUNCATE_INPUT_TOKENS, + ModelConfigurations.TASK_SETTINGS, + validationException + ); + + if (validationException.validationErrors().isEmpty() == false) { + throw validationException; + } + + return of(topNDocumentsOnly, returnDocuments, truncateInputTokens); + } + + /** + * Creates a new {@link IbmWatsonxRerankTaskSettings} + * by preferring non-null fields from the request settings over the original settings. + * + * @param originalSettings the settings stored as part of the inference entity configuration + * @param requestTaskSettings the settings passed in within the task_settings field of the request + * @return a constructed {@link IbmWatsonxRerankTaskSettings} + */ + public static IbmWatsonxRerankTaskSettings of( + IbmWatsonxRerankTaskSettings originalSettings, + IbmWatsonxRerankTaskSettings requestTaskSettings + ) { + return new IbmWatsonxRerankTaskSettings( + requestTaskSettings.getTopNDocumentsOnly() != null + ? requestTaskSettings.getTopNDocumentsOnly() + : originalSettings.getTopNDocumentsOnly(), + requestTaskSettings.getReturnDocuments() != null + ? requestTaskSettings.getReturnDocuments() + : originalSettings.getReturnDocuments(), + requestTaskSettings.getTruncateInputTokens() != null + ? requestTaskSettings.getTruncateInputTokens() + : originalSettings.getTruncateInputTokens() + ); + } + + public static IbmWatsonxRerankTaskSettings of(Integer topNDocumentsOnly, Boolean returnDocuments, Integer maxChunksPerDoc) { + return new IbmWatsonxRerankTaskSettings(topNDocumentsOnly, returnDocuments, maxChunksPerDoc); + } + + private final Integer topNDocumentsOnly; + private final Boolean returnDocuments; + private final Integer truncateInputTokens; + + public IbmWatsonxRerankTaskSettings(StreamInput in) throws IOException { + this(in.readOptionalInt(), in.readOptionalBoolean(), in.readOptionalInt()); + } + + public IbmWatsonxRerankTaskSettings( + @Nullable Integer topNDocumentsOnly, + @Nullable Boolean doReturnDocuments, + @Nullable Integer truncateInputTokens + ) { + this.topNDocumentsOnly = topNDocumentsOnly; + this.returnDocuments = doReturnDocuments; + this.truncateInputTokens = truncateInputTokens; + } + + @Override + public boolean isEmpty() { + return topNDocumentsOnly == null && returnDocuments == null && truncateInputTokens == null; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + if (topNDocumentsOnly != null) { + builder.field(TOP_N_DOCS_ONLY, topNDocumentsOnly); + } + if (returnDocuments != null) { + builder.field(RETURN_DOCUMENTS, returnDocuments); + } + if (truncateInputTokens != null) { + builder.field(TRUNCATE_INPUT_TOKENS, truncateInputTokens); + } + builder.endObject(); + return builder; + } + + @Override + public String getWriteableName() { + return NAME; + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return TransportVersions.ML_INFERENCE_IBM_WATSONX_RERANK_ADDED; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalInt(topNDocumentsOnly); + out.writeOptionalBoolean(returnDocuments); + out.writeOptionalInt(truncateInputTokens); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + IbmWatsonxRerankTaskSettings that = (IbmWatsonxRerankTaskSettings) o; + return Objects.equals(returnDocuments, that.returnDocuments) + && Objects.equals(topNDocumentsOnly, that.topNDocumentsOnly) + && Objects.equals(truncateInputTokens, that.truncateInputTokens); + } + + @Override + public int hashCode() { + return Objects.hash(returnDocuments, topNDocumentsOnly, truncateInputTokens); + } + + public static String invalidInputTypeMessage(InputType inputType) { + return Strings.format("received invalid input type value [%s]", inputType.toString()); + } + + public Boolean getDoesReturnDocuments() { + return returnDocuments; + } + + public Integer getTopNDocumentsOnly() { + return topNDocumentsOnly; + } + + public Boolean getReturnDocuments() { + return returnDocuments; + } + + public Integer getTruncateInputTokens() { + return truncateInputTokens; + } + + @Override + public TaskSettings updatedTaskSettings(Map newSettings) { + IbmWatsonxRerankTaskSettings updatedSettings = IbmWatsonxRerankTaskSettings.fromMap(new HashMap<>(newSettings)); + return IbmWatsonxRerankTaskSettings.of(this, updatedSettings); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java new file mode 100644 index 0000000000000..8278b76a1cee4 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java @@ -0,0 +1,60 @@ +/* + * 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.inference.external.request.ibmwatsonx.rerank; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxRerankRequestEntity; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.xpack.inference.MatchersUtils.equalToIgnoringWhitespaceInJsonString; + +public class IbmWatsonxRerankRequestEntityTests extends ESTestCase { + public void testXContent_Request() throws IOException { + IbmWatsonxRerankTaskSettings taskSettings = new IbmWatsonxRerankTaskSettings(5, true, 100); + var entity = new IbmWatsonxRerankRequestEntity( + "database", + List.of("greenland", "google", "john", "mysql", "potter", "grammar"), + taskSettings, + "model", + "project_id" + ); + + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); + entity.toXContent(builder, null); + String xContentResult = Strings.toString(builder); + + assertThat(xContentResult, equalToIgnoringWhitespaceInJsonString(""" + {"model_id":"model", + "query":"database", + "inputs":[ + {"text":"greenland"}, + {"text":"google"}, + {"text":"john"}, + {"text":"mysql"}, + {"text":"potter"}, + {"text":"grammar"} + ], + "project_id":"project_id", + "parameters":{ + "truncate_input_tokens":100, + "return_options":{ + "inputs":true, + "top_n":5 + } + } + } + """)); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java new file mode 100644 index 0000000000000..8c95a01bc3230 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java @@ -0,0 +1,107 @@ +/* + * 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.inference.external.request.ibmwatsonx.rerank; + +import org.apache.http.HttpHeaders; +import org.apache.http.client.methods.HttpPost; +import org.elasticsearch.core.Strings; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxRerankRequest; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModelTests; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.xpack.inference.external.http.Utils.entityAsMap; +import static org.hamcrest.Matchers.aMapWithSize; +import static org.hamcrest.Matchers.endsWith; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +public class IbmWatsonxRerankRequestTests extends ESTestCase { + private static final String AUTH_HEADER_VALUE = "foo"; + + public void testCreateRequest() throws IOException { + var model = "model"; + var projectId = "project_id"; + URI uri = null; + try { + uri = new URI("http://abc.com"); + } catch (Exception ignored) {} + var apiVersion = "2023-05-04"; + var apiKey = "api_key"; + var query = "database"; + List input = List.of("greenland", "google", "john", "mysql", "potter", "grammar"); + + var request = createRequest(model, projectId, uri, apiVersion, apiKey, query, input); + var httpRequest = request.createHttpRequest(); + + assertThat(httpRequest.httpRequestBase(), instanceOf(HttpPost.class)); + var httpPost = (HttpPost) httpRequest.httpRequestBase(); + + assertThat(httpPost.getURI().toString(), endsWith(Strings.format("%s=%s", "version", apiVersion))); + assertThat(httpPost.getLastHeader(HttpHeaders.CONTENT_TYPE).getValue(), is(XContentType.JSON.mediaType())); + + var requestMap = entityAsMap(httpPost.getEntity().getContent()); + assertThat(requestMap, aMapWithSize(5)); + assertThat( + requestMap, + is( + + Map.of( + "project_id", + "project_id", + "model_id", + "model", + "inputs", + List.of( + Map.of("text", "greenland"), + Map.of("text", "google"), + Map.of("text", "john"), + Map.of("text", "mysql"), + Map.of("text", "potter"), + Map.of("text", "grammar") + ), + "query", + "database", + "parameters", + Map.of("return_options", Map.of("top_n", 2, "inputs", true), "truncate_input_tokens", 100) + ) + ) + ); + } + + public static IbmWatsonxRerankRequest createRequest( + String model, + String projectId, + URI uri, + String apiVersion, + String apiKey, + String query, + List input + ) { + var embeddingsModel = IbmWatsonxRerankModelTests.createModel(model, projectId, uri, apiVersion, apiKey); + + return new IbmWatsonxRerankWithoutAuthRequest(query, input, embeddingsModel); + } + + private static class IbmWatsonxRerankWithoutAuthRequest extends IbmWatsonxRerankRequest { + IbmWatsonxRerankWithoutAuthRequest(String query, List input, IbmWatsonxRerankModel model) { + super(query, input, model); + } + + @Override + public void decorateWithAuth(HttpPost httpPost) { + httpPost.setHeader(HttpHeaders.AUTHORIZATION, AUTH_HEADER_VALUE); + } + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java new file mode 100644 index 0000000000000..6b59f25896a48 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java @@ -0,0 +1,166 @@ +/* + * 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.inference.external.response.ibmwatsonx; + +import org.apache.http.HttpResponse; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.inference.results.RankedDocsResults; +import org.elasticsearch.xpack.inference.external.http.HttpResult; +import org.hamcrest.MatcherAssert; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; + +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; + +public class IbmWatsonxRankedResponseEntityTests extends ESTestCase { + + public void testResponseLiteral() throws IOException { + InferenceServiceResults parsedResults = IbmWatsonxRankedResponseEntity.fromResponse( + new HttpResult(mock(HttpResponse.class), responseLiteral.getBytes(StandardCharsets.UTF_8)) + ); + + MatcherAssert.assertThat(parsedResults, instanceOf(RankedDocsResults.class)); + List expected = responseLiteralDocs(); + for (int i = 0; i < ((RankedDocsResults) parsedResults).getRankedDocs().size(); i++) { + assertEquals(((RankedDocsResults) parsedResults).getRankedDocs().get(i).index(), expected.get(i).index()); + } + } + + public void testGeneratedResponse() throws IOException { + int numDocs = randomIntBetween(1, 10); + + List expected = new ArrayList<>(numDocs); + StringBuilder responseBuilder = new StringBuilder(); + + responseBuilder.append("{"); + responseBuilder.append("\"results\": ["); + List indices = linear(numDocs); + List scores = linearFloats(numDocs); + for (int i = 0; i < numDocs; i++) { + int index = indices.remove(randomInt(indices.size() - 1)); + + responseBuilder.append("{"); + responseBuilder.append("\"index\":").append(index).append(","); + responseBuilder.append("\"score\":").append(scores.get(i).toString()).append("}"); + expected.add(new RankedDocsResults.RankedDoc(index, scores.get(i), null)); + if (i < numDocs - 1) { + responseBuilder.append(","); + } + } + responseBuilder.append("]"); + + responseBuilder.append(randomIntBetween(1, 10)).append("}"); + + InferenceServiceResults parsedResults = IbmWatsonxRankedResponseEntity.fromResponse( + new HttpResult(mock(HttpResponse.class), responseBuilder.toString().getBytes(StandardCharsets.UTF_8)) + ); + MatcherAssert.assertThat(parsedResults, instanceOf(RankedDocsResults.class)); + for (int i = 0; i < ((RankedDocsResults) parsedResults).getRankedDocs().size(); i++) { + assertEquals(((RankedDocsResults) parsedResults).getRankedDocs().get(i).index(), expected.get(i).index()); + } + } + + private ArrayList responseLiteralDocs() { + var list = new ArrayList(); + + list.add(new RankedDocsResults.RankedDoc(2, 0.98005307F, null)); + list.add(new RankedDocsResults.RankedDoc(3, 0.27904198F, null)); + list.add(new RankedDocsResults.RankedDoc(0, 0.10194652F, null)); + return list; + } + + private final String responseLiteral = """ + { + "results": [ + { + "index": 2, + "score": 0.98005307 + }, + { + "index": 3, + "score": 0.27904198 + }, + { + "index": 0, + "score": 0.10194652 + } + ] + } + """; + + public void testResponseLiteralWithDocuments() throws IOException { + InferenceServiceResults parsedResults = IbmWatsonxRankedResponseEntity.fromResponse( + new HttpResult(mock(HttpResponse.class), responseLiteralWithDocuments.getBytes(StandardCharsets.UTF_8)) + ); + + MatcherAssert.assertThat(parsedResults, instanceOf(RankedDocsResults.class)); + MatcherAssert.assertThat(((RankedDocsResults) parsedResults).getRankedDocs(), is(responseLiteralDocsWithText)); + } + + private final String responseLiteralWithDocuments = """ + { + "results": [ + { + "input": { + "text": "Washington, D.C.." + }, + "index": 2, + "score": 0.98005307 + }, + { + "input": { + "text": "Capital punishment has existed in the United States since before the United States was a country. " + }, + "index": 3, + "score": 0.27904198 + }, + { + "input": { + "text": "Carson City is the capital city of the American state of Nevada." + }, + "index": 0, + "score": 0.10194652 + } + ] + } + """; + + private final List responseLiteralDocsWithText = List.of( + new RankedDocsResults.RankedDoc(2, 0.98005307F, "Washington, D.C.."), + new RankedDocsResults.RankedDoc( + 3, + 0.27904198F, + "Capital punishment has existed in the United States since before the United States was a country. " + ), + new RankedDocsResults.RankedDoc(0, 0.10194652F, "Carson City is the capital city of the American state of Nevada.") + ); + + private ArrayList linear(int n) { + ArrayList list = new ArrayList<>(); + for (int i = 0; i <= n; i++) { + list.add(i); + } + return list; + } + + // creates a list of doubles of monotonically decreasing magnitude + private ArrayList linearFloats(int n) { + ArrayList list = new ArrayList<>(); + float startValue = 1.0f; + float decrement = startValue / n + 1; + for (int i = 0; i <= n; i++) { + list.add(startValue - (i * decrement)); + } + return list; + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java index ff99101fc4ee5..99b7b3868b7f4 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java @@ -50,6 +50,7 @@ import org.elasticsearch.xpack.inference.services.ServiceFields; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModelTests; +import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModelTests; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; @@ -154,6 +155,42 @@ public void testParseRequestConfig_CreatesAIbmWatsonxEmbeddingsModel() throws IO } } + public void testParseRequestConfig_CreatesAIbmWatsonxRerankModel() throws IOException { + try (var service = createIbmWatsonxService()) { + ActionListener modelListener = ActionListener.wrap(model -> { + assertThat(model, instanceOf(IbmWatsonxRerankModel.class)); + + var rerankModel = (IbmWatsonxRerankModel) model; + assertThat(rerankModel.getServiceSettings().modelId(), is(modelId)); + assertThat(rerankModel.getServiceSettings().projectId(), is(projectId)); + assertThat(rerankModel.getServiceSettings().apiVersion(), is(apiVersion)); + assertThat(rerankModel.getSecretSettings().apiKey().toString(), is(apiKey)); + }, e -> fail("Model parsing should have succeeded, but failed: " + e.getMessage())); + + service.parseRequestConfig( + "id", + TaskType.RERANK, + getRequestConfigMap( + new HashMap<>( + Map.of( + ServiceFields.MODEL_ID, + modelId, + IbmWatsonxServiceFields.PROJECT_ID, + projectId, + ServiceFields.URL, + url, + IbmWatsonxServiceFields.API_VERSION, + apiVersion + ) + ), + new HashMap<>(Map.of()), + getSecretSettingsMap(apiKey) + ), + modelListener + ); + } + } + public void testParseRequestConfig_CreatesAIbmWatsonxEmbeddingsModelWhenChunkingSettingsProvided() throws IOException { try (var service = createIbmWatsonxService()) { ActionListener modelListener = ActionListener.wrap(model -> { diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java new file mode 100644 index 0000000000000..0138952c11e07 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java @@ -0,0 +1,28 @@ +/* + * 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.inference.services.ibmwatsonx.rerank; + +import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.inference.services.settings.DefaultSecretSettings; + +import java.net.URI; + +public class IbmWatsonxRerankModelTests extends ESTestCase { + public static IbmWatsonxRerankModel createModel(String model, String projectId, URI uri, String apiVersion, String apiKey) { + return new IbmWatsonxRerankModel( + "id", + TaskType.RERANK, + "service", + new IbmWatsonxRerankServiceSettings(uri, apiVersion, model, projectId, null), + new IbmWatsonxRerankTaskSettings(2, true, 100), + new DefaultSecretSettings(new SecureString(apiKey.toCharArray())) + ); + } +} From daddb623f95aef543db30f79b3e635fe094fd369 Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Fri, 31 Jan 2025 10:30:55 +0100 Subject: [PATCH 24/24] revert rebasing update --- docs/changelog/117176.yaml | 5 - docs/changelog/121109.yaml | 6 - docs/changelog/121325.yaml | 5 - muted-tests.yml | 12 -- rest-api-spec/build.gradle | 1 - .../rest-api-spec/test/mapping/10_copy_to.yml | 41 ---- .../org/elasticsearch/TransportVersions.java | 1 - .../index/mapper/DocumentParserContext.java | 8 - .../index/mapper/MapperFeatures.java | 4 +- .../DefaultMappingParametersHandler.java | 7 +- .../test/rest/ESRestTestCase.java | 12 +- .../xpack/esql/action/EsqlCapabilities.java | 4 +- .../InferenceNamedWriteablesProvider.java | 13 -- .../ibmwatsonx/IbmWatsonxActionCreator.java | 14 +- .../ibmwatsonx/IbmWatsonxActionVisitor.java | 3 - .../IbmWatsonxRerankRequestManager.java | 72 ------- .../ibmwatsonx/IbmWatsonxResponseHandler.java | 1 + .../ibmwatsonx/IbmWatsonxRerankRequest.java | 110 ---------- .../IbmWatsonxRerankRequestEntity.java | 77 ------- .../request/ibmwatsonx/IbmWatsonxUtils.java | 1 - .../IbmWatsonxRankedResponseEntity.java | 157 -------------- .../services/ibmwatsonx/IbmWatsonxModel.java | 7 - .../ibmwatsonx/IbmWatsonxService.java | 10 - .../rerank/IbmWatsonxRerankModel.java | 121 ----------- .../IbmWatsonxRerankServiceSettings.java | 190 ----------------- .../rerank/IbmWatsonxRerankTaskSettings.java | 192 ------------------ .../IbmWatsonxRerankRequestEntityTests.java | 60 ------ .../rerank/IbmWatsonxRerankRequestTests.java | 107 ---------- .../IbmWatsonxRankedResponseEntityTests.java | 166 --------------- .../ibmwatsonx/IbmWatsonxServiceTests.java | 37 ---- .../rerank/IbmWatsonxRerankModelTests.java | 28 --- ...ReindexDataStreamIndexTransportAction.java | 3 +- 32 files changed, 16 insertions(+), 1459 deletions(-) delete mode 100644 docs/changelog/117176.yaml delete mode 100644 docs/changelog/121109.yaml delete mode 100644 docs/changelog/121325.yaml delete mode 100644 rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java delete mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java delete mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java delete mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java delete mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java diff --git a/docs/changelog/117176.yaml b/docs/changelog/117176.yaml deleted file mode 100644 index 26e0d3635bc9e..0000000000000 --- a/docs/changelog/117176.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 117176 -summary: Integrate IBM watsonx to Inference API for re-ranking task -area: Experiences -type: enhancement -issues: [] diff --git a/docs/changelog/121109.yaml b/docs/changelog/121109.yaml deleted file mode 100644 index 6492eccbf975a..0000000000000 --- a/docs/changelog/121109.yaml +++ /dev/null @@ -1,6 +0,0 @@ -pr: 121109 -summary: Fix propagation of dynamic mapping parameter when applying `copy_to` -area: Mapping -type: bug -issues: - - 113049 diff --git a/docs/changelog/121325.yaml b/docs/changelog/121325.yaml deleted file mode 100644 index 9a9edc67d19fa..0000000000000 --- a/docs/changelog/121325.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 121325 -summary: '`ReindexDataStreamIndex` bug in assertion caused by reference equality' -area: Data streams -type: bug -issues: [] diff --git a/muted-tests.yml b/muted-tests.yml index f6413da66de9d..e8ef5992d3e95 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -379,18 +379,6 @@ tests: - class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT method: test {yaml=reference/rest-api/common-options/line_125} issue: https://github.com/elastic/elasticsearch/issues/121338 -- class: org.elasticsearch.smoketest.DocsClientYamlTestSuiteIT - method: test {yaml=reference/snapshot-restore/apis/get-snapshot-api/line_751} - issue: https://github.com/elastic/elasticsearch/issues/121345 -- class: org.elasticsearch.xpack.security.profile.ProfileIntegTests - method: testHasPrivileges - issue: https://github.com/elastic/elasticsearch/issues/121346 -- class: org.elasticsearch.xpack.security.profile.ProfileIntegTests - method: testActivateProfile - issue: https://github.com/elastic/elasticsearch/issues/121151 -- class: org.elasticsearch.smoketest.SmokeTestMultiNodeClientYamlTestSuiteIT - method: test {yaml=search.vectors/42_knn_search_int4_flat/Vector similarity with filter only} - issue: https://github.com/elastic/elasticsearch/issues/121350 # Examples: # diff --git a/rest-api-spec/build.gradle b/rest-api-spec/build.gradle index 67f0b5a0714a9..0069e34fe949e 100644 --- a/rest-api-spec/build.gradle +++ b/rest-api-spec/build.gradle @@ -73,5 +73,4 @@ tasks.named("yamlRestCompatTestTransform").configure ({ task -> task.skipTest("index/92_metrics_auto_subobjects/Root without subobjects with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0") task.skipTest("index/91_metrics_no_subobjects/Metrics object indexing with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0") task.skipTest("index/91_metrics_no_subobjects/Root without subobjects with synthetic source", "_source.mode mapping attribute is no-op since 9.0.0") - task.skipTest("indices.create/20_synthetic_source/synthetic_source with copy_to inside nested object", "temporary until backported") }) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml deleted file mode 100644 index 58f09ec71ad61..0000000000000 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/mapping/10_copy_to.yml +++ /dev/null @@ -1,41 +0,0 @@ ---- -copy_to from object with dynamic strict to dynamic field: - - requires: - cluster_features: ["mapper.copy_to.dynamic_handling"] - reason: requires a fix - - - do: - indices.create: - index: test - body: - mappings: - properties: - one: - dynamic: strict - properties: - k: - type: keyword - copy_to: two.k - - - do: - index: - index: test - id: 1 - refresh: true - body: - one: - k: "hey" - - - do: - search: - index: test - body: - docvalue_fields: [ "two.k.keyword" ] - - - match: - hits.hits.0._source: - one: - k: "hey" - - match: - hits.hits.0.fields: - two.k.keyword: [ "hey" ] diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 1144f94795713..8f747a59ae5e0 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -171,7 +171,6 @@ static TransportVersion def(int id) { public static final TransportVersion LINEAR_RETRIEVER_SUPPORT = def(8_837_00_0); public static final TransportVersion TIMEOUT_GET_PARAM_FOR_RESOLVE_CLUSTER = def(8_838_00_0); public static final TransportVersion INFERENCE_REQUEST_ADAPTIVE_RATE_LIMITING = def(8_839_00_0); - public static final TransportVersion ML_INFERENCE_IBM_WATSONX_RERANK_ADDED = def(8_840_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java index 127ec05b25e63..ba9e902fee5d9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParserContext.java @@ -50,13 +50,6 @@ private Wrapper(ObjectMapper parent, DocumentParserContext in) { this.in = in; } - // Used to create a copy_to context. - // It is important to reset `dynamic` here since it is possible that we copy into a completely different object. - private Wrapper(RootObjectMapper root, DocumentParserContext in) { - super(root, ObjectMapper.Dynamic.getRootDynamic(in.mappingLookup()), in); - this.in = in; - } - @Override public Iterable nonRootDocuments() { return in.nonRootDocuments(); @@ -718,7 +711,6 @@ in synthetic _source (to be consistent with stored _source). ContentPath path = new ContentPath(); XContentParser parser = DotExpandingXContentParser.expandDots(new CopyToParser(copyToField, parser()), path); - return new Wrapper(root(), this) { @Override public ContentPath path() { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java index 7567fae7d73e6..0935e219fb5c0 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperFeatures.java @@ -30,7 +30,6 @@ public class MapperFeatures implements FeatureSpecification { public static final NodeFeature META_FETCH_FIELDS_ERROR_CODE_CHANGED = new NodeFeature("meta_fetch_fields_error_code_changed"); public static final NodeFeature SPARSE_VECTOR_STORE_SUPPORT = new NodeFeature("mapper.sparse_vector.store_support"); public static final NodeFeature SORT_FIELDS_CHECK_FOR_NESTED_OBJECT_FIX = new NodeFeature("mapper.nested.sorting_fields_check_fix"); - public static final NodeFeature DYNAMIC_HANDLING_IN_COPY_TO = new NodeFeature("mapper.copy_to.dynamic_handling"); @Override public Set getTestFeatures() { @@ -46,9 +45,8 @@ public Set getTestFeatures() { CONSTANT_KEYWORD_SYNTHETIC_SOURCE_WRITE_FIX, META_FETCH_FIELDS_ERROR_CODE_CHANGED, SPARSE_VECTOR_STORE_SUPPORT, - COUNTED_KEYWORD_SYNTHETIC_SOURCE_NATIVE_SUPPORT, SORT_FIELDS_CHECK_FOR_NESTED_OBJECT_FIX, - DYNAMIC_HANDLING_IN_COPY_TO, + COUNTED_KEYWORD_SYNTHETIC_SOURCE_NATIVE_SUPPORT, SourceFieldMapper.SYNTHETIC_RECOVERY_SOURCE, ObjectMapper.SUBOBJECTS_FALSE_MAPPING_UPDATE_FIX ); diff --git a/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java b/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java index 04cb9467270d4..b639108ea6ad2 100644 --- a/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java +++ b/test/framework/src/main/java/org/elasticsearch/logsdb/datageneration/datasource/DefaultMappingParametersHandler.java @@ -11,6 +11,7 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.ObjectMapper; +import org.elasticsearch.logsdb.datageneration.fields.DynamicMapping; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; @@ -49,7 +50,11 @@ private Supplier> keywordMapping( // We only add copy_to to keywords because we get into trouble with numeric fields that are copied to dynamic fields. // If first copied value is numeric, dynamic field is created with numeric field type and then copy of text values fail. // Actual value being copied does not influence the core logic of copy_to anyway. - if (ESTestCase.randomDouble() <= 0.05) { + // + // TODO + // We don't use copy_to on fields that are inside an object with dynamic: strict + // because we'll hit https://github.com/elastic/elasticsearch/issues/113049. + if (request.dynamicMapping() != DynamicMapping.FORBIDDEN && ESTestCase.randomDouble() <= 0.05) { var options = request.eligibleCopyToFields() .stream() .filter(f -> f.equals(request.fieldName()) == false) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 40b2bae2fc4b1..5738ab60f47eb 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -169,16 +169,14 @@ public abstract class ESRestTestCase extends ESTestCase { private static final String EXPECTED_ROLLUP_WARNING_MESSAGE = "The rollup functionality will be removed in Elasticsearch 10.0. See docs for more information."; public static final RequestOptions.Builder ROLLUP_REQUESTS_OPTIONS = RequestOptions.DEFAULT.toBuilder().setWarningsHandler(warnings -> { + // Either no warning, because of bwc integration test OR + // the expected warning, because on current version if (warnings.isEmpty()) { return false; - } else { - // Sometimes multiple rollup deprecation warnings. Transport actions can be invoked multiple time on different nodes. - for (String warning : warnings) { - if (EXPECTED_ROLLUP_WARNING_MESSAGE.equals(warning) == false) { - return true; - } - } + } else if (warnings.size() == 1 && EXPECTED_ROLLUP_WARNING_MESSAGE.equals(warnings.get(0))) { return false; + } else { + return true; } }); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 25518220e308b..20de3e443107d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -20,8 +20,6 @@ import java.util.Locale; import java.util.Set; -import static org.elasticsearch.xpack.esql.core.plugin.EsqlCorePlugin.AGGREGATE_METRIC_DOUBLE_FEATURE_FLAG; - /** * A {@link Set} of "capabilities" supported by the {@link RestEsqlQueryAction} * and {@link RestEsqlAsyncQueryAction} APIs. These are exposed over the @@ -786,7 +784,7 @@ public enum Cap { /** * Support for aggregate_metric_double type */ - AGGREGATE_METRIC_DOUBLE(AGGREGATE_METRIC_DOUBLE_FEATURE_FLAG.isEnabled()); + AGGREGATE_METRIC_DOUBLE; private final boolean enabled; diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java index e8dc763116707..6fc9870034018 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java @@ -75,8 +75,6 @@ import org.elasticsearch.xpack.inference.services.huggingface.HuggingFaceServiceSettings; import org.elasticsearch.xpack.inference.services.huggingface.elser.HuggingFaceElserServiceSettings; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsServiceSettings; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankServiceSettings; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; import org.elasticsearch.xpack.inference.services.jinaai.JinaAIServiceSettings; import org.elasticsearch.xpack.inference.services.jinaai.embeddings.JinaAIEmbeddingsServiceSettings; import org.elasticsearch.xpack.inference.services.jinaai.embeddings.JinaAIEmbeddingsTaskSettings; @@ -366,17 +364,6 @@ private static void addIbmWatsonxNamedWritables(List namedWriteables) { diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java index 6b1097256e97f..7cad7c42bdcf1 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionCreator.java @@ -12,11 +12,9 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.SenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.sender.IbmWatsonxEmbeddingsRequestManager; -import org.elasticsearch.xpack.inference.external.http.sender.IbmWatsonxRerankRequestManager; import org.elasticsearch.xpack.inference.external.http.sender.Sender; import org.elasticsearch.xpack.inference.services.ServiceComponents; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import java.util.Map; import java.util.Objects; @@ -24,6 +22,7 @@ import static org.elasticsearch.xpack.inference.external.action.ActionUtils.constructFailedToSendRequestMessage; public class IbmWatsonxActionCreator implements IbmWatsonxActionVisitor { + private final Sender sender; private final ServiceComponents serviceComponents; @@ -42,17 +41,6 @@ public ExecutableAction create(IbmWatsonxEmbeddingsModel model, Map taskSettings) { - var overriddenModel = IbmWatsonxRerankModel.of(model, taskSettings); - var requestCreator = IbmWatsonxRerankRequestManager.of(overriddenModel, serviceComponents.threadPool()); - var failedToSendRequestErrorMessage = constructFailedToSendRequestMessage( - overriddenModel.getServiceSettings().uri(), - "Ibm Watsonx rerank" - ); - return new SenderExecutableAction(sender, requestCreator, failedToSendRequestErrorMessage); - } - protected IbmWatsonxEmbeddingsRequestManager getEmbeddingsRequestManager( IbmWatsonxEmbeddingsModel model, Truncator truncator, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java index 474533040e0c3..0a13ec2fb4645 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/ibmwatsonx/IbmWatsonxActionVisitor.java @@ -9,12 +9,9 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import java.util.Map; public interface IbmWatsonxActionVisitor { ExecutableAction create(IbmWatsonxEmbeddingsModel model, Map taskSettings); - - ExecutableAction create(IbmWatsonxRerankModel model, Map taskSettings); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java deleted file mode 100644 index f503771510e72..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/IbmWatsonxRerankRequestManager.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.inference.external.http.sender; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.inference.InferenceServiceResults; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xpack.inference.external.http.retry.RequestSender; -import org.elasticsearch.xpack.inference.external.http.retry.ResponseHandler; -import org.elasticsearch.xpack.inference.external.ibmwatsonx.IbmWatsonxResponseHandler; -import org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxRerankRequest; -import org.elasticsearch.xpack.inference.external.response.ibmwatsonx.IbmWatsonxRankedResponseEntity; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; - -import java.util.List; -import java.util.Objects; -import java.util.function.Supplier; - -public class IbmWatsonxRerankRequestManager extends IbmWatsonxRequestManager { - private static final Logger logger = LogManager.getLogger(IbmWatsonxRerankRequestManager.class); - private static final ResponseHandler HANDLER = createIbmWatsonxResponseHandler(); - - private static ResponseHandler createIbmWatsonxResponseHandler() { - return new IbmWatsonxResponseHandler( - "ibm watsonx rerank", - (request, response) -> IbmWatsonxRankedResponseEntity.fromResponse(response) - ); - } - - public static IbmWatsonxRerankRequestManager of(IbmWatsonxRerankModel model, ThreadPool threadPool) { - return new IbmWatsonxRerankRequestManager(Objects.requireNonNull(model), Objects.requireNonNull(threadPool)); - } - - private final IbmWatsonxRerankModel model; - - public IbmWatsonxRerankRequestManager(IbmWatsonxRerankModel model, ThreadPool threadPool) { - super(threadPool, model); - this.model = model; - } - - @Override - public void execute( - InferenceInputs inferenceInputs, - RequestSender requestSender, - Supplier hasRequestCompletedFunction, - ActionListener listener - ) { - var rerankInput = QueryAndDocsInputs.of(inferenceInputs); - - execute( - new ExecutableInferenceRequest( - requestSender, - logger, - getRerankRequest(rerankInput.getQuery(), rerankInput.getChunks(), model), - HANDLER, - hasRequestCompletedFunction, - listener - ) - ); - } - - protected IbmWatsonxRerankRequest getRerankRequest(String query, List chunks, IbmWatsonxRerankModel model) { - return new IbmWatsonxRerankRequest(query, chunks, model); - } -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java index 1f28a8cd61026..6d1d3fb2a4f91 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/ibmwatsonx/IbmWatsonxResponseHandler.java @@ -17,6 +17,7 @@ import static org.elasticsearch.core.Strings.format; public class IbmWatsonxResponseHandler extends BaseResponseHandler { + public IbmWatsonxResponseHandler(String requestType, ResponseParser parseFunction) { super(requestType, parseFunction, IbmWatsonxErrorResponseEntity::fromResponse); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java deleted file mode 100644 index cfc1f367be45c..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequest.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.inference.external.request.ibmwatsonx; - -import org.apache.http.HttpHeaders; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.entity.ByteArrayEntity; -import org.elasticsearch.common.Strings; -import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.inference.external.request.HttpRequest; -import org.elasticsearch.xpack.inference.external.request.Request; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; - -import java.net.URI; -import java.net.URISyntaxException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Objects; - -public class IbmWatsonxRerankRequest implements IbmWatsonxRequest { - - private final String query; - private final List input; - private final IbmWatsonxRerankTaskSettings taskSettings; - private final IbmWatsonxRerankModel model; - - public IbmWatsonxRerankRequest(String query, List input, IbmWatsonxRerankModel model) { - Objects.requireNonNull(model); - - this.input = Objects.requireNonNull(input); - this.query = Objects.requireNonNull(query); - taskSettings = model.getTaskSettings(); - this.model = model; - } - - @Override - public HttpRequest createHttpRequest() { - URI uri; - - try { - uri = new URI(model.uri().toString()); - } catch (URISyntaxException ex) { - throw new IllegalArgumentException("cannot parse URI patter"); - } - - HttpPost httpPost = new HttpPost(uri); - - ByteArrayEntity byteEntity = new ByteArrayEntity( - Strings.toString( - new IbmWatsonxRerankRequestEntity( - query, - input, - taskSettings, - model.getServiceSettings().modelId(), - model.getServiceSettings().projectId() - ) - ).getBytes(StandardCharsets.UTF_8) - ); - - httpPost.setEntity(byteEntity); - httpPost.setHeader(HttpHeaders.CONTENT_TYPE, XContentType.JSON.mediaType()); - - decorateWithAuth(httpPost); - - return new HttpRequest(httpPost, getInferenceEntityId()); - } - - public void decorateWithAuth(HttpPost httpPost) { - IbmWatsonxRequest.decorateWithBearerToken(httpPost, model.getSecretSettings(), model.getInferenceEntityId()); - } - - @Override - public String getInferenceEntityId() { - return model.getInferenceEntityId(); - } - - @Override - public URI getURI() { - return model.uri(); - } - - @Override - public Request truncate() { - return this; - } - - public String getQuery() { - return query; - } - - public List getInput() { - return input; - } - - public IbmWatsonxRerankModel getModel() { - return model; - } - - @Override - public boolean[] getTruncationInfo() { - return null; - } - -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java deleted file mode 100644 index 36e5951ebdc15..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxRerankRequestEntity.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.inference.external.request.ibmwatsonx; - -import org.elasticsearch.xcontent.ToXContentObject; -import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; - -import java.io.IOException; -import java.util.List; -import java.util.Objects; - -public record IbmWatsonxRerankRequestEntity( - String query, - List inputs, - IbmWatsonxRerankTaskSettings taskSettings, - String modelId, - String projectId -) implements ToXContentObject { - - private static final String INPUTS_FIELD = "inputs"; - private static final String QUERY_FIELD = "query"; - private static final String MODEL_ID_FIELD = "model_id"; - private static final String PROJECT_ID_FIELD = "project_id"; - - public IbmWatsonxRerankRequestEntity { - Objects.requireNonNull(query); - Objects.requireNonNull(inputs); - Objects.requireNonNull(modelId); - Objects.requireNonNull(projectId); - Objects.requireNonNull(taskSettings); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - - builder.field(MODEL_ID_FIELD, modelId); - builder.field(QUERY_FIELD, query); - builder.startArray(INPUTS_FIELD); - for (String input : inputs) { - builder.startObject(); - builder.field("text", input); - builder.endObject(); - } - builder.endArray(); - builder.field(PROJECT_ID_FIELD, projectId); - - builder.startObject("parameters"); - { - if (taskSettings.getTruncateInputTokens() != null) { - builder.field("truncate_input_tokens", taskSettings.getTruncateInputTokens()); - } - - builder.startObject("return_options"); - { - if (taskSettings.getDoesReturnDocuments() != null) { - builder.field("inputs", taskSettings.getDoesReturnDocuments()); - } - if (taskSettings.getTopNDocumentsOnly() != null) { - builder.field("top_n", taskSettings.getTopNDocumentsOnly()); - } - } - builder.endObject(); - } - builder.endObject(); - - builder.endObject(); - - return builder; - } -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java index 91679288e5ae3..a506a33385dfb 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/IbmWatsonxUtils.java @@ -13,7 +13,6 @@ public class IbmWatsonxUtils { public static final String V1 = "v1"; public static final String TEXT = "text"; public static final String EMBEDDINGS = "embeddings"; - public static final String RERANKS = "reranks"; private IbmWatsonxUtils() {} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java deleted file mode 100644 index 05f369bd8961e..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntity.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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. - * - * this file was contributed to by a generative AI - */ - -package org.elasticsearch.xpack.inference.external.response.ibmwatsonx; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; -import org.elasticsearch.inference.InferenceServiceResults; -import org.elasticsearch.xcontent.XContentFactory; -import org.elasticsearch.xcontent.XContentParser; -import org.elasticsearch.xcontent.XContentParserConfiguration; -import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.core.inference.results.RankedDocsResults; -import org.elasticsearch.xpack.inference.external.http.HttpResult; - -import java.io.IOException; - -import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; -import static org.elasticsearch.common.xcontent.XContentParserUtils.parseList; -import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownField; -import static org.elasticsearch.common.xcontent.XContentParserUtils.throwUnknownToken; -import static org.elasticsearch.xpack.inference.external.response.XContentUtils.moveToFirstToken; -import static org.elasticsearch.xpack.inference.external.response.XContentUtils.positionParserAtTokenAfterField; - -public class IbmWatsonxRankedResponseEntity { - - private static final Logger logger = LogManager.getLogger(IbmWatsonxRankedResponseEntity.class); - - /** - * Parses the Ibm Watsonx ranked response. - * - * For a request like: - * "model": "rerank-english-v2.0", - * "query": "database", - * "return_documents": true, - * "top_n": 3, - * "input": ["greenland", "google","john", "mysql","potter", "grammar"] - *

- * The response will look like (without whitespace): - * { - * "rerank": [ - * { - * "index": 3, - * "relevance_score": 0.7989932 - * }, - * { - * "index": 5, - * "relevance_score": 0.61281824 - * }, - * { - * "index": 1, - * "relevance_score": 0.5762553 - * }, - * { - * "index": 4, - * "relevance_score": 0.47395563 - * }, - * { - * "index": 0, - * "relevance_score": 0.4338926 - * }, - * { - * "index": 2, - * "relevance_score": 0.42638257 - * } - * ], - * } - * - * @param response the http response from ibm watsonx - * @return the parsed response - * @throws IOException if there is an error parsing the response - */ - public static InferenceServiceResults fromResponse(HttpResult response) throws IOException { - var parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler(LoggingDeprecationHandler.INSTANCE); - - try (XContentParser jsonParser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, response.body())) { - moveToFirstToken(jsonParser); - - XContentParser.Token token = jsonParser.currentToken(); - ensureExpectedToken(XContentParser.Token.START_OBJECT, token, jsonParser); - - positionParserAtTokenAfterField(jsonParser, "results", FAILED_TO_FIND_FIELD_TEMPLATE); // TODO error message - - token = jsonParser.currentToken(); - if (token == XContentParser.Token.START_ARRAY) { - return new RankedDocsResults(parseList(jsonParser, IbmWatsonxRankedResponseEntity::parseRankedDocObject)); - } else { - throwUnknownToken(token, jsonParser); - } - - // This should never be reached. The above code should either return successfully or hit the throwUnknownToken - // or throw a parsing exception - throw new IllegalStateException("Reached an invalid state while parsing the Watsonx response"); - } - } - - private static RankedDocsResults.RankedDoc parseRankedDocObject(XContentParser parser) throws IOException { - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); - int index = -1; - float score = -1; - String documentText = null; - parser.nextToken(); - while (parser.currentToken() != XContentParser.Token.END_OBJECT) { - if (parser.currentToken() == XContentParser.Token.FIELD_NAME) { - switch (parser.currentName()) { - case "index": - parser.nextToken(); // move to VALUE_NUMBER - index = parser.intValue(); - parser.nextToken(); // move to next FIELD_NAME or END_OBJECT - break; - case "score": - parser.nextToken(); // move to VALUE_NUMBER - score = parser.floatValue(); - parser.nextToken(); // move to next FIELD_NAME or END_OBJECT - break; - case "input": - parser.nextToken(); // move to START_OBJECT; document text is wrapped in an object - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); - do { - if (parser.currentToken() == XContentParser.Token.FIELD_NAME && parser.currentName().equals("text")) { - parser.nextToken(); // move to VALUE_STRING - documentText = parser.text(); - } - } while (parser.nextToken() != XContentParser.Token.END_OBJECT); - parser.nextToken();// move past END_OBJECT - // parser should now be at the next FIELD_NAME or END_OBJECT - break; - default: - throwUnknownField(parser.currentName(), parser); - } - } else { - parser.nextToken(); - } - } - - if (index == -1) { - logger.warn("Failed to find required field [index] in Watsonx rerank response"); - } - if (score == -1) { - logger.warn("Failed to find required field [relevance_score] in Watsonx rerank response"); - } - // documentText may or may not be present depending on the request parameter - - return new RankedDocsResults.RankedDoc(index, score, documentText); - } - - private IbmWatsonxRankedResponseEntity() {} - - static String FAILED_TO_FIND_FIELD_TEMPLATE = "Failed to find required field [%s] in Watsonx rerank response"; -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java index 09706f70e3684..4f0b425cdaa51 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxModel.java @@ -12,7 +12,6 @@ import org.elasticsearch.inference.ModelConfigurations; import org.elasticsearch.inference.ModelSecrets; import org.elasticsearch.inference.ServiceSettings; -import org.elasticsearch.inference.TaskSettings; import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.ibmwatsonx.IbmWatsonxActionVisitor; @@ -39,12 +38,6 @@ public IbmWatsonxModel(IbmWatsonxModel model, ServiceSettings serviceSettings) { rateLimitServiceSettings = model.rateLimitServiceSettings(); } - public IbmWatsonxModel(IbmWatsonxModel model, TaskSettings taskSettings) { - super(model, taskSettings); - - rateLimitServiceSettings = model.rateLimitServiceSettings(); - } - public abstract ExecutableAction accept(IbmWatsonxActionVisitor creator, Map taskSettings, InputType inputType); public IbmWatsonxRateLimitServiceSettings rateLimitServiceSettings() { diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java index 3fa423c2dae19..477225f00d22b 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java @@ -41,7 +41,6 @@ import org.elasticsearch.xpack.inference.services.ServiceUtils; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsServiceSettings; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import org.elasticsearch.xpack.inference.services.validation.ModelValidatorBuilder; import java.util.EnumSet; @@ -139,15 +138,6 @@ private static IbmWatsonxModel createModel( secretSettings, context ); - case RERANK -> new IbmWatsonxRerankModel( - inferenceEntityId, - taskType, - NAME, - serviceSettings, - taskSettings, - secretSettings, - context - ); default -> throw new ElasticsearchStatusException(failureMessage, RestStatus.BAD_REQUEST); }; } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java deleted file mode 100644 index cb4c509d88c2b..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModel.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.inference.services.ibmwatsonx.rerank; - -import org.apache.http.client.utils.URIBuilder; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.inference.InputType; -import org.elasticsearch.inference.ModelConfigurations; -import org.elasticsearch.inference.ModelSecrets; -import org.elasticsearch.inference.TaskType; -import org.elasticsearch.xpack.inference.external.action.ExecutableAction; -import org.elasticsearch.xpack.inference.external.action.ibmwatsonx.IbmWatsonxActionVisitor; -import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxModel; -import org.elasticsearch.xpack.inference.services.settings.DefaultSecretSettings; - -import java.net.URI; -import java.net.URISyntaxException; -import java.util.Map; - -import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.ML; -import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.RERANKS; -import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.TEXT; -import static org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxUtils.V1; - -public class IbmWatsonxRerankModel extends IbmWatsonxModel { - public static IbmWatsonxRerankModel of(IbmWatsonxRerankModel model, Map taskSettings) { - var requestTaskSettings = IbmWatsonxRerankTaskSettings.fromMap(taskSettings); - return new IbmWatsonxRerankModel(model, IbmWatsonxRerankTaskSettings.of(model.getTaskSettings(), requestTaskSettings)); - } - - public IbmWatsonxRerankModel( - String modelId, - TaskType taskType, - String service, - Map serviceSettings, - Map taskSettings, - @Nullable Map secrets, - ConfigurationParseContext context - ) { - this( - modelId, - taskType, - service, - IbmWatsonxRerankServiceSettings.fromMap(serviceSettings, context), - IbmWatsonxRerankTaskSettings.fromMap(taskSettings), - DefaultSecretSettings.fromMap(secrets) - ); - } - - // should only be used for testing - IbmWatsonxRerankModel( - String modelId, - TaskType taskType, - String service, - IbmWatsonxRerankServiceSettings serviceSettings, - IbmWatsonxRerankTaskSettings taskSettings, - @Nullable DefaultSecretSettings secretSettings - ) { - super( - new ModelConfigurations(modelId, taskType, service, serviceSettings, taskSettings), - new ModelSecrets(secretSettings), - serviceSettings - ); - } - - private IbmWatsonxRerankModel(IbmWatsonxRerankModel model, IbmWatsonxRerankTaskSettings taskSettings) { - super(model, taskSettings); - } - - @Override - public IbmWatsonxRerankServiceSettings getServiceSettings() { - return (IbmWatsonxRerankServiceSettings) super.getServiceSettings(); - } - - @Override - public IbmWatsonxRerankTaskSettings getTaskSettings() { - return (IbmWatsonxRerankTaskSettings) super.getTaskSettings(); - } - - @Override - public DefaultSecretSettings getSecretSettings() { - return (DefaultSecretSettings) super.getSecretSettings(); - } - - public URI uri() { - URI uri; - try { - uri = buildUri(this.getServiceSettings().uri().toString(), this.getServiceSettings().apiVersion()); - } catch (URISyntaxException e) { - throw new RuntimeException(e); - } - - return uri; - } - - /** - * Accepts a visitor to create an executable action. The returned action will not return documents in the response. - * @param visitor _ - * @param taskSettings _ - * @param inputType ignored for rerank task - * @return the rerank action - */ - @Override - public ExecutableAction accept(IbmWatsonxActionVisitor visitor, Map taskSettings, InputType inputType) { - return visitor.create(this, taskSettings); - } - - public static URI buildUri(String uri, String apiVersion) throws URISyntaxException { - return new URIBuilder().setScheme("https") - .setHost(uri) - .setPathSegments(ML, V1, TEXT, RERANKS) - .setParameter("version", apiVersion) - .build(); - } -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java deleted file mode 100644 index 969622f9ba54f..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankServiceSettings.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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.inference.services.ibmwatsonx.rerank; - -import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; -import org.elasticsearch.common.ValidationException; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.inference.ModelConfigurations; -import org.elasticsearch.inference.ServiceSettings; -import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxRateLimitServiceSettings; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxService; -import org.elasticsearch.xpack.inference.services.settings.FilteredXContentObject; -import org.elasticsearch.xpack.inference.services.settings.RateLimitSettings; - -import java.io.IOException; -import java.net.URI; -import java.util.Map; -import java.util.Objects; - -import static org.elasticsearch.xpack.inference.services.ServiceFields.MODEL_ID; -import static org.elasticsearch.xpack.inference.services.ServiceFields.URL; -import static org.elasticsearch.xpack.inference.services.ServiceUtils.convertToUri; -import static org.elasticsearch.xpack.inference.services.ServiceUtils.createUri; -import static org.elasticsearch.xpack.inference.services.ServiceUtils.extractRequiredString; -import static org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxServiceFields.API_VERSION; -import static org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxServiceFields.PROJECT_ID; - -public class IbmWatsonxRerankServiceSettings extends FilteredXContentObject implements ServiceSettings, IbmWatsonxRateLimitServiceSettings { - public static final String NAME = "ibm_watsonx_rerank_service_settings"; - - /** - * Rate limits are defined at - * Watson Machine Learning plans. - * For Lite plan, you've 120 requests per minute. - */ - private static final RateLimitSettings DEFAULT_RATE_LIMIT_SETTINGS = new RateLimitSettings(120); - - public static IbmWatsonxRerankServiceSettings fromMap(Map map, ConfigurationParseContext context) { - ValidationException validationException = new ValidationException(); - - String url = extractRequiredString(map, URL, ModelConfigurations.SERVICE_SETTINGS, validationException); - URI uri = convertToUri(url, URL, ModelConfigurations.SERVICE_SETTINGS, validationException); - String apiVersion = extractRequiredString(map, API_VERSION, ModelConfigurations.SERVICE_SETTINGS, validationException); - - String modelId = extractRequiredString(map, MODEL_ID, ModelConfigurations.SERVICE_SETTINGS, validationException); - String projectId = extractRequiredString(map, PROJECT_ID, ModelConfigurations.SERVICE_SETTINGS, validationException); - - RateLimitSettings rateLimitSettings = RateLimitSettings.of( - map, - DEFAULT_RATE_LIMIT_SETTINGS, - validationException, - IbmWatsonxService.NAME, - context - ); - - if (validationException.validationErrors().isEmpty() == false) { - throw validationException; - } - - return new IbmWatsonxRerankServiceSettings(uri, apiVersion, modelId, projectId, rateLimitSettings); - } - - private final URI uri; - - private final String apiVersion; - - private final String modelId; - - private final String projectId; - - private final RateLimitSettings rateLimitSettings; - - public IbmWatsonxRerankServiceSettings( - URI uri, - String apiVersion, - String modelId, - String projectId, - @Nullable RateLimitSettings rateLimitSettings - ) { - this.uri = uri; - this.apiVersion = apiVersion; - this.projectId = projectId; - this.modelId = modelId; - this.rateLimitSettings = Objects.requireNonNullElse(rateLimitSettings, DEFAULT_RATE_LIMIT_SETTINGS); - } - - public IbmWatsonxRerankServiceSettings(StreamInput in) throws IOException { - this.uri = createUri(in.readString()); - this.apiVersion = in.readString(); - this.modelId = in.readString(); - this.projectId = in.readString(); - this.rateLimitSettings = new RateLimitSettings(in); - - } - - public URI uri() { - return uri; - } - - public String apiVersion() { - return apiVersion; - } - - @Override - public String modelId() { - return modelId; - } - - public String projectId() { - return projectId; - } - - @Override - public RateLimitSettings rateLimitSettings() { - return rateLimitSettings; - } - - @Override - public String getWriteableName() { - return NAME; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - - toXContentFragmentOfExposedFields(builder, params); - - builder.endObject(); - return builder; - } - - @Override - protected XContentBuilder toXContentFragmentOfExposedFields(XContentBuilder builder, Params params) throws IOException { - builder.field(URL, uri.toString()); - - builder.field(API_VERSION, apiVersion); - - builder.field(MODEL_ID, modelId); - - builder.field(PROJECT_ID, projectId); - - rateLimitSettings.toXContent(builder, params); - - return builder; - } - - @Override - public TransportVersion getMinimalSupportedVersion() { - return TransportVersions.ML_INFERENCE_IBM_WATSONX_RERANK_ADDED; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(uri.toString()); - out.writeString(apiVersion); - - out.writeString(modelId); - out.writeString(projectId); - - rateLimitSettings.writeTo(out); - } - - @Override - public boolean equals(Object object) { - if (this == object) return true; - if (object == null || getClass() != object.getClass()) return false; - IbmWatsonxRerankServiceSettings that = (IbmWatsonxRerankServiceSettings) object; - return Objects.equals(uri, that.uri) - && Objects.equals(apiVersion, that.apiVersion) - && Objects.equals(modelId, that.modelId) - && Objects.equals(projectId, that.projectId) - && Objects.equals(rateLimitSettings, that.rateLimitSettings); - } - - @Override - public int hashCode() { - return Objects.hash(uri, apiVersion, modelId, projectId, rateLimitSettings); - } -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java deleted file mode 100644 index 12f4b8f6fa33e..0000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankTaskSettings.java +++ /dev/null @@ -1,192 +0,0 @@ -/* - * 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.inference.services.ibmwatsonx.rerank; - -import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.ValidationException; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.inference.InputType; -import org.elasticsearch.inference.ModelConfigurations; -import org.elasticsearch.inference.TaskSettings; -import org.elasticsearch.xcontent.XContentBuilder; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; - -import static org.elasticsearch.xpack.inference.services.ServiceUtils.extractOptionalBoolean; -import static org.elasticsearch.xpack.inference.services.ServiceUtils.extractOptionalPositiveInteger; - -public class IbmWatsonxRerankTaskSettings implements TaskSettings { - - public static final String NAME = "ibm_watsonx_rerank_task_settings"; - public static final String RETURN_DOCUMENTS = "return_documents"; - public static final String TOP_N_DOCS_ONLY = "top_n"; - public static final String TRUNCATE_INPUT_TOKENS = "truncate_input_tokens"; - - static final IbmWatsonxRerankTaskSettings EMPTY_SETTINGS = new IbmWatsonxRerankTaskSettings(null, null, null); - - public static IbmWatsonxRerankTaskSettings fromMap(Map map) { - ValidationException validationException = new ValidationException(); - - if (map == null || map.isEmpty()) { - return EMPTY_SETTINGS; - } - - Boolean returnDocuments = extractOptionalBoolean(map, RETURN_DOCUMENTS, validationException); - Integer topNDocumentsOnly = extractOptionalPositiveInteger( - map, - TOP_N_DOCS_ONLY, - ModelConfigurations.TASK_SETTINGS, - validationException - ); - Integer truncateInputTokens = extractOptionalPositiveInteger( - map, - TRUNCATE_INPUT_TOKENS, - ModelConfigurations.TASK_SETTINGS, - validationException - ); - - if (validationException.validationErrors().isEmpty() == false) { - throw validationException; - } - - return of(topNDocumentsOnly, returnDocuments, truncateInputTokens); - } - - /** - * Creates a new {@link IbmWatsonxRerankTaskSettings} - * by preferring non-null fields from the request settings over the original settings. - * - * @param originalSettings the settings stored as part of the inference entity configuration - * @param requestTaskSettings the settings passed in within the task_settings field of the request - * @return a constructed {@link IbmWatsonxRerankTaskSettings} - */ - public static IbmWatsonxRerankTaskSettings of( - IbmWatsonxRerankTaskSettings originalSettings, - IbmWatsonxRerankTaskSettings requestTaskSettings - ) { - return new IbmWatsonxRerankTaskSettings( - requestTaskSettings.getTopNDocumentsOnly() != null - ? requestTaskSettings.getTopNDocumentsOnly() - : originalSettings.getTopNDocumentsOnly(), - requestTaskSettings.getReturnDocuments() != null - ? requestTaskSettings.getReturnDocuments() - : originalSettings.getReturnDocuments(), - requestTaskSettings.getTruncateInputTokens() != null - ? requestTaskSettings.getTruncateInputTokens() - : originalSettings.getTruncateInputTokens() - ); - } - - public static IbmWatsonxRerankTaskSettings of(Integer topNDocumentsOnly, Boolean returnDocuments, Integer maxChunksPerDoc) { - return new IbmWatsonxRerankTaskSettings(topNDocumentsOnly, returnDocuments, maxChunksPerDoc); - } - - private final Integer topNDocumentsOnly; - private final Boolean returnDocuments; - private final Integer truncateInputTokens; - - public IbmWatsonxRerankTaskSettings(StreamInput in) throws IOException { - this(in.readOptionalInt(), in.readOptionalBoolean(), in.readOptionalInt()); - } - - public IbmWatsonxRerankTaskSettings( - @Nullable Integer topNDocumentsOnly, - @Nullable Boolean doReturnDocuments, - @Nullable Integer truncateInputTokens - ) { - this.topNDocumentsOnly = topNDocumentsOnly; - this.returnDocuments = doReturnDocuments; - this.truncateInputTokens = truncateInputTokens; - } - - @Override - public boolean isEmpty() { - return topNDocumentsOnly == null && returnDocuments == null && truncateInputTokens == null; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - if (topNDocumentsOnly != null) { - builder.field(TOP_N_DOCS_ONLY, topNDocumentsOnly); - } - if (returnDocuments != null) { - builder.field(RETURN_DOCUMENTS, returnDocuments); - } - if (truncateInputTokens != null) { - builder.field(TRUNCATE_INPUT_TOKENS, truncateInputTokens); - } - builder.endObject(); - return builder; - } - - @Override - public String getWriteableName() { - return NAME; - } - - @Override - public TransportVersion getMinimalSupportedVersion() { - return TransportVersions.ML_INFERENCE_IBM_WATSONX_RERANK_ADDED; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalInt(topNDocumentsOnly); - out.writeOptionalBoolean(returnDocuments); - out.writeOptionalInt(truncateInputTokens); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - IbmWatsonxRerankTaskSettings that = (IbmWatsonxRerankTaskSettings) o; - return Objects.equals(returnDocuments, that.returnDocuments) - && Objects.equals(topNDocumentsOnly, that.topNDocumentsOnly) - && Objects.equals(truncateInputTokens, that.truncateInputTokens); - } - - @Override - public int hashCode() { - return Objects.hash(returnDocuments, topNDocumentsOnly, truncateInputTokens); - } - - public static String invalidInputTypeMessage(InputType inputType) { - return Strings.format("received invalid input type value [%s]", inputType.toString()); - } - - public Boolean getDoesReturnDocuments() { - return returnDocuments; - } - - public Integer getTopNDocumentsOnly() { - return topNDocumentsOnly; - } - - public Boolean getReturnDocuments() { - return returnDocuments; - } - - public Integer getTruncateInputTokens() { - return truncateInputTokens; - } - - @Override - public TaskSettings updatedTaskSettings(Map newSettings) { - IbmWatsonxRerankTaskSettings updatedSettings = IbmWatsonxRerankTaskSettings.fromMap(new HashMap<>(newSettings)); - return IbmWatsonxRerankTaskSettings.of(this, updatedSettings); - } -} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java deleted file mode 100644 index 8278b76a1cee4..0000000000000 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestEntityTests.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.inference.external.request.ibmwatsonx.rerank; - -import org.elasticsearch.common.Strings; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.XContentFactory; -import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxRerankRequestEntity; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankTaskSettings; - -import java.io.IOException; -import java.util.List; - -import static org.elasticsearch.xpack.inference.MatchersUtils.equalToIgnoringWhitespaceInJsonString; - -public class IbmWatsonxRerankRequestEntityTests extends ESTestCase { - public void testXContent_Request() throws IOException { - IbmWatsonxRerankTaskSettings taskSettings = new IbmWatsonxRerankTaskSettings(5, true, 100); - var entity = new IbmWatsonxRerankRequestEntity( - "database", - List.of("greenland", "google", "john", "mysql", "potter", "grammar"), - taskSettings, - "model", - "project_id" - ); - - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); - entity.toXContent(builder, null); - String xContentResult = Strings.toString(builder); - - assertThat(xContentResult, equalToIgnoringWhitespaceInJsonString(""" - {"model_id":"model", - "query":"database", - "inputs":[ - {"text":"greenland"}, - {"text":"google"}, - {"text":"john"}, - {"text":"mysql"}, - {"text":"potter"}, - {"text":"grammar"} - ], - "project_id":"project_id", - "parameters":{ - "truncate_input_tokens":100, - "return_options":{ - "inputs":true, - "top_n":5 - } - } - } - """)); - } -} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java deleted file mode 100644 index 8c95a01bc3230..0000000000000 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/ibmwatsonx/rerank/IbmWatsonxRerankRequestTests.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * 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.inference.external.request.ibmwatsonx.rerank; - -import org.apache.http.HttpHeaders; -import org.apache.http.client.methods.HttpPost; -import org.elasticsearch.core.Strings; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.inference.external.request.ibmwatsonx.IbmWatsonxRerankRequest; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModelTests; - -import java.io.IOException; -import java.net.URI; -import java.util.List; -import java.util.Map; - -import static org.elasticsearch.xpack.inference.external.http.Utils.entityAsMap; -import static org.hamcrest.Matchers.aMapWithSize; -import static org.hamcrest.Matchers.endsWith; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; - -public class IbmWatsonxRerankRequestTests extends ESTestCase { - private static final String AUTH_HEADER_VALUE = "foo"; - - public void testCreateRequest() throws IOException { - var model = "model"; - var projectId = "project_id"; - URI uri = null; - try { - uri = new URI("http://abc.com"); - } catch (Exception ignored) {} - var apiVersion = "2023-05-04"; - var apiKey = "api_key"; - var query = "database"; - List input = List.of("greenland", "google", "john", "mysql", "potter", "grammar"); - - var request = createRequest(model, projectId, uri, apiVersion, apiKey, query, input); - var httpRequest = request.createHttpRequest(); - - assertThat(httpRequest.httpRequestBase(), instanceOf(HttpPost.class)); - var httpPost = (HttpPost) httpRequest.httpRequestBase(); - - assertThat(httpPost.getURI().toString(), endsWith(Strings.format("%s=%s", "version", apiVersion))); - assertThat(httpPost.getLastHeader(HttpHeaders.CONTENT_TYPE).getValue(), is(XContentType.JSON.mediaType())); - - var requestMap = entityAsMap(httpPost.getEntity().getContent()); - assertThat(requestMap, aMapWithSize(5)); - assertThat( - requestMap, - is( - - Map.of( - "project_id", - "project_id", - "model_id", - "model", - "inputs", - List.of( - Map.of("text", "greenland"), - Map.of("text", "google"), - Map.of("text", "john"), - Map.of("text", "mysql"), - Map.of("text", "potter"), - Map.of("text", "grammar") - ), - "query", - "database", - "parameters", - Map.of("return_options", Map.of("top_n", 2, "inputs", true), "truncate_input_tokens", 100) - ) - ) - ); - } - - public static IbmWatsonxRerankRequest createRequest( - String model, - String projectId, - URI uri, - String apiVersion, - String apiKey, - String query, - List input - ) { - var embeddingsModel = IbmWatsonxRerankModelTests.createModel(model, projectId, uri, apiVersion, apiKey); - - return new IbmWatsonxRerankWithoutAuthRequest(query, input, embeddingsModel); - } - - private static class IbmWatsonxRerankWithoutAuthRequest extends IbmWatsonxRerankRequest { - IbmWatsonxRerankWithoutAuthRequest(String query, List input, IbmWatsonxRerankModel model) { - super(query, input, model); - } - - @Override - public void decorateWithAuth(HttpPost httpPost) { - httpPost.setHeader(HttpHeaders.AUTHORIZATION, AUTH_HEADER_VALUE); - } - } -} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java deleted file mode 100644 index 6b59f25896a48..0000000000000 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/response/ibmwatsonx/IbmWatsonxRankedResponseEntityTests.java +++ /dev/null @@ -1,166 +0,0 @@ -/* - * 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.inference.external.response.ibmwatsonx; - -import org.apache.http.HttpResponse; -import org.elasticsearch.inference.InferenceServiceResults; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.core.inference.results.RankedDocsResults; -import org.elasticsearch.xpack.inference.external.http.HttpResult; -import org.hamcrest.MatcherAssert; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; - -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; -import static org.mockito.Mockito.mock; - -public class IbmWatsonxRankedResponseEntityTests extends ESTestCase { - - public void testResponseLiteral() throws IOException { - InferenceServiceResults parsedResults = IbmWatsonxRankedResponseEntity.fromResponse( - new HttpResult(mock(HttpResponse.class), responseLiteral.getBytes(StandardCharsets.UTF_8)) - ); - - MatcherAssert.assertThat(parsedResults, instanceOf(RankedDocsResults.class)); - List expected = responseLiteralDocs(); - for (int i = 0; i < ((RankedDocsResults) parsedResults).getRankedDocs().size(); i++) { - assertEquals(((RankedDocsResults) parsedResults).getRankedDocs().get(i).index(), expected.get(i).index()); - } - } - - public void testGeneratedResponse() throws IOException { - int numDocs = randomIntBetween(1, 10); - - List expected = new ArrayList<>(numDocs); - StringBuilder responseBuilder = new StringBuilder(); - - responseBuilder.append("{"); - responseBuilder.append("\"results\": ["); - List indices = linear(numDocs); - List scores = linearFloats(numDocs); - for (int i = 0; i < numDocs; i++) { - int index = indices.remove(randomInt(indices.size() - 1)); - - responseBuilder.append("{"); - responseBuilder.append("\"index\":").append(index).append(","); - responseBuilder.append("\"score\":").append(scores.get(i).toString()).append("}"); - expected.add(new RankedDocsResults.RankedDoc(index, scores.get(i), null)); - if (i < numDocs - 1) { - responseBuilder.append(","); - } - } - responseBuilder.append("]"); - - responseBuilder.append(randomIntBetween(1, 10)).append("}"); - - InferenceServiceResults parsedResults = IbmWatsonxRankedResponseEntity.fromResponse( - new HttpResult(mock(HttpResponse.class), responseBuilder.toString().getBytes(StandardCharsets.UTF_8)) - ); - MatcherAssert.assertThat(parsedResults, instanceOf(RankedDocsResults.class)); - for (int i = 0; i < ((RankedDocsResults) parsedResults).getRankedDocs().size(); i++) { - assertEquals(((RankedDocsResults) parsedResults).getRankedDocs().get(i).index(), expected.get(i).index()); - } - } - - private ArrayList responseLiteralDocs() { - var list = new ArrayList(); - - list.add(new RankedDocsResults.RankedDoc(2, 0.98005307F, null)); - list.add(new RankedDocsResults.RankedDoc(3, 0.27904198F, null)); - list.add(new RankedDocsResults.RankedDoc(0, 0.10194652F, null)); - return list; - } - - private final String responseLiteral = """ - { - "results": [ - { - "index": 2, - "score": 0.98005307 - }, - { - "index": 3, - "score": 0.27904198 - }, - { - "index": 0, - "score": 0.10194652 - } - ] - } - """; - - public void testResponseLiteralWithDocuments() throws IOException { - InferenceServiceResults parsedResults = IbmWatsonxRankedResponseEntity.fromResponse( - new HttpResult(mock(HttpResponse.class), responseLiteralWithDocuments.getBytes(StandardCharsets.UTF_8)) - ); - - MatcherAssert.assertThat(parsedResults, instanceOf(RankedDocsResults.class)); - MatcherAssert.assertThat(((RankedDocsResults) parsedResults).getRankedDocs(), is(responseLiteralDocsWithText)); - } - - private final String responseLiteralWithDocuments = """ - { - "results": [ - { - "input": { - "text": "Washington, D.C.." - }, - "index": 2, - "score": 0.98005307 - }, - { - "input": { - "text": "Capital punishment has existed in the United States since before the United States was a country. " - }, - "index": 3, - "score": 0.27904198 - }, - { - "input": { - "text": "Carson City is the capital city of the American state of Nevada." - }, - "index": 0, - "score": 0.10194652 - } - ] - } - """; - - private final List responseLiteralDocsWithText = List.of( - new RankedDocsResults.RankedDoc(2, 0.98005307F, "Washington, D.C.."), - new RankedDocsResults.RankedDoc( - 3, - 0.27904198F, - "Capital punishment has existed in the United States since before the United States was a country. " - ), - new RankedDocsResults.RankedDoc(0, 0.10194652F, "Carson City is the capital city of the American state of Nevada.") - ); - - private ArrayList linear(int n) { - ArrayList list = new ArrayList<>(); - for (int i = 0; i <= n; i++) { - list.add(i); - } - return list; - } - - // creates a list of doubles of monotonically decreasing magnitude - private ArrayList linearFloats(int n) { - ArrayList list = new ArrayList<>(); - float startValue = 1.0f; - float decrement = startValue / n + 1; - for (int i = 0; i <= n; i++) { - list.add(startValue - (i * decrement)); - } - return list; - } -} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java index 99b7b3868b7f4..ff99101fc4ee5 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxServiceTests.java @@ -50,7 +50,6 @@ import org.elasticsearch.xpack.inference.services.ServiceFields; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModel; import org.elasticsearch.xpack.inference.services.ibmwatsonx.embeddings.IbmWatsonxEmbeddingsModelTests; -import org.elasticsearch.xpack.inference.services.ibmwatsonx.rerank.IbmWatsonxRerankModel; import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModelTests; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; @@ -155,42 +154,6 @@ public void testParseRequestConfig_CreatesAIbmWatsonxEmbeddingsModel() throws IO } } - public void testParseRequestConfig_CreatesAIbmWatsonxRerankModel() throws IOException { - try (var service = createIbmWatsonxService()) { - ActionListener modelListener = ActionListener.wrap(model -> { - assertThat(model, instanceOf(IbmWatsonxRerankModel.class)); - - var rerankModel = (IbmWatsonxRerankModel) model; - assertThat(rerankModel.getServiceSettings().modelId(), is(modelId)); - assertThat(rerankModel.getServiceSettings().projectId(), is(projectId)); - assertThat(rerankModel.getServiceSettings().apiVersion(), is(apiVersion)); - assertThat(rerankModel.getSecretSettings().apiKey().toString(), is(apiKey)); - }, e -> fail("Model parsing should have succeeded, but failed: " + e.getMessage())); - - service.parseRequestConfig( - "id", - TaskType.RERANK, - getRequestConfigMap( - new HashMap<>( - Map.of( - ServiceFields.MODEL_ID, - modelId, - IbmWatsonxServiceFields.PROJECT_ID, - projectId, - ServiceFields.URL, - url, - IbmWatsonxServiceFields.API_VERSION, - apiVersion - ) - ), - new HashMap<>(Map.of()), - getSecretSettingsMap(apiKey) - ), - modelListener - ); - } - } - public void testParseRequestConfig_CreatesAIbmWatsonxEmbeddingsModelWhenChunkingSettingsProvided() throws IOException { try (var service = createIbmWatsonxService()) { ActionListener modelListener = ActionListener.wrap(model -> { diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java deleted file mode 100644 index 0138952c11e07..0000000000000 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/rerank/IbmWatsonxRerankModelTests.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.inference.services.ibmwatsonx.rerank; - -import org.elasticsearch.common.settings.SecureString; -import org.elasticsearch.inference.TaskType; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.inference.services.settings.DefaultSecretSettings; - -import java.net.URI; - -public class IbmWatsonxRerankModelTests extends ESTestCase { - public static IbmWatsonxRerankModel createModel(String model, String projectId, URI uri, String apiVersion, String apiKey) { - return new IbmWatsonxRerankModel( - "id", - TaskType.RERANK, - "service", - new IbmWatsonxRerankServiceSettings(uri, apiVersion, model, projectId, null), - new IbmWatsonxRerankTaskSettings(2, true, 100), - new DefaultSecretSettings(new SecureString(apiKey.toCharArray())) - ); - } -} diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java index e8110886cbd5a..8c12011ca4bb1 100644 --- a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java +++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamIndexTransportAction.java @@ -54,7 +54,6 @@ import java.util.Locale; import java.util.Map; -import java.util.Objects; import static org.elasticsearch.cluster.metadata.IndexMetadata.APIBlock.WRITE; @@ -373,7 +372,7 @@ private void sanityCheck( listener.delegateFailureAndWrap((delegate, ignored) -> { getIndexDocCount(sourceIndexName, parentTaskId, delegate.delegateFailureAndWrap((delegate1, sourceCount) -> { getIndexDocCount(destIndexName, parentTaskId, delegate1.delegateFailureAndWrap((delegate2, destCount) -> { - assert Objects.equals(sourceCount, destCount) + assert sourceCount == destCount : String.format( Locale.ROOT, "source index [%s] has %d docs and dest [%s] has %d docs",