diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java deleted file mode 100644 index 42ec98209a21b..0000000000000 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/ResponseRewriter.java +++ /dev/null @@ -1,122 +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", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.action.fieldcaps; - -import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; -import org.elasticsearch.core.Predicates; - -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; - -/** - * Applies field type filters to field caps responses that come from earlier versions of ES - * that do not support filtering directly. - */ -final class ResponseRewriter { - - public static Map rewriteOldResponses( - TransportVersion version, - Map input, - String[] filters, - String[] allowedTypes - ) { - if (version.onOrAfter(TransportVersions.V_8_2_0)) { - return input; // nothing needs to be done - } - Function transformer = buildTransformer(input, filters, allowedTypes); - 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( - Map input, - String[] filters, - String[] allowedTypes - ) { - Predicate test = Predicates.always(); - Set objects = null; - Set nestedObjects = null; - if (allowedTypes.length > 0) { - Set at = Set.of(allowedTypes); - test = test.and(ifc -> at.contains(ifc.type())); - } - for (String filter : filters) { - if ("-parent".equals(filter)) { - test = test.and(fc -> fc.type().equals("nested") == false && fc.type().equals("object") == false); - } - if ("-metadata".equals(filter)) { - test = test.and(fc -> fc.isMetadatafield() == false); - } - if ("+metadata".equals(filter)) { - test = test.and(IndexFieldCapabilities::isMetadatafield); - } - if ("-nested".equals(filter)) { - if (nestedObjects == null) { - nestedObjects = findTypes("nested", input); - } - Set no = nestedObjects; - test = test.and(fc -> isNestedField(fc.name(), 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.name(), o)); - } - } - Predicate finalTest = test; - return fc -> { - if (finalTest.test(fc) == false) { - return null; - } - return fc; - }; - } - - private static Set findTypes(String type, Map fieldCaps) { - return fieldCaps.entrySet() - .stream() - .filter(entry -> type.equals(entry.getValue().type())) - .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 4ae5e01569333..74201f106f7d6 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -469,7 +469,7 @@ private static FieldCapabilitiesResponse merge( } else { subIndices = ArrayUtil.copyOfSubArray(indices, lastPendingIndex, i); } - innerMerge(subIndices, fieldsBuilder, request, indexResponses[lastPendingIndex]); + innerMerge(subIndices, fieldsBuilder, indexResponses[lastPendingIndex]); lastPendingIndex = i; } } @@ -589,15 +589,9 @@ private static String nextIndex(Iterator iter, Set filtered) { private static void innerMerge( String[] indices, Map> responseMapBuilder, - FieldCapabilitiesRequest request, FieldCapabilitiesIndexResponse response ) { - Map fields = ResponseRewriter.rewriteOldResponses( - response.getOriginVersion(), - response.get(), - request.filters(), - request.types() - ); + Map fields = response.get(); for (Map.Entry entry : fields.entrySet()) { final String field = entry.getKey(); final IndexFieldCapabilities fieldCap = entry.getValue(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java index a73e9635a53b3..a364764f8d028 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java @@ -9,7 +9,7 @@ package org.elasticsearch.action.admin.cluster.reroute; -import org.elasticsearch.TransportVersions; +import org.elasticsearch.TransportVersion; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -132,7 +132,7 @@ public void testToXContentWithDeprecatedClusterState() { "nodes_versions": [ { "node_id": "node0", - "transport_version": "8000099", + "transport_version": "%s", "mappings_versions": { ".system-index": { "version": 1, @@ -218,6 +218,7 @@ public void testToXContentWithDeprecatedClusterState() { Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current(), + TransportVersion.current(), IndexVersion.current(), IndexVersion.current() ), @@ -333,7 +334,7 @@ private static ClusterState createClusterState() { .nodes(new DiscoveryNodes.Builder().add(node0).masterNodeId(node0.getId()).build()) .putCompatibilityVersions( node0.getId(), - TransportVersions.V_8_0_0, + TransportVersion.current(), Map.of(".system-index", new SystemIndexDescriptor.MappingsVersion(1, 0)) ) .metadata( diff --git a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java b/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java deleted file mode 100644 index 8ed87475f2856..0000000000000 --- a/server/src/test/java/org/elasticsearch/action/fieldcaps/ResponseRewriterTests.java +++ /dev/null @@ -1,152 +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", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.action.fieldcaps; - -import org.elasticsearch.TransportVersions; -import org.elasticsearch.common.Strings; -import org.elasticsearch.test.ESTestCase; - -import java.util.Map; - -public class ResponseRewriterTests extends ESTestCase { - - public void testExcludeMetadata() { - Map oldResponse = Map.of( - "field", - fieldCaps("field", "keyword", false), - "_index", - fieldCaps("_index", "_index", true) - ); - - Map rewritten = ResponseRewriter.rewriteOldResponses( - TransportVersions.V_8_0_0, - oldResponse, - new String[] { "-metadata" }, - Strings.EMPTY_ARRAY - ); - - assertTrue(rewritten.containsKey("field")); - assertFalse(rewritten.containsKey("_index")); - } - - public void testIncludeOnlyMetadata() { - Map oldResponse = Map.of( - "field", - fieldCaps("field", "keyword", false), - "_index", - fieldCaps("_index", "_index", true) - ); - - Map rewritten = ResponseRewriter.rewriteOldResponses( - TransportVersions.V_8_0_0, - oldResponse, - new String[] { "+metadata" }, - Strings.EMPTY_ARRAY - ); - - assertFalse(rewritten.containsKey("field")); - assertTrue(rewritten.containsKey("_index")); - } - - public void testExcludeNested() { - Map oldResponse = Map.of( - "field", - fieldCaps("field", "keyword", false), - "parent", - fieldCaps("parent", "nested", false), - "parent.child", - fieldCaps("parent.child", "keyword", false) - ); - - Map rewritten = ResponseRewriter.rewriteOldResponses( - TransportVersions.V_8_0_0, - oldResponse, - new String[] { "-nested" }, - Strings.EMPTY_ARRAY - ); - - 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( - TransportVersions.V_8_0_0, - oldResponse, - new String[] { "-multifield" }, - Strings.EMPTY_ARRAY - ); - - assertTrue(rewritten.containsKey("field")); - assertFalse(rewritten.containsKey("field.keyword")); - assertTrue(rewritten.containsKey("parent.child")); - } - - 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( - TransportVersions.V_8_0_0, - oldResponse, - new String[] { "-parent" }, - Strings.EMPTY_ARRAY - ); - - assertTrue(rewritten.containsKey("field")); - assertFalse(rewritten.containsKey("parent")); - 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( - TransportVersions.V_8_0_0, - oldResponse, - Strings.EMPTY_ARRAY, - new String[] { "text", "keyword" } - ); - - assertTrue(rewritten.containsKey("text")); - assertTrue(rewritten.containsKey("keyword")); - assertFalse(rewritten.containsKey("long")); - } - - private static IndexFieldCapabilities fieldCaps(String name, String type, boolean isMetadata) { - return new IndexFieldCapabilitiesBuilder(name, type).isMetadataField(isMetadata).build(); - } - -} diff --git a/server/src/test/java/org/elasticsearch/action/search/OpenPointInTimeRequestTests.java b/server/src/test/java/org/elasticsearch/action/search/OpenPointInTimeRequestTests.java index fa89a5c71e3a3..33ed1944fb978 100644 --- a/server/src/test/java/org/elasticsearch/action/search/OpenPointInTimeRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/OpenPointInTimeRequestTests.java @@ -9,19 +9,11 @@ package org.elasticsearch.action.search; -import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.test.AbstractWireSerializingTestCase; -import org.elasticsearch.test.TransportVersionUtils; import java.io.IOException; -import java.util.List; import static org.hamcrest.Matchers.equalTo; @@ -95,19 +87,4 @@ protected OpenPointInTimeRequest mutateInstance(OpenPointInTimeRequest in) throw default -> throw new AssertionError("Unknown option"); }; } - - public void testUseDefaultConcurrentForOldVersion() throws Exception { - TransportVersion previousVersion = TransportVersionUtils.getPreviousVersion(TransportVersions.V_8_9_X); - try (BytesStreamOutput output = new BytesStreamOutput()) { - TransportVersion version = TransportVersionUtils.randomVersionBetween(random(), TransportVersions.V_8_0_0, previousVersion); - output.setTransportVersion(version); - OpenPointInTimeRequest original = createTestInstance(); - original.writeTo(output); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), new NamedWriteableRegistry(List.of()))) { - in.setTransportVersion(version); - OpenPointInTimeRequest copy = new OpenPointInTimeRequest(in); - assertThat(copy.maxConcurrentShardRequests(), equalTo(5)); - } - } - } } diff --git a/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java b/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java index da7876b63db61..6cf88644f9765 100644 --- a/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java +++ b/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java @@ -11,7 +11,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.cluster.Diff; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; @@ -620,7 +619,7 @@ public void testMissingValue() throws Exception { public void testReadWriteArray() throws IOException { BytesStreamOutput output = new BytesStreamOutput(); - output.setTransportVersion(randomFrom(TransportVersion.current(), TransportVersions.V_8_0_0)); + output.setTransportVersion(TransportVersion.current()); Settings settings = Settings.builder().putList("foo.bar", "0", "1", "2", "3").put("foo.bar.baz", "baz").build(); settings.writeTo(output); StreamInput in = StreamInput.wrap(BytesReference.toBytes(output.bytes())); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/TypeParsersTests.java b/server/src/test/java/org/elasticsearch/index/mapper/TypeParsersTests.java index 5b0f823ac1e17..e3c3fb2cdcddf 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/TypeParsersTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/TypeParsersTests.java @@ -10,7 +10,6 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; @@ -88,11 +87,7 @@ public void testMultiFieldWithinMultiField() throws IOException { Map fieldNodeCopy = XContentHelper.convertToMap(BytesReference.bytes(mapping), true, mapping.contentType()).v2(); IndexVersion version = IndexVersionUtils.randomVersionBetween(random(), IndexVersions.V_8_0_0, IndexVersion.current()); - TransportVersion transportVersion = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersion.current() - ); + TransportVersion transportVersion = TransportVersionUtils.randomCompatibleVersion(random()); MappingParserContext context = new MappingParserContext( null, type -> typeParser, diff --git a/server/src/test/java/org/elasticsearch/search/vectors/AbstractKnnVectorQueryBuilderTestCase.java b/server/src/test/java/org/elasticsearch/search/vectors/AbstractKnnVectorQueryBuilderTestCase.java index 97cbc435bf251..0306d694cdc68 100644 --- a/server/src/test/java/org/elasticsearch/search/vectors/AbstractKnnVectorQueryBuilderTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/vectors/AbstractKnnVectorQueryBuilderTestCase.java @@ -413,26 +413,6 @@ public void testMustRewrite() throws IOException { assertThat(rewrittenQuery, instanceOf(MatchNoneQueryBuilder.class)); } - public void testBWCVersionSerializationFilters() throws IOException { - KnnVectorQueryBuilder query = createTestQueryBuilder(); - VectorData vectorData = VectorData.fromFloats(query.queryVector().asFloatVector()); - KnnVectorQueryBuilder queryNoFilters = new KnnVectorQueryBuilder( - query.getFieldName(), - vectorData, - null, - query.numCands(), - null, - null, - null - ).queryName(query.queryName()).boost(query.boost()); - TransportVersion beforeFilterVersion = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersions.V_8_1_0 - ); - assertBWCSerialization(query, queryNoFilters, beforeFilterVersion); - } - public void testBWCVersionSerializationSimilarity() throws IOException { KnnVectorQueryBuilder query = createTestQueryBuilder(); VectorData vectorData = VectorData.fromFloats(query.queryVector().asFloatVector()); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/XPackInfoRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/XPackInfoRequestTests.java index 4aea054c83f2d..baea261a7c6be 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/XPackInfoRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/protocol/xpack/XPackInfoRequestTests.java @@ -8,11 +8,9 @@ package org.elasticsearch.protocol.xpack; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.protocol.xpack.XPackInfoRequest.Category; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.TransportVersionUtils; import java.util.EnumSet; import java.util.List; @@ -25,16 +23,6 @@ public void testSerializeToCurrentVersion() throws Exception { assertSerialization(TransportVersion.current()); } - public void testSerializeUsing7xVersion() throws Exception { - assertSerialization( - TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_7_8_1, - TransportVersionUtils.getPreviousVersion(TransportVersions.V_8_0_0) - ) - ); - } - private void assertSerialization(TransportVersion version) throws java.io.IOException { final XPackInfoRequest request = new XPackInfoRequest(); final List categories = randomSubsetOf(List.of(Category.values())); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetTrainedModelsStatsActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetTrainedModelsStatsActionResponseTests.java index 1f58d2522a150..17b52f0e72633 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetTrainedModelsStatsActionResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/GetTrainedModelsStatsActionResponseTests.java @@ -7,16 +7,13 @@ package org.elasticsearch.xpack.core.ml.action; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.ingest.IngestStats; import org.elasticsearch.xpack.core.action.util.QueryPage; import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase; import org.elasticsearch.xpack.core.ml.action.GetTrainedModelsStatsAction.Response; -import org.elasticsearch.xpack.core.ml.inference.assignment.AssignmentStats; import org.elasticsearch.xpack.core.ml.inference.assignment.AssignmentStatsTests; -import org.elasticsearch.xpack.core.ml.inference.assignment.Priority; import org.elasticsearch.xpack.core.ml.inference.trainedmodel.InferenceStatsTests; import org.elasticsearch.xpack.core.ml.inference.trainedmodel.TrainedModelSizeStatsTests; @@ -96,564 +93,6 @@ protected Writeable.Reader instanceReader() { @Override protected Response mutateInstanceForVersion(Response instance, TransportVersion version) { - if (version.before(TransportVersions.V_8_0_0)) { - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - null, - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - null - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_1_0)) { - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - null, - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - null, - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - 0, - null, - 0, - 0, - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - null, - null, - 0L, - 0L, - null, - null - ) - ) - .toList(), - Priority.NORMAL - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_2_0)) { - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - null, - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - null, - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - nodeStats.getErrorCount(), - null, - nodeStats.getRejectedExecutionCount(), - nodeStats.getTimeoutCount(), - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - nodeStats.getThreadsPerAllocation(), - nodeStats.getNumberOfAllocations(), - 0L, - 0L, - null, - null - ) - ) - .toList(), - Priority.NORMAL - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_4_0)) { - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - null, - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - null, - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - nodeStats.getErrorCount(), - null, - nodeStats.getRejectedExecutionCount(), - nodeStats.getTimeoutCount(), - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - nodeStats.getThreadsPerAllocation(), - nodeStats.getNumberOfAllocations(), - nodeStats.getPeakThroughput(), - nodeStats.getThroughputLastPeriod(), - nodeStats.getAvgInferenceTimeLastPeriod(), - null - ) - ) - .toList(), - Priority.NORMAL - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_5_0)) { - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - stats.getDeploymentStats().getCacheSize(), - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - null, - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - nodeStats.getErrorCount(), - nodeStats.getCacheHitCount().orElse(null), - nodeStats.getRejectedExecutionCount(), - nodeStats.getTimeoutCount(), - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - nodeStats.getThreadsPerAllocation(), - nodeStats.getNumberOfAllocations(), - nodeStats.getPeakThroughput(), - nodeStats.getThroughputLastPeriod(), - nodeStats.getAvgInferenceTimeLastPeriod(), - nodeStats.getCacheHitCountLastPeriod().orElse(null) - ) - ) - .toList(), - Priority.NORMAL - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_6_0)) { - // priority added - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - stats.getDeploymentStats().getCacheSize(), - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - nodeStats.getAvgInferenceTimeExcludingCacheHit().orElse(null), - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - nodeStats.getErrorCount(), - nodeStats.getCacheHitCount().orElse(null), - nodeStats.getRejectedExecutionCount(), - nodeStats.getTimeoutCount(), - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - nodeStats.getThreadsPerAllocation(), - nodeStats.getNumberOfAllocations(), - nodeStats.getPeakThroughput(), - nodeStats.getThroughputLastPeriod(), - nodeStats.getAvgInferenceTimeLastPeriod(), - nodeStats.getCacheHitCountLastPeriod().orElse(null) - ) - ) - .toList(), - Priority.NORMAL - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_8_0)) { - // deployment_id added - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - stats.getDeploymentStats().getCacheSize(), - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - nodeStats.getAvgInferenceTimeExcludingCacheHit().orElse(null), - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - nodeStats.getErrorCount(), - nodeStats.getCacheHitCount().orElse(null), - nodeStats.getRejectedExecutionCount(), - nodeStats.getTimeoutCount(), - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - nodeStats.getThreadsPerAllocation(), - nodeStats.getNumberOfAllocations(), - nodeStats.getPeakThroughput(), - nodeStats.getThroughputLastPeriod(), - nodeStats.getAvgInferenceTimeLastPeriod(), - nodeStats.getCacheHitCountLastPeriod().orElse(null) - ) - ) - .toList(), - stats.getDeploymentStats().getPriority() - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } else if (version.before(TransportVersions.V_8_15_0)) { - // added ByteStats to IngestStats.PipelineStat - return new Response( - new QueryPage<>( - instance.getResources() - .results() - .stream() - .map( - stats -> new Response.TrainedModelStats( - stats.getModelId(), - stats.getModelSizeStats(), - new IngestStats( - stats.getIngestStats().totalStats(), - stats.getIngestStats() - .pipelineStats() - .stream() - .map( - pipelineStat -> new IngestStats.PipelineStat( - ProjectId.DEFAULT, - pipelineStat.pipelineId(), - pipelineStat.stats(), - IngestStats.ByteStats.IDENTITY - ) - ) - .toList(), - stats.getIngestStats().processorStats() - ), - stats.getPipelineCount(), - stats.getInferenceStats(), - stats.getDeploymentStats() == null - ? null - : new AssignmentStats( - stats.getDeploymentStats().getDeploymentId(), - stats.getDeploymentStats().getModelId(), - stats.getDeploymentStats().getThreadsPerAllocation(), - stats.getDeploymentStats().getNumberOfAllocations(), - null, - stats.getDeploymentStats().getQueueCapacity(), - stats.getDeploymentStats().getCacheSize(), - stats.getDeploymentStats().getStartTime(), - stats.getDeploymentStats() - .getNodeStats() - .stream() - .map( - nodeStats -> new AssignmentStats.NodeStats( - nodeStats.getNode(), - nodeStats.getInferenceCount().orElse(null), - nodeStats.getAvgInferenceTime().orElse(null), - nodeStats.getAvgInferenceTimeExcludingCacheHit().orElse(null), - nodeStats.getLastAccess(), - nodeStats.getPendingCount(), - nodeStats.getErrorCount(), - nodeStats.getCacheHitCount().orElse(null), - nodeStats.getRejectedExecutionCount(), - nodeStats.getTimeoutCount(), - nodeStats.getRoutingState(), - nodeStats.getStartTime(), - nodeStats.getThreadsPerAllocation(), - nodeStats.getNumberOfAllocations(), - nodeStats.getPeakThroughput(), - nodeStats.getThroughputLastPeriod(), - nodeStats.getAvgInferenceTimeLastPeriod(), - nodeStats.getCacheHitCountLastPeriod().orElse(null) - ) - ) - .toList(), - stats.getDeploymentStats().getPriority() - ) - ) - ) - .toList(), - instance.getResources().count(), - RESULTS_FIELD - ) - ); - } return instance; } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderWireSerializationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderWireSerializationTests.java index 6e3aad0c8bc11..d2257c97704c1 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderWireSerializationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderWireSerializationTests.java @@ -8,7 +8,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; @@ -74,9 +73,6 @@ public static AggProvider createRandomValidAggProvider() { @Override protected AggProvider mutateInstanceForVersion(AggProvider instance, TransportVersion version) { - if (version.before(TransportVersions.V_8_0_0)) { - return new AggProvider(instance.getAggs(), instance.getParsedAggs(), instance.getParsingException(), false); - } return instance; } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponseTests.java index 73e12fe2625c3..db90ee6601fa0 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponseTests.java @@ -44,7 +44,6 @@ import static java.util.Collections.emptySet; import static org.elasticsearch.xpack.core.security.authz.permission.RemoteClusterPermissions.ROLE_REMOTE_CLUSTER_PRIVS; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; public class GetUserPrivilegesResponseTests extends ESTestCase { @@ -85,41 +84,6 @@ public void testSerializationForCurrentVersion() throws Exception { assertThat(copy, equalTo(original)); } - public void testSerializationWithRemoteIndicesThrowsOnUnsupportedVersions() throws IOException { - final BytesStreamOutput out = new BytesStreamOutput(); - final TransportVersion versionBeforeAdvancedRemoteClusterSecurity = TransportVersionUtils.getPreviousVersion( - TransportVersions.V_8_8_0 - ); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeAdvancedRemoteClusterSecurity - ); - out.setTransportVersion(version); - - final GetUserPrivilegesResponse original = randomResponse(true, false); - if (original.hasRemoteIndicesPrivileges()) { - final var ex = expectThrows(IllegalArgumentException.class, () -> original.writeTo(out)); - assertThat( - ex.getMessage(), - containsString( - "versions of Elasticsearch before [" - + TransportVersions.V_8_8_0.toReleaseVersion() - + "] can't handle remote indices privileges and attempted to send to [" - + version.toReleaseVersion() - + "]" - ) - ); - } else { - original.writeTo(out); - final NamedWriteableRegistry registry = new NamedWriteableRegistry(new XPackClientPlugin().getNamedWriteables()); - StreamInput in = new NamedWriteableAwareStreamInput(ByteBufferStreamInput.wrap(BytesReference.toBytes(out.bytes())), registry); - in.setTransportVersion(out.getTransportVersion()); - final GetUserPrivilegesResponse copy = new GetUserPrivilegesResponse(in); - assertThat(copy, equalTo(original)); - } - } - public void testEqualsAndHashCode() throws IOException { final GetUserPrivilegesResponse response = randomResponse(); final EqualsHashCodeTestUtils.CopyFunction copy = original -> new GetUserPrivilegesResponse( diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationSerializationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationSerializationTests.java index 07bc8489fa500..f5b61a203de3a 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationSerializationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationSerializationTests.java @@ -7,12 +7,9 @@ package org.elasticsearch.xpack.core.security.authc; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.TransportVersionUtils; -import org.elasticsearch.transport.RemoteClusterPortSettings; import org.elasticsearch.xpack.core.security.user.ElasticUser; import org.elasticsearch.xpack.core.security.user.InternalUsers; import org.elasticsearch.xpack.core.security.user.KibanaSystemUser; @@ -81,40 +78,6 @@ public void testWriteToAndReadFromWithCrossClusterAccess() throws Exception { assertThat(readFrom, equalTo(authentication)); } - public void testWriteToWithCrossClusterAccessThrowsOnUnsupportedVersion() throws Exception { - final Authentication authentication = randomBoolean() - ? AuthenticationTestHelper.builder().crossClusterAccess().build() - : AuthenticationTestHelper.builder().build(); - - final BytesStreamOutput out = new BytesStreamOutput(); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersionUtils.getPreviousVersion(RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY) - ); - out.setTransportVersion(version); - - if (authentication.isCrossClusterAccess()) { - final var ex = expectThrows(IllegalArgumentException.class, () -> authentication.writeTo(out)); - assertThat( - ex.getMessage(), - containsString( - "versions of Elasticsearch before [" - + RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY.toReleaseVersion() - + "] can't handle cross cluster access authentication and attempted to send to [" - + out.getTransportVersion().toReleaseVersion() - + "]" - ) - ); - } else { - authentication.writeTo(out); - final StreamInput in = out.bytes().streamInput(); - in.setTransportVersion(out.getTransportVersion()); - final Authentication readFrom = new Authentication(in); - assertThat(readFrom, equalTo(authentication.maybeRewriteForOlderVersion(out.getTransportVersion()))); - } - } - public void testWriteToAndReadFromWithCloudApiKeyAuthentication() throws Exception { final Authentication authentication = Authentication.newCloudAuthentication( Authentication.AuthenticationType.API_KEY, @@ -147,7 +110,7 @@ public void testWriteToWithCloudApiKeyThrowsOnUnsupportedVersion() { try (BytesStreamOutput out = new BytesStreamOutput()) { final TransportVersion version = TransportVersionUtils.randomVersionBetween( random(), - TransportVersions.V_8_0_0, + TransportVersion.minimumCompatible(), TransportVersionUtils.getPreviousVersion(SECURITY_CLOUD_API_KEY_REALM_AND_TYPE) ); out.setTransportVersion(version); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationTests.java index 11815cc3a3683..025016c7c3710 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authc/AuthenticationTests.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.core.security.authc; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -51,7 +50,6 @@ import static org.elasticsearch.xpack.core.security.authz.permission.RemoteClusterPermissions.ROLE_MONITOR_STATS; import static org.elasticsearch.xpack.core.security.authz.permission.RemoteClusterPermissions.ROLE_REMOTE_CLUSTER_PRIVS; import static org.hamcrest.Matchers.anEmptyMap; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.hasKey; @@ -677,13 +675,12 @@ public void testDomainSerialize() throws Exception { } try (BytesStreamOutput out = new BytesStreamOutput()) { - out.setTransportVersion(TransportVersions.V_8_0_0); + out.setTransportVersion(TransportVersion.minimumCompatible()); test.writeTo(out); StreamInput in = out.bytes().streamInput(); - in.setTransportVersion(TransportVersions.V_8_0_0); + in.setTransportVersion(TransportVersion.minimumCompatible()); Authentication testBack = new Authentication(in); - assertThat(testBack.getDomain(), nullValue()); - assertThat(testBack.isAssignedToDomain(), is(false)); + assertThat(testBack.getDomain(), equalTo(test.getDomain())); } } @@ -865,38 +862,6 @@ public void testBwcWithStoredAuthenticationHeaders() throws IOException { assertThat(authenticationV7.encode(), equalTo(headerV7)); } - public void testMaybeRewriteForOlderVersionWithCrossClusterAccessThrowsOnUnsupportedVersion() { - final Authentication authentication = randomBoolean() - ? AuthenticationTestHelper.builder().crossClusterAccess().build() - : AuthenticationTestHelper.builder().build(); - - final TransportVersion versionBeforeCrossClusterAccessRealm = TransportVersionUtils.getPreviousVersion( - RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY - ); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeCrossClusterAccessRealm - ); - - if (authentication.isCrossClusterAccess()) { - final var ex = expectThrows(IllegalArgumentException.class, () -> authentication.maybeRewriteForOlderVersion(version)); - assertThat( - ex.getMessage(), - containsString( - "versions of Elasticsearch before [" - + RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY.toReleaseVersion() - + "] can't handle cross cluster access authentication and attempted to rewrite for [" - + version.toReleaseVersion() - + "]" - ) - ); - } else { - // Assert that rewriting took place; the details of rewriting logic are checked in other tests - assertThat(authentication.maybeRewriteForOlderVersion(version), not(equalTo(authentication))); - } - } - public void testMaybeRewriteForOlderVersionWithCrossClusterAccessRewritesAuthenticationInMetadata() throws IOException { final TransportVersion crossClusterAccessRealmVersion = RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY; @@ -977,25 +942,6 @@ public void testCopyWithFilteredMetadataFields() { assertThat(authentication.copyWithEmptyMetadata().getAuthenticatingSubject().getMetadata(), is(anEmptyMap())); } - public void testMaybeRewriteForOlderVersionErasesDomainForVersionsBeforeDomains() { - final TransportVersion olderVersion = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersionUtils.getPreviousVersion(Authentication.VERSION_REALM_DOMAINS) - ); - final Authentication authentication = AuthenticationTestHelper.builder() - .realm() // randomize to test both when realm is null on the original auth and non-null, instead of setting `underDomain` - .transportVersion(TransportVersionUtils.randomVersionBetween(random(), Authentication.VERSION_REALM_DOMAINS, null)) - .build(); - assertThat(authentication.getEffectiveSubject().getTransportVersion().after(olderVersion), is(true)); - - final Authentication actual = authentication.maybeRewriteForOlderVersion(olderVersion); - - assertThat(actual.getEffectiveSubject().getTransportVersion(), equalTo(olderVersion)); - assertThat(actual.getAuthenticatingSubject().getRealm().getDomain(), nullValue()); - assertThat(actual.getEffectiveSubject().getRealm().getDomain(), nullValue()); - } - public void testMaybeRewriteForOlderVersionDoesNotEraseDomainForVersionsAfterDomains() { final TransportVersion olderVersion = TransportVersionUtils.randomVersionBetween( random(), diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptorTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptorTests.java index 1bd0206beccf7..133f9cc8f06f0 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptorTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/RoleDescriptorTests.java @@ -647,158 +647,6 @@ public void testSerializationForCurrentVersion() throws Exception { assertThat(serialized, equalTo(descriptor)); } - public void testSerializationWithRemoteIndicesWithElderVersion() throws IOException { - final TransportVersion versionBeforeRemoteIndices = TransportVersionUtils.getPreviousVersion(TransportVersions.V_8_8_0); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeRemoteIndices - ); - final BytesStreamOutput output = new BytesStreamOutput(); - output.setTransportVersion(version); - - final RoleDescriptor descriptor = RoleDescriptorTestHelper.builder() - .allowReservedMetadata(true) - .allowRemoteIndices(true) - .allowRestriction(false) - .allowDescription(false) - .allowRemoteClusters(false) - .build(); - - descriptor.writeTo(output); - final NamedWriteableRegistry registry = new NamedWriteableRegistry(new XPackClientPlugin().getNamedWriteables()); - StreamInput streamInput = new NamedWriteableAwareStreamInput( - ByteBufferStreamInput.wrap(BytesReference.toBytes(output.bytes())), - registry - ); - streamInput.setTransportVersion(version); - final RoleDescriptor serialized = new RoleDescriptor(streamInput); - if (descriptor.hasRemoteIndicesPrivileges()) { - assertThat( - serialized, - equalTo( - new RoleDescriptor( - descriptor.getName(), - descriptor.getClusterPrivileges(), - descriptor.getIndicesPrivileges(), - descriptor.getApplicationPrivileges(), - descriptor.getConditionalClusterPrivileges(), - descriptor.getRunAs(), - descriptor.getMetadata(), - descriptor.getTransientMetadata(), - null, - null, - descriptor.getRestriction(), - descriptor.getDescription() - ) - ) - ); - } else { - assertThat(descriptor, equalTo(serialized)); - } - } - - public void testSerializationWithRemoteClusterWithElderVersion() throws IOException { - final TransportVersion versionBeforeRemoteCluster = TransportVersionUtils.getPreviousVersion(ROLE_REMOTE_CLUSTER_PRIVS); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeRemoteCluster - ); - final BytesStreamOutput output = new BytesStreamOutput(); - output.setTransportVersion(version); - - final RoleDescriptor descriptor = RoleDescriptorTestHelper.builder() - .allowReservedMetadata(true) - .allowRemoteIndices(false) - .allowRestriction(false) - .allowDescription(false) - .allowRemoteClusters(true) - .build(); - descriptor.writeTo(output); - final NamedWriteableRegistry registry = new NamedWriteableRegistry(new XPackClientPlugin().getNamedWriteables()); - StreamInput streamInput = new NamedWriteableAwareStreamInput( - ByteBufferStreamInput.wrap(BytesReference.toBytes(output.bytes())), - registry - ); - streamInput.setTransportVersion(version); - final RoleDescriptor serialized = new RoleDescriptor(streamInput); - if (descriptor.hasRemoteClusterPermissions()) { - assertThat( - serialized, - equalTo( - new RoleDescriptor( - descriptor.getName(), - descriptor.getClusterPrivileges(), - descriptor.getIndicesPrivileges(), - descriptor.getApplicationPrivileges(), - descriptor.getConditionalClusterPrivileges(), - descriptor.getRunAs(), - descriptor.getMetadata(), - descriptor.getTransientMetadata(), - descriptor.getRemoteIndicesPrivileges(), - null, - descriptor.getRestriction(), - descriptor.getDescription() - ) - ) - ); - } else { - assertThat(descriptor, equalTo(serialized)); - assertThat(descriptor.getRemoteClusterPermissions(), equalTo(RemoteClusterPermissions.NONE)); - } - } - - public void testSerializationWithWorkflowsRestrictionAndUnsupportedVersions() throws IOException { - final TransportVersion versionBeforeWorkflowsRestriction = TransportVersionUtils.getPreviousVersion(WORKFLOWS_RESTRICTION_VERSION); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeWorkflowsRestriction - ); - final BytesStreamOutput output = new BytesStreamOutput(); - output.setTransportVersion(version); - - final RoleDescriptor descriptor = RoleDescriptorTestHelper.builder() - .allowReservedMetadata(true) - .allowRemoteIndices(false) - .allowRestriction(true) - .allowDescription(false) - .allowRemoteClusters(false) - .build(); - descriptor.writeTo(output); - final NamedWriteableRegistry registry = new NamedWriteableRegistry(new XPackClientPlugin().getNamedWriteables()); - StreamInput streamInput = new NamedWriteableAwareStreamInput( - ByteBufferStreamInput.wrap(BytesReference.toBytes(output.bytes())), - registry - ); - streamInput.setTransportVersion(version); - final RoleDescriptor serialized = new RoleDescriptor(streamInput); - if (descriptor.hasWorkflowsRestriction()) { - assertThat( - serialized, - equalTo( - new RoleDescriptor( - descriptor.getName(), - descriptor.getClusterPrivileges(), - descriptor.getIndicesPrivileges(), - descriptor.getApplicationPrivileges(), - descriptor.getConditionalClusterPrivileges(), - descriptor.getRunAs(), - descriptor.getMetadata(), - descriptor.getTransientMetadata(), - descriptor.getRemoteIndicesPrivileges(), - descriptor.getRemoteClusterPermissions(), - null, - descriptor.getDescription() - ) - ) - ); - } else { - assertThat(descriptor, equalTo(serialized)); - } - } - public void testParseRoleWithRestrictionFailsWhenAllowRestrictionIsFalse() { final String json = """ { @@ -844,50 +692,6 @@ public void testParseRoleWithRestrictionWhenAllowRestrictionIsTrue() throws IOEx assertThat(role.getRestriction().getWorkflows(), arrayContaining("search_application")); } - public void testSerializationWithDescriptionAndUnsupportedVersions() throws IOException { - final TransportVersion versionBeforeRoleDescription = TransportVersionUtils.getPreviousVersion(SECURITY_ROLE_DESCRIPTION); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeRoleDescription - ); - final BytesStreamOutput output = new BytesStreamOutput(); - output.setTransportVersion(version); - - final RoleDescriptor descriptor = RoleDescriptorTestHelper.builder().allowDescription(true).build(); - descriptor.writeTo(output); - final NamedWriteableRegistry registry = new NamedWriteableRegistry(new XPackClientPlugin().getNamedWriteables()); - StreamInput streamInput = new NamedWriteableAwareStreamInput( - ByteBufferStreamInput.wrap(BytesReference.toBytes(output.bytes())), - registry - ); - streamInput.setTransportVersion(version); - final RoleDescriptor serialized = new RoleDescriptor(streamInput); - if (descriptor.hasDescription()) { - assertThat( - serialized, - equalTo( - new RoleDescriptor( - descriptor.getName(), - descriptor.getClusterPrivileges(), - descriptor.getIndicesPrivileges(), - descriptor.getApplicationPrivileges(), - descriptor.getConditionalClusterPrivileges(), - descriptor.getRunAs(), - descriptor.getMetadata(), - descriptor.getTransientMetadata(), - descriptor.getRemoteIndicesPrivileges(), - descriptor.getRemoteClusterPermissions(), - descriptor.getRestriction(), - null - ) - ) - ); - } else { - assertThat(descriptor, equalTo(serialized)); - } - } - public void testParseRoleWithDescriptionFailsWhenAllowDescriptionIsFalse() { final String json = """ { diff --git a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/DeleteSamlServiceProviderRequestTests.java b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/DeleteSamlServiceProviderRequestTests.java index 110605b6a6de5..4757b463ca0e6 100644 --- a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/DeleteSamlServiceProviderRequestTests.java +++ b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/DeleteSamlServiceProviderRequestTests.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.idp.action; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.test.TransportVersionUtils; @@ -27,11 +26,7 @@ public void testSerialization() throws IOException { randomAlphaOfLengthBetween(1, 100), randomFrom(WriteRequest.RefreshPolicy.values()) ); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersion.current() - ); + final TransportVersion version = TransportVersionUtils.randomCompatibleVersion(random()); final DeleteSamlServiceProviderRequest read = copyWriteable( request, new NamedWriteableRegistry(List.of()), diff --git a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/PutSamlServiceProviderRequestTests.java b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/PutSamlServiceProviderRequestTests.java index c9b1008ae795a..7c39a48c2223e 100644 --- a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/PutSamlServiceProviderRequestTests.java +++ b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/action/PutSamlServiceProviderRequestTests.java @@ -9,7 +9,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.common.Strings; @@ -85,11 +84,7 @@ public void testValidateRequiredFields() { public void testSerialization() throws IOException { final SamlServiceProviderDocument doc = SamlServiceProviderTestUtils.randomDocument(); final PutSamlServiceProviderRequest request = new PutSamlServiceProviderRequest(doc, RefreshPolicy.NONE); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersion.current() - ); + final TransportVersion version = TransportVersionUtils.randomCompatibleVersion(random()); final PutSamlServiceProviderRequest read = copyWriteable( request, new NamedWriteableRegistry(List.of()), diff --git a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/sp/SamlServiceProviderDocumentTests.java b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/sp/SamlServiceProviderDocumentTests.java index a7e9ca438b33c..f9ac39a8748d2 100644 --- a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/sp/SamlServiceProviderDocumentTests.java +++ b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/sp/SamlServiceProviderDocumentTests.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.idp.saml.sp; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -109,7 +108,7 @@ public void testSerializationBeforeExtensionAttributes() throws Exception { ) : TransportVersionUtils.randomVersionBetween( random(), - TransportVersions.V_8_0_0, + TransportVersion.minimumCompatible(), TransportVersionUtils.getPreviousVersion(IDP_CUSTOM_SAML_ATTRIBUTES_ALLOW_LIST_PATCH) ); final SamlServiceProviderDocument copy = copyWriteable( diff --git a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/support/SamlAuthenticationStateTests.java b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/support/SamlAuthenticationStateTests.java index 61018fe77d47e..21087c42a0c30 100644 --- a/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/support/SamlAuthenticationStateTests.java +++ b/x-pack/plugin/identity-provider/src/test/java/org/elasticsearch/xpack/idp/saml/support/SamlAuthenticationStateTests.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.idp.saml.support; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.xcontent.XContentHelper; @@ -88,11 +87,7 @@ private SamlAuthenticationState assertXContentRoundTrip(SamlAuthenticationState } private SamlAuthenticationState assertSerializationRoundTrip(SamlAuthenticationState state) throws IOException { - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - TransportVersion.current() - ); + final TransportVersion version = TransportVersionUtils.randomCompatibleVersion(random()); final SamlAuthenticationState read = copyWriteable( state, new NamedWriteableRegistry(List.of()), diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityContextTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityContextTests.java index db1d2051a6806..6910df95e6c6c 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityContextTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityContextTests.java @@ -8,7 +8,6 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -38,7 +37,6 @@ import java.io.EOFException; import java.io.IOException; import java.io.UncheckedIOException; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; @@ -193,43 +191,6 @@ public void testExecuteAfterRewritingAuthentication() throws IOException { assertEquals(original, securityContext.getAuthentication()); } - public void testExecuteAfterRewritingAuthenticationWillConditionallyRewriteNewApiKeyMetadata() throws IOException { - final Map metadata = new HashMap<>(); - metadata.put(AuthenticationField.API_KEY_ID_KEY, randomAlphaOfLengthBetween(1, 10)); - metadata.put(AuthenticationField.API_KEY_NAME_KEY, randomBoolean() ? null : randomAlphaOfLengthBetween(1, 10)); - metadata.put(AuthenticationField.API_KEY_ROLE_DESCRIPTORS_KEY, new BytesArray("{\"a role\": {\"cluster\": [\"all\"]}}")); - metadata.put( - AuthenticationField.API_KEY_LIMITED_ROLE_DESCRIPTORS_KEY, - new BytesArray("{\"limitedBy role\": {\"cluster\": [\"all\"]}}") - ); - - final Authentication original = AuthenticationTestHelper.builder() - .apiKey() - .metadata(metadata) - .transportVersion(TransportVersions.V_8_0_0) - .build(); - original.writeToContext(threadContext); - - // If target is old node, rewrite new style API key metadata to old format - securityContext.executeAfterRewritingAuthentication(originalCtx -> { - Authentication authentication = securityContext.getAuthentication(); - assertEquals( - Map.of("a role", Map.of("cluster", List.of("all"))), - authentication.getAuthenticatingSubject().getMetadata().get(AuthenticationField.API_KEY_ROLE_DESCRIPTORS_KEY) - ); - assertEquals( - Map.of("limitedBy role", Map.of("cluster", List.of("all"))), - authentication.getAuthenticatingSubject().getMetadata().get(AuthenticationField.API_KEY_LIMITED_ROLE_DESCRIPTORS_KEY) - ); - }, TransportVersions.V_7_8_0); - - // If target is new node, no need to rewrite the new style API key metadata - securityContext.executeAfterRewritingAuthentication(originalCtx -> { - Authentication authentication = securityContext.getAuthentication(); - assertSame(original.getAuthenticatingSubject().getMetadata(), authentication.getAuthenticatingSubject().getMetadata()); - }, TransportVersionUtils.randomVersionBetween(random(), VERSION_API_KEY_ROLES_AS_BYTES, TransportVersion.current())); - } - public void testExecuteAfterRewritingAuthenticationWillConditionallyRewriteOldApiKeyMetadata() throws IOException { final Authentication original = AuthenticationTestHelper.builder().apiKey().transportVersion(TransportVersions.V_7_8_0).build(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/SecurityServerTransportInterceptorTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/SecurityServerTransportInterceptorTests.java index 1e17459613a2c..3bf995edee9eb 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/SecurityServerTransportInterceptorTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/SecurityServerTransportInterceptorTests.java @@ -8,7 +8,6 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionResponse.Empty; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; @@ -36,7 +35,6 @@ import org.elasticsearch.test.TransportVersionUtils; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.RemoteClusterPortSettings; import org.elasticsearch.transport.RemoteConnectionManager.RemoteClusterAliasWithCredentials; import org.elasticsearch.transport.SendRequestTransportException; import org.elasticsearch.transport.Transport; @@ -966,102 +964,6 @@ public void sendRequest( assertThat(securityContext.getThreadContext().getHeader(CROSS_CLUSTER_ACCESS_CREDENTIALS_HEADER_KEY), nullValue()); } - public void testSendWithCrossClusterAccessHeadersThrowsOnOldConnection() throws Exception { - final Authentication authentication = AuthenticationTestHelper.builder() - .user(new User(randomAlphaOfLengthBetween(3, 10), randomArray(0, 4, String[]::new, () -> randomAlphaOfLengthBetween(1, 20)))) - .realm() - .build(); - authentication.writeToContext(threadContext); - final String remoteClusterAlias = randomAlphaOfLengthBetween(5, 10); - final String encodedApiKey = randomAlphaOfLengthBetween(10, 42); - final String remoteClusterCredential = ApiKeyService.withApiKeyPrefix(encodedApiKey); - - final SecurityServerTransportInterceptor interceptor = new SecurityServerTransportInterceptor( - settings, - threadPool, - mockSslService(), - securityContext, - destructiveOperations, - new CrossClusterAccessTransportInterceptor( - settings, - threadPool, - mock(AuthenticationService.class), - mock(AuthorizationService.class), - securityContext, - mock(CrossClusterAccessAuthenticationService.class), - crossClusterApiKeySignatureManager, - mockLicenseState, - ignored -> Optional.of( - new RemoteClusterAliasWithCredentials(remoteClusterAlias, new SecureString(encodedApiKey.toCharArray())) - ) - ) - ); - - final AsyncSender sender = interceptor.interceptSender(new AsyncSender() { - @Override - public void sendRequest( - Transport.Connection connection, - String action, - TransportRequest request, - TransportRequestOptions options, - TransportResponseHandler handler - ) { - fail("sender should not be called"); - } - }); - final Transport.Connection connection = mock(Transport.Connection.class); - final TransportVersion versionBeforeCrossClusterAccessRealm = TransportVersionUtils.getPreviousVersion( - RemoteClusterPortSettings.TRANSPORT_VERSION_ADVANCED_REMOTE_CLUSTER_SECURITY - ); - final TransportVersion version = TransportVersionUtils.randomVersionBetween( - random(), - TransportVersions.V_8_0_0, - versionBeforeCrossClusterAccessRealm - ); - when(connection.getTransportVersion()).thenReturn(version); - final AtomicBoolean calledHandleException = new AtomicBoolean(false); - final AtomicReference actualException = new AtomicReference<>(); - sender.sendRequest(connection, "action", mock(TransportRequest.class), null, new TransportResponseHandler<>() { - @Override - public Executor executor() { - return TransportResponseHandler.TRANSPORT_WORKER; - } - - @Override - public void handleResponse(TransportResponse response) { - fail("should not receive a response"); - } - - @Override - public void handleException(TransportException exp) { - if (calledHandleException.compareAndSet(false, true) == false) { - fail("handle exception called more than once"); - } - actualException.set(exp); - } - - @Override - public TransportResponse read(StreamInput in) { - fail("should not receive a response"); - return null; - } - }); - assertThat(actualException.get(), instanceOf(SendRequestTransportException.class)); - assertThat(actualException.get().getCause(), instanceOf(IllegalArgumentException.class)); - assertThat( - actualException.get().getCause().getMessage(), - equalTo( - "Settings for remote cluster [" - + remoteClusterAlias - + "] indicate cross cluster access headers should be sent but target cluster version [" - + connection.getTransportVersion().toReleaseVersion() - + "] does not support receiving them" - ) - ); - assertThat(securityContext.getThreadContext().getHeader(CROSS_CLUSTER_ACCESS_SUBJECT_INFO_HEADER_KEY), nullValue()); - assertThat(securityContext.getThreadContext().getHeader(CROSS_CLUSTER_ACCESS_CREDENTIALS_HEADER_KEY), nullValue()); - } - public void testSendRemoteRequestFailsIfUserHasNoRemoteIndicesPrivileges() throws Exception { final Authentication authentication = AuthenticationTestHelper.builder() .user(new User(randomAlphaOfLengthBetween(3, 10), randomRoles()))