From abe01d8395a215be57b8838d76630f67840d4245 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Thu, 27 Jan 2022 17:19:13 +0000 Subject: [PATCH 01/11] Add filters to field caps + bwc --- .../fieldcaps/FieldCapabilitiesFetcher.java | 163 ++++++++----- .../FieldCapabilitiesNodeRequest.java | 18 ++ .../fieldcaps/FieldCapabilitiesRequest.java | 21 ++ .../fieldcaps/IndexFieldCapabilities.java | 13 ++ .../action/fieldcaps/RequestDispatcher.java | 1 + .../action/fieldcaps/ResponseRewriter.java | 120 ++++++++++ .../TransportFieldCapabilitiesAction.java | 31 ++- .../index/mapper/FieldTypeLookup.java | 6 + .../index/mapper/MapperService.java | 4 + .../index/mapper/MappingLookup.java | 3 + .../index/query/SearchExecutionContext.java | 4 + .../FieldCapabilitiesFilterTests.java | 215 ++++++++++++++++++ .../FieldCapabilitiesNodeRequestTests.java | 32 ++- .../FieldCapabilitiesRequestTests.java | 18 +- .../fieldcaps/ResponseRewriterTests.java | 148 ++++++++++++ 15 files changed, 718 insertions(+), 79 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java create mode 100644 server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java create mode 100644 server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java index 7eac2f54c1137..90dd04ff0e88a 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java @@ -45,6 +45,7 @@ class FieldCapabilitiesFetcher { FieldCapabilitiesIndexResponse fetch( ShardId shardId, String[] fieldPatterns, + String[] filters, QueryBuilder indexFilter, long nowInMillis, Map runtimeFields @@ -66,67 +67,91 @@ FieldCapabilitiesIndexResponse fetch( return new FieldCapabilitiesIndexResponse(shardId.getIndexName(), Collections.emptyMap(), false); } - Set fieldNames = new HashSet<>(); - for (String pattern : fieldPatterns) { - fieldNames.addAll(searchExecutionContext.getMatchingFieldNames(pattern)); - } - Predicate fieldPredicate = indicesService.getFieldFilter().apply(shardId.getIndexName()); - Map responseMap = new HashMap<>(); - for (String field : fieldNames) { - MappedFieldType ft = searchExecutionContext.getFieldType(field); - boolean isMetadataField = searchExecutionContext.isMetadataField(field); - if (isMetadataField || fieldPredicate.test(ft.name())) { - IndexFieldCapabilities fieldCap = new IndexFieldCapabilities( - field, - ft.familyTypeName(), - isMetadataField, - ft.isSearchable(), - ft.isAggregatable(), - ft.isDimension(), - ft.getMetricType(), - ft.meta() - ); - responseMap.put(field, fieldCap); - } else { - continue; - } - // Check the ancestor of the field to find nested and object fields. - // Runtime fields are excluded since they can override any path. - // TODO find a way to do this that does not require an instanceof check - if (ft instanceof RuntimeField == false) { - int dotIndex = ft.name().lastIndexOf('.'); - while (dotIndex > -1) { - String parentField = ft.name().substring(0, dotIndex); - if (responseMap.containsKey(parentField)) { - // we added this path on another field already - break; - } - // checks if the parent field contains sub-fields - if (searchExecutionContext.getFieldType(parentField) == null) { - // no field type, it must be an object field - String type = searchExecutionContext.nestedLookup().getNestedMappers().get(parentField) != null - ? "nested" - : "object"; - IndexFieldCapabilities fieldCap = new IndexFieldCapabilities( - parentField, - type, - false, - false, - false, - false, - null, - Collections.emptyMap() - ); - responseMap.put(parentField, fieldCap); - } - dotIndex = parentField.lastIndexOf('.'); + return retrieveFieldCaps(shardId.getIndexName(), searchExecutionContext, fieldPatterns, filters, fieldPredicate); + } + } + + public static FieldCapabilitiesIndexResponse retrieveFieldCaps( + String indexName, + SearchExecutionContext context, + String[] fieldPatterns, + String[] filters, + Predicate indexFieldfilter + ) { + + Set fieldNames = new HashSet<>(); + for (String pattern : fieldPatterns) { + fieldNames.addAll(context.getMatchingFieldNames(pattern)); + } + + boolean includeParentObjects = checkIncludeParents(filters); + + FieldCapsFilter filter = buildFilter(indexFieldfilter, filters); + Map responseMap = new HashMap<>(); + for (String field : fieldNames) { + MappedFieldType ft = context.getFieldType(field); + if (filter.matches(ft, context)) { + IndexFieldCapabilities fieldCap = new IndexFieldCapabilities( + field, + ft.familyTypeName(), + context.isMetadataField(field), + ft.isSearchable(), + ft.isAggregatable(), + ft.isDimension(), + ft.getMetricType(), + ft.meta() + ); + responseMap.put(field, fieldCap); + } else { + continue; + } + + // Check the ancestor of the field to find nested and object fields. + // Runtime fields are excluded since they can override any path. + // TODO find a way to do this that does not require an instanceof check + if (ft instanceof RuntimeField == false && includeParentObjects) { + int dotIndex = ft.name().lastIndexOf('.'); + while (dotIndex > -1) { + String parentField = ft.name().substring(0, dotIndex); + if (responseMap.containsKey(parentField)) { + // we added this path on another field already + break; + } + // checks if the parent field contains sub-fields + if (context.getFieldType(parentField) == null) { + // no field type, it must be an object field + String type = context.nestedLookup().getNestedMappers().get(parentField) != null ? "nested" : "object"; + IndexFieldCapabilities fieldCap = new IndexFieldCapabilities( + parentField, + type, + false, + false, + false, + false, + null, + Collections.emptyMap() + ); + responseMap.put(parentField, fieldCap); } + dotIndex = parentField.lastIndexOf('.'); } } - return new FieldCapabilitiesIndexResponse(shardId.getIndexName(), responseMap, true); } + return new FieldCapabilitiesIndexResponse(indexName, responseMap, true); + } + + private static boolean checkIncludeParents(String[] filters) { + for (String filter : filters) { + if ("-parent".equals(filter)) { + return false; + } + if ("parent".equals(filter)) { + return true; + } + } + return true; } private boolean canMatchShard( @@ -144,4 +169,32 @@ private boolean canMatchShard( return SearchService.queryStillMatchesAfterRewrite(searchRequest, searchExecutionContext); } + private interface FieldCapsFilter { + boolean matches(MappedFieldType fieldType, SearchExecutionContext context); + + default FieldCapsFilter and(FieldCapsFilter other) { + return (ft, context) -> matches(ft, context) && other.matches(ft, context); + } + } + + private static FieldCapsFilter buildFilter(Predicate fieldFilter, String[] filters) { + // security filters don't exclude metadata fields + FieldCapsFilter fcf = (ft, c) -> fieldFilter.test(ft.name()) || c.isMetadataField(ft.name()); + for (String filter : filters) { + if ("parent".equals(filter) || "-parent".equals(filter)) { + continue; + } + FieldCapsFilter next = switch (filter) { + case "+metadata" -> (ft, c) -> c.isMetadataField(ft.name()); + case "-metadata" -> (ft, c) -> c.isMetadataField(ft.name()) == false; + case "-nested" -> (ft, c) -> c.nestedLookup().getNestedParent(ft.name()) == null; + case "+dimension" -> (ft, c) -> ft.isDimension(); + case "-multifield" -> (ft, c) -> c.isMultiField(ft.name()) == false; + default -> throw new IllegalArgumentException("Unknown field caps filter [" + filter + "]"); + }; + fcf = fcf.and(next); + } + return fcf; + } + } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java index a35ed069c63b9..be1d8495c19d3 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java @@ -8,11 +8,13 @@ package org.elasticsearch.action.fieldcaps; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.query.QueryBuilder; @@ -28,6 +30,7 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque private final List shardIds; private final String[] fields; + private final String[] filters; private final OriginalIndices originalIndices; private final QueryBuilder indexFilter; private final long nowInMillis; @@ -37,6 +40,11 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque super(in); shardIds = in.readList(ShardId::new); fields = in.readStringArray(); + if (in.getVersion().onOrAfter(Version.V_8_1_0)) { + filters = in.readStringArray(); + } else { + filters = Strings.EMPTY_ARRAY; + } originalIndices = OriginalIndices.readOriginalIndices(in); indexFilter = in.readOptionalNamedWriteable(QueryBuilder.class); nowInMillis = in.readLong(); @@ -46,6 +54,7 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque FieldCapabilitiesNodeRequest( List shardIds, String[] fields, + String[] filters, OriginalIndices originalIndices, QueryBuilder indexFilter, long nowInMillis, @@ -53,6 +62,7 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque ) { this.shardIds = Objects.requireNonNull(shardIds); this.fields = fields; + this.filters = filters; this.originalIndices = originalIndices; this.indexFilter = indexFilter; this.nowInMillis = nowInMillis; @@ -63,6 +73,10 @@ public String[] fields() { return fields; } + public String[] filters() { + return filters; + } + public OriginalIndices originalIndices() { return originalIndices; } @@ -98,6 +112,9 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeList(shardIds); out.writeStringArray(fields); + if (out.getVersion().onOrAfter(Version.V_8_1_0)) { + out.writeStringArray(filters); + } OriginalIndices.writeOriginalIndices(originalIndices, out); out.writeOptionalNamedWriteable(indexFilter); out.writeLong(nowInMillis); @@ -117,6 +134,7 @@ public boolean equals(Object o) { return nowInMillis == that.nowInMillis && shardIds.equals(that.shardIds) && Arrays.equals(fields, that.fields) + && Arrays.equals(filters, that.filters) && Objects.equals(originalIndices, that.originalIndices) && Objects.equals(indexFilter, that.indexFilter) && Objects.equals(runtimeFields, that.runtimeFields); 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 f66ef34972385..9ff313f9c0b8b 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.fieldcaps; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; @@ -35,6 +36,7 @@ public final class FieldCapabilitiesRequest extends ActionRequest implements Ind private String[] indices = Strings.EMPTY_ARRAY; private IndicesOptions indicesOptions = DEFAULT_INDICES_OPTIONS; private String[] fields = Strings.EMPTY_ARRAY; + private String[] filters = Strings.EMPTY_ARRAY; private boolean includeUnmapped = false; // pkg private API mainly for cross cluster search to signal that we do multiple reductions ie. the results should not be merged private boolean mergeResults = true; @@ -52,6 +54,9 @@ public FieldCapabilitiesRequest(StreamInput in) throws IOException { indexFilter = in.readOptionalNamedWriteable(QueryBuilder.class); nowInMillis = in.readOptionalLong(); runtimeFields = in.readMap(); + if (in.getVersion().onOrAfter(Version.V_8_1_0)) { + filters = in.readStringArray(); + } } public FieldCapabilitiesRequest() {} @@ -86,6 +91,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalNamedWriteable(indexFilter); out.writeOptionalLong(nowInMillis); out.writeMap(runtimeFields); + if (out.getVersion().onOrAfter(Version.V_8_1_0)) { + out.writeStringArray(filters); + } } @Override @@ -117,6 +125,15 @@ public String[] fields() { return fields; } + public FieldCapabilitiesRequest filters(String... filters) { + this.filters = filters; + return this; + } + + public String[] filters() { + return filters; + } + /** * The list of indices to lookup */ @@ -213,6 +230,7 @@ public boolean equals(Object o) { && Arrays.equals(fields, that.fields) && Objects.equals(indexFilter, that.indexFilter) && Objects.equals(nowInMillis, that.nowInMillis) + && Arrays.equals(filters, that.filters) && Objects.equals(runtimeFields, that.runtimeFields); } @@ -221,6 +239,7 @@ public int hashCode() { int result = Objects.hash(indicesOptions, includeUnmapped, mergeResults, indexFilter, nowInMillis, runtimeFields); result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(fields); + result = 31 * result + Arrays.hashCode(filters); return result; } @@ -230,6 +249,8 @@ public String getDescription() { Strings.collectionToDelimitedStringWithLimit(Arrays.asList(indices), ",", "", "", 1024, stringBuilder); stringBuilder.append("], fields["); Strings.collectionToDelimitedStringWithLimit(Arrays.asList(fields), ",", "", "", 1024, stringBuilder); + stringBuilder.append("], filters["); + stringBuilder.append(Strings.collectionToDelimitedString(Arrays.asList(filters), ",")); stringBuilder.append("]"); return stringBuilder.toString(); } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/IndexFieldCapabilities.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/IndexFieldCapabilities.java index 674a8e7da8456..6cefa39f7f6ab 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/IndexFieldCapabilities.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/IndexFieldCapabilities.java @@ -35,6 +35,19 @@ public class IndexFieldCapabilities implements Writeable { private final TimeSeriesParams.MetricType metricType; private final Map meta; + public static IndexFieldCapabilities withMetadata(IndexFieldCapabilities input, boolean isMetadata) { + return new IndexFieldCapabilities( + input.getName(), + input.getType(), + isMetadata, + input.isSearchable, + input.isAggregatable, + input.isDimension, + input.metricType, + input.meta + ); + } + /** * @param name The name of the field. * @param type The type associated with the field. diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java index dc2b99afa46cd..1257a172f331a 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java @@ -174,6 +174,7 @@ private void sendRequestToNode(String nodeId, List shardIds) { final FieldCapabilitiesNodeRequest nodeRequest = new FieldCapabilitiesNodeRequest( shardIds, fieldCapsRequest.fields(), + fieldCapsRequest.filters(), originalIndices, fieldCapsRequest.indexFilter(), nowInMillis, diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java new file mode 100644 index 0000000000000..3acc06c2c84eb --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java @@ -0,0 +1,120 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.fieldcaps; + +import org.elasticsearch.Version; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +final class ResponseRewriter { + + public static Map rewriteOldResponses( + Version version, + Map input, + String[] filters, + Predicate isMetadata + ) { + if (version.onOrAfter(Version.V_8_1_0)) { + return input; // nothing needs to be done + } + Function transformer = buildTransformer(version, input, filters, isMetadata); + Map rewritten = new HashMap<>(); + for (var entry : input.entrySet()) { + IndexFieldCapabilities fc = transformer.apply(entry.getValue()); + if (fc != null) { + rewritten.put(entry.getKey(), fc); + } + } + return rewritten; + } + + private static Function buildTransformer( + Version version, + Map input, + String[] filters, + Predicate isMetadata + ) { + boolean checkMetadata = version.before(Version.V_7_13_0); + Predicate test = ifc -> true; + Set objects = null; + Set nestedObjects = null; + for (String filter : filters) { + if ("-parent".equals(filter)) { + test = test.and(fc -> fc.getType().equals("nested") == false && fc.getType().equals("object") == false); + } + if ("-metadata".equals(filter)) { + test = test.and(fc -> fc.isMetadatafield() == false); + } + if ("+metadata".equals(filter)) { + test = test.and(IndexFieldCapabilities::isMetadatafield); + } + if ("+dimension".equals(filter)) { + test = test.and(IndexFieldCapabilities::isDimension); + } + if ("-nested".equals(filter)) { + if (nestedObjects == null) { + nestedObjects = findTypes("nested", input); + } + Set no = nestedObjects; + test = test.and(fc -> isNestedField(fc.getName(), no) == false); + } + if ("-multifield".equals(filter)) { + // immediate parent is not an object field + if (objects == null) { + objects = findTypes("object", input); + } + Set o = objects; + test = test.and(fc -> isNotMultifield(fc.getName(), o)); + } + } + Predicate finalTest = test; + return fc -> { + IndexFieldCapabilities rewritten = fc; + if (checkMetadata) { + rewritten = IndexFieldCapabilities.withMetadata(fc, isMetadata.test(fc.getName())); + } + if (finalTest.test(rewritten) == false) { + return null; + } + return rewritten; + }; + } + + private static Set findTypes(String type, Map fieldCaps) { + return fieldCaps.entrySet() + .stream() + .filter(entry -> type.equals(entry.getValue().getType())) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + } + + private static boolean isNestedField(String field, Set nestedParents) { + for (String parent : nestedParents) { + if (field.startsWith(parent + ".") || field.equals(parent)) { + return true; + } + } + return false; + } + + private static boolean isNotMultifield(String field, Set objectFields) { + int lastDotPos = field.lastIndexOf("."); + if (lastDotPos == -1) { + return true; + } + String parent = field.substring(0, lastDotPos); + return objectFields.contains(parent); + } + +} 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 7429ec5e8b50a..2526699b4a739 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -9,7 +9,6 @@ package org.elasticsearch.action.fieldcaps; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.OriginalIndices; @@ -170,12 +169,7 @@ private Runnable createResponseMerger( // fork off to the management pool for merging the responses as the operation can run for longer than is acceptable // on a transport thread in case of large numbers of indices and/or fields threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION) - .submit( - ActionRunnable.supply( - listener, - () -> merge(indexResponses, request.includeUnmapped(), new ArrayList<>(failures)) - ) - ); + .submit(ActionRunnable.supply(listener, () -> merge(indexResponses, request, new ArrayList<>(failures)))); } else { listener.onResponse( new FieldCapabilitiesResponse(new ArrayList<>(indexResponses.values()), new ArrayList<>(failures)) @@ -204,6 +198,7 @@ private static FieldCapabilitiesRequest prepareRemoteRequest( remoteRequest.indicesOptions(originalIndices.indicesOptions()); remoteRequest.indices(originalIndices.indices()); remoteRequest.fields(request.fields()); + remoteRequest.filters(request.filters()); remoteRequest.runtimeFields(request.runtimeFields()); remoteRequest.indexFilter(request.indexFilter()); remoteRequest.nowInMillis(nowInMillis); @@ -212,18 +207,18 @@ private static FieldCapabilitiesRequest prepareRemoteRequest( private FieldCapabilitiesResponse merge( Map indexResponses, - boolean includeUnmapped, + FieldCapabilitiesRequest request, List failures ) { String[] indices = indexResponses.keySet().stream().sorted().toArray(String[]::new); final Map> responseMapBuilder = new HashMap<>(); for (FieldCapabilitiesIndexResponse response : indexResponses.values()) { - innerMerge(responseMapBuilder, response); + innerMerge(responseMapBuilder, request, response); } final Map> responseMap = new HashMap<>(); for (Map.Entry> entry : responseMapBuilder.entrySet()) { final Map typeMapBuilder = entry.getValue(); - if (includeUnmapped) { + if (request.includeUnmapped()) { addUnmappedFields(indices, entry.getKey(), typeMapBuilder); } boolean multiTypes = typeMapBuilder.size() > 1; @@ -250,14 +245,17 @@ private void addUnmappedFields(String[] indices, String field, Map> responseMapBuilder, + FieldCapabilitiesRequest request, FieldCapabilitiesIndexResponse response ) { - for (Map.Entry entry : response.get().entrySet()) { + Map fields = ResponseRewriter.rewriteOldResponses( + response.getOriginVersion(), + response.get(), + request.filters(), + metadataFieldPred + ); + for (Map.Entry entry : fields.entrySet()) { final String field = entry.getKey(); - // best effort to detect metadata field coming from older nodes - final boolean isMetadataField = response.getOriginVersion().onOrAfter(Version.V_7_13_0) - ? entry.getValue().isMetadatafield() - : metadataFieldPred.test(field); final IndexFieldCapabilities fieldCap = entry.getValue(); Map typeMap = responseMapBuilder.computeIfAbsent(field, f -> new HashMap<>()); FieldCapabilities.Builder builder = typeMap.computeIfAbsent( @@ -266,7 +264,7 @@ private void innerMerge( ); builder.add( response.getIndexName(), - isMetadataField, + fieldCap.isMetadatafield(), fieldCap.isSearchable(), fieldCap.isAggregatable(), fieldCap.isDimension(), @@ -341,6 +339,7 @@ public void messageReceived(FieldCapabilitiesNodeRequest request, TransportChann final FieldCapabilitiesIndexResponse response = fieldCapabilitiesFetcher.fetch( shardId, request.fields(), + request.filters(), request.indexFilter(), request.nowInMillis(), request.runtimeFields() diff --git a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java index af51b42deda80..c67d64bd2048c 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/FieldTypeLookup.java @@ -25,6 +25,7 @@ final class FieldTypeLookup { private final Map fullNameToFieldType = new HashMap<>(); private final Map dynamicFieldTypes = new HashMap<>(); + private final Set runtimeFieldNames = new HashSet<>(); /** * A map from field name to all fields whose content has been copied into it @@ -80,6 +81,7 @@ final class FieldTypeLookup { for (MappedFieldType fieldType : RuntimeField.collectFieldTypes(runtimeFields).values()) { // this will override concrete fields with runtime fields that have the same name fullNameToFieldType.put(fieldType.name(), fieldType); + runtimeFieldNames.add(fieldType.name()); } } @@ -104,6 +106,10 @@ MappedFieldType get(String field) { return getDynamicField(field); } + boolean isRuntimeField(String field) { + return runtimeFieldNames.contains(field); + } + // for testing int getMaxParentPathDots() { return maxParentPathDots; diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 8d5ff6df45b00..d2615641d468f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -510,6 +510,10 @@ public boolean isMetadataField(String field) { return mapperRegistry.getMetadataMapperParsers(indexVersionCreated).containsKey(field); } + public boolean isMultiField(String field) { + return mappingLookup().isMultiField(field); + } + public synchronized List reloadSearchAnalyzers(AnalysisRegistry registry) throws IOException { logger.info("reloading search analyzers"); // refresh indexAnalyzers and search analyzers diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java index 5ce6b05556459..e35b283af0197 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -336,6 +336,9 @@ public NestedLookup nestedLookup() { } public boolean isMultiField(String field) { + if (fieldTypeLookup.isRuntimeField(field)) { + return false; + } String sourceParent = parentObject(field); return sourceParent != null && fieldMappers.containsKey(sourceParent); } diff --git a/server/src/main/java/org/elasticsearch/index/query/SearchExecutionContext.java b/server/src/main/java/org/elasticsearch/index/query/SearchExecutionContext.java index 457f7160306eb..ee584dd9acb38 100644 --- a/server/src/main/java/org/elasticsearch/index/query/SearchExecutionContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/SearchExecutionContext.java @@ -374,6 +374,10 @@ public boolean isMetadataField(String field) { return mapperService.isMetadataField(field); } + public boolean isMultiField(String field) { + return mapperService.isMultiField(field); + } + public Set sourcePath(String fullName) { return mappingLookup.sourcePaths(fullName); } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java new file mode 100644 index 0000000000000..b8053016dfa8a --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java @@ -0,0 +1,215 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.fieldcaps; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MapperServiceTestCase; +import org.elasticsearch.index.query.SearchExecutionContext; + +import java.io.IOException; +import java.util.function.Predicate; + +public class FieldCapabilitiesFilterTests extends MapperServiceTestCase { + + public void testExcludeNestedFields() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "field1" : { "type" : "keyword" }, + "field2" : { + "type" : "nested", + "properties" : { + "field3" : { "type" : "keyword" } + } + }, + "field4" : { "type" : "keyword" } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "-nested" }, + f -> true + ); + + assertNotNull(response.getField("field1")); + assertNotNull(response.getField("field4")); + assertNull(response.getField("field2")); + assertNull(response.getField("field2.field3")); + } + + public void testDimensionFieldsOnly() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "field1" : { "type" : "keyword" }, + "field2" : { + "type" : "keyword", + "time_series_dimension" : "true" + } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "+dimension" }, + f -> true + ); + + assertNull(response.getField("field1")); + assertNotNull(response.getField("field2")); + assertNull(response.getField("_index")); + } + + public void testMetadataFilters() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "field1" : { "type" : "keyword" }, + "field2" : { "type" : "keyword" } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + + { + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "+metadata" }, + f -> true + ); + assertNotNull(response.getField("_index")); + assertNull(response.getField("field1")); + } + { + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "-metadata" }, + f -> true + ); + assertNull(response.getField("_index")); + assertNotNull(response.getField("field1")); + } + } + + public void testExcludeMultifields() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "field1" : { + "type" : "text", + "fields" : { + "keyword" : { "type" : "keyword" } + } + }, + "field2" : { "type" : "keyword" } + }, + "runtime" : { + "field2.keyword" : { "type" : "keyword" } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "-multifield" }, + f -> true + ); + assertNotNull(response.getField("field1")); + assertNull(response.getField("field1.keyword")); + assertNotNull(response.getField("field2")); + assertNotNull(response.getField("field2.keyword")); + assertNotNull(response.getField("_index")); + } + + public void testDontIncludeParentInfo() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "parent" : { + "properties" : { + "field1" : { "type" : "keyword" }, + "field2" : { "type" : "keyword" } + } + } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "-parent" }, + f -> true + ); + assertNotNull(response.getField("parent.field1")); + assertNotNull(response.getField("parent.field2")); + assertNull(response.getField("parent")); + } + + public void testSecurityFilter() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "permitted1" : { "type" : "keyword" }, + "permitted2" : { "type" : "keyword" }, + "forbidden" : { "type" : "keyword" } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + Predicate securityFilter = f -> f.startsWith("permitted"); + + { + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + Strings.EMPTY_ARRAY, + securityFilter + ); + + assertNotNull(response.getField("permitted1")); + assertNull(response.getField("forbidden")); + assertNotNull(response.getField("_index")); // security filter doesn't apply to metadata + } + + { + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + new String[] { "-metadata" }, + securityFilter + ); + + assertNotNull(response.getField("permitted1")); + assertNull(response.getField("forbidden")); + assertNull(response.getField("_index")); // -metadata filter applies on top + } + } +} diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java index 8810cdb3a7a25..dde25ae32e386 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; @@ -31,6 +32,7 @@ public class FieldCapabilitiesNodeRequestTests extends AbstractWireSerializingTe protected FieldCapabilitiesNodeRequest createTestInstance() { List randomShards = randomShardIds(randomIntBetween(1, 5)); String[] randomFields = randomFields(randomIntBetween(1, 20)); + String[] randomFilter = randomBoolean() ? Strings.EMPTY_ARRAY : new String[] { "-nested" }; OriginalIndices originalIndices = randomOriginalIndices(randomIntBetween(0, 20)); QueryBuilder indexFilter = randomBoolean() ? QueryBuilders.termQuery("field", randomAlphaOfLength(5)) : null; @@ -40,7 +42,15 @@ protected FieldCapabilitiesNodeRequest createTestInstance() { ? Collections.singletonMap(randomAlphaOfLength(5), randomAlphaOfLength(5)) : null; - return new FieldCapabilitiesNodeRequest(randomShards, randomFields, originalIndices, indexFilter, nowInMillis, runtimeFields); + return new FieldCapabilitiesNodeRequest( + randomShards, + randomFields, + randomFilter, + originalIndices, + indexFilter, + nowInMillis, + runtimeFields + ); } private List randomShardIds(int numShards) { @@ -81,12 +91,13 @@ protected Writeable.Reader instanceReader() { @Override protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeRequest instance) throws IOException { - switch (random().nextInt(5)) { + switch (random().nextInt(6)) { case 0 -> { List shardIds = randomShardIds(instance.shardIds().size() + 1); return new FieldCapabilitiesNodeRequest( shardIds, instance.fields(), + instance.filters(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), @@ -98,6 +109,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque return new FieldCapabilitiesNodeRequest( instance.shardIds(), fields, + instance.filters(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), @@ -109,6 +121,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque return new FieldCapabilitiesNodeRequest( instance.shardIds(), instance.fields(), + instance.filters(), originalIndices, instance.indexFilter(), instance.nowInMillis(), @@ -120,6 +133,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque return new FieldCapabilitiesNodeRequest( instance.shardIds(), instance.fields(), + instance.filters(), instance.originalIndices(), indexFilter, instance.nowInMillis(), @@ -131,6 +145,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque return new FieldCapabilitiesNodeRequest( instance.shardIds(), instance.fields(), + instance.filters(), instance.originalIndices(), instance.indexFilter(), nowInMillis, @@ -144,12 +159,25 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque return new FieldCapabilitiesNodeRequest( instance.shardIds(), instance.fields(), + instance.filters(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), runtimeFields ); } + case 6 -> { + String[] randomFilter = randomBoolean() ? Strings.EMPTY_ARRAY : new String[] { "-nested" }; + return new FieldCapabilitiesNodeRequest( + instance.shardIds(), + instance.fields(), + randomFilter, + instance.originalIndices(), + instance.indexFilter(), + instance.nowInMillis(), + instance.runtimeFields() + ); + } default -> throw new IllegalStateException("The test should only allow 5 parameters mutated"); } } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java index 566fa43a8da62..73d17412ac643 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; @@ -146,14 +147,18 @@ public void testValidation() { public void testGetDescription() { final FieldCapabilitiesRequest request = new FieldCapabilitiesRequest(); - assertThat(request.getDescription(), equalTo("indices[], fields[]")); + assertThat(request.getDescription(), equalTo("indices[], fields[], filters[]")); request.fields("a", "b"); - assertThat(request.getDescription(), anyOf(equalTo("indices[], fields[a,b]"), equalTo("indices[], fields[b,a]"))); + assertThat(request.getDescription(), + anyOf(equalTo("indices[], fields[a,b], filters[]"), equalTo("indices[], fields[b,a], filters[]"))); request.indices("x", "y", "z"); request.fields("a"); - assertThat(request.getDescription(), equalTo("indices[x,y,z], fields[a]")); + assertThat(request.getDescription(), equalTo("indices[x,y,z], fields[a], filters[]")); + + request.filters("-metadata", "-multifields"); + assertThat(request.getDescription(), endsWith("filters[-metadata,-multifields]")); final String[] lots = new String[between(1024, 2048)]; for (int i = 0; i < lots.length; i++) { @@ -162,6 +167,7 @@ public void testGetDescription() { request.indices("x", "y", "z"); request.fields(lots); + request.filters(Strings.EMPTY_ARRAY); assertThat( request.getDescription(), allOf( @@ -173,7 +179,7 @@ public void testGetDescription() { ); assertThat( request.getDescription().length(), - lessThanOrEqualTo(1024 + ("indices[x,y,z], fields[" + "s9999,... (9999 in total, 9999 omitted)]").length()) + lessThanOrEqualTo(1024 + ("indices[x,y,z], fields[" + "s9999,... (9999 in total, 9999 omitted)], filters[]").length()) ); request.fields("a"); @@ -185,12 +191,12 @@ public void testGetDescription() { containsString("..."), containsString(lots.length + " in total"), containsString("omitted"), - endsWith("], fields[a]") + endsWith("], fields[a], filters[]") ) ); assertThat( request.getDescription().length(), - lessThanOrEqualTo(1024 + ("indices[" + "s9999,... (9999 in total, 9999 omitted)], fields[a]").length()) + lessThanOrEqualTo(1024 + ("indices[" + "s9999,... (9999 in total, 9999 omitted)], fields[a], filters[]").length()) ); final FieldCapabilitiesRequest randomRequest = createTestInstance(); diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java new file mode 100644 index 0000000000000..d7750dc86919b --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java @@ -0,0 +1,148 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.fieldcaps; + +import org.elasticsearch.Version; +import org.elasticsearch.test.ESTestCase; + +import java.util.Collections; +import java.util.Map; + +public class ResponseRewriterTests extends ESTestCase { + + public void testExcludeMetadata() { + Map oldResponse = Map.of( + "field", + new IndexFieldCapabilities("field", "keyword", false, true, true, false, null, Collections.emptyMap()), + "_index", + new IndexFieldCapabilities("_index", "_index", true, true, true, false, null, Collections.emptyMap()) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + new String[] { "-metadata" }, + f -> f.startsWith("_") + ); + + assertTrue(rewritten.containsKey("field")); + assertFalse(rewritten.containsKey("_index")); + } + + public void testIncludeOnlyMetadata() { + Map oldResponse = Map.of( + "field", + new IndexFieldCapabilities("field", "keyword", false, true, true, false, null, Collections.emptyMap()), + "_index", + new IndexFieldCapabilities("_index", "_index", true, true, true, false, null, Collections.emptyMap()) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + new String[] { "+metadata" }, + f -> f.startsWith("_") + ); + + assertFalse(rewritten.containsKey("field")); + assertTrue(rewritten.containsKey("_index")); + } + + public void testExcludeNested() { + Map oldResponse = Map.of( + "field", + new IndexFieldCapabilities("field", "keyword", false, true, true, false, null, Collections.emptyMap()), + "parent", + new IndexFieldCapabilities("parent", "nested", false, false, false, false, null, Collections.emptyMap()), + "parent.child", + new IndexFieldCapabilities("parent.child", "keyword", false, true, true, false, null, Collections.emptyMap()) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + new String[] { "-nested" }, + f -> f.startsWith("_") + ); + + assertTrue(rewritten.containsKey("field")); + assertFalse(rewritten.containsKey("parent.child")); + assertFalse(rewritten.containsKey("parent")); + } + + public void testExcludeMultifield() { + Map oldResponse = Map.of( + "field", + fieldCaps("field", "text", false), + "field.keyword", + fieldCaps("field.keyword", "keyword", false), + "parent", + fieldCaps("parent", "object", false), + "parent.child", + fieldCaps("parent.child", "keyword", false) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + new String[] { "-multifield" }, + f -> f.startsWith("_") + ); + + assertTrue(rewritten.containsKey("field")); + assertFalse(rewritten.containsKey("field.keyword")); + assertTrue(rewritten.containsKey("parent.child")); + } + + public void testIncludeOnlyDimensions() { + Map oldResponse = Map.of( + "field1", + fieldCaps("field1", "text", false), + "field2", + new IndexFieldCapabilities("field2", "keyword", false, true, true, true, null, Collections.emptyMap()) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + new String[] { "+dimension" }, + f -> f.startsWith("_") + ); + + assertFalse(rewritten.containsKey("field1")); + assertTrue(rewritten.containsKey("field2")); + } + + public void testExcludeParents() { + Map oldResponse = Map.of( + "field", + fieldCaps("field", "text", false), + "parent", + fieldCaps("parent", "object", false), + "parent.child", + fieldCaps("parent.child", "keyword", false) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + new String[] { "-parent" }, + f -> f.startsWith("_") + ); + + assertTrue(rewritten.containsKey("field")); + assertFalse(rewritten.containsKey("parent")); + assertTrue(rewritten.containsKey("parent.child")); + } + + private static IndexFieldCapabilities fieldCaps(String name, String type, boolean isMetadata) { + return new IndexFieldCapabilities(name, type, isMetadata, true, true, false, null, Collections.emptyMap()); + } + +} From 3771d63fffad4231a8c0323b947c634d39b4731b Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Mon, 31 Jan 2022 14:45:24 +0000 Subject: [PATCH 02/11] WIP --- .../rest-api-spec/api/field_caps.json | 4 +++ .../{30_filter.yml => 30_index_filter.yml} | 0 .../test/field_caps/50_fieldtype_filter.yml | 0 .../fieldcaps/FieldCapabilitiesFetcher.java | 2 +- .../action/fieldcaps/ResponseRewriter.java | 4 +-- .../support/TimeSeriesMetricsService.java | 1 + .../FieldCapabilitiesFilterTests.java | 9 +++-- .../fieldcaps/ResponseRewriterTests.java | 33 +++++++++++++------ 8 files changed, 38 insertions(+), 15 deletions(-) rename rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/{30_filter.yml => 30_index_filter.yml} (100%) create mode 100644 rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index f1f5e3992ab4a..c15288eec012a 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -63,6 +63,10 @@ "type":"boolean", "default":false, "description":"Indicates whether unmapped fields should be included in the response." + }, + "filters": { + "type":"list", + "description":"An optional set of filters: can include +metadata,-metadata,-nested,-multifield,+timeseries,parent" } }, "body":{ diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/30_filter.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/30_index_filter.yml similarity index 100% rename from rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/30_filter.yml rename to rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/30_index_filter.yml diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java index 90dd04ff0e88a..aab6d02beb700 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java @@ -188,7 +188,7 @@ private static FieldCapsFilter buildFilter(Predicate fieldFilter, String case "+metadata" -> (ft, c) -> c.isMetadataField(ft.name()); case "-metadata" -> (ft, c) -> c.isMetadataField(ft.name()) == false; case "-nested" -> (ft, c) -> c.nestedLookup().getNestedParent(ft.name()) == null; - case "+dimension" -> (ft, c) -> ft.isDimension(); + case "+timeseries" -> (ft, c) -> ft.isDimension() || ft.getMetricType() != null; case "-multifield" -> (ft, c) -> c.isMultiField(ft.name()) == false; default -> throw new IllegalArgumentException("Unknown field caps filter [" + filter + "]"); }; diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java index 3acc06c2c84eb..6d3bef4616c4b 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java @@ -59,8 +59,8 @@ private static Function buildTra if ("+metadata".equals(filter)) { test = test.and(IndexFieldCapabilities::isMetadatafield); } - if ("+dimension".equals(filter)) { - test = test.and(IndexFieldCapabilities::isDimension); + if ("+timeseries".equals(filter)) { + test = test.and(ifc -> ifc.isDimension() || ifc.getMetricType() != null); } if ("-nested".equals(filter)) { if (nestedObjects == null) { diff --git a/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java b/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java index 45a72b57a2f93..15a30fa3ba02c 100644 --- a/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java +++ b/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java @@ -55,6 +55,7 @@ public void newMetrics(String[] indices, IndicesOptions indicesOptions, ActionLi request.indices(indices); request.fields("*"); request.indicesOptions(indicesOptions); + request.filters("+dimension"); client.fieldCaps(request, listener.map(this::newMetrics)); } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java index b8053016dfa8a..36b858e5b7674 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java @@ -49,7 +49,7 @@ public void testExcludeNestedFields() throws IOException { assertNull(response.getField("field2.field3")); } - public void testDimensionFieldsOnly() throws IOException { + public void testTimeSeriesFieldsOnly() throws IOException { MapperService mapperService = createMapperService(""" { "_doc" : { "properties" : { @@ -57,6 +57,10 @@ public void testDimensionFieldsOnly() throws IOException { "field2" : { "type" : "keyword", "time_series_dimension" : "true" + }, + "field3" : { + "type" : "long", + "time_series_metric" : "counter" } } } } @@ -67,12 +71,13 @@ public void testDimensionFieldsOnly() throws IOException { "index", sec, new String[] { "*" }, - new String[] { "+dimension" }, + new String[] { "+timeseries" }, f -> true ); assertNull(response.getField("field1")); assertNotNull(response.getField("field2")); + assertNotNull(response.getField("field3")); assertNull(response.getField("_index")); } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java index d7750dc86919b..41cc4a148204d 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.fieldcaps; import org.elasticsearch.Version; +import org.elasticsearch.index.mapper.TimeSeriesParams; import org.elasticsearch.test.ESTestCase; import java.util.Collections; @@ -19,9 +20,9 @@ public class ResponseRewriterTests extends ESTestCase { public void testExcludeMetadata() { Map oldResponse = Map.of( "field", - new IndexFieldCapabilities("field", "keyword", false, true, true, false, null, Collections.emptyMap()), + fieldCaps("field", "keyword", false), "_index", - new IndexFieldCapabilities("_index", "_index", true, true, true, false, null, Collections.emptyMap()) + fieldCaps("_index", "_index", true) ); Map rewritten = ResponseRewriter.rewriteOldResponses( @@ -38,9 +39,9 @@ public void testExcludeMetadata() { public void testIncludeOnlyMetadata() { Map oldResponse = Map.of( "field", - new IndexFieldCapabilities("field", "keyword", false, true, true, false, null, Collections.emptyMap()), + fieldCaps("field", "keyword", false), "_index", - new IndexFieldCapabilities("_index", "_index", true, true, true, false, null, Collections.emptyMap()) + fieldCaps("_index", "_index", true) ); Map rewritten = ResponseRewriter.rewriteOldResponses( @@ -57,11 +58,11 @@ public void testIncludeOnlyMetadata() { public void testExcludeNested() { Map oldResponse = Map.of( "field", - new IndexFieldCapabilities("field", "keyword", false, true, true, false, null, Collections.emptyMap()), + fieldCaps("field", "keyword", false), "parent", - new IndexFieldCapabilities("parent", "nested", false, false, false, false, null, Collections.emptyMap()), + fieldCaps("parent", "nested", false), "parent.child", - new IndexFieldCapabilities("parent.child", "keyword", false, true, true, false, null, Collections.emptyMap()) + fieldCaps("parent.child", "keyword", false) ); Map rewritten = ResponseRewriter.rewriteOldResponses( @@ -100,23 +101,35 @@ public void testExcludeMultifield() { assertTrue(rewritten.containsKey("parent.child")); } - public void testIncludeOnlyDimensions() { + public void testIncludeOnlyTimeSeriesValues() { Map oldResponse = Map.of( "field1", fieldCaps("field1", "text", false), "field2", - new IndexFieldCapabilities("field2", "keyword", false, true, true, true, null, Collections.emptyMap()) + new IndexFieldCapabilities("field2", "keyword", false, true, true, true, null, Collections.emptyMap()), + "field3", + new IndexFieldCapabilities( + "field3", + "long", + false, + true, + true, + false, + TimeSeriesParams.MetricType.counter, + Collections.emptyMap() + ) ); Map rewritten = ResponseRewriter.rewriteOldResponses( Version.V_8_0_0, oldResponse, - new String[] { "+dimension" }, + new String[] { "+timeseries" }, f -> f.startsWith("_") ); assertFalse(rewritten.containsKey("field1")); assertTrue(rewritten.containsKey("field2")); + assertTrue(rewritten.containsKey("field3")); } public void testExcludeParents() { From 5a6868e4e54c25752438901f04eb634380ed4fe1 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Mon, 7 Feb 2022 11:42:12 +0000 Subject: [PATCH 03/11] nuke timeseries filter as it's not useful for now --- .../rest-api-spec/api/field_caps.json | 2 +- .../fieldcaps/FieldCapabilitiesFetcher.java | 1 - .../action/fieldcaps/ResponseRewriter.java | 3 -- .../support/TimeSeriesMetricsService.java | 1 - .../FieldCapabilitiesFilterTests.java | 32 ------------------- .../fieldcaps/ResponseRewriterTests.java | 32 ------------------- 6 files changed, 1 insertion(+), 70 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index c15288eec012a..f17e8bb22eee6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -66,7 +66,7 @@ }, "filters": { "type":"list", - "description":"An optional set of filters: can include +metadata,-metadata,-nested,-multifield,+timeseries,parent" + "description":"An optional set of filters: can include +metadata,-metadata,-nested,-multifield,parent" } }, "body":{ diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java index aab6d02beb700..1719fd758e15f 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java @@ -188,7 +188,6 @@ private static FieldCapsFilter buildFilter(Predicate fieldFilter, String case "+metadata" -> (ft, c) -> c.isMetadataField(ft.name()); case "-metadata" -> (ft, c) -> c.isMetadataField(ft.name()) == false; case "-nested" -> (ft, c) -> c.nestedLookup().getNestedParent(ft.name()) == null; - case "+timeseries" -> (ft, c) -> ft.isDimension() || ft.getMetricType() != null; case "-multifield" -> (ft, c) -> c.isMultiField(ft.name()) == false; default -> throw new IllegalArgumentException("Unknown field caps filter [" + filter + "]"); }; diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java index 6d3bef4616c4b..9cacaadd12ef3 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java @@ -59,9 +59,6 @@ private static Function buildTra if ("+metadata".equals(filter)) { test = test.and(IndexFieldCapabilities::isMetadatafield); } - if ("+timeseries".equals(filter)) { - test = test.and(ifc -> ifc.isDimension() || ifc.getMetricType() != null); - } if ("-nested".equals(filter)) { if (nestedObjects == null) { nestedObjects = findTypes("nested", input); diff --git a/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java b/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java index 15a30fa3ba02c..45a72b57a2f93 100644 --- a/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java +++ b/server/src/main/java/org/elasticsearch/timeseries/support/TimeSeriesMetricsService.java @@ -55,7 +55,6 @@ public void newMetrics(String[] indices, IndicesOptions indicesOptions, ActionLi request.indices(indices); request.fields("*"); request.indicesOptions(indicesOptions); - request.filters("+dimension"); client.fieldCaps(request, listener.map(this::newMetrics)); } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java index 36b858e5b7674..1591517aa05a0 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java @@ -49,38 +49,6 @@ public void testExcludeNestedFields() throws IOException { assertNull(response.getField("field2.field3")); } - public void testTimeSeriesFieldsOnly() throws IOException { - MapperService mapperService = createMapperService(""" - { "_doc" : { - "properties" : { - "field1" : { "type" : "keyword" }, - "field2" : { - "type" : "keyword", - "time_series_dimension" : "true" - }, - "field3" : { - "type" : "long", - "time_series_metric" : "counter" - } - } - } } - """); - SearchExecutionContext sec = createSearchExecutionContext(mapperService); - - FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( - "index", - sec, - new String[] { "*" }, - new String[] { "+timeseries" }, - f -> true - ); - - assertNull(response.getField("field1")); - assertNotNull(response.getField("field2")); - assertNotNull(response.getField("field3")); - assertNull(response.getField("_index")); - } - public void testMetadataFilters() throws IOException { MapperService mapperService = createMapperService(""" { "_doc" : { diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java index 41cc4a148204d..7d50b688bd764 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java @@ -9,7 +9,6 @@ package org.elasticsearch.action.fieldcaps; import org.elasticsearch.Version; -import org.elasticsearch.index.mapper.TimeSeriesParams; import org.elasticsearch.test.ESTestCase; import java.util.Collections; @@ -101,37 +100,6 @@ public void testExcludeMultifield() { assertTrue(rewritten.containsKey("parent.child")); } - public void testIncludeOnlyTimeSeriesValues() { - Map oldResponse = Map.of( - "field1", - fieldCaps("field1", "text", false), - "field2", - new IndexFieldCapabilities("field2", "keyword", false, true, true, true, null, Collections.emptyMap()), - "field3", - new IndexFieldCapabilities( - "field3", - "long", - false, - true, - true, - false, - TimeSeriesParams.MetricType.counter, - Collections.emptyMap() - ) - ); - - Map rewritten = ResponseRewriter.rewriteOldResponses( - Version.V_8_0_0, - oldResponse, - new String[] { "+timeseries" }, - f -> f.startsWith("_") - ); - - assertFalse(rewritten.containsKey("field1")); - assertTrue(rewritten.containsKey("field2")); - assertTrue(rewritten.containsKey("field3")); - } - public void testExcludeParents() { Map oldResponse = Map.of( "field", From 86f932e86bb571da52c7d9977ba0e4764a8d8134 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Mon, 7 Feb 2022 13:13:03 +0000 Subject: [PATCH 04/11] add yaml tests --- .../test/field_caps/50_fieldtype_filter.yml | 187 ++++++++++++++++++ .../action/RestFieldCapabilitiesAction.java | 1 + 2 files changed, 188 insertions(+) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml index e69de29bb2d1d..e33be1c6bcf38 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml @@ -0,0 +1,187 @@ +--- +setup: + - skip: + version: "- 8.1.99" + reason: Field type filters were added in 8.2 + - do: + indices.create: + index: test1 + body: + mappings: + properties: + text: + type: text + fields: + keyword: + type: keyword + keyword: + type: keyword + number: + type: double + geo: + type: geo_point + misc: + type: text + object: + type: object + properties: + nested1 : + type : text + index: false + nested2: + type: float + doc_values: false + level1: + type: nested + properties: + level2: + type: object + properties: + leaf1: + type: text + index: false + runtime: + misc.keyword: + type: keyword + + - do: + indices.create: + index: test2 + body: + mappings: + properties: + text: + type: text + keyword: + type: keyword + number: + type: double + date: + type: date + geo: + type: geo_point + object: + type: object + properties: + nested1 : + type : text + index: true + nested2: + type: float + doc_values: true + level1: + type: nested + properties: + level2: + type: object + properties: + leaf1: + type: text + index: false + - do: + indices.create: + index: test3 + body: + mappings: + properties: + text: + type: text + keyword: + type: keyword + number: + type: long + date: + type: date + non_indexed_date: + type: date + index: false + non_indexed_keyword: + type: keyword + index: false + non_indexed_boolean: + type: boolean + index: false + non_indexed_ip: + type: ip + index: false + non_indexed_geo_point: + type: geo_point + index: false + geo: + type: keyword + object: + type: nested + properties: + nested1 : + type : long + index: false + nested2: + type: keyword + doc_values: false +--- +"No filters includes all the following fields": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + + - is_true: fields.object + - is_true: fields.text + - is_true: fields.text\\.keyword + - is_true: fields._seq_no + - is_true: fields.level1\\.level2\\.leaf1 + - is_true: fields.level1 + +--- +"Exclude parent objects": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + filters: '-parent' + + - is_true: fields.object\\.nested1 + - is_false: fields.object + +--- +"Exclude metadata fields": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + filters: '-metadata' + + - is_false: fields._seq_no + +--- +"Exclude non-metadata fields": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + filters: '+metadata' + + - is_true: fields._seq_no + - is_false: fields.text + +--- +"Exclude nested fields": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + filters: '-nested' + + - is_false: fields.level1 + - is_false: fields.level1\\.level2\\.leaf1 + +--- +"Exclude multifields": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + filters: '-multifield' + + - is_false: fields.text\\.keyword + - is_true: fields.misc\\.keyword diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java index eb98197793017..7020b122f92df 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java @@ -50,6 +50,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC fieldRequest.indicesOptions(IndicesOptions.fromRequest(request, fieldRequest.indicesOptions())); fieldRequest.includeUnmapped(request.paramAsBoolean("include_unmapped", false)); + fieldRequest.filters(request.paramAsStringArray("filters", Strings.EMPTY_ARRAY)); request.withContentOrSourceParamParserOrNull(parser -> { if (parser != null) { PARSER.parse(parser, fieldRequest, null); From 267922277add91e92ae96468746391acab1b35a6 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 8 Feb 2022 09:41:51 +0000 Subject: [PATCH 05/11] Add filtering on field type --- .../rest-api-spec/api/field_caps.json | 4 +++ .../test/field_caps/50_fieldtype_filter.yml | 13 ++++++++ .../fieldcaps/FieldCapabilitiesFetcher.java | 12 +++++-- .../FieldCapabilitiesNodeRequest.java | 17 ++++++++-- .../fieldcaps/FieldCapabilitiesRequest.java | 20 +++++++++-- .../action/fieldcaps/RequestDispatcher.java | 1 + .../action/fieldcaps/ResponseRewriter.java | 9 ++++- .../TransportFieldCapabilitiesAction.java | 3 ++ .../action/RestFieldCapabilitiesAction.java | 1 + .../FieldCapabilitiesFilterTests.java | 33 +++++++++++++++++++ .../FieldCapabilitiesNodeRequestTests.java | 28 ++++++++++++++-- .../FieldCapabilitiesRequestTests.java | 16 +++++---- .../fieldcaps/ResponseRewriterTests.java | 26 +++++++++++++++ 13 files changed, 165 insertions(+), 18 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index f17e8bb22eee6..c6fa20f178a81 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -67,6 +67,10 @@ "filters": { "type":"list", "description":"An optional set of filters: can include +metadata,-metadata,-nested,-multifield,parent" + }, + "types": { + "type": "list", + "description":"Only return results for fields that have one of the types in the list" } }, "body":{ diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml index e33be1c6bcf38..8b3a04bd4dc57 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml @@ -185,3 +185,16 @@ setup: - is_false: fields.text\\.keyword - is_true: fields.misc\\.keyword + +--- +"Field type filters": + - do: + field_caps: + index: 'test1,test2,test3' + fields: '*' + types: 'text,keyword,long' + + - is_false: fields.date + - is_false: fields.non_indexed_boolean + - is_true: fields.non_indexed_keyword + - is_true: fields.misc diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java index 1719fd758e15f..b2815dbb05a09 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFetcher.java @@ -46,6 +46,7 @@ FieldCapabilitiesIndexResponse fetch( ShardId shardId, String[] fieldPatterns, String[] filters, + String[] fieldTypes, QueryBuilder indexFilter, long nowInMillis, Map runtimeFields @@ -69,7 +70,7 @@ FieldCapabilitiesIndexResponse fetch( Predicate fieldPredicate = indicesService.getFieldFilter().apply(shardId.getIndexName()); - return retrieveFieldCaps(shardId.getIndexName(), searchExecutionContext, fieldPatterns, filters, fieldPredicate); + return retrieveFieldCaps(shardId.getIndexName(), searchExecutionContext, fieldPatterns, filters, fieldTypes, fieldPredicate); } } @@ -78,6 +79,7 @@ public static FieldCapabilitiesIndexResponse retrieveFieldCaps( SearchExecutionContext context, String[] fieldPatterns, String[] filters, + String[] types, Predicate indexFieldfilter ) { @@ -88,7 +90,7 @@ public static FieldCapabilitiesIndexResponse retrieveFieldCaps( boolean includeParentObjects = checkIncludeParents(filters); - FieldCapsFilter filter = buildFilter(indexFieldfilter, filters); + FieldCapsFilter filter = buildFilter(indexFieldfilter, filters, types); Map responseMap = new HashMap<>(); for (String field : fieldNames) { MappedFieldType ft = context.getFieldType(field); @@ -177,9 +179,13 @@ default FieldCapsFilter and(FieldCapsFilter other) { } } - private static FieldCapsFilter buildFilter(Predicate fieldFilter, String[] filters) { + private static FieldCapsFilter buildFilter(Predicate fieldFilter, String[] filters, String[] fieldTypes) { // security filters don't exclude metadata fields FieldCapsFilter fcf = (ft, c) -> fieldFilter.test(ft.name()) || c.isMetadataField(ft.name()); + if (fieldTypes.length > 0) { + Set acceptedTypes = Set.of(fieldTypes); + fcf = fcf.and((ft, c) -> acceptedTypes.contains(ft.familyTypeName())); + } for (String filter : filters) { if ("parent".equals(filter) || "-parent".equals(filter)) { continue; diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java index be1d8495c19d3..6280b0266ba4c 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java @@ -31,6 +31,7 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque private final List shardIds; private final String[] fields; private final String[] filters; + private final String[] allowedTypes; private final OriginalIndices originalIndices; private final QueryBuilder indexFilter; private final long nowInMillis; @@ -40,10 +41,12 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque super(in); shardIds = in.readList(ShardId::new); fields = in.readStringArray(); - if (in.getVersion().onOrAfter(Version.V_8_1_0)) { + if (in.getVersion().onOrAfter(Version.V_8_2_0)) { filters = in.readStringArray(); + allowedTypes = in.readStringArray(); } else { filters = Strings.EMPTY_ARRAY; + allowedTypes = Strings.EMPTY_ARRAY; } originalIndices = OriginalIndices.readOriginalIndices(in); indexFilter = in.readOptionalNamedWriteable(QueryBuilder.class); @@ -55,6 +58,7 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque List shardIds, String[] fields, String[] filters, + String[] allowedTypes, OriginalIndices originalIndices, QueryBuilder indexFilter, long nowInMillis, @@ -63,6 +67,7 @@ class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesReque this.shardIds = Objects.requireNonNull(shardIds); this.fields = fields; this.filters = filters; + this.allowedTypes = allowedTypes; this.originalIndices = originalIndices; this.indexFilter = indexFilter; this.nowInMillis = nowInMillis; @@ -77,6 +82,10 @@ public String[] filters() { return filters; } + public String[] allowedTypes() { + return allowedTypes; + } + public OriginalIndices originalIndices() { return originalIndices; } @@ -112,8 +121,9 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeList(shardIds); out.writeStringArray(fields); - if (out.getVersion().onOrAfter(Version.V_8_1_0)) { + if (out.getVersion().onOrAfter(Version.V_8_2_0)) { out.writeStringArray(filters); + out.writeStringArray(allowedTypes); } OriginalIndices.writeOriginalIndices(originalIndices, out); out.writeOptionalNamedWriteable(indexFilter); @@ -135,6 +145,7 @@ public boolean equals(Object o) { && shardIds.equals(that.shardIds) && Arrays.equals(fields, that.fields) && Arrays.equals(filters, that.filters) + && Arrays.equals(allowedTypes, that.allowedTypes) && Objects.equals(originalIndices, that.originalIndices) && Objects.equals(indexFilter, that.indexFilter) && Objects.equals(runtimeFields, that.runtimeFields); @@ -145,6 +156,8 @@ public int hashCode() { int result = Objects.hash(originalIndices, indexFilter, nowInMillis, runtimeFields); result = 31 * result + shardIds.hashCode(); result = 31 * result + Arrays.hashCode(fields); + result = 31 * result + Arrays.hashCode(filters); + result = 31 * result + Arrays.hashCode(allowedTypes); return result; } } 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 9ff313f9c0b8b..0397f61515fd7 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java @@ -37,6 +37,7 @@ public final class FieldCapabilitiesRequest extends ActionRequest implements Ind private IndicesOptions indicesOptions = DEFAULT_INDICES_OPTIONS; private String[] fields = Strings.EMPTY_ARRAY; private String[] filters = Strings.EMPTY_ARRAY; + private String[] allowedTypes = Strings.EMPTY_ARRAY; private boolean includeUnmapped = false; // pkg private API mainly for cross cluster search to signal that we do multiple reductions ie. the results should not be merged private boolean mergeResults = true; @@ -54,8 +55,9 @@ public FieldCapabilitiesRequest(StreamInput in) throws IOException { indexFilter = in.readOptionalNamedWriteable(QueryBuilder.class); nowInMillis = in.readOptionalLong(); runtimeFields = in.readMap(); - if (in.getVersion().onOrAfter(Version.V_8_1_0)) { + if (in.getVersion().onOrAfter(Version.V_8_2_0)) { filters = in.readStringArray(); + allowedTypes = in.readStringArray(); } } @@ -91,8 +93,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalNamedWriteable(indexFilter); out.writeOptionalLong(nowInMillis); out.writeMap(runtimeFields); - if (out.getVersion().onOrAfter(Version.V_8_1_0)) { + if (out.getVersion().onOrAfter(Version.V_8_2_0)) { out.writeStringArray(filters); + out.writeStringArray(allowedTypes); } } @@ -134,6 +137,15 @@ public String[] filters() { return filters; } + public FieldCapabilitiesRequest allowedTypes(String... types) { + this.allowedTypes = types; + return this; + } + + public String[] allowedTypes() { + return allowedTypes; + } + /** * The list of indices to lookup */ @@ -231,6 +243,7 @@ public boolean equals(Object o) { && Objects.equals(indexFilter, that.indexFilter) && Objects.equals(nowInMillis, that.nowInMillis) && Arrays.equals(filters, that.filters) + && Arrays.equals(allowedTypes, that.allowedTypes) && Objects.equals(runtimeFields, that.runtimeFields); } @@ -240,6 +253,7 @@ public int hashCode() { result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(fields); result = 31 * result + Arrays.hashCode(filters); + result = 31 * result + Arrays.hashCode(allowedTypes); return result; } @@ -251,6 +265,8 @@ public String getDescription() { Strings.collectionToDelimitedStringWithLimit(Arrays.asList(fields), ",", "", "", 1024, stringBuilder); stringBuilder.append("], filters["); stringBuilder.append(Strings.collectionToDelimitedString(Arrays.asList(filters), ",")); + stringBuilder.append("], types["); + stringBuilder.append(Strings.collectionToDelimitedString(Arrays.asList(allowedTypes), ",")); stringBuilder.append("]"); return stringBuilder.toString(); } diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java index 1257a172f331a..f9d5cff2471b4 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/RequestDispatcher.java @@ -175,6 +175,7 @@ private void sendRequestToNode(String nodeId, List shardIds) { shardIds, fieldCapsRequest.fields(), fieldCapsRequest.filters(), + fieldCapsRequest.allowedTypes(), originalIndices, fieldCapsRequest.indexFilter(), nowInMillis, diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java index 9cacaadd12ef3..6d3b1c1b0604a 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java @@ -23,12 +23,14 @@ public static Map rewriteOldResponses( Version version, Map input, String[] filters, + String[] allowedTypes, Predicate isMetadata ) { if (version.onOrAfter(Version.V_8_1_0)) { return input; // nothing needs to be done } - Function transformer = buildTransformer(version, input, filters, isMetadata); + Function transformer + = buildTransformer(version, input, filters, allowedTypes, isMetadata); Map rewritten = new HashMap<>(); for (var entry : input.entrySet()) { IndexFieldCapabilities fc = transformer.apply(entry.getValue()); @@ -43,12 +45,17 @@ private static Function buildTra Version version, Map input, String[] filters, + String[] allowedTypes, Predicate isMetadata ) { boolean checkMetadata = version.before(Version.V_7_13_0); Predicate test = ifc -> true; Set objects = null; Set nestedObjects = null; + if (allowedTypes.length > 0) { + Set at = Set.of(allowedTypes); + test = test.and(ifc -> at.contains(ifc.getType())); + } for (String filter : filters) { if ("-parent".equals(filter)) { test = test.and(fc -> fc.getType().equals("nested") == false && fc.getType().equals("object") == false); 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 b58068897575a..ddabc0ab6de79 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -207,6 +207,7 @@ private static FieldCapabilitiesRequest prepareRemoteRequest( remoteRequest.indices(originalIndices.indices()); remoteRequest.fields(request.fields()); remoteRequest.filters(request.filters()); + remoteRequest.allowedTypes(request.allowedTypes()); remoteRequest.runtimeFields(request.runtimeFields()); remoteRequest.indexFilter(request.indexFilter()); remoteRequest.nowInMillis(nowInMillis); @@ -260,6 +261,7 @@ private void innerMerge( response.getOriginVersion(), response.get(), request.filters(), + request.allowedTypes(), metadataFieldPred ); for (Map.Entry entry : fields.entrySet()) { @@ -348,6 +350,7 @@ public void messageReceived(FieldCapabilitiesNodeRequest request, TransportChann shardId, request.fields(), request.filters(), + request.allowedTypes(), request.indexFilter(), request.nowInMillis(), request.runtimeFields() diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java index 7020b122f92df..f39b3d3a3479c 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestFieldCapabilitiesAction.java @@ -51,6 +51,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC fieldRequest.indicesOptions(IndicesOptions.fromRequest(request, fieldRequest.indicesOptions())); fieldRequest.includeUnmapped(request.paramAsBoolean("include_unmapped", false)); fieldRequest.filters(request.paramAsStringArray("filters", Strings.EMPTY_ARRAY)); + fieldRequest.allowedTypes(request.paramAsStringArray("types", Strings.EMPTY_ARRAY)); request.withContentOrSourceParamParserOrNull(parser -> { if (parser != null) { PARSER.parse(parser, fieldRequest, null); diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java index 1591517aa05a0..25f64107e5b23 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesFilterTests.java @@ -40,6 +40,7 @@ public void testExcludeNestedFields() throws IOException { sec, new String[] { "*" }, new String[] { "-nested" }, + Strings.EMPTY_ARRAY, f -> true ); @@ -66,6 +67,7 @@ public void testMetadataFilters() throws IOException { sec, new String[] { "*" }, new String[] { "+metadata" }, + Strings.EMPTY_ARRAY, f -> true ); assertNotNull(response.getField("_index")); @@ -77,6 +79,7 @@ public void testMetadataFilters() throws IOException { sec, new String[] { "*" }, new String[] { "-metadata" }, + Strings.EMPTY_ARRAY, f -> true ); assertNull(response.getField("_index")); @@ -108,6 +111,7 @@ public void testExcludeMultifields() throws IOException { sec, new String[] { "*" }, new String[] { "-multifield" }, + Strings.EMPTY_ARRAY, f -> true ); assertNotNull(response.getField("field1")); @@ -137,6 +141,7 @@ public void testDontIncludeParentInfo() throws IOException { sec, new String[] { "*" }, new String[] { "-parent" }, + Strings.EMPTY_ARRAY, f -> true ); assertNotNull(response.getField("parent.field1")); @@ -163,6 +168,7 @@ public void testSecurityFilter() throws IOException { sec, new String[] { "*" }, Strings.EMPTY_ARRAY, + Strings.EMPTY_ARRAY, securityFilter ); @@ -177,6 +183,7 @@ public void testSecurityFilter() throws IOException { sec, new String[] { "*" }, new String[] { "-metadata" }, + Strings.EMPTY_ARRAY, securityFilter ); @@ -185,4 +192,30 @@ public void testSecurityFilter() throws IOException { assertNull(response.getField("_index")); // -metadata filter applies on top } } + + public void testFieldTypeFiltering() throws IOException { + MapperService mapperService = createMapperService(""" + { "_doc" : { + "properties" : { + "field1" : { "type" : "keyword" }, + "field2" : { "type" : "long" }, + "field3" : { "type" : "text" } + } + } } + """); + SearchExecutionContext sec = createSearchExecutionContext(mapperService); + + FieldCapabilitiesIndexResponse response = FieldCapabilitiesFetcher.retrieveFieldCaps( + "index", + sec, + new String[] { "*" }, + Strings.EMPTY_ARRAY, + new String[] { "text", "keyword" }, + f -> true + ); + assertNotNull(response.getField("field1")); + assertNull(response.getField("field2")); + assertNotNull(response.getField("field3")); + assertNull(response.getField("_index")); + } } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java index dde25ae32e386..549bd4e299737 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequestTests.java @@ -33,6 +33,7 @@ protected FieldCapabilitiesNodeRequest createTestInstance() { List randomShards = randomShardIds(randomIntBetween(1, 5)); String[] randomFields = randomFields(randomIntBetween(1, 20)); String[] randomFilter = randomBoolean() ? Strings.EMPTY_ARRAY : new String[] { "-nested" }; + String[] randomTypeFilter = randomBoolean() ? Strings.EMPTY_ARRAY : new String[] { "keyword" }; OriginalIndices originalIndices = randomOriginalIndices(randomIntBetween(0, 20)); QueryBuilder indexFilter = randomBoolean() ? QueryBuilders.termQuery("field", randomAlphaOfLength(5)) : null; @@ -46,6 +47,7 @@ protected FieldCapabilitiesNodeRequest createTestInstance() { randomShards, randomFields, randomFilter, + randomTypeFilter, originalIndices, indexFilter, nowInMillis, @@ -91,13 +93,14 @@ protected Writeable.Reader instanceReader() { @Override protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeRequest instance) throws IOException { - switch (random().nextInt(6)) { + switch (random().nextInt(7)) { case 0 -> { List shardIds = randomShardIds(instance.shardIds().size() + 1); return new FieldCapabilitiesNodeRequest( shardIds, instance.fields(), instance.filters(), + instance.allowedTypes(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), @@ -110,6 +113,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque instance.shardIds(), fields, instance.filters(), + instance.allowedTypes(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), @@ -122,6 +126,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque instance.shardIds(), instance.fields(), instance.filters(), + instance.allowedTypes(), originalIndices, instance.indexFilter(), instance.nowInMillis(), @@ -134,6 +139,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque instance.shardIds(), instance.fields(), instance.filters(), + instance.allowedTypes(), instance.originalIndices(), indexFilter, instance.nowInMillis(), @@ -146,6 +152,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque instance.shardIds(), instance.fields(), instance.filters(), + instance.allowedTypes(), instance.originalIndices(), instance.indexFilter(), nowInMillis, @@ -160,6 +167,7 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque instance.shardIds(), instance.fields(), instance.filters(), + instance.allowedTypes(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), @@ -167,18 +175,32 @@ protected FieldCapabilitiesNodeRequest mutateInstance(FieldCapabilitiesNodeReque ); } case 6 -> { - String[] randomFilter = randomBoolean() ? Strings.EMPTY_ARRAY : new String[] { "-nested" }; + String[] randomFilter = instance.filters().length > 0 ? Strings.EMPTY_ARRAY : new String[] { "-nested" }; return new FieldCapabilitiesNodeRequest( instance.shardIds(), instance.fields(), randomFilter, + instance.allowedTypes(), instance.originalIndices(), instance.indexFilter(), instance.nowInMillis(), instance.runtimeFields() ); } - default -> throw new IllegalStateException("The test should only allow 5 parameters mutated"); + case 7 -> { + String[] randomType = instance.allowedTypes().length > 0 ? Strings.EMPTY_ARRAY : new String[] { "text" }; + return new FieldCapabilitiesNodeRequest( + instance.shardIds(), + instance.fields(), + instance.filters(), + randomType, + instance.originalIndices(), + instance.indexFilter(), + instance.nowInMillis(), + instance.runtimeFields() + ); + } + default -> throw new IllegalStateException("The test should only allow 7 parameters mutated"); } } } diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java index 73d17412ac643..15ae9ba62d1fe 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java @@ -147,18 +147,20 @@ public void testValidation() { public void testGetDescription() { final FieldCapabilitiesRequest request = new FieldCapabilitiesRequest(); - assertThat(request.getDescription(), equalTo("indices[], fields[], filters[]")); + assertThat(request.getDescription(), equalTo("indices[], fields[], filters[], types[]")); request.fields("a", "b"); assertThat(request.getDescription(), - anyOf(equalTo("indices[], fields[a,b], filters[]"), equalTo("indices[], fields[b,a], filters[]"))); + anyOf( + equalTo("indices[], fields[a,b], filters[], types[]"), + equalTo("indices[], fields[b,a], filters[], types[]"))); request.indices("x", "y", "z"); request.fields("a"); - assertThat(request.getDescription(), equalTo("indices[x,y,z], fields[a], filters[]")); + assertThat(request.getDescription(), equalTo("indices[x,y,z], fields[a], filters[], types[]")); request.filters("-metadata", "-multifields"); - assertThat(request.getDescription(), endsWith("filters[-metadata,-multifields]")); + assertThat(request.getDescription(), endsWith("filters[-metadata,-multifields], types[]")); final String[] lots = new String[between(1024, 2048)]; for (int i = 0; i < lots.length; i++) { @@ -179,7 +181,7 @@ public void testGetDescription() { ); assertThat( request.getDescription().length(), - lessThanOrEqualTo(1024 + ("indices[x,y,z], fields[" + "s9999,... (9999 in total, 9999 omitted)], filters[]").length()) + lessThanOrEqualTo(1024 + ("indices[x,y,z], fields[" + "s9999,... (9999 in total, 9999 omitted)], filters[], types[]").length()) ); request.fields("a"); @@ -191,12 +193,12 @@ public void testGetDescription() { containsString("..."), containsString(lots.length + " in total"), containsString("omitted"), - endsWith("], fields[a], filters[]") + endsWith("], fields[a], filters[], types[]") ) ); assertThat( request.getDescription().length(), - lessThanOrEqualTo(1024 + ("indices[" + "s9999,... (9999 in total, 9999 omitted)], fields[a], filters[]").length()) + lessThanOrEqualTo(1024 + ("indices[" + "s9999,... (9999 in total, 9999 omitted)], fields[a], filters[], types[]").length()) ); final FieldCapabilitiesRequest randomRequest = createTestInstance(); diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java index 7d50b688bd764..d8375ed8e5555 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.fieldcaps; import org.elasticsearch.Version; +import org.elasticsearch.common.Strings; import org.elasticsearch.test.ESTestCase; import java.util.Collections; @@ -28,6 +29,7 @@ public void testExcludeMetadata() { Version.V_8_0_0, oldResponse, new String[] { "-metadata" }, + Strings.EMPTY_ARRAY, f -> f.startsWith("_") ); @@ -47,6 +49,7 @@ public void testIncludeOnlyMetadata() { Version.V_8_0_0, oldResponse, new String[] { "+metadata" }, + Strings.EMPTY_ARRAY, f -> f.startsWith("_") ); @@ -68,6 +71,7 @@ public void testExcludeNested() { Version.V_8_0_0, oldResponse, new String[] { "-nested" }, + Strings.EMPTY_ARRAY, f -> f.startsWith("_") ); @@ -92,6 +96,7 @@ public void testExcludeMultifield() { Version.V_8_0_0, oldResponse, new String[] { "-multifield" }, + Strings.EMPTY_ARRAY, f -> f.startsWith("_") ); @@ -114,6 +119,7 @@ public void testExcludeParents() { Version.V_8_0_0, oldResponse, new String[] { "-parent" }, + Strings.EMPTY_ARRAY, f -> f.startsWith("_") ); @@ -122,6 +128,26 @@ public void testExcludeParents() { assertTrue(rewritten.containsKey("parent.child")); } + public void testAllowedTypes() { + Map oldResponse = Map.of( + "text", fieldCaps("text", "text", false), + "long", fieldCaps("long", "long", false), + "keyword", fieldCaps("keyword", "keyword", false) + ); + + Map rewritten = ResponseRewriter.rewriteOldResponses( + Version.V_8_0_0, + oldResponse, + Strings.EMPTY_ARRAY, + new String[] { "text", "keyword" }, + f -> f.startsWith("_") + ); + + assertTrue(rewritten.containsKey("text")); + assertTrue(rewritten.containsKey("keyword")); + assertFalse(rewritten.containsKey("long")); + } + private static IndexFieldCapabilities fieldCaps(String name, String type, boolean isMetadata) { return new IndexFieldCapabilities(name, type, isMetadata, true, true, false, null, Collections.emptyMap()); } From 32a5c3501bf4f3295a32d5f65465f049311ffddd Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 8 Feb 2022 09:47:14 +0000 Subject: [PATCH 06/11] Docs --- docs/reference/search/field-caps.asciidoc | 9 +++++++++ .../src/main/resources/rest-api-spec/api/field_caps.json | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/reference/search/field-caps.asciidoc b/docs/reference/search/field-caps.asciidoc index 7cbb583bab800..580553b027fa2 100644 --- a/docs/reference/search/field-caps.asciidoc +++ b/docs/reference/search/field-caps.asciidoc @@ -77,6 +77,15 @@ include::{es-repo-dir}/rest-api/common-parms.asciidoc[tag=index-ignore-unavailab (Optional, Boolean) If `true`, unmapped fields are included in the response. Defaults to `false`. +`filters`:: +(Optional, string) Comma-separated list of filters to apply to the response. The +following filters are supported: +metadata,-metadata,-parent,-nested,-multifield + +`types`:: +(Optional, string) Comma-separated list of field types to include. Any fields that +do not match one of these types will be excluded from the results. Defaults to empty, +meaning that all field types are returned. + [[search-field-caps-api-request-body]] ==== {api-request-body-title} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json index c6fa20f178a81..934ef3daa44aa 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/field_caps.json @@ -66,7 +66,7 @@ }, "filters": { "type":"list", - "description":"An optional set of filters: can include +metadata,-metadata,-nested,-multifield,parent" + "description":"An optional set of filters: can include +metadata,-metadata,-nested,-multifield,-parent" }, "types": { "type": "list", From 851755f80718979427362984185b417876859ddc Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 8 Feb 2022 09:57:50 +0000 Subject: [PATCH 07/11] Update docs/changelog/83636.yaml --- docs/changelog/83636.yaml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 docs/changelog/83636.yaml diff --git a/docs/changelog/83636.yaml b/docs/changelog/83636.yaml new file mode 100644 index 0000000000000..5ca5352e12422 --- /dev/null +++ b/docs/changelog/83636.yaml @@ -0,0 +1,6 @@ +pr: 83636 +summary: Add filtering to fieldcaps endpoint +area: Search +type: enhancement +issues: + - 82966 From 6c0d3e9b30f03671d5679928f6575286f82f11a7 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 8 Feb 2022 10:10:02 +0000 Subject: [PATCH 08/11] spotless --- .../elasticsearch/action/fieldcaps/ResponseRewriter.java | 9 +++++++-- .../action/fieldcaps/FieldCapabilitiesRequestTests.java | 8 ++++---- .../action/fieldcaps/ResponseRewriterTests.java | 9 ++++++--- 3 files changed, 17 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java index 6d3b1c1b0604a..59c1caa80c2e6 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java @@ -29,8 +29,13 @@ public static Map rewriteOldResponses( if (version.onOrAfter(Version.V_8_1_0)) { return input; // nothing needs to be done } - Function transformer - = buildTransformer(version, input, filters, allowedTypes, isMetadata); + Function transformer = buildTransformer( + version, + input, + filters, + allowedTypes, + isMetadata + ); Map rewritten = new HashMap<>(); for (var entry : input.entrySet()) { IndexFieldCapabilities fc = transformer.apply(entry.getValue()); diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java index 15ae9ba62d1fe..dc72d180ef0c2 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequestTests.java @@ -150,10 +150,10 @@ public void testGetDescription() { assertThat(request.getDescription(), equalTo("indices[], fields[], filters[], types[]")); request.fields("a", "b"); - assertThat(request.getDescription(), - anyOf( - equalTo("indices[], fields[a,b], filters[], types[]"), - equalTo("indices[], fields[b,a], filters[], types[]"))); + assertThat( + request.getDescription(), + anyOf(equalTo("indices[], fields[a,b], filters[], types[]"), equalTo("indices[], fields[b,a], filters[], types[]")) + ); request.indices("x", "y", "z"); request.fields("a"); diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java index d8375ed8e5555..830f9bb42912b 100644 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java +++ b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java @@ -130,9 +130,12 @@ public void testExcludeParents() { public void testAllowedTypes() { Map oldResponse = Map.of( - "text", fieldCaps("text", "text", false), - "long", fieldCaps("long", "long", false), - "keyword", fieldCaps("keyword", "keyword", false) + "text", + fieldCaps("text", "text", false), + "long", + fieldCaps("long", "long", false), + "keyword", + fieldCaps("keyword", "keyword", false) ); Map rewritten = ResponseRewriter.rewriteOldResponses( From 196016f4c767de4b22264e2dba7a894651769509 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Tue, 8 Feb 2022 12:17:36 +0000 Subject: [PATCH 09/11] rename --- x-pack/qa/runtime-fields/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/qa/runtime-fields/build.gradle b/x-pack/qa/runtime-fields/build.gradle index 4ab43335caa80..ddebd593c0843 100644 --- a/x-pack/qa/runtime-fields/build.gradle +++ b/x-pack/qa/runtime-fields/build.gradle @@ -70,7 +70,7 @@ subprojects { /////// TO FIX /////// 'search.aggregation/40_range/Date range', //source only date field should also emit values for numbers, it expects strings only 'search/115_multiple_field_collapsing/two levels fields collapsing', // Field collapsing on a runtime field does not work - 'field_caps/30_filter/Field caps with index filter', // We don't support filtering field caps on runtime fields. What should we do? + 'field_caps/30_index_filter/Field caps with index filter', // We don't support filtering field caps on runtime fields. What should we do? 'search.aggregation/220_filters_bucket/cache busting', // runtime keyword does not support split_queries_on_whitespace 'search/140_pre_filter_search_shards/pre_filter_shard_size with shards that have no hit', //completion suggester does not return options when the context field is a geo_point runtime field From 2a9f5d7b26d1b1e1d074829852943a53551ebbbe Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Wed, 9 Feb 2022 11:48:53 +0000 Subject: [PATCH 10/11] Add test including fields restriction --- .../test/field_caps/50_fieldtype_filter.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml index 8b3a04bd4dc57..303c5e78d8981 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml @@ -198,3 +198,15 @@ setup: - is_false: fields.non_indexed_boolean - is_true: fields.non_indexed_keyword - is_true: fields.misc + +--- +"Field type filters with field name restrictions": + - do: + field_caps: + index: 'test1,test2,test3' + fields: 'non_*' + types: 'text,keyword,long' + + - is_false: fields.non_indexed_boolean + - is_true: fields.non_indexed_keyword + - is_false: fields.misc From 7d6553015ec52ceb0126f1c7ca73914859abf4f5 Mon Sep 17 00:00:00 2001 From: Alan Woodward Date: Thu, 10 Feb 2022 11:42:40 +0000 Subject: [PATCH 11/11] Extend tests to use a fields list --- .../rest-api-spec/test/field_caps/50_fieldtype_filter.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml index 303c5e78d8981..cfed4f68ea5e7 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/field_caps/50_fieldtype_filter.yml @@ -204,9 +204,10 @@ setup: - do: field_caps: index: 'test1,test2,test3' - fields: 'non_*' + fields: 'non_*,text' types: 'text,keyword,long' - is_false: fields.non_indexed_boolean - is_true: fields.non_indexed_keyword - is_false: fields.misc + - is_true: fields.text