diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/CCSFieldCapabilitiesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/CCSFieldCapabilitiesIT.java index 541fe0f869f26..315ff4446742b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/CCSFieldCapabilitiesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/CCSFieldCapabilitiesIT.java @@ -10,6 +10,7 @@ package org.elasticsearch.search.fieldcaps; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ResolvedIndexExpressions; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesFailure; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; import org.elasticsearch.client.internal.Client; @@ -21,11 +22,13 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Map; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.aMapWithSize; import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.arrayContainingInAnyOrder; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyArray; import static org.hamcrest.Matchers.equalTo; @@ -33,6 +36,7 @@ import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; public class CCSFieldCapabilitiesIT extends AbstractMultiClustersTestCase { @@ -266,4 +270,35 @@ public void testReturnAllLocal() { } } } + + public void testResolvedTo() { + String localIndex = "index-local"; + String remoteIndex = "index-remote"; + String remoteClusterAlias = "remote_cluster"; + populateIndices(localIndex, remoteIndex, remoteClusterAlias, false); + String remoteIndexWithCluster = String.join(":", remoteClusterAlias, remoteIndex); + FieldCapabilitiesResponse response = client().prepareFieldCaps(localIndex, remoteIndexWithCluster) + .setFields("*") + .setIncludeResolvedTo(true) + .get(); + + assertThat(response.getIndices(), arrayContainingInAnyOrder(localIndex, remoteIndexWithCluster)); + + ResolvedIndexExpressions local = response.getResolvedLocally(); + assertThat(local, notNullValue()); + assertThat(local.expressions(), hasSize(1)); + + List localIndicesList = local.getLocalIndicesList(); + assertThat(localIndicesList, hasSize(1)); + assertThat(localIndicesList, containsInAnyOrder(localIndex)); + + Map remote = response.getResolvedRemotely(); + assertThat(remote, notNullValue()); + assertThat(remote, aMapWithSize(1)); + assertThat(remote.keySet(), contains(remoteClusterAlias)); + + List remoteIndicesList = remote.get(remoteClusterAlias).getLocalIndicesList(); + assertThat(remoteIndicesList, hasSize(1)); + assertThat(remoteIndicesList, containsInAnyOrder(remoteIndex)); + } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java index b9fcec556213a..418e2eae6393b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/fieldcaps/FieldCapabilitiesIT.java @@ -102,6 +102,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; @@ -913,6 +914,13 @@ public void testIndexMode() throws Exception { assertThat(actualIndexModes, equalTo(indexModes)); } + public void testNoneExpressionIndices() { + // The auth code injects the pattern ["*", "-*"] which effectively means a request that requests no indices + FieldCapabilitiesResponse response = client().prepareFieldCaps("*", "-*").setFields("*").get(); + + assertThat(response.getIndices().length, is(0)); + } + private void assertIndices(FieldCapabilitiesResponse response, String... indices) { assertNotNull(response.getIndices()); Arrays.sort(indices); diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java index b399835372476..c09c36f34f498 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java @@ -40,6 +40,7 @@ public final class FieldCapabilitiesRequest extends LegacyActionRequest implemen public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpenAndForbidClosed(); private static final TransportVersion FIELD_CAPS_ADD_CLUSTER_ALIAS = TransportVersion.fromName("field_caps_add_cluster_alias"); + private static final TransportVersion RESOLVED_FIELDS_CAPS = TransportVersion.fromName("resolved_fields_caps"); private String clusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY; @@ -58,6 +59,8 @@ public final class FieldCapabilitiesRequest extends LegacyActionRequest implemen */ private transient boolean includeIndices = false; + private boolean includeResolvedTo = false; + /** * Controls whether all local indices should be returned if no remotes matched * See {@link org.elasticsearch.transport.RemoteClusterService#groupIndices} returnLocalAll argument. @@ -91,6 +94,11 @@ public FieldCapabilitiesRequest(StreamInput in) throws IOException { } else { clusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY; } + if (in.getTransportVersion().supports(RESOLVED_FIELDS_CAPS)) { + includeResolvedTo = in.readBoolean(); + } else { + includeResolvedTo = false; + } } public FieldCapabilitiesRequest() {} @@ -141,6 +149,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getTransportVersion().supports(FIELD_CAPS_ADD_CLUSTER_ALIAS)) { out.writeOptionalString(clusterAlias); } + if (out.getTransportVersion().supports(RESOLVED_FIELDS_CAPS)) { + out.writeBoolean(includeResolvedTo); + } } @Override @@ -219,6 +230,11 @@ public FieldCapabilitiesRequest includeIndices(boolean includeIndices) { return this; } + public FieldCapabilitiesRequest includeResolvedTo(boolean includeResolvedTo) { + this.includeResolvedTo = includeResolvedTo; + return this; + } + public FieldCapabilitiesRequest returnLocalAll(boolean returnLocalAll) { this.returnLocalAll = returnLocalAll; return this; @@ -252,6 +268,10 @@ public boolean includeIndices() { return includeIndices; } + public boolean includeResolvedTo() { + return includeResolvedTo; + } + public boolean returnLocalAll() { return returnLocalAll; } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java index 4437895c7e08d..9e79c52ce8ae1 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestBuilder.java @@ -63,4 +63,9 @@ public FieldCapabilitiesRequestBuilder setReturnLocalAll(boolean returnLocalAll) request().returnLocalAll(returnLocalAll); return this; } + + public FieldCapabilitiesRequestBuilder setIncludeResolvedTo(boolean resolvedTo) { + request().includeResolvedTo(resolvedTo); + return this; + } } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java index e3fcd5bbd200d..6c5cf995f7fd1 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponse.java @@ -9,7 +9,9 @@ package org.elasticsearch.action.fieldcaps; +import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ResolvedIndexExpressions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.StreamInput; @@ -19,7 +21,9 @@ import org.elasticsearch.xcontent.ToXContent; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -35,34 +39,41 @@ public class FieldCapabilitiesResponse extends ActionResponse implements Chunked private static final ParseField FAILED_INDICES_FIELD = new ParseField("failed_indices"); public static final ParseField FAILURES_FIELD = new ParseField("failures"); + private static final TransportVersion RESOLVED_FIELDS_CAPS = TransportVersion.fromName("resolved_fields_caps"); + private final String[] indices; + private final ResolvedIndexExpressions resolvedLocally; + private final Map resolvedRemotely; private final Map> fields; private final List failures; private final List indexResponses; - public FieldCapabilitiesResponse( - String[] indices, - Map> fields, - List failures - ) { - this(indices, fields, Collections.emptyList(), failures); - } - - public FieldCapabilitiesResponse(String[] indices, Map> fields) { - this(indices, fields, Collections.emptyList(), Collections.emptyList()); + public static FieldCapabilitiesResponse empty() { + return new FieldCapabilitiesResponse( + Strings.EMPTY_ARRAY, + null, + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyList(), + Collections.emptyList() + ); } - public FieldCapabilitiesResponse(List indexResponses, List failures) { - this(Strings.EMPTY_ARRAY, Collections.emptyMap(), indexResponses, failures); + public static FieldCapabilitiesResponse.Builder builder() { + return new FieldCapabilitiesResponse.Builder(); } private FieldCapabilitiesResponse( String[] indices, + ResolvedIndexExpressions resolvedLocally, + Map resolvedRemotely, Map> fields, List indexResponses, List failures ) { this.fields = Objects.requireNonNull(fields); + this.resolvedLocally = resolvedLocally; + this.resolvedRemotely = Objects.requireNonNull(resolvedRemotely); this.indexResponses = Objects.requireNonNull(indexResponses); this.indices = indices; this.failures = failures; @@ -70,6 +81,13 @@ private FieldCapabilitiesResponse( public FieldCapabilitiesResponse(StreamInput in) throws IOException { this.indices = in.readStringArray(); + if (in.getTransportVersion().supports(RESOLVED_FIELDS_CAPS)) { + this.resolvedLocally = in.readOptionalWriteable(ResolvedIndexExpressions::new); + this.resolvedRemotely = in.readImmutableMap(StreamInput::readString, ResolvedIndexExpressions::new); + } else { + this.resolvedLocally = null; + this.resolvedRemotely = Collections.emptyMap(); + } this.fields = in.readMap(FieldCapabilitiesResponse::readField); this.indexResponses = FieldCapabilitiesIndexResponse.readList(in); this.failures = in.readCollectionAsList(FieldCapabilitiesFailure::new); @@ -115,6 +133,20 @@ public List getIndexResponses() { return indexResponses; } + /** + * Locally resolved index expressions + */ + public ResolvedIndexExpressions getResolvedLocally() { + return resolvedLocally; + } + + /** + * Locally resolved index expressions + */ + public Map getResolvedRemotely() { + return resolvedRemotely; + } + /** * * Get the field capabilities per type for the provided {@code field}. @@ -141,6 +173,10 @@ private static Map readField(StreamInput in) throws I @Override public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(indices); + if (out.getTransportVersion().supports(RESOLVED_FIELDS_CAPS)) { + out.writeOptionalWriteable(resolvedLocally); + out.writeMap(resolvedRemotely, StreamOutput::writeWriteable); + } out.writeMap(fields, FieldCapabilitiesResponse::writeField); FieldCapabilitiesIndexResponse.writeList(out, indexResponses); out.writeCollection(failures); @@ -182,6 +218,8 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; FieldCapabilitiesResponse that = (FieldCapabilitiesResponse) o; return Arrays.equals(indices, that.indices) + && Objects.equals(resolvedLocally, that.resolvedLocally) + && Objects.equals(resolvedRemotely, that.resolvedRemotely) && Objects.equals(fields, that.fields) && Objects.equals(indexResponses, that.indexResponses) && Objects.equals(failures, that.failures); @@ -189,7 +227,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - int result = Objects.hash(fields, indexResponses, failures); + int result = Objects.hash(resolvedLocally, resolvedRemotely, fields, indexResponses, failures); result = 31 * result + Arrays.hashCode(indices); return result; } @@ -201,4 +239,46 @@ public String toString() { } return Strings.toString(this); } + + public static class Builder { + private String[] indices = Strings.EMPTY_ARRAY; + private ResolvedIndexExpressions resolvedLocally; + private Map resolvedRemotely = Collections.emptyMap(); + private Map> fields = Collections.emptyMap(); + private List indexResponses = Collections.emptyList(); + private List failures = Collections.emptyList(); + + private Builder() {} + + public Builder withIndices(String[] indices) { + this.indices = indices; + return this; + } + + public Builder withResolved(ResolvedIndexExpressions resolvedLocally, Map resolvedRemotely) { + this.resolvedLocally = resolvedLocally; + this.resolvedRemotely = resolvedRemotely; + return this; + } + + public Builder withFields(Map> fields) { + this.fields = fields; + return this; + } + + public Builder withIndexResponses(Collection indexResponses) { + this.indexResponses = new ArrayList<>(indexResponses); + return this; + } + + public Builder withFailures(List failures) { + this.failures = failures; + return this; + } + + public FieldCapabilitiesResponse build() { + return new FieldCapabilitiesResponse(indices, resolvedLocally, resolvedRemotely, fields, indexResponses, failures); + } + } + } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java index 91c5c1a212dd9..b24065fd08c09 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -19,18 +19,22 @@ import org.elasticsearch.action.ActionType; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.RemoteClusterActionType; +import org.elasticsearch.action.ResolvedIndexExpression; +import org.elasticsearch.action.ResolvedIndexExpressions; import org.elasticsearch.action.support.AbstractThreadedActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.cluster.ProjectState; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.ProjectMetadata; import org.elasticsearch.cluster.project.ProjectResolver; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.util.Maps; @@ -67,6 +71,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; @@ -150,19 +155,43 @@ private void doExecuteForked(Task task, FieldCapabilitiesRequest request, Action final Map remoteClusterIndices = transportService.getRemoteClusterService() .groupIndices(request.indicesOptions(), request.indices(), request.returnLocalAll()); final OriginalIndices localIndices = remoteClusterIndices.remove(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY); - final String[] concreteIndices; - if (localIndices == null) { - // in the case we have one or more remote indices but no local we don't expand to all local indices and just do remote indices - concreteIndices = Strings.EMPTY_ARRAY; - } else { - concreteIndices = indexNameExpressionResolver.concreteIndexNames(projectState.metadata(), localIndices); + + final List resolvedLocallyList = new ArrayList<>(); + // in the case we have one or more remote indices but no local we don't expand to all local indices and just do remote indices + if (localIndices != null) { + ProjectMetadata projectMetadata = projectState.metadata(); + IndicesOptions indicesOptions = localIndices.indicesOptions(); + String[] localIndexNames = localIndices.indices(); + if (localIndexNames.length == 0) { + String[] concreteIndexNames = indexNameExpressionResolver.concreteIndexNames(projectMetadata, indicesOptions); + resolvedLocallyList.add(createResolvedIndexExpression(Metadata.ALL, concreteIndexNames)); + } else if (IndexNameExpressionResolver.isNoneExpression(localIndexNames) == false) { + for (String localIndexName : localIndexNames) { + String[] concreteIndexNames = indexNameExpressionResolver.concreteIndexNames( + projectMetadata, + indicesOptions, + localIndices.includeDataStreams(), + localIndexName + ); + resolvedLocallyList.add(createResolvedIndexExpression(localIndexName, concreteIndexNames)); + } + } } + String[] concreteIndices = resolvedLocallyList.stream() + .map(r -> r.localExpressions().expressions()) + .flatMap(Set::stream) + .distinct() + .toArray(String[]::new); if (concreteIndices.length == 0 && remoteClusterIndices.isEmpty()) { - listener.onResponse(new FieldCapabilitiesResponse(new String[0], Collections.emptyMap())); + listener.onResponse(FieldCapabilitiesResponse.empty()); return; } + if (request.includeResolvedTo() == false) { + resolvedLocallyList.clear(); + } + checkIndexBlocks(projectState, concreteIndices); final FailureCollector indexFailures = new FailureCollector(); final Map indexResponses = new HashMap<>(); @@ -230,12 +259,14 @@ private void doExecuteForked(Task task, FieldCapabilitiesRequest request, Action LOGGER.trace("clear index responses on cancellation submitted"); } }); + + Map resolvedRemotely = new ConcurrentHashMap<>(); try (RefCountingRunnable refs = new RefCountingRunnable(() -> { finishedOrCancelled.set(true); if (fieldCapTask.notifyIfCancelled(listener)) { releaseResourcesOnCancel.run(); } else { - mergeIndexResponses(request, fieldCapTask, indexResponses, indexFailures, listener); + mergeIndexResponses(request, fieldCapTask, indexResponses, indexFailures, resolvedLocallyList, resolvedRemotely, listener); } })) { // local cluster @@ -263,6 +294,12 @@ private void doExecuteForked(Task task, FieldCapabilitiesRequest request, Action OriginalIndices originalIndices = remoteIndices.getValue(); FieldCapabilitiesRequest remoteRequest = prepareRemoteRequest(clusterAlias, request, originalIndices, nowInMillis); ActionListener remoteListener = ActionListener.wrap(response -> { + + if (request.includeResolvedTo()) { + ResolvedIndexExpressions resolvedOnRemoteProject = response.getResolvedLocally(); + resolvedRemotely.put(clusterAlias, resolvedOnRemoteProject); + } + for (FieldCapabilitiesIndexResponse resp : response.getIndexResponses()) { String indexName = RemoteClusterAware.buildRemoteIndexName(clusterAlias, resp.getIndexName()); handleIndexResponse.accept( @@ -319,6 +356,18 @@ private void doExecuteForked(Task task, FieldCapabilitiesRequest request, Action } } + private static ResolvedIndexExpression createResolvedIndexExpression(String original, String[] concreteIndexNames) { + return new ResolvedIndexExpression( + original, + new ResolvedIndexExpression.LocalExpressions( + Set.of(concreteIndexNames), + ResolvedIndexExpression.LocalIndexResolutionResult.SUCCESS, + null + ), + Collections.emptySet() + ); + } + public static Executor buildSingleThreadedExecutor(Executor searchCoordinationExecutor, Logger logger) { final ThrottledTaskRunner throttledTaskRunner = new ThrottledTaskRunner("field_caps", 1, searchCoordinationExecutor); return r -> throttledTaskRunner.enqueueTask(new ActionListener<>() { @@ -360,14 +409,26 @@ private static void mergeIndexResponses( CancellableTask task, Map indexResponses, FailureCollector indexFailures, + List resolvedLocallyList, + Map resolvedRemotely, ActionListener listener ) { + ResolvedIndexExpressions resolvedLocally = new ResolvedIndexExpressions(resolvedLocallyList); List failures = indexFailures.build(indexResponses.keySet()); if (indexResponses.size() > 0) { if (request.isMergeResults()) { - ActionListener.completeWith(listener, () -> merge(indexResponses, task, request, failures)); + ActionListener.completeWith( + listener, + () -> merge(indexResponses, resolvedLocally, resolvedRemotely, task, request, failures) + ); } else { - listener.onResponse(new FieldCapabilitiesResponse(new ArrayList<>(indexResponses.values()), failures)); + listener.onResponse( + FieldCapabilitiesResponse.builder() + .withIndexResponses(indexResponses.values()) + .withResolved(resolvedLocally, resolvedRemotely) + .withFailures(failures) + .build() + ); } } else { // we have no responses at all, maybe because of errors @@ -381,13 +442,16 @@ private static void mergeIndexResponses( failure -> failure.getException() instanceof IllegalStateException ise && ise.getCause() instanceof ElasticsearchTimeoutException )) { - listener.onResponse(new FieldCapabilitiesResponse(Collections.emptyList(), failures)); + listener.onResponse( + FieldCapabilitiesResponse.builder().withResolved(resolvedLocally, resolvedRemotely).withFailures(failures).build() + ); } else { // throw back the first exception listener.onFailure(failures.get(0).getException()); } } else { - listener.onResponse(new FieldCapabilitiesResponse(Collections.emptyList(), Collections.emptyList())); + // TODO[aifgi]: is this correct? + listener.onResponse(FieldCapabilitiesResponse.empty()); } } } @@ -410,6 +474,7 @@ public static FieldCapabilitiesRequest prepareRemoteRequest( remoteRequest.indexFilter(request.indexFilter()); remoteRequest.nowInMillis(nowInMillis); remoteRequest.includeEmptyFields(request.includeEmptyFields()); + remoteRequest.includeResolvedTo(request.includeResolvedTo()); return remoteRequest; } @@ -421,6 +486,8 @@ private static boolean hasSameMappingHash(FieldCapabilitiesIndexResponse r1, Fie private static FieldCapabilitiesResponse merge( Map indexResponsesMap, + ResolvedIndexExpressions resolvedLocally, + Map resolvedRemotely, CancellableTask task, FieldCapabilitiesRequest request, List failures @@ -464,7 +531,13 @@ private static FieldCapabilitiesResponse merge( ); } } - return new FieldCapabilitiesResponse(indices, Collections.unmodifiableMap(fields), failures); + + return FieldCapabilitiesResponse.builder() + .withIndices(indices) + .withResolved(resolvedLocally, resolvedRemotely) + .withFields(fields) + .withFailures(failures) + .build(); } private static boolean shouldLogException(Exception e) { diff --git a/server/src/main/resources/transport/definitions/referable/resolved_fields_caps.csv b/server/src/main/resources/transport/definitions/referable/resolved_fields_caps.csv new file mode 100644 index 0000000000000..49307e681caa8 --- /dev/null +++ b/server/src/main/resources/transport/definitions/referable/resolved_fields_caps.csv @@ -0,0 +1 @@ +9194000 diff --git a/server/src/main/resources/transport/upper_bounds/9.3.csv b/server/src/main/resources/transport/upper_bounds/9.3.csv index f560c6e8d5593..42a0f3d7e6b8a 100644 --- a/server/src/main/resources/transport/upper_bounds/9.3.csv +++ b/server/src/main/resources/transport/upper_bounds/9.3.csv @@ -1 +1 @@ -available_processors_in_os_stats,9193000 +resolved_fields_caps,9194000 diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java index cddd9dda00250..5eb0e03521521 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesResponseTests.java @@ -52,7 +52,7 @@ protected FieldCapabilitiesResponse createTestInstance() { var indexMode = randomFrom(IndexMode.values()); responses.add(new FieldCapabilitiesIndexResponse("index_" + i, null, fieldCaps, randomBoolean(), indexMode)); } - randomResponse = new FieldCapabilitiesResponse(responses, Collections.emptyList()); + randomResponse = FieldCapabilitiesResponse.builder().withIndexResponses(responses).build(); return randomResponse; } @@ -87,7 +87,7 @@ protected FieldCapabilitiesResponse mutateInstance(FieldCapabilitiesResponse res ); } } - return new FieldCapabilitiesResponse(null, mutatedResponses, Collections.emptyList()); + return FieldCapabilitiesResponse.builder().withFields(mutatedResponses).build(); } public void testFailureSerialization() throws IOException { @@ -143,7 +143,7 @@ public static FieldCapabilitiesResponse createResponseWithFailures() { failures.get(failures.size() - 1).addIndex(index); } } - return new FieldCapabilitiesResponse(indices, Collections.emptyMap(), failures); + return FieldCapabilitiesResponse.builder().withIndices(indices).withFailures(failures).build(); } private static FieldCapabilitiesResponse randomCCSResponse(List indexResponses) { @@ -153,7 +153,7 @@ private static FieldCapabilitiesResponse randomCCSResponse(List failureMap = List.of( new FieldCapabilitiesFailure(new String[] { "errorindex", "errorindex2" }, new IllegalArgumentException("test")) ); - return new FieldCapabilitiesResponse(new String[] { "index1", "index2", "index3", "index4" }, responses, failureMap); + return FieldCapabilitiesResponse.builder() + .withIndices(new String[] { "index1", "index2", "index3", "index4" }) + .withFields(responses) + .withFailures(failureMap) + .build(); } public void testChunking() { diff --git a/test/framework/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapsUtils.java b/test/framework/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapsUtils.java index 84c057d3b6a81..4a181d816451b 100644 --- a/test/framework/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapsUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapsUtils.java @@ -61,7 +61,12 @@ public static FieldCapabilitiesFailure parseFailure(XContentParser parser) throw .collect(Collectors.toMap(Tuple::v1, Tuple::v2)); List indices = a[1] == null ? Collections.emptyList() : (List) a[1]; List failures = a[2] == null ? Collections.emptyList() : (List) a[2]; - return new FieldCapabilitiesResponse(indices.toArray(String[]::new), responseMap, failures); + + return FieldCapabilitiesResponse.builder() + .withIndices(indices.toArray(String[]::new)) + .withFields(responseMap) + .withFailures(failures) + .build(); } ); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/ClassificationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/ClassificationTests.java index d14bd8c8c8196..549a5f5714b8b 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/ClassificationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/dataframe/analyses/ClassificationTests.java @@ -427,7 +427,7 @@ public void testFieldCardinalityLimitsIsNonEmpty() { } public void testGetResultMappings_DependentVariableMappingIsAbsent() { - FieldCapabilitiesResponse fieldCapabilitiesResponse = new FieldCapabilitiesResponse(new String[0], Collections.emptyMap()); + FieldCapabilitiesResponse fieldCapabilitiesResponse = FieldCapabilitiesResponse.empty(); expectThrows( ElasticsearchStatusException.class, () -> new Classification("foo").getResultMappings("results", fieldCapabilitiesResponse) @@ -435,10 +435,9 @@ public void testGetResultMappings_DependentVariableMappingIsAbsent() { } public void testGetResultMappings_DependentVariableMappingHasNoTypes() { - FieldCapabilitiesResponse fieldCapabilitiesResponse = new FieldCapabilitiesResponse( - new String[0], - Collections.singletonMap("foo", Collections.emptyMap()) - ); + FieldCapabilitiesResponse fieldCapabilitiesResponse = FieldCapabilitiesResponse.builder() + .withFields(Collections.singletonMap("foo", Collections.emptyMap())) + .build(); expectThrows( ElasticsearchStatusException.class, () -> new Classification("foo").getResultMappings("results", fieldCapabilitiesResponse) @@ -459,10 +458,9 @@ public void testGetResultMappings_DependentVariableMappingIsPresent() { Map.of("type", "double") ) ); - FieldCapabilitiesResponse fieldCapabilitiesResponse = new FieldCapabilitiesResponse( - new String[0], - Collections.singletonMap("foo", Collections.singletonMap("dummy", createFieldCapabilities("foo", "dummy"))) - ); + FieldCapabilitiesResponse fieldCapabilitiesResponse = FieldCapabilitiesResponse.builder() + .withFields(Collections.singletonMap("foo", Collections.singletonMap("dummy", createFieldCapabilities("foo", "dummy")))) + .build(); Map resultMappings = new Classification("foo").getResultMappings("results", fieldCapabilitiesResponse); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResolveFieldsAction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResolveFieldsAction.java index 47574a6530837..e242a3d6d98ae 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResolveFieldsAction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlResolveFieldsAction.java @@ -46,8 +46,6 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -148,12 +146,7 @@ private void doExecuteForked(Task task, FieldCapabilitiesRequest request, Action if (concreteIndices.length == 0 && remoteClusterIndices.isEmpty()) { // No indices at all! - listener.onResponse( - new EsqlResolveFieldsResponse( - new FieldCapabilitiesResponse(new String[0], Collections.emptyMap()), - minTransportVersion.get() - ) - ); + listener.onResponse(new EsqlResolveFieldsResponse(FieldCapabilitiesResponse.empty(), minTransportVersion.get())); return; } @@ -335,7 +328,9 @@ private static void finishHim( ) { List failures = indexFailures.build(indexResponses.keySet()); if (indexResponses.isEmpty() == false) { - listener.onResponse(new FieldCapabilitiesResponse(new ArrayList<>(indexResponses.values()), failures)); + listener.onResponse( + FieldCapabilitiesResponse.builder().withIndexResponses(indexResponses.values()).withFailures(failures).build() + ); } else { // we have no responses at all, maybe because of errors if (indexFailures.isEmpty() == false) { @@ -348,13 +343,13 @@ private static void finishHim( failure -> failure.getException() instanceof IllegalStateException ise && ise.getCause() instanceof ElasticsearchTimeoutException )) { - listener.onResponse(new FieldCapabilitiesResponse(Collections.emptyList(), failures)); + listener.onResponse(FieldCapabilitiesResponse.builder().withFailures(failures).build()); } else { // throw back the first exception listener.onFailure(failures.get(0).getException()); } } else { - listener.onResponse(new FieldCapabilitiesResponse(Collections.emptyList(), Collections.emptyList())); + listener.onResponse(FieldCapabilitiesResponse.empty()); } } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java index 1a096df1b0648..8067a8f0a7050 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -3152,13 +3152,14 @@ public void testResolveInsist_multiIndexFieldPartiallyMappedWithSingleKeywordTyp IndexResolution resolution = IndexResolver.mergedMappings( "foo, bar", new IndexResolver.FieldsInfo( - new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse("foo", messageResponseMap("keyword")), - fieldCapabilitiesIndexResponse("bar", Map.of()) - ), - List.of() - ), + FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse("foo", messageResponseMap("keyword")), + fieldCapabilitiesIndexResponse("bar", Map.of()) + ) + ) + .build(), true, true ) @@ -3179,13 +3180,14 @@ public void testResolveInsist_multiIndexFieldExistsWithSingleTypeButIsNotKeyword IndexResolution resolution = IndexResolver.mergedMappings( "foo, bar", new IndexResolver.FieldsInfo( - new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), - fieldCapabilitiesIndexResponse("bar", Map.of()) - ), - List.of() - ), + FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), + fieldCapabilitiesIndexResponse("bar", Map.of()) + ) + ) + .build(), true, true ) @@ -3207,14 +3209,15 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesNoKeyw IndexResolution resolution = IndexResolver.mergedMappings( "foo, bar", new IndexResolver.FieldsInfo( - new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), - fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")), - fieldCapabilitiesIndexResponse("bazz", Map.of()) - ), - List.of() - ), + FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), + fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")), + fieldCapabilitiesIndexResponse("bazz", Map.of()) + ) + ) + .build(), true, true ) @@ -3235,13 +3238,14 @@ public void testResolveInsist_multiIndexSameMapping_fieldIsMapped() { IndexResolution resolution = IndexResolver.mergedMappings( "foo, bar", new IndexResolver.FieldsInfo( - new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), - fieldCapabilitiesIndexResponse("bar", messageResponseMap("long")) - ), - List.of() - ), + FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), + fieldCapabilitiesIndexResponse("bar", messageResponseMap("long")) + ) + ) + .build(), true, true ) @@ -3260,15 +3264,16 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithKe IndexResolution resolution = IndexResolver.mergedMappings( "foo, bar", new IndexResolver.FieldsInfo( - new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), - fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")), - fieldCapabilitiesIndexResponse("bazz", messageResponseMap("keyword")), - fieldCapabilitiesIndexResponse("qux", Map.of()) - ), - List.of() - ), + FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), + fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")), + fieldCapabilitiesIndexResponse("bazz", messageResponseMap("keyword")), + fieldCapabilitiesIndexResponse("qux", Map.of()) + ) + ) + .build(), true, true ) @@ -3289,14 +3294,15 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithCa IndexResolution resolution = IndexResolver.mergedMappings( "foo, bar", new IndexResolver.FieldsInfo( - new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), - fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")), - fieldCapabilitiesIndexResponse("bazz", Map.of()) - ), - List.of() - ), + FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse("foo", messageResponseMap("long")), + fieldCapabilitiesIndexResponse("bar", messageResponseMap("date")), + fieldCapabilitiesIndexResponse("bazz", Map.of()) + ) + ) + .build(), true, true ) @@ -3313,10 +3319,11 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithCa } public void testResolveDenseVector() { - FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse( - List.of(fieldCapabilitiesIndexResponse("foo", Map.of("v", new IndexFieldCapabilitiesBuilder("v", "dense_vector").build()))), - List.of() - ); + FieldCapabilitiesResponse caps = FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of(fieldCapabilitiesIndexResponse("foo", Map.of("v", new IndexFieldCapabilitiesBuilder("v", "dense_vector").build()))) + ) + .build(); { IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true)); var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER)); @@ -3332,15 +3339,16 @@ public void testResolveDenseVector() { } public void testResolveAggregateMetricDouble() { - FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse( - List.of( - fieldCapabilitiesIndexResponse( - "foo", - Map.of("v", new IndexFieldCapabilitiesBuilder("v", "aggregate_metric_double").build()) + FieldCapabilitiesResponse caps = FieldCapabilitiesResponse.builder() + .withIndexResponses( + List.of( + fieldCapabilitiesIndexResponse( + "foo", + Map.of("v", new IndexFieldCapabilitiesBuilder("v", "aggregate_metric_double").build()) + ) ) - ), - List.of() - ); + ) + .build(); { IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true)); var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER)); @@ -3801,7 +3809,11 @@ private static LogicalPlan analyzeWithEmptyFieldCapsResponse(String query) throw List idxResponses = List.of( new FieldCapabilitiesIndexResponse("idx", "idx", Map.of(), true, IndexMode.STANDARD) ); - IndexResolver.FieldsInfo caps = new IndexResolver.FieldsInfo(new FieldCapabilitiesResponse(idxResponses, List.of()), true, true); + IndexResolver.FieldsInfo caps = new IndexResolver.FieldsInfo( + FieldCapabilitiesResponse.builder().withIndexResponses(idxResponses).build(), + true, + true + ); IndexResolution resolution = IndexResolver.mergedMappings("test*", caps); var analyzer = analyzer(resolution, TEST_VERIFIER, configuration(query)); return analyze(query, analyzer); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolverTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolverTests.java index 9cb735b955d09..fc85cb2bf5791 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolverTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolverTests.java @@ -506,9 +506,9 @@ protected void fieldCaps.put(e.getKey(), f); } var indexResponse = new FieldCapabilitiesIndexResponse(alias, null, fieldCaps, true, IndexMode.STANDARD); - response = new FieldCapabilitiesResponse(List.of(indexResponse), List.of()); + response = FieldCapabilitiesResponse.builder().withIndexResponses(List.of(indexResponse)).build(); } else { - response = new FieldCapabilitiesResponse(List.of(), List.of()); + response = FieldCapabilitiesResponse.empty(); } threadPool().executor(ThreadPool.Names.SEARCH_COORDINATION) .execute( diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java index bf8434e3c11c5..db24318314443 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/telemetry/PlanExecutorMetricsTests.java @@ -150,7 +150,7 @@ public void testFailedMetric() { // simulate a valid field_caps response so we can parse and correctly analyze de query listener.onResponse( new EsqlResolveFieldsResponse( - new FieldCapabilitiesResponse(indexFieldCapabilities(indices), List.of()), + FieldCapabilitiesResponse.builder().withIndexResponses(indexFieldCapabilities(indices)).build(), TransportVersion.current() ) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java index fbb22c49af331..7fe7fb2150b02 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java @@ -52,7 +52,7 @@ private void resolve(String esTypeName, TimeSeriesParams.MetricType metricType, ) ); - FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse(idxResponses, List.of()); + FieldCapabilitiesResponse caps = FieldCapabilitiesResponse.builder().withIndexResponses(idxResponses).build(); // IndexResolver uses EsqlDataTypeRegistry directly IndexResolution resolution = IndexResolver.mergedMappings("idx-*", new IndexResolver.FieldsInfo(caps, true, true)); EsField f = resolution.get().mapping().get(field); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DestinationIndexTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DestinationIndexTests.java index b7646f430726a..a3ebdadb20db8 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DestinationIndexTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DestinationIndexTests.java @@ -285,14 +285,20 @@ private Map testCreateDestinationIndex(DataFrameAnalysis analysi doAnswer(callListenerOnResponse(getMappingsResponse)).when(client) .execute(eq(GetMappingsAction.INSTANCE), getMappingsRequestCaptor.capture(), any()); - FieldCapabilitiesResponse fieldCapabilitiesResponse = new FieldCapabilitiesResponse(new String[0], new HashMap<>() { - { - put(NUMERICAL_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - put(OUTER_FIELD + "." + INNER_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - put(ALIAS_TO_NUMERICAL_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - put(ALIAS_TO_NESTED_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - } - }); + FieldCapabilitiesResponse fieldCapabilitiesResponse = FieldCapabilitiesResponse.builder() + .withFields( + Map.of( + NUMERICAL_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")), + OUTER_FIELD + "." + INNER_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")), + ALIAS_TO_NUMERICAL_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")), + ALIAS_TO_NESTED_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")) + ) + ) + .build(); doAnswer(callListenerOnResponse(fieldCapabilitiesResponse)).when(client) .execute(eq(TransportFieldCapabilitiesAction.TYPE), fieldCapabilitiesRequestCaptor.capture(), any()); @@ -615,14 +621,20 @@ private Map testUpdateMappingsToDestIndex(DataFrameAnalysis anal doAnswer(callListenerOnResponse(AcknowledgedResponse.TRUE)).when(client) .execute(eq(TransportPutMappingAction.TYPE), putMappingRequestCaptor.capture(), any()); - FieldCapabilitiesResponse fieldCapabilitiesResponse = new FieldCapabilitiesResponse(new String[0], new HashMap<>() { - { - put(NUMERICAL_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - put(OUTER_FIELD + "." + INNER_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - put(ALIAS_TO_NUMERICAL_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - put(ALIAS_TO_NESTED_FIELD, singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer"))); - } - }); + FieldCapabilitiesResponse fieldCapabilitiesResponse = FieldCapabilitiesResponse.builder() + .withFields( + Map.of( + NUMERICAL_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")), + OUTER_FIELD + "." + INNER_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")), + ALIAS_TO_NUMERICAL_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")), + ALIAS_TO_NESTED_FIELD, + singletonMap("integer", createFieldCapabilities(NUMERICAL_FIELD, "integer")) + ) + ) + .build(); doAnswer(callListenerOnResponse(fieldCapabilitiesResponse)).when(client) .execute(eq(TransportFieldCapabilitiesAction.TYPE), fieldCapabilitiesRequestCaptor.capture(), any()); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorTests.java index f028f39c6069f..4b9c25d71f452 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorTests.java @@ -1724,7 +1724,7 @@ private MockFieldCapsResponseBuilder addField(String field, boolean isMetadataFi } private FieldCapabilitiesResponse build() { - return new FieldCapabilitiesResponse(new String[] { "test" }, fieldCaps); + return FieldCapabilitiesResponse.builder().withIndices(new String[] { "test" }).withFields(fieldCaps).build(); } } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java index 0610721c04537..27a4adb910bf0 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java @@ -600,7 +600,7 @@ private static IndexResolution mergedMappings( return IndexResolver.mergedMappings( SqlDataTypeRegistry.INSTANCE, indexPattern, - new FieldCapabilitiesResponse(indexNames, fieldCaps) + FieldCapabilitiesResponse.builder().withIndices(indexNames).withFields(fieldCaps).build() ); } @@ -612,7 +612,7 @@ private static List separateMappings( return IndexResolver.separateMappings( SqlDataTypeRegistry.INSTANCE, javaRegex, - new FieldCapabilitiesResponse(indexNames, fieldCaps), + FieldCapabilitiesResponse.builder().withIndices(indexNames).withFields(fieldCaps).build(), null ); } diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/common/DocumentConversionUtilsTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/common/DocumentConversionUtilsTests.java index 18af78d704646..49b58f0dde2b4 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/common/DocumentConversionUtilsTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/common/DocumentConversionUtilsTests.java @@ -87,16 +87,18 @@ public void testRemoveInternalFields() { } public void testExtractFieldMappings() { - FieldCapabilitiesResponse response = new FieldCapabilitiesResponse( - new String[] { "some-index" }, - Map.ofEntries( - entry("field-1", Map.of("keyword", createFieldCapabilities("field-1", "keyword"))), - entry( - "field-2", - Map.of("long", createFieldCapabilities("field-2", "long"), "keyword", createFieldCapabilities("field-2", "keyword")) + FieldCapabilitiesResponse response = FieldCapabilitiesResponse.builder() + .withIndices(new String[] { "some-index" }) + .withFields( + Map.ofEntries( + entry("field-1", Map.of("keyword", createFieldCapabilities("field-1", "keyword"))), + entry( + "field-2", + Map.of("long", createFieldCapabilities("field-2", "long"), "keyword", createFieldCapabilities("field-2", "keyword")) + ) ) ) - ); + .build(); assertThat( DocumentConversionUtils.extractFieldMappings(response), diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/pivot/SchemaUtilTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/pivot/SchemaUtilTests.java index d65428a3912de..580e3fb2bdbfa 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/pivot/SchemaUtilTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/pivot/SchemaUtilTests.java @@ -287,7 +287,10 @@ protected void responseMap.put(field, singletonMap(field, createFieldCapabilities(field, type))); } - final FieldCapabilitiesResponse response = new FieldCapabilitiesResponse(fieldCapsRequest.indices(), responseMap); + final FieldCapabilitiesResponse response = FieldCapabilitiesResponse.builder() + .withIndices(fieldCapsRequest.indices()) + .withFields(responseMap) + .build(); listener.onResponse((Response) response); return; }