فهرست منبع

Remove the RestHighLevelClient and some supporting classes (#102305)

Joe Gallo 1 سال پیش
والد
کامیت
94b58f5f94

+ 0 - 5
client/rest-high-level/build.gradle

@@ -61,11 +61,6 @@ tasks.named('forbiddenApisMain').configure {
   signaturesFiles += files('src/main/resources/forbidden/rest-high-level-signatures.txt')
 }
 
-tasks.named('splitPackagesAudit').configure {
-  // the client package should be owned by the client, but server has some classes there too
-  ignoreClasses 'org.elasticsearch.client.*'
-}
-
 // we don't have tests now, as HLRC is in the process of being removed
 tasks.named("test").configure {enabled = false }
 

+ 0 - 12
client/rest-high-level/roles.yml

@@ -1,12 +0,0 @@
-admin:
-  cluster:
-    - all
-  indices:
-    - names: '*'
-      privileges:
-        - all
-  run_as: [ '*' ]
-  applications:
-    - application: '*'
-      privileges: [ '*' ]
-      resources: [ '*' ]

+ 0 - 528
client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java

@@ -1,528 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0 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.client;
-
-import org.apache.http.HttpEntity;
-import org.apache.http.client.methods.HttpPost;
-import org.apache.http.client.methods.HttpPut;
-import org.apache.http.entity.ContentType;
-import org.apache.http.nio.entity.NByteArrayEntity;
-import org.apache.lucene.util.BytesRef;
-import org.elasticsearch.action.DocWriteRequest;
-import org.elasticsearch.action.bulk.BulkRequest;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.action.search.SearchRequest;
-import org.elasticsearch.action.support.ActiveShardCount;
-import org.elasticsearch.action.support.IndicesOptions;
-import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
-import org.elasticsearch.action.update.UpdateRequest;
-import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.bytes.BytesReference;
-import org.elasticsearch.common.lucene.uid.Versions;
-import org.elasticsearch.common.xcontent.XContentHelper;
-import org.elasticsearch.core.Nullable;
-import org.elasticsearch.core.SuppressForbidden;
-import org.elasticsearch.core.TimeValue;
-import org.elasticsearch.index.VersionType;
-import org.elasticsearch.index.seqno.SequenceNumbers;
-import org.elasticsearch.rest.action.search.RestSearchAction;
-import org.elasticsearch.xcontent.DeprecationHandler;
-import org.elasticsearch.xcontent.NamedXContentRegistry;
-import org.elasticsearch.xcontent.ToXContent;
-import org.elasticsearch.xcontent.XContentBuilder;
-import org.elasticsearch.xcontent.XContentParser;
-import org.elasticsearch.xcontent.XContentType;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.charset.Charset;
-import java.util.HashMap;
-import java.util.Locale;
-import java.util.Map;
-import java.util.StringJoiner;
-
-final class RequestConverters {
-    static final XContentType REQUEST_BODY_CONTENT_TYPE = XContentType.JSON;
-
-    private RequestConverters() {
-        // Contains only status utility methods
-    }
-
-    static Request bulk(BulkRequest bulkRequest) throws IOException {
-        Request request = new Request(HttpPost.METHOD_NAME, "/_bulk");
-
-        Params parameters = new Params();
-        parameters.withTimeout(bulkRequest.timeout());
-        parameters.withRefreshPolicy(bulkRequest.getRefreshPolicy());
-        parameters.withPipeline(bulkRequest.pipeline());
-        parameters.withRouting(bulkRequest.routing());
-        // Bulk API only supports newline delimited JSON or Smile. Before executing
-        // the bulk, we need to check that all requests have the same content-type
-        // and this content-type is supported by the Bulk API.
-        XContentType bulkContentType = null;
-        for (int i = 0; i < bulkRequest.numberOfActions(); i++) {
-            DocWriteRequest<?> action = bulkRequest.requests().get(i);
-
-            DocWriteRequest.OpType opType = action.opType();
-            if (opType == DocWriteRequest.OpType.INDEX || opType == DocWriteRequest.OpType.CREATE) {
-                bulkContentType = enforceSameContentType((IndexRequest) action, bulkContentType);
-
-            } else if (opType == DocWriteRequest.OpType.UPDATE) {
-                UpdateRequest updateRequest = (UpdateRequest) action;
-                if (updateRequest.doc() != null) {
-                    bulkContentType = enforceSameContentType(updateRequest.doc(), bulkContentType);
-                }
-                if (updateRequest.upsertRequest() != null) {
-                    bulkContentType = enforceSameContentType(updateRequest.upsertRequest(), bulkContentType);
-                }
-            }
-        }
-
-        if (bulkContentType == null) {
-            bulkContentType = XContentType.JSON;
-        }
-
-        final byte separator = bulkContentType.xContent().streamSeparator();
-        final ContentType requestContentType = createContentType(bulkContentType);
-
-        ByteArrayOutputStream content = new ByteArrayOutputStream();
-        for (DocWriteRequest<?> action : bulkRequest.requests()) {
-            DocWriteRequest.OpType opType = action.opType();
-
-            try (XContentBuilder metadata = XContentBuilder.builder(bulkContentType.xContent())) {
-                metadata.startObject();
-                {
-                    metadata.startObject(opType.getLowercase());
-                    if (Strings.hasLength(action.index())) {
-                        metadata.field("_index", action.index());
-                    }
-                    if (Strings.hasLength(action.id())) {
-                        metadata.field("_id", action.id());
-                    }
-                    if (Strings.hasLength(action.routing())) {
-                        metadata.field("routing", action.routing());
-                    }
-                    if (action.version() != Versions.MATCH_ANY) {
-                        metadata.field("version", action.version());
-                    }
-
-                    VersionType versionType = action.versionType();
-                    if (versionType != VersionType.INTERNAL) {
-                        if (versionType == VersionType.EXTERNAL) {
-                            metadata.field("version_type", "external");
-                        } else if (versionType == VersionType.EXTERNAL_GTE) {
-                            metadata.field("version_type", "external_gte");
-                        }
-                    }
-
-                    if (action.ifSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) {
-                        metadata.field("if_seq_no", action.ifSeqNo());
-                        metadata.field("if_primary_term", action.ifPrimaryTerm());
-                    }
-
-                    if (opType == DocWriteRequest.OpType.INDEX || opType == DocWriteRequest.OpType.CREATE) {
-                        IndexRequest indexRequest = (IndexRequest) action;
-                        if (Strings.hasLength(indexRequest.getPipeline())) {
-                            metadata.field("pipeline", indexRequest.getPipeline());
-                        }
-                    } else if (opType == DocWriteRequest.OpType.UPDATE) {
-                        UpdateRequest updateRequest = (UpdateRequest) action;
-                        if (updateRequest.retryOnConflict() > 0) {
-                            metadata.field("retry_on_conflict", updateRequest.retryOnConflict());
-                        }
-                        if (updateRequest.fetchSource() != null) {
-                            metadata.field("_source", updateRequest.fetchSource());
-                        }
-                    }
-                    metadata.endObject();
-                }
-                metadata.endObject();
-
-                BytesRef metadataSource = BytesReference.bytes(metadata).toBytesRef();
-                content.write(metadataSource.bytes, metadataSource.offset, metadataSource.length);
-                content.write(separator);
-            }
-
-            BytesRef source = null;
-            if (opType == DocWriteRequest.OpType.INDEX || opType == DocWriteRequest.OpType.CREATE) {
-                IndexRequest indexRequest = (IndexRequest) action;
-                BytesReference indexSource = indexRequest.source();
-                XContentType indexXContentType = indexRequest.getContentType();
-
-                try (
-                    XContentParser parser = XContentHelper.createParser(
-                        /*
-                         * EMPTY and THROW are fine here because we just call
-                         * copyCurrentStructure which doesn't touch the
-                         * registry or deprecation.
-                         */
-                        NamedXContentRegistry.EMPTY,
-                        DeprecationHandler.THROW_UNSUPPORTED_OPERATION,
-                        indexSource,
-                        indexXContentType
-                    )
-                ) {
-                    try (XContentBuilder builder = XContentBuilder.builder(bulkContentType.xContent())) {
-                        builder.copyCurrentStructure(parser);
-                        source = BytesReference.bytes(builder).toBytesRef();
-                    }
-                }
-            } else if (opType == DocWriteRequest.OpType.UPDATE) {
-                source = XContentHelper.toXContent((UpdateRequest) action, bulkContentType, false).toBytesRef();
-            }
-
-            if (source != null) {
-                content.write(source.bytes, source.offset, source.length);
-                content.write(separator);
-            }
-        }
-        request.addParameters(parameters.asMap());
-        request.setEntity(new NByteArrayEntity(content.toByteArray(), 0, content.size(), requestContentType));
-        return request;
-    }
-
-    static Request index(IndexRequest indexRequest) {
-        String method = Strings.hasLength(indexRequest.id()) ? HttpPut.METHOD_NAME : HttpPost.METHOD_NAME;
-
-        String endpoint;
-        if (indexRequest.opType() == DocWriteRequest.OpType.CREATE) {
-            endpoint = endpoint(indexRequest.index(), "_create", indexRequest.id());
-        } else {
-            endpoint = endpoint(indexRequest.index(), indexRequest.id());
-        }
-
-        Request request = new Request(method, endpoint);
-
-        Params parameters = new Params();
-        parameters.withRouting(indexRequest.routing());
-        parameters.withTimeout(indexRequest.timeout());
-        parameters.withVersion(indexRequest.version());
-        parameters.withVersionType(indexRequest.versionType());
-        parameters.withIfSeqNo(indexRequest.ifSeqNo());
-        parameters.withIfPrimaryTerm(indexRequest.ifPrimaryTerm());
-        parameters.withPipeline(indexRequest.getPipeline());
-        parameters.withRefreshPolicy(indexRequest.getRefreshPolicy());
-        parameters.withWaitForActiveShards(indexRequest.waitForActiveShards());
-        parameters.withRequireAlias(indexRequest.isRequireAlias());
-
-        BytesRef source = indexRequest.source().toBytesRef();
-        ContentType contentType = createContentType(indexRequest.getContentType());
-        request.addParameters(parameters.asMap());
-        request.setEntity(new NByteArrayEntity(source.bytes, source.offset, source.length, contentType));
-        return request;
-    }
-
-    /**
-     * Convert a {@linkplain SearchRequest} into a {@linkplain Request}.
-     * @param searchRequest the request to convert
-     * @param searchEndpoint the name of the search endpoint. {@literal _search}
-     *    for standard searches and {@literal _rollup_search} for rollup
-     *    searches.
-     */
-    static Request search(SearchRequest searchRequest, String searchEndpoint) throws IOException {
-        Request request = new Request(HttpPost.METHOD_NAME, endpoint(searchRequest.indices(), searchEndpoint));
-
-        Params params = new Params();
-        addSearchRequestParams(params, searchRequest);
-
-        if (searchRequest.source() != null) {
-            request.setEntity(createEntity(searchRequest.source(), REQUEST_BODY_CONTENT_TYPE));
-        }
-        request.addParameters(params.asMap());
-        return request;
-    }
-
-    private static void addSearchRequestParams(Params params, SearchRequest searchRequest) {
-        params.putParam(RestSearchAction.TYPED_KEYS_PARAM, "true");
-        params.withRouting(searchRequest.routing());
-        params.withPreference(searchRequest.preference());
-        if (SearchRequest.DEFAULT_INDICES_OPTIONS.equals(searchRequest.indicesOptions()) == false) {
-            params.withIndicesOptions(searchRequest.indicesOptions());
-        }
-        params.withSearchType(searchRequest.searchType().name().toLowerCase(Locale.ROOT));
-        if (searchRequest.isCcsMinimizeRoundtrips() != SearchRequest.defaultCcsMinimizeRoundtrips(searchRequest)) {
-            params.putParam("ccs_minimize_roundtrips", Boolean.toString(searchRequest.isCcsMinimizeRoundtrips()));
-        }
-        if (searchRequest.getPreFilterShardSize() != null) {
-            params.putParam("pre_filter_shard_size", Integer.toString(searchRequest.getPreFilterShardSize()));
-        }
-        params.withMaxConcurrentShardRequests(searchRequest.getMaxConcurrentShardRequests());
-        if (searchRequest.requestCache() != null) {
-            params.withRequestCache(searchRequest.requestCache());
-        }
-        if (searchRequest.allowPartialSearchResults() != null) {
-            params.withAllowPartialResults(searchRequest.allowPartialSearchResults());
-        }
-        params.withBatchedReduceSize(searchRequest.getBatchedReduceSize());
-        if (searchRequest.scroll() != null) {
-            params.putParam("scroll", searchRequest.scroll().keepAlive());
-        }
-    }
-
-    private static HttpEntity createEntity(ToXContent toXContent, XContentType xContentType) throws IOException {
-        return createEntity(toXContent, xContentType, ToXContent.EMPTY_PARAMS);
-    }
-
-    private static HttpEntity createEntity(ToXContent toXContent, XContentType xContentType, ToXContent.Params toXContentParams)
-        throws IOException {
-        BytesRef source = XContentHelper.toXContent(toXContent, xContentType, toXContentParams, false).toBytesRef();
-        return new NByteArrayEntity(source.bytes, source.offset, source.length, createContentType(xContentType));
-    }
-
-    private static String endpoint(String index, String type, String id) {
-        return new EndpointBuilder().addPathPart(index, type, id).build();
-    }
-
-    private static String endpoint(String index, String id) {
-        return new EndpointBuilder().addPathPart(index, "_doc", id).build();
-    }
-
-    private static String endpoint(String[] indices, String endpoint) {
-        return new EndpointBuilder().addCommaSeparatedPathParts(indices).addPathPartAsIs(endpoint).build();
-    }
-
-    /**
-     * Returns a {@link ContentType} from a given {@link XContentType}.
-     *
-     * @param xContentType the {@link XContentType}
-     * @return the {@link ContentType}
-     */
-    @SuppressForbidden(reason = "Only allowed place to convert a XContentType to a ContentType")
-    private static ContentType createContentType(final XContentType xContentType) {
-        return ContentType.create(xContentType.mediaTypeWithoutParameters(), (Charset) null);
-    }
-
-    /**
-     * Utility class to help with common parameter names and patterns. Wraps
-     * a {@link Request} and adds the parameters to it directly.
-     */
-    private static class Params {
-        private final Map<String, String> parameters = new HashMap<>();
-
-        Params() {}
-
-        Params putParam(String name, String value) {
-            if (Strings.hasLength(value)) {
-                parameters.put(name, value);
-            }
-            return this;
-        }
-
-        Params putParam(String key, TimeValue value) {
-            if (value != null) {
-                return putParam(key, value.getStringRep());
-            }
-            return this;
-        }
-
-        Map<String, String> asMap() {
-            return parameters;
-        }
-
-        Params withPipeline(String pipeline) {
-            return putParam("pipeline", pipeline);
-        }
-
-        Params withPreference(String preference) {
-            return putParam("preference", preference);
-        }
-
-        Params withSearchType(String searchType) {
-            return putParam("search_type", searchType);
-        }
-
-        Params withMaxConcurrentShardRequests(int maxConcurrentShardRequests) {
-            return putParam("max_concurrent_shard_requests", Integer.toString(maxConcurrentShardRequests));
-        }
-
-        Params withBatchedReduceSize(int batchedReduceSize) {
-            return putParam("batched_reduce_size", Integer.toString(batchedReduceSize));
-        }
-
-        Params withRequestCache(boolean requestCache) {
-            return putParam("request_cache", Boolean.toString(requestCache));
-        }
-
-        Params withAllowPartialResults(boolean allowPartialSearchResults) {
-            return putParam("allow_partial_search_results", Boolean.toString(allowPartialSearchResults));
-        }
-
-        Params withRefreshPolicy(RefreshPolicy refreshPolicy) {
-            if (refreshPolicy != RefreshPolicy.NONE) {
-                return putParam("refresh", refreshPolicy.getValue());
-            }
-            return this;
-        }
-
-        Params withRouting(String routing) {
-            return putParam("routing", routing);
-        }
-
-        Params withTimeout(TimeValue timeout) {
-            return putParam("timeout", timeout);
-        }
-
-        Params withVersion(long version) {
-            if (version != Versions.MATCH_ANY) {
-                return putParam("version", Long.toString(version));
-            }
-            return this;
-        }
-
-        Params withVersionType(VersionType versionType) {
-            if (versionType != VersionType.INTERNAL) {
-                return putParam("version_type", versionType.name().toLowerCase(Locale.ROOT));
-            }
-            return this;
-        }
-
-        Params withIfSeqNo(long ifSeqNo) {
-            if (ifSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) {
-                return putParam("if_seq_no", Long.toString(ifSeqNo));
-            }
-            return this;
-        }
-
-        Params withIfPrimaryTerm(long ifPrimaryTerm) {
-            if (ifPrimaryTerm != SequenceNumbers.UNASSIGNED_PRIMARY_TERM) {
-                return putParam("if_primary_term", Long.toString(ifPrimaryTerm));
-            }
-            return this;
-        }
-
-        Params withWaitForActiveShards(ActiveShardCount activeShardCount) {
-            return withWaitForActiveShards(activeShardCount, ActiveShardCount.DEFAULT);
-        }
-
-        Params withWaitForActiveShards(ActiveShardCount activeShardCount, ActiveShardCount defaultActiveShardCount) {
-            if (activeShardCount != null && activeShardCount != defaultActiveShardCount) {
-                return putParam("wait_for_active_shards", activeShardCount.toString().toLowerCase(Locale.ROOT));
-            }
-            return this;
-        }
-
-        Params withRequireAlias(boolean requireAlias) {
-            if (requireAlias) {
-                return putParam("require_alias", Boolean.toString(requireAlias));
-            }
-            return this;
-        }
-
-        Params withIndicesOptions(IndicesOptions indicesOptions) {
-            if (indicesOptions != null) {
-                withIgnoreUnavailable(indicesOptions.ignoreUnavailable());
-                putParam("allow_no_indices", Boolean.toString(indicesOptions.allowNoIndices()));
-                String expandWildcards;
-                if (indicesOptions.expandWildcardExpressions() == false) {
-                    expandWildcards = "none";
-                } else {
-                    StringJoiner joiner = new StringJoiner(",");
-                    if (indicesOptions.expandWildcardsOpen()) {
-                        joiner.add("open");
-                    }
-                    if (indicesOptions.expandWildcardsClosed()) {
-                        joiner.add("closed");
-                    }
-                    expandWildcards = joiner.toString();
-                }
-                putParam("expand_wildcards", expandWildcards);
-                putParam("ignore_throttled", Boolean.toString(indicesOptions.ignoreThrottled()));
-            }
-            return this;
-        }
-
-        Params withIgnoreUnavailable(boolean ignoreUnavailable) {
-            // Always explicitly place the ignore_unavailable value.
-            putParam("ignore_unavailable", Boolean.toString(ignoreUnavailable));
-            return this;
-        }
-    }
-
-    /**
-     * Ensure that the {@link IndexRequest}'s content type is supported by the Bulk API and that it conforms
-     * to the current {@link BulkRequest}'s content type (if it's known at the time of this method get called).
-     *
-     * @return the {@link IndexRequest}'s content type
-     */
-    private static XContentType enforceSameContentType(IndexRequest indexRequest, @Nullable XContentType xContentType) {
-        XContentType requestContentType = indexRequest.getContentType();
-        if (requestContentType.canonical() != XContentType.JSON && requestContentType.canonical() != XContentType.SMILE) {
-            throw new IllegalArgumentException(
-                "Unsupported content-type found for request with content-type ["
-                    + requestContentType
-                    + "], only JSON and SMILE are supported"
-            );
-        }
-        if (xContentType == null) {
-            return requestContentType;
-        }
-        if (requestContentType.canonical() != xContentType.canonical()) {
-            throw new IllegalArgumentException(
-                "Mismatching content-type found for request with content-type ["
-                    + requestContentType
-                    + "], previous requests have content-type ["
-                    + xContentType
-                    + "]"
-            );
-        }
-        return xContentType;
-    }
-
-    /**
-     * Utility class to build request's endpoint given its parts as strings
-     */
-    private static class EndpointBuilder {
-
-        private final StringJoiner joiner = new StringJoiner("/", "/", "");
-
-        EndpointBuilder addPathPart(String... parts) {
-            for (String part : parts) {
-                if (Strings.hasLength(part)) {
-                    joiner.add(encodePart(part));
-                }
-            }
-            return this;
-        }
-
-        EndpointBuilder addCommaSeparatedPathParts(String[] parts) {
-            addPathPart(String.join(",", parts));
-            return this;
-        }
-
-        EndpointBuilder addPathPartAsIs(String... parts) {
-            for (String part : parts) {
-                if (Strings.hasLength(part)) {
-                    joiner.add(part);
-                }
-            }
-            return this;
-        }
-
-        private String build() {
-            return joiner.toString();
-        }
-
-        private static String encodePart(String pathPart) {
-            try {
-                // encode each part (e.g. index, type and id) separately before merging them into the path
-                // we prepend "/" to the path part to make this path absolute, otherwise there can be issues with
-                // paths that start with `-` or contain `:`
-                // the authority must be an empty string and not null, else paths that being with slashes could have them
-                // misinterpreted as part of the authority.
-                URI uri = new URI(null, "", "/" + pathPart, null, null);
-                // manually encode any slash that each part may contain
-                return uri.getRawPath().substring(1).replaceAll("/", "%2F");
-            } catch (URISyntaxException e) {
-                throw new IllegalArgumentException("Path part [" + pathPart + "] couldn't be encoded", e);
-            }
-        }
-    }
-}

+ 0 - 969
client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java

@@ -1,969 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0 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.client;
-
-import org.apache.http.Header;
-import org.apache.http.HttpEntity;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.ElasticsearchStatusException;
-import org.elasticsearch.action.ActionListener;
-import org.elasticsearch.action.ActionRequest;
-import org.elasticsearch.action.ActionRequestValidationException;
-import org.elasticsearch.action.bulk.BulkRequest;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.index.IndexRequest;
-import org.elasticsearch.action.index.IndexResponse;
-import org.elasticsearch.action.search.SearchRequest;
-import org.elasticsearch.action.search.SearchResponse;
-import org.elasticsearch.action.support.PlainActionFuture;
-import org.elasticsearch.aggregations.bucket.adjacency.AdjacencyMatrixAggregationBuilder;
-import org.elasticsearch.aggregations.bucket.adjacency.ParsedAdjacencyMatrix;
-import org.elasticsearch.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder;
-import org.elasticsearch.aggregations.bucket.histogram.ParsedAutoDateHistogram;
-import org.elasticsearch.aggregations.bucket.timeseries.ParsedTimeSeries;
-import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
-import org.elasticsearch.aggregations.pipeline.DerivativePipelineAggregationBuilder;
-import org.elasticsearch.client.analytics.ParsedStringStats;
-import org.elasticsearch.client.analytics.ParsedTopMetrics;
-import org.elasticsearch.client.analytics.StringStatsAggregationBuilder;
-import org.elasticsearch.client.analytics.TopMetricsAggregationBuilder;
-import org.elasticsearch.client.core.MainResponse;
-import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.util.concurrent.FutureUtils;
-import org.elasticsearch.common.util.concurrent.ListenableFuture;
-import org.elasticsearch.core.CheckedConsumer;
-import org.elasticsearch.core.CheckedFunction;
-import org.elasticsearch.plugins.spi.NamedXContentProvider;
-import org.elasticsearch.rest.RestResponse;
-import org.elasticsearch.rest.RestStatus;
-import org.elasticsearch.search.aggregations.Aggregation;
-import org.elasticsearch.search.aggregations.bucket.composite.CompositeAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.composite.ParsedComposite;
-import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.filter.ParsedFilter;
-import org.elasticsearch.search.aggregations.bucket.filter.ParsedFilters;
-import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGridAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileGridAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.geogrid.ParsedGeoHashGrid;
-import org.elasticsearch.search.aggregations.bucket.geogrid.ParsedGeoTileGrid;
-import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.global.ParsedGlobal;
-import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.histogram.HistogramAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.histogram.ParsedDateHistogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.ParsedHistogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.ParsedVariableWidthHistogram;
-import org.elasticsearch.search.aggregations.bucket.histogram.VariableWidthHistogramAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.missing.MissingAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.missing.ParsedMissing;
-import org.elasticsearch.search.aggregations.bucket.nested.NestedAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.nested.ParsedNested;
-import org.elasticsearch.search.aggregations.bucket.nested.ParsedReverseNested;
-import org.elasticsearch.search.aggregations.bucket.nested.ReverseNestedAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.range.DateRangeAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.range.GeoDistanceAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.range.IpRangeAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.range.ParsedBinaryRange;
-import org.elasticsearch.search.aggregations.bucket.range.ParsedDateRange;
-import org.elasticsearch.search.aggregations.bucket.range.ParsedGeoDistance;
-import org.elasticsearch.search.aggregations.bucket.range.ParsedRange;
-import org.elasticsearch.search.aggregations.bucket.range.RangeAggregationBuilder;
-import org.elasticsearch.search.aggregations.bucket.sampler.InternalSampler;
-import org.elasticsearch.search.aggregations.bucket.sampler.ParsedSampler;
-import org.elasticsearch.search.aggregations.bucket.terms.DoubleTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.LongRareTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedDoubleTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedLongRareTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedLongTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedSignificantLongTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedSignificantStringTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedStringRareTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.ParsedStringTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.SignificantLongTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.SignificantStringTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.StringRareTerms;
-import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
-import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.ExtendedStatsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.GeoBoundsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.GeoCentroidAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.InternalHDRPercentiles;
-import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.InternalTDigestPercentiles;
-import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.MedianAbsoluteDeviationAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.MinAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.ParsedAvg;
-import org.elasticsearch.search.aggregations.metrics.ParsedCardinality;
-import org.elasticsearch.search.aggregations.metrics.ParsedExtendedStats;
-import org.elasticsearch.search.aggregations.metrics.ParsedGeoBounds;
-import org.elasticsearch.search.aggregations.metrics.ParsedGeoCentroid;
-import org.elasticsearch.search.aggregations.metrics.ParsedHDRPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.ParsedHDRPercentiles;
-import org.elasticsearch.search.aggregations.metrics.ParsedMax;
-import org.elasticsearch.search.aggregations.metrics.ParsedMedianAbsoluteDeviation;
-import org.elasticsearch.search.aggregations.metrics.ParsedMin;
-import org.elasticsearch.search.aggregations.metrics.ParsedScriptedMetric;
-import org.elasticsearch.search.aggregations.metrics.ParsedStats;
-import org.elasticsearch.search.aggregations.metrics.ParsedSum;
-import org.elasticsearch.search.aggregations.metrics.ParsedTDigestPercentileRanks;
-import org.elasticsearch.search.aggregations.metrics.ParsedTDigestPercentiles;
-import org.elasticsearch.search.aggregations.metrics.ParsedTopHits;
-import org.elasticsearch.search.aggregations.metrics.ParsedValueCount;
-import org.elasticsearch.search.aggregations.metrics.ParsedWeightedAvg;
-import org.elasticsearch.search.aggregations.metrics.ScriptedMetricAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.StatsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.SumAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.TopHitsAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder;
-import org.elasticsearch.search.aggregations.metrics.WeightedAvgAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.ExtendedStatsBucketPipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.InternalBucketMetricValue;
-import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
-import org.elasticsearch.search.aggregations.pipeline.ParsedBucketMetricValue;
-import org.elasticsearch.search.aggregations.pipeline.ParsedDerivative;
-import org.elasticsearch.search.aggregations.pipeline.ParsedExtendedStatsBucket;
-import org.elasticsearch.search.aggregations.pipeline.ParsedPercentilesBucket;
-import org.elasticsearch.search.aggregations.pipeline.ParsedSimpleValue;
-import org.elasticsearch.search.aggregations.pipeline.ParsedStatsBucket;
-import org.elasticsearch.search.aggregations.pipeline.PercentilesBucketPipelineAggregationBuilder;
-import org.elasticsearch.search.aggregations.pipeline.StatsBucketPipelineAggregationBuilder;
-import org.elasticsearch.search.suggest.Suggest;
-import org.elasticsearch.search.suggest.completion.CompletionSuggestion;
-import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder;
-import org.elasticsearch.search.suggest.phrase.PhraseSuggestion;
-import org.elasticsearch.search.suggest.phrase.PhraseSuggestionBuilder;
-import org.elasticsearch.search.suggest.term.TermSuggestion;
-import org.elasticsearch.search.suggest.term.TermSuggestionBuilder;
-import org.elasticsearch.xcontent.ContextParser;
-import org.elasticsearch.xcontent.DeprecationHandler;
-import org.elasticsearch.xcontent.NamedXContentRegistry;
-import org.elasticsearch.xcontent.ParseField;
-import org.elasticsearch.xcontent.XContentParser;
-import org.elasticsearch.xcontent.XContentParserConfiguration;
-import org.elasticsearch.xcontent.XContentType;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.ServiceLoader;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static java.util.Collections.emptySet;
-import static java.util.stream.Collectors.toList;
-
-/**
- * High level REST client that wraps an instance of the low level {@link RestClient} and allows to build requests and read responses. The
- * {@link RestClient} instance is internally built based on the provided {@link RestClientBuilder} and it gets closed automatically when
- * closing the {@link RestHighLevelClient} instance that wraps it.
- *
- * @deprecated The High Level Rest Client is deprecated in favor of the
- * <a href="https://www.elastic.co/guide/en/elasticsearch/client/java-api-client/current/introduction.html">
- * Elasticsearch Java API Client</a>
- */
-@Deprecated(since = "7.16.0", forRemoval = true)
-@SuppressWarnings("removal")
-public class RestHighLevelClient implements Closeable {
-
-    private static final Logger logger = LogManager.getLogger(RestHighLevelClient.class);
-    /**
-     * Environment variable determining whether to send the 7.x compatibility header
-     */
-    private static final String API_VERSIONING_ENV_VARIABLE = "ELASTIC_CLIENT_APIVERSIONING";
-
-    // To be called using performClientRequest and performClientRequestAsync to ensure version compatibility check
-    private final RestClient client;
-    private final XContentParserConfiguration parserConfig;
-    private final CheckedConsumer<RestClient, IOException> doClose;
-    private final boolean useAPICompatibility;
-
-    /** Do not access directly but through getVersionValidationFuture() */
-    private volatile ListenableFuture<Optional<String>> versionValidationFuture;
-
-    /**
-     * Creates a {@link RestHighLevelClient} given the low level {@link RestClient} that it should use to perform requests and
-     * a list of entries that allow to parse custom response sections added to Elasticsearch through plugins.
-     * This constructor can be called by subclasses in case an externally created low-level REST client needs to be provided.
-     * The consumer argument allows to control what needs to be done when the {@link #close()} method is called.
-     * Also subclasses can provide parsers for custom response sections added to Elasticsearch through plugins.
-     */
-    protected RestHighLevelClient(
-        RestClient restClient,
-        CheckedConsumer<RestClient, IOException> doClose,
-        List<NamedXContentRegistry.Entry> namedXContentEntries
-    ) {
-        this(restClient, doClose, namedXContentEntries, null);
-    }
-
-    /**
-     * Creates a {@link RestHighLevelClient} given the low level {@link RestClient} that it should use to perform requests and
-     * a list of entries that allow to parse custom response sections added to Elasticsearch through plugins.
-     * This constructor can be called by subclasses in case an externally created low-level REST client needs to be provided.
-     * The consumer argument allows to control what needs to be done when the {@link #close()} method is called.
-     * Also subclasses can provide parsers for custom response sections added to Elasticsearch through plugins.
-     */
-    private RestHighLevelClient(
-        RestClient restClient,
-        CheckedConsumer<RestClient, IOException> doClose,
-        List<NamedXContentRegistry.Entry> namedXContentEntries,
-        Boolean useAPICompatibility
-    ) {
-        this.client = Objects.requireNonNull(restClient, "restClient must not be null");
-        this.doClose = Objects.requireNonNull(doClose, "doClose consumer must not be null");
-        NamedXContentRegistry registry = new NamedXContentRegistry(
-            Stream.of(getDefaultNamedXContents().stream(), getProvidedNamedXContents().stream(), namedXContentEntries.stream())
-                .flatMap(Function.identity())
-                .collect(toList())
-        );
-        /*
-         * Ignores deprecation warnings. This is appropriate because it is only
-         * used to parse responses from Elasticsearch. Any deprecation warnings
-         * emitted there just mean that you are talking to an old version of
-         * Elasticsearch. There isn't anything you can do about the deprecation.
-         */
-        this.parserConfig = XContentParserConfiguration.EMPTY.withRegistry(registry)
-            .withDeprecationHandler(DeprecationHandler.IGNORE_DEPRECATIONS);
-        if (useAPICompatibility == null && "true".equals(System.getenv(API_VERSIONING_ENV_VARIABLE))) {
-            this.useAPICompatibility = true;
-        } else {
-            this.useAPICompatibility = Boolean.TRUE.equals(useAPICompatibility);
-        }
-    }
-
-    /**
-     * Returns the low-level client that the current high-level client instance is using to perform requests
-     */
-    public final RestClient getLowLevelClient() {
-        return client;
-    }
-
-    public final XContentParserConfiguration getParserConfig() {
-        return parserConfig;
-    }
-
-    @Override
-    public final void close() throws IOException {
-        doClose.accept(client);
-    }
-
-    /**
-     * Asynchronously executes a bulk request using the Bulk API.
-     * See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html">Bulk API on elastic.co</a>
-     * @param bulkRequest the request
-     * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
-     * @param listener the listener to be notified upon request completion
-     * @return cancellable that may be used to cancel the request
-     */
-    public final Cancellable bulkAsync(BulkRequest bulkRequest, RequestOptions options, ActionListener<BulkResponse> listener) {
-        return performRequestAsyncAndParseEntity(
-            bulkRequest,
-            RequestConverters::bulk,
-            options,
-            BulkResponse::fromXContent,
-            listener,
-            emptySet()
-        );
-    }
-
-    /**
-     * Index a document using the Index API.
-     * See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-index_.html">Index API on elastic.co</a>
-     * @param indexRequest the request
-     * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
-     * @return the response
-     */
-    public final IndexResponse index(IndexRequest indexRequest, RequestOptions options) throws IOException {
-        return performRequestAndParseEntity(indexRequest, RequestConverters::index, options, IndexResponse::fromXContent, emptySet());
-    }
-
-    /**
-     * Asynchronously executes a search using the Search API.
-     * See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/search-search.html">Search API on elastic.co</a>
-     * @param searchRequest the request
-     * @param options the request options (e.g. headers), use {@link RequestOptions#DEFAULT} if nothing needs to be customized
-     * @param listener the listener to be notified upon request completion
-     * @return cancellable that may be used to cancel the request
-     */
-    public final Cancellable searchAsync(SearchRequest searchRequest, RequestOptions options, ActionListener<SearchResponse> listener) {
-        return performRequestAsyncAndParseEntity(
-            searchRequest,
-            r -> RequestConverters.search(r, "_search"),
-            options,
-            SearchResponse::fromXContent,
-            listener,
-            emptySet()
-        );
-    }
-
-    /**
-     * @deprecated If creating a new HLRC ReST API call, consider creating new actions instead of reusing server actions.
-     */
-    @Deprecated
-    private <Req extends ActionRequest, Resp> Resp performRequestAndParseEntity(
-        Req request,
-        CheckedFunction<Req, Request, IOException> requestConverter,
-        RequestOptions options,
-        CheckedFunction<XContentParser, Resp, IOException> entityParser,
-        Set<Integer> ignores
-    ) throws IOException {
-        return performRequest(request, requestConverter, options, response -> parseEntity(response.getEntity(), entityParser), ignores);
-    }
-
-    /**
-     * @deprecated If creating a new HLRC ReST API call, consider creating new actions instead of reusing server actions.
-     */
-    @Deprecated
-    private <Req extends ActionRequest, Resp> Resp performRequest(
-        Req request,
-        CheckedFunction<Req, Request, IOException> requestConverter,
-        RequestOptions options,
-        CheckedFunction<Response, Resp, IOException> responseConverter,
-        Set<Integer> ignores
-    ) throws IOException {
-        ActionRequestValidationException validationException = request.validate();
-        if (validationException != null && validationException.validationErrors().isEmpty() == false) {
-            throw validationException;
-        }
-        return internalPerformRequest(request, requestConverter, options, responseConverter, ignores);
-    }
-
-    /**
-     * Provides common functionality for performing a request.
-     */
-    private <Req, Resp> Resp internalPerformRequest(
-        Req request,
-        CheckedFunction<Req, Request, IOException> requestConverter,
-        RequestOptions options,
-        CheckedFunction<Response, Resp, IOException> responseConverter,
-        Set<Integer> ignores
-    ) throws IOException {
-        Request req = requestConverter.apply(request);
-        req.setOptions(options);
-        Response response;
-        try {
-            response = performClientRequest(req);
-        } catch (ResponseException e) {
-            if (ignores.contains(e.getResponse().getStatusLine().getStatusCode())) {
-                try {
-                    return responseConverter.apply(e.getResponse());
-                } catch (Exception innerException) {
-                    // the exception is ignored as we now try to parse the response as an error.
-                    // this covers cases like get where 404 can either be a valid document not found response,
-                    // or an error for which parsing is completely different. We try to consider the 404 response as a valid one
-                    // first. If parsing of the response breaks, we fall back to parsing it as an error.
-                    throw parseResponseException(e);
-                }
-            }
-            throw parseResponseException(e);
-        }
-
-        try {
-            return responseConverter.apply(response);
-        } catch (Exception e) {
-            throw new IOException("Unable to parse response body for " + response, e);
-        }
-    }
-
-    /**
-     * @deprecated If creating a new HLRC ReST API call, consider creating new actions instead of reusing server actions.
-     * @return Cancellable instance that may be used to cancel the request
-     */
-    @Deprecated
-    private <Req extends ActionRequest, Resp> Cancellable performRequestAsyncAndParseEntity(
-        Req request,
-        CheckedFunction<Req, Request, IOException> requestConverter,
-        RequestOptions options,
-        CheckedFunction<XContentParser, Resp, IOException> entityParser,
-        ActionListener<Resp> listener,
-        Set<Integer> ignores
-    ) {
-        return performRequestAsync(
-            request,
-            requestConverter,
-            options,
-            response -> parseEntity(response.getEntity(), entityParser),
-            listener,
-            ignores
-        );
-    }
-
-    /**
-     * @deprecated If creating a new HLRC ReST API call, consider creating new actions instead of reusing server actions.
-     * @return Cancellable instance that may be used to cancel the request
-     */
-    @Deprecated
-    private <Req extends ActionRequest, Resp> Cancellable performRequestAsync(
-        Req request,
-        CheckedFunction<Req, Request, IOException> requestConverter,
-        RequestOptions options,
-        CheckedFunction<Response, Resp, IOException> responseConverter,
-        ActionListener<Resp> listener,
-        Set<Integer> ignores
-    ) {
-        ActionRequestValidationException validationException = request.validate();
-        if (validationException != null && validationException.validationErrors().isEmpty() == false) {
-            listener.onFailure(validationException);
-            return Cancellable.NO_OP;
-        }
-        return internalPerformRequestAsync(request, requestConverter, options, responseConverter, listener, ignores);
-    }
-
-    /**
-     * Provides common functionality for asynchronously performing a request.
-     * @return Cancellable instance that may be used to cancel the request
-     */
-    private <Req, Resp> Cancellable internalPerformRequestAsync(
-        Req request,
-        CheckedFunction<Req, Request, IOException> requestConverter,
-        RequestOptions options,
-        CheckedFunction<Response, Resp, IOException> responseConverter,
-        ActionListener<Resp> listener,
-        Set<Integer> ignores
-    ) {
-        Request req;
-        try {
-            req = requestConverter.apply(request);
-        } catch (Exception e) {
-            listener.onFailure(e);
-            return Cancellable.NO_OP;
-        }
-        req.setOptions(options);
-
-        ResponseListener responseListener = wrapResponseListener(responseConverter, listener, ignores);
-        return performClientRequestAsync(req, responseListener);
-    }
-
-    private <Resp> ResponseListener wrapResponseListener(
-        CheckedFunction<Response, Resp, IOException> responseConverter,
-        ActionListener<Resp> actionListener,
-        Set<Integer> ignores
-    ) {
-        return new ResponseListener() {
-            @Override
-            public void onSuccess(Response response) {
-                try {
-                    actionListener.onResponse(responseConverter.apply(response));
-                } catch (Exception e) {
-                    IOException ioe = new IOException("Unable to parse response body for " + response, e);
-                    onFailure(ioe);
-                }
-            }
-
-            @Override
-            public void onFailure(Exception exception) {
-                if (exception instanceof ResponseException responseException) {
-                    Response response = responseException.getResponse();
-                    if (ignores.contains(response.getStatusLine().getStatusCode())) {
-                        try {
-                            actionListener.onResponse(responseConverter.apply(response));
-                        } catch (Exception innerException) {
-                            // the exception is ignored as we now try to parse the response as an error.
-                            // this covers cases like get where 404 can either be a valid document not found response,
-                            // or an error for which parsing is completely different. We try to consider the 404 response as a valid one
-                            // first. If parsing of the response breaks, we fall back to parsing it as an error.
-                            actionListener.onFailure(parseResponseException(responseException));
-                        }
-                    } else {
-                        actionListener.onFailure(parseResponseException(responseException));
-                    }
-                } else {
-                    actionListener.onFailure(exception);
-                }
-            }
-        };
-    }
-
-    /**
-     * Converts a {@link ResponseException} obtained from the low level REST client into an {@link ElasticsearchException}.
-     * If a response body was returned, tries to parse it as an error returned from Elasticsearch.
-     * If no response body was returned or anything goes wrong while parsing the error, returns a new {@link ElasticsearchStatusException}
-     * that wraps the original {@link ResponseException}. The potential exception obtained while parsing is added to the returned
-     * exception as a suppressed exception. This method is guaranteed to not throw any exception eventually thrown while parsing.
-     */
-    private ElasticsearchStatusException parseResponseException(ResponseException responseException) {
-        Response response = responseException.getResponse();
-        HttpEntity entity = response.getEntity();
-        ElasticsearchStatusException elasticsearchException;
-        RestStatus restStatus = RestStatus.fromCode(response.getStatusLine().getStatusCode());
-
-        if (entity == null) {
-            elasticsearchException = new ElasticsearchStatusException(responseException.getMessage(), restStatus, responseException);
-        } else {
-            try {
-                elasticsearchException = parseEntity(entity, RestResponse::errorFromXContent);
-                elasticsearchException.addSuppressed(responseException);
-            } catch (Exception e) {
-                elasticsearchException = new ElasticsearchStatusException("Unable to parse response body", restStatus, responseException);
-                elasticsearchException.addSuppressed(e);
-            }
-        }
-        return elasticsearchException;
-    }
-
-    private <Resp> Resp parseEntity(final HttpEntity entity, final CheckedFunction<XContentParser, Resp, IOException> entityParser)
-        throws IOException {
-        if (entity == null) {
-            throw new IllegalStateException("Response body expected but not returned");
-        }
-        if (entity.getContentType() == null) {
-            throw new IllegalStateException("Elasticsearch didn't return the [Content-Type] header, unable to parse response body");
-        }
-        XContentType xContentType = XContentType.fromMediaType(entity.getContentType().getValue());
-        if (xContentType == null) {
-            throw new IllegalStateException("Unsupported Content-Type: " + entity.getContentType().getValue());
-        }
-        try (XContentParser parser = xContentType.xContent().createParser(parserConfig, entity.getContent())) {
-            return entityParser.apply(parser);
-        }
-    }
-
-    private enum EntityType {
-        JSON() {
-            @Override
-            public String header() {
-                return "application/json";
-            }
-
-            @Override
-            public String compatibleHeader() {
-                return "application/vnd.elasticsearch+json; compatible-with=7";
-            }
-        },
-        NDJSON() {
-            @Override
-            public String header() {
-                return "application/x-ndjson";
-            }
-
-            @Override
-            public String compatibleHeader() {
-                return "application/vnd.elasticsearch+x-ndjson; compatible-with=7";
-            }
-        },
-        STAR() {
-            @Override
-            public String header() {
-                return "application/*";
-            }
-
-            @Override
-            public String compatibleHeader() {
-                return "application/vnd.elasticsearch+json; compatible-with=7";
-            }
-        },
-        YAML() {
-            @Override
-            public String header() {
-                return "application/yaml";
-            }
-
-            @Override
-            public String compatibleHeader() {
-                return "application/vnd.elasticsearch+yaml; compatible-with=7";
-            }
-        },
-        SMILE() {
-            @Override
-            public String header() {
-                return "application/smile";
-            }
-
-            @Override
-            public String compatibleHeader() {
-                return "application/vnd.elasticsearch+smile; compatible-with=7";
-            }
-        },
-        CBOR() {
-            @Override
-            public String header() {
-                return "application/cbor";
-            }
-
-            @Override
-            public String compatibleHeader() {
-                return "application/vnd.elasticsearch+cbor; compatible-with=7";
-            }
-        };
-
-        public abstract String header();
-
-        public abstract String compatibleHeader();
-
-        @Override
-        public String toString() {
-            return header();
-        }
-    }
-
-    private Cancellable performClientRequestAsync(Request request, ResponseListener listener) {
-        // Add compatibility request headers if compatibility mode has been enabled
-        if (this.useAPICompatibility) {
-            modifyRequestForCompatibility(request);
-        }
-
-        ListenableFuture<Optional<String>> versionCheck = getVersionValidationFuture();
-
-        // Create a future that tracks cancellation of this method's result and forwards cancellation to the actual LLRC request.
-        CompletableFuture<Void> cancellationForwarder = new CompletableFuture<>();
-        Cancellable result = new Cancellable() {
-            @Override
-            public void cancel() {
-                // Raise the flag by completing the future
-                FutureUtils.cancel(cancellationForwarder);
-            }
-
-            @Override
-            void runIfNotCancelled(Runnable runnable) {
-                if (cancellationForwarder.isCancelled()) {
-                    throw newCancellationException();
-                }
-                runnable.run();
-            }
-        };
-
-        // Send the request after we have done the version compatibility check. Note that if it has already happened, the listener will
-        // be called immediately on the same thread with no asynchronous scheduling overhead.
-        versionCheck.addListener(new ActionListener<>() {
-            @Override
-            public void onResponse(Optional<String> validation) {
-                if (validation.isPresent() == false) {
-                    // Send the request and propagate cancellation
-                    Cancellable call = client.performRequestAsync(request, listener);
-                    cancellationForwarder.whenComplete((r, t) ->
-                    // Forward cancellation to the actual request (no need to check parameters as the
-                    // only way for cancellationForwarder to be completed is by being cancelled).
-                    call.cancel());
-                } else {
-                    // Version validation wasn't successful, fail the request with the validation result.
-                    listener.onFailure(new ElasticsearchException(validation.get()));
-                }
-            }
-
-            @Override
-            public void onFailure(Exception e) {
-                // Propagate validation request failure. This will be transient since `getVersionValidationFuture` clears the validation
-                // future if the request fails, leading to retries at the next HLRC request (see comments below).
-                listener.onFailure(e);
-            }
-        });
-
-        return result;
-    }
-
-    /**
-     * Go through all the request's existing headers, looking for {@code headerName} headers and if they exist,
-     * changing them to use version compatibility. If no request headers are changed, modify the entity type header if appropriate
-     */
-    private boolean addCompatibilityFor(RequestOptions.Builder newOptions, Header entityHeader, String headerName) {
-        // Modify any existing "Content-Type" headers on the request to use the version compatibility, if available
-        boolean contentTypeModified = false;
-        for (Header header : new ArrayList<>(newOptions.getHeaders())) {
-            if (headerName.equalsIgnoreCase(header.getName()) == false) {
-                continue;
-            }
-            contentTypeModified = contentTypeModified || modifyHeader(newOptions, header, headerName);
-        }
-
-        // If there were no request-specific headers, modify the request entity's header to be compatible
-        if (entityHeader != null && contentTypeModified == false) {
-            contentTypeModified = modifyHeader(newOptions, entityHeader, headerName);
-        }
-
-        return contentTypeModified;
-    }
-
-    /**
-     * Modify the given header to be version compatible, if necessary.
-     * Returns true if a modification was made, false otherwise.
-     */
-    private boolean modifyHeader(RequestOptions.Builder newOptions, Header header, String headerName) {
-        for (EntityType type : EntityType.values()) {
-            final String headerValue = header.getValue();
-            if (headerValue.startsWith(type.header())) {
-                String newHeaderValue = headerValue.replace(type.header(), type.compatibleHeader());
-                newOptions.removeHeader(header.getName());
-                newOptions.addHeader(headerName, newHeaderValue);
-                return true;
-            }
-        }
-        return false;
-    }
-
-    /**
-     * Make all necessary changes to support API compatibility for the given request. This includes
-     * modifying the "Content-Type" and "Accept" headers if present, or modifying the header based
-     * on the request's entity type.
-     */
-    private void modifyRequestForCompatibility(Request request) {
-        final Header entityHeader = request.getEntity() == null ? null : request.getEntity().getContentType();
-        final RequestOptions.Builder newOptions = request.getOptions().toBuilder();
-
-        addCompatibilityFor(newOptions, entityHeader, "Content-Type");
-        if (request.getOptions().containsHeader("Accept")) {
-            addCompatibilityFor(newOptions, entityHeader, "Accept");
-        } else {
-            // There is no entity, and no existing accept header, but we still need one
-            // with compatibility, so use the compatible JSON (default output) format
-            newOptions.addHeader("Accept", EntityType.JSON.compatibleHeader());
-        }
-        request.setOptions(newOptions);
-    }
-
-    private Response performClientRequest(Request request) throws IOException {
-        // Add compatibility request headers if compatibility mode has been enabled
-        if (this.useAPICompatibility) {
-            modifyRequestForCompatibility(request);
-        }
-
-        Optional<String> versionValidation;
-        try {
-            final var future = new PlainActionFuture<Optional<String>>();
-            getVersionValidationFuture().addListener(future);
-            versionValidation = future.get();
-        } catch (InterruptedException | ExecutionException e) {
-            // Unlikely to happen
-            throw new ElasticsearchException(e);
-        }
-
-        if (versionValidation.isPresent() == false) {
-            return client.performRequest(request);
-        } else {
-            throw new ElasticsearchException(versionValidation.get());
-        }
-    }
-
-    /**
-     * Returns a future that asynchronously validates the Elasticsearch product version. Its result is an optional string: if empty then
-     * validation was successful, if present it contains the validation error. API requests should be chained to this future and check
-     * the validation result before going further.
-     * <p>
-     * This future is a memoization of the first successful request to the "/" endpoint and the subsequent compatibility check
-     * ({@see #versionValidationFuture}). Further client requests reuse its result.
-     * <p>
-     * If the version check request fails (e.g. network error), {@link #versionValidationFuture} is cleared so that a new validation
-     * request is sent at the next HLRC request. This allows retries to happen while avoiding a busy retry loop (LLRC retries on the node
-     * pool still happen).
-     */
-    private ListenableFuture<Optional<String>> getVersionValidationFuture() {
-        ListenableFuture<Optional<String>> currentFuture = this.versionValidationFuture;
-        if (currentFuture != null) {
-            return currentFuture;
-        } else {
-            synchronized (this) {
-                // Re-check in synchronized block
-                currentFuture = this.versionValidationFuture;
-                if (currentFuture != null) {
-                    return currentFuture;
-                }
-                ListenableFuture<Optional<String>> future = new ListenableFuture<>();
-                this.versionValidationFuture = future;
-
-                // Asynchronously call the info endpoint and complete the future with the version validation result.
-                Request req = new Request("GET", "/");
-                // These status codes are nominal in the context of product version verification
-                req.addParameter("ignore", "401,403");
-                client.performRequestAsync(req, new ResponseListener() {
-                    @Override
-                    public void onSuccess(Response response) {
-                        Optional<String> validation;
-                        try {
-                            validation = getVersionValidation(response);
-                        } catch (Exception e) {
-                            logger.error("Failed to parse info response", e);
-                            validation = Optional.of(
-                                "Failed to parse info response. Check logs for detailed information - " + e.getMessage()
-                            );
-                        }
-                        future.onResponse(validation);
-                    }
-
-                    @Override
-                    public void onFailure(Exception exception) {
-
-                        // Fail the requests (this one and the ones waiting for it) and clear the future
-                        // so that we retry the next time the client executes a request.
-                        versionValidationFuture = null;
-                        future.onFailure(exception);
-                    }
-                });
-
-                return future;
-            }
-        }
-    }
-
-    /**
-     * Validates that the response info() is a compatible Elasticsearch version.
-     *
-     * @return an optional string. If empty, version is compatible. Otherwise, it's the message to return to the application.
-     */
-    private Optional<String> getVersionValidation(Response response) throws IOException {
-        // Let requests go through if the client doesn't have permissions for the info endpoint.
-        int statusCode = response.getStatusLine().getStatusCode();
-        if (statusCode == 401 || statusCode == 403) {
-            return Optional.empty();
-        }
-
-        MainResponse mainResponse;
-        try {
-            mainResponse = parseEntity(response.getEntity(), MainResponse::fromXContent);
-        } catch (ResponseException e) {
-            throw parseResponseException(e);
-        }
-
-        String version = mainResponse.getVersion().getNumber();
-        if (Strings.hasLength(version) == false) {
-            return Optional.of("Missing version.number in info response");
-        }
-
-        String[] parts = version.split("\\.");
-        if (parts.length < 2) {
-            return Optional.of("Wrong version.number format in info response");
-        }
-
-        int major = Integer.parseInt(parts[0]);
-        int minor = Integer.parseInt(parts[1]);
-
-        if (major < 6) {
-            return Optional.of("Elasticsearch version 6 or more is required");
-        }
-
-        if (major == 6 || (major == 7 && minor < 14)) {
-            if ("You Know, for Search".equalsIgnoreCase(mainResponse.getTagline()) == false) {
-                return Optional.of("Invalid or missing tagline [" + mainResponse.getTagline() + "]");
-            }
-
-            return Optional.empty();
-        }
-
-        String header = response.getHeader("X-Elastic-Product");
-        if (header == null) {
-            return Optional.of(
-                "Missing [X-Elastic-Product] header. Please check that you are connecting to an Elasticsearch "
-                    + "instance, and that any networking filters are preserving that header."
-            );
-        }
-
-        if ("Elasticsearch".equals(header) == false) {
-            return Optional.of("Invalid value [" + header + "] for [X-Elastic-Product] header.");
-        }
-
-        return Optional.empty();
-    }
-
-    private static List<NamedXContentRegistry.Entry> getDefaultNamedXContents() {
-        Map<String, ContextParser<Object, ? extends Aggregation>> map = new HashMap<>();
-        map.put(CardinalityAggregationBuilder.NAME, (p, c) -> ParsedCardinality.fromXContent(p, (String) c));
-        map.put(InternalHDRPercentiles.NAME, (p, c) -> ParsedHDRPercentiles.fromXContent(p, (String) c));
-        map.put(InternalHDRPercentileRanks.NAME, (p, c) -> ParsedHDRPercentileRanks.fromXContent(p, (String) c));
-        map.put(InternalTDigestPercentiles.NAME, (p, c) -> ParsedTDigestPercentiles.fromXContent(p, (String) c));
-        map.put(InternalTDigestPercentileRanks.NAME, (p, c) -> ParsedTDigestPercentileRanks.fromXContent(p, (String) c));
-        map.put(PercentilesBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedPercentilesBucket.fromXContent(p, (String) c));
-        map.put(MedianAbsoluteDeviationAggregationBuilder.NAME, (p, c) -> ParsedMedianAbsoluteDeviation.fromXContent(p, (String) c));
-        map.put(MinAggregationBuilder.NAME, (p, c) -> ParsedMin.fromXContent(p, (String) c));
-        map.put(MaxAggregationBuilder.NAME, (p, c) -> ParsedMax.fromXContent(p, (String) c));
-        map.put(SumAggregationBuilder.NAME, (p, c) -> ParsedSum.fromXContent(p, (String) c));
-        map.put(AvgAggregationBuilder.NAME, (p, c) -> ParsedAvg.fromXContent(p, (String) c));
-        map.put(WeightedAvgAggregationBuilder.NAME, (p, c) -> ParsedWeightedAvg.fromXContent(p, (String) c));
-        map.put(ValueCountAggregationBuilder.NAME, (p, c) -> ParsedValueCount.fromXContent(p, (String) c));
-        map.put(InternalSimpleValue.NAME, (p, c) -> ParsedSimpleValue.fromXContent(p, (String) c));
-        map.put(DerivativePipelineAggregationBuilder.NAME, (p, c) -> ParsedDerivative.fromXContent(p, (String) c));
-        map.put(InternalBucketMetricValue.NAME, (p, c) -> ParsedBucketMetricValue.fromXContent(p, (String) c));
-        map.put(StatsAggregationBuilder.NAME, (p, c) -> ParsedStats.fromXContent(p, (String) c));
-        map.put(StatsBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedStatsBucket.fromXContent(p, (String) c));
-        map.put(ExtendedStatsAggregationBuilder.NAME, (p, c) -> ParsedExtendedStats.fromXContent(p, (String) c));
-        map.put(ExtendedStatsBucketPipelineAggregationBuilder.NAME, (p, c) -> ParsedExtendedStatsBucket.fromXContent(p, (String) c));
-        map.put(GeoBoundsAggregationBuilder.NAME, (p, c) -> ParsedGeoBounds.fromXContent(p, (String) c));
-        map.put(GeoCentroidAggregationBuilder.NAME, (p, c) -> ParsedGeoCentroid.fromXContent(p, (String) c));
-        map.put(HistogramAggregationBuilder.NAME, (p, c) -> ParsedHistogram.fromXContent(p, (String) c));
-        map.put(DateHistogramAggregationBuilder.NAME, (p, c) -> ParsedDateHistogram.fromXContent(p, (String) c));
-        map.put(AutoDateHistogramAggregationBuilder.NAME, (p, c) -> ParsedAutoDateHistogram.fromXContent(p, (String) c));
-        map.put(VariableWidthHistogramAggregationBuilder.NAME, (p, c) -> ParsedVariableWidthHistogram.fromXContent(p, (String) c));
-        map.put(StringTerms.NAME, (p, c) -> ParsedStringTerms.fromXContent(p, (String) c));
-        map.put(LongTerms.NAME, (p, c) -> ParsedLongTerms.fromXContent(p, (String) c));
-        map.put(DoubleTerms.NAME, (p, c) -> ParsedDoubleTerms.fromXContent(p, (String) c));
-        map.put(LongRareTerms.NAME, (p, c) -> ParsedLongRareTerms.fromXContent(p, (String) c));
-        map.put(StringRareTerms.NAME, (p, c) -> ParsedStringRareTerms.fromXContent(p, (String) c));
-        map.put(MissingAggregationBuilder.NAME, (p, c) -> ParsedMissing.fromXContent(p, (String) c));
-        map.put(NestedAggregationBuilder.NAME, (p, c) -> ParsedNested.fromXContent(p, (String) c));
-        map.put(ReverseNestedAggregationBuilder.NAME, (p, c) -> ParsedReverseNested.fromXContent(p, (String) c));
-        map.put(GlobalAggregationBuilder.NAME, (p, c) -> ParsedGlobal.fromXContent(p, (String) c));
-        map.put(FilterAggregationBuilder.NAME, (p, c) -> ParsedFilter.fromXContent(p, (String) c));
-        map.put(InternalSampler.PARSER_NAME, (p, c) -> ParsedSampler.fromXContent(p, (String) c));
-        map.put(GeoHashGridAggregationBuilder.NAME, (p, c) -> ParsedGeoHashGrid.fromXContent(p, (String) c));
-        map.put(GeoTileGridAggregationBuilder.NAME, (p, c) -> ParsedGeoTileGrid.fromXContent(p, (String) c));
-        map.put(RangeAggregationBuilder.NAME, (p, c) -> ParsedRange.fromXContent(p, (String) c));
-        map.put(DateRangeAggregationBuilder.NAME, (p, c) -> ParsedDateRange.fromXContent(p, (String) c));
-        map.put(GeoDistanceAggregationBuilder.NAME, (p, c) -> ParsedGeoDistance.fromXContent(p, (String) c));
-        map.put(FiltersAggregationBuilder.NAME, (p, c) -> ParsedFilters.fromXContent(p, (String) c));
-        map.put(AdjacencyMatrixAggregationBuilder.NAME, (p, c) -> ParsedAdjacencyMatrix.fromXContent(p, (String) c));
-        map.put(SignificantLongTerms.NAME, (p, c) -> ParsedSignificantLongTerms.fromXContent(p, (String) c));
-        map.put(SignificantStringTerms.NAME, (p, c) -> ParsedSignificantStringTerms.fromXContent(p, (String) c));
-        map.put(ScriptedMetricAggregationBuilder.NAME, (p, c) -> ParsedScriptedMetric.fromXContent(p, (String) c));
-        map.put(IpRangeAggregationBuilder.NAME, (p, c) -> ParsedBinaryRange.fromXContent(p, (String) c));
-        map.put(TopHitsAggregationBuilder.NAME, (p, c) -> ParsedTopHits.fromXContent(p, (String) c));
-        map.put(CompositeAggregationBuilder.NAME, (p, c) -> ParsedComposite.fromXContent(p, (String) c));
-        map.put(StringStatsAggregationBuilder.NAME, (p, c) -> ParsedStringStats.PARSER.parse(p, (String) c));
-        map.put(TopMetricsAggregationBuilder.NAME, (p, c) -> ParsedTopMetrics.PARSER.parse(p, (String) c));
-        map.put(TimeSeriesAggregationBuilder.NAME, (p, c) -> ParsedTimeSeries.fromXContent(p, (String) (c)));
-        List<NamedXContentRegistry.Entry> entries = map.entrySet()
-            .stream()
-            .map(entry -> new NamedXContentRegistry.Entry(Aggregation.class, new ParseField(entry.getKey()), entry.getValue()))
-            .collect(Collectors.toList());
-        entries.add(
-            new NamedXContentRegistry.Entry(
-                Suggest.Suggestion.class,
-                new ParseField(TermSuggestionBuilder.SUGGESTION_NAME),
-                (parser, context) -> TermSuggestion.fromXContent(parser, (String) context)
-            )
-        );
-        entries.add(
-            new NamedXContentRegistry.Entry(
-                Suggest.Suggestion.class,
-                new ParseField(PhraseSuggestionBuilder.SUGGESTION_NAME),
-                (parser, context) -> PhraseSuggestion.fromXContent(parser, (String) context)
-            )
-        );
-        entries.add(
-            new NamedXContentRegistry.Entry(
-                Suggest.Suggestion.class,
-                new ParseField(CompletionSuggestionBuilder.SUGGESTION_NAME),
-                (parser, context) -> CompletionSuggestion.fromXContent(parser, (String) context)
-            )
-        );
-        return entries;
-    }
-
-    /**
-     * Loads and returns the {@link NamedXContentRegistry.Entry} parsers provided by plugins.
-     */
-    private static List<NamedXContentRegistry.Entry> getProvidedNamedXContents() {
-        List<NamedXContentRegistry.Entry> entries = new ArrayList<>();
-        for (NamedXContentProvider service : ServiceLoader.load(NamedXContentProvider.class)) {
-            entries.addAll(service.getNamedXContentParsers());
-        }
-        return entries;
-    }
-}

+ 0 - 220
client/rest-high-level/src/main/java/org/elasticsearch/client/core/MainResponse.java

@@ -1,220 +0,0 @@
-/*
- * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
- * or more contributor license agreements. Licensed under the Elastic License
- * 2.0 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.client.core;
-
-import org.elasticsearch.xcontent.ConstructingObjectParser;
-import org.elasticsearch.xcontent.ParseField;
-import org.elasticsearch.xcontent.XContentParser;
-
-import java.util.Objects;
-
-public class MainResponse {
-
-    private static final ConstructingObjectParser<MainResponse, Void> PARSER = new ConstructingObjectParser<>(
-        MainResponse.class.getName(),
-        true,
-        args -> {
-            return new MainResponse((String) args[0], (Version) args[1], (String) args[2], (String) args[3], (String) args[4]);
-        }
-    );
-
-    static {
-        PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("name"));
-        PARSER.declareObject(ConstructingObjectParser.constructorArg(), Version.PARSER, new ParseField("version"));
-        PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("cluster_name"));
-        PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("cluster_uuid"));
-        PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("tagline"));
-
-    }
-
-    private final String nodeName;
-    private final Version version;
-    private final String clusterName;
-    private final String clusterUuid;
-    private final String tagline;
-
-    public MainResponse(String nodeName, Version version, String clusterName, String clusterUuid, String tagline) {
-        this.nodeName = nodeName;
-        this.version = version;
-        this.clusterName = clusterName;
-        this.clusterUuid = clusterUuid;
-        this.tagline = tagline;
-    }
-
-    public String getNodeName() {
-        return nodeName;
-    }
-
-    public Version getVersion() {
-        return version;
-    }
-
-    public String getClusterName() {
-        return clusterName;
-    }
-
-    public String getClusterUuid() {
-        return clusterUuid;
-    }
-
-    public String getTagline() {
-        return tagline;
-    }
-
-    public static MainResponse fromXContent(XContentParser parser) {
-        return PARSER.apply(parser, null);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        MainResponse that = (MainResponse) o;
-        return nodeName.equals(that.nodeName)
-            && version.equals(that.version)
-            && clusterName.equals(that.clusterName)
-            && clusterUuid.equals(that.clusterUuid)
-            && tagline.equals(that.tagline);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(nodeName, version, clusterName, clusterUuid, tagline);
-    }
-
-    public static class Version {
-        private static final ConstructingObjectParser<Version, Void> PARSER = new ConstructingObjectParser<>(
-            Version.class.getName(),
-            true,
-            args -> {
-                return new Version(
-                    (String) args[0],
-                    (String) args[1],
-                    (String) args[2],
-                    (String) args[3],
-                    (String) args[4],
-                    (Boolean) args[5],
-                    (String) args[6],
-                    (String) args[7],
-                    (String) args[8]
-                );
-            }
-        );
-
-        static {
-            PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("number"));
-            PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField("build_flavor"));
-            PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField("build_type"));
-            PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("build_hash"));
-            PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("build_date"));
-            PARSER.declareBoolean(ConstructingObjectParser.constructorArg(), new ParseField("build_snapshot"));
-            PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("lucene_version"));
-            PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("minimum_wire_compatibility_version"));
-            PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField("minimum_index_compatibility_version"));
-        }
-        private final String number;
-        private final String buildFlavor;
-        private final String buildType;
-        private final String buildHash;
-        private final String buildDate;
-        private final boolean isSnapshot;
-        private final String luceneVersion;
-        private final String minimumWireCompatibilityVersion;
-        private final String minimumIndexCompatibilityVersion;
-
-        public Version(
-            String number,
-            String buildFlavor,
-            String buildType,
-            String buildHash,
-            String buildDate,
-            boolean isSnapshot,
-            String luceneVersion,
-            String minimumWireCompatibilityVersion,
-            String minimumIndexCompatibilityVersion
-        ) {
-            this.number = number;
-            this.buildFlavor = buildFlavor;
-            this.buildType = buildType;
-            this.buildHash = buildHash;
-            this.buildDate = buildDate;
-            this.isSnapshot = isSnapshot;
-            this.luceneVersion = luceneVersion;
-            this.minimumWireCompatibilityVersion = minimumWireCompatibilityVersion;
-            this.minimumIndexCompatibilityVersion = minimumIndexCompatibilityVersion;
-        }
-
-        public String getNumber() {
-            return number;
-        }
-
-        public String getBuildFlavor() {
-            return buildFlavor;
-        }
-
-        public String getBuildType() {
-            return buildType;
-        }
-
-        public String getBuildHash() {
-            return buildHash;
-        }
-
-        public String getBuildDate() {
-            return buildDate;
-        }
-
-        public boolean isSnapshot() {
-            return isSnapshot;
-        }
-
-        public String getLuceneVersion() {
-            return luceneVersion;
-        }
-
-        public String getMinimumWireCompatibilityVersion() {
-            return minimumWireCompatibilityVersion;
-        }
-
-        public String getMinimumIndexCompatibilityVersion() {
-            return minimumIndexCompatibilityVersion;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-            Version version = (Version) o;
-            return isSnapshot == version.isSnapshot
-                && number.equals(version.number)
-                && Objects.equals(buildFlavor, version.buildFlavor)
-                && Objects.equals(buildType, version.buildType)
-                && buildHash.equals(version.buildHash)
-                && buildDate.equals(version.buildDate)
-                && luceneVersion.equals(version.luceneVersion)
-                && minimumWireCompatibilityVersion.equals(version.minimumWireCompatibilityVersion)
-                && minimumIndexCompatibilityVersion.equals(version.minimumIndexCompatibilityVersion);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(
-                number,
-                buildFlavor,
-                buildType,
-                buildHash,
-                buildDate,
-                isSnapshot,
-                luceneVersion,
-                minimumWireCompatibilityVersion,
-                minimumIndexCompatibilityVersion
-            );
-        }
-    }
-}

+ 0 - 23
client/rest-high-level/testnode.crt

@@ -1,23 +0,0 @@
------BEGIN CERTIFICATE-----
-MIID0zCCArugAwIBAgIJALi5bDfjMszLMA0GCSqGSIb3DQEBCwUAMEgxDDAKBgNV
-BAoTA29yZzEWMBQGA1UECxMNZWxhc3RpY3NlYXJjaDEgMB4GA1UEAxMXRWxhc3Rp
-Y3NlYXJjaCBUZXN0IE5vZGUwHhcNMTUwOTIzMTg1MjU3WhcNMTkwOTIyMTg1MjU3
-WjBIMQwwCgYDVQQKEwNvcmcxFjAUBgNVBAsTDWVsYXN0aWNzZWFyY2gxIDAeBgNV
-BAMTF0VsYXN0aWNzZWFyY2ggVGVzdCBOb2RlMIIBIjANBgkqhkiG9w0BAQEFAAOC
-AQ8AMIIBCgKCAQEA3rGZ1QbsW0+MuyrSLmMfDFKtLBkIFW8V0gRuurFg1PUKKNR1
-Mq2tMVwjjYETAU/UY0iKZOzjgvYPKhDTYBTte/WHR1ZK4CYVv7TQX/gtFQG/ge/c
-7u0sLch9p7fbd+/HZiLS/rBEZDIohvgUvzvnA8+OIYnw4kuxKo/5iboAIS41klMg
-/lATm8V71LMY68inht71/ZkQoAHKgcR9z4yNYvQ1WqKG8DG8KROXltll3sTrKbl5
-zJhn660es/1ZnR6nvwt6xnSTl/mNHMjkfv1bs4rJ/py3qPxicdoSIn/KyojUcgHV
-F38fuAy2CQTdjVG5fWj9iz+mQvLm3+qsIYQdFwIDAQABo4G/MIG8MAkGA1UdEwQC
-MAAwHQYDVR0OBBYEFEMMWLWQi/g83PzlHYqAVnty5L7HMIGPBgNVHREEgYcwgYSC
-CWxvY2FsaG9zdIIVbG9jYWxob3N0LmxvY2FsZG9tYWluggpsb2NhbGhvc3Q0ghds
-b2NhbGhvc3Q0LmxvY2FsZG9tYWluNIIKbG9jYWxob3N0NoIXbG9jYWxob3N0Ni5s
-b2NhbGRvbWFpbjaHBH8AAAGHEAAAAAAAAAAAAAAAAAAAAAEwDQYJKoZIhvcNAQEL
-BQADggEBAMjGGXT8Nt1tbl2GkiKtmiuGE2Ej66YuZ37WSJViaRNDVHLlg87TCcHe
-k2rdO+6sFqQbbzEfwQ05T7xGmVu7tm54HwKMRugoQ3wct0bQC5wEWYN+oMDvSyO6
-M28mZwWb4VtR2IRyWP+ve5DHwTM9mxWa6rBlGzsQqH6YkJpZojzqk/mQTug+Y8aE
-mVoqRIPMHq9ob+S9qd5lp09+MtYpwPfTPx/NN+xMEooXWW/ARfpGhWPkg/FuCu4z
-1tFmCqHgNcWirzMm3dQpF78muE9ng6OB2MXQwL4VgnVkxmlZNHbkR2v/t8MyZJxC
-y4g6cTMM3S/UMt5/+aIB2JAuMKyuD+A=
------END CERTIFICATE-----

BIN
client/rest-high-level/testnode.jks