From 83da48cdbbedaf0fe15fe7ce64a14abd2b7be720 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Thu, 9 May 2019 12:17:32 -0700 Subject: [PATCH 1/3] Remove the 6.7 version constants. --- .../reindex/TransportUpdateByQueryAction.java | 10 +-- .../main/java/org/elasticsearch/Build.java | 7 +- .../elasticsearch/ElasticsearchException.java | 8 +- .../main/java/org/elasticsearch/Version.java | 16 ---- .../index/engine/ReadOnlyEngine.java | 2 +- .../recovery/RecoverySourceHandler.java | 4 +- .../java/org/elasticsearch/BuildTests.java | 34 --------- .../ExceptionSerializationTests.java | 2 +- .../java/org/elasticsearch/VersionTests.java | 1 - .../action/shard/ShardStateActionTests.java | 6 +- .../cluster/block/ClusterBlockTests.java | 43 +---------- .../xpack/core/ccr/AutoFollowStats.java | 25 ++----- .../action/PutAutoFollowPatternAction.java | 33 +------- .../core/ccr/action/PutFollowAction.java | 9 +-- .../deprecation/DeprecationInfoAction.java | 12 +-- .../xpack/core/ml/MlMetadata.java | 20 +---- .../xpack/core/ml/datafeed/AggProvider.java | 37 +-------- .../xpack/core/ml/datafeed/QueryProvider.java | 35 +-------- .../core/ml/datafeed/AggProviderTests.java | 69 ----------------- .../core/ml/datafeed/QueryProviderTests.java | 75 ------------------- .../security/authc/TokenServiceTests.java | 2 +- 21 files changed, 39 insertions(+), 411 deletions(-) diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/TransportUpdateByQueryAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/TransportUpdateByQueryAction.java index 5ea175d11a7cb..410ae1b51116d 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/TransportUpdateByQueryAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/TransportUpdateByQueryAction.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.reindex; import org.apache.logging.log4j.Logger; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.ActionFilters; @@ -82,18 +81,13 @@ protected void doExecute(Task task, UpdateByQueryRequest request, ActionListener */ static class AsyncIndexBySearchAction extends AbstractAsyncBulkByScrollAction { - private final boolean useSeqNoForCAS; - AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client, ThreadPool threadPool, TransportUpdateByQueryAction action, UpdateByQueryRequest request, ClusterState clusterState, ActionListener listener) { super(task, - // not all nodes support sequence number powered optimistic concurrency control, we fall back to version - clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_6_7_0) == false, - // all nodes support sequence number powered optimistic concurrency control and we can use it - clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_6_7_0), + // use sequence number powered optimistic concurrency control + false, true, logger, client, threadPool, action, request, listener); - useSeqNoForCAS = clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_6_7_0); } @Override diff --git a/server/src/main/java/org/elasticsearch/Build.java b/server/src/main/java/org/elasticsearch/Build.java index 1b1cd8d3e720a..1a1ee2744f77a 100644 --- a/server/src/main/java/org/elasticsearch/Build.java +++ b/server/src/main/java/org/elasticsearch/Build.java @@ -254,12 +254,7 @@ public static void writeBuild(Build build, StreamOutput out) throws IOException out.writeString(build.flavor().displayName()); } if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - final Type buildType; - if (out.getVersion().before(Version.V_6_7_0) && build.type() == Type.DOCKER) { - buildType = Type.TAR; - } else { - buildType = build.type(); - } + final Type buildType = build.type(); out.writeString(buildType.displayName()); } out.writeString(build.shortHash()); diff --git a/server/src/main/java/org/elasticsearch/ElasticsearchException.java b/server/src/main/java/org/elasticsearch/ElasticsearchException.java index 861228d221778..6a00ac2a1c879 100644 --- a/server/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/server/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -1010,19 +1010,19 @@ private enum ElasticsearchExceptionHandle { COORDINATION_STATE_REJECTED_EXCEPTION(org.elasticsearch.cluster.coordination.CoordinationStateRejectedException.class, org.elasticsearch.cluster.coordination.CoordinationStateRejectedException::new, 150, Version.V_7_0_0), SNAPSHOT_IN_PROGRESS_EXCEPTION(org.elasticsearch.snapshots.SnapshotInProgressException.class, - org.elasticsearch.snapshots.SnapshotInProgressException::new, 151, Version.V_6_7_0), + org.elasticsearch.snapshots.SnapshotInProgressException::new, 151, UNKNOWN_VERSION_ADDED), NO_SUCH_REMOTE_CLUSTER_EXCEPTION(org.elasticsearch.transport.NoSuchRemoteClusterException.class, - org.elasticsearch.transport.NoSuchRemoteClusterException::new, 152, Version.V_6_7_0), + org.elasticsearch.transport.NoSuchRemoteClusterException::new, 152, UNKNOWN_VERSION_ADDED), RETENTION_LEASE_ALREADY_EXISTS_EXCEPTION( org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException.class, org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException::new, 153, - Version.V_6_7_0), + UNKNOWN_VERSION_ADDED), RETENTION_LEASE_NOT_FOUND_EXCEPTION( org.elasticsearch.index.seqno.RetentionLeaseNotFoundException.class, org.elasticsearch.index.seqno.RetentionLeaseNotFoundException::new, 154, - Version.V_6_7_0); + UNKNOWN_VERSION_ADDED); final Class exceptionClass; final CheckedFunction constructor; diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index c2d927f457bd1..7492d802b614b 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -104,14 +104,6 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_6_6_1 = new Version(V_6_6_1_ID, org.apache.lucene.util.Version.LUCENE_7_6_0); public static final int V_6_6_2_ID = 6060299; public static final Version V_6_6_2 = new Version(V_6_6_2_ID, org.apache.lucene.util.Version.LUCENE_7_6_0); - public static final int V_6_7_0_ID = 6070099; - public static final Version V_6_7_0 = new Version(V_6_7_0_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); - public static final int V_6_7_1_ID = 6070199; - public static final Version V_6_7_1 = new Version(V_6_7_1_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); - public static final int V_6_7_2_ID = 6070299; - public static final Version V_6_7_2 = new Version(V_6_7_2_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); - public static final int V_6_7_3_ID = 6070399; - public static final Version V_6_7_3 = new Version(V_6_7_3_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); public static final int V_6_8_0_ID = 6080099; public static final Version V_6_8_0 = new Version(V_6_8_0_ID, org.apache.lucene.util.Version.LUCENE_7_7_0); public static final int V_7_0_0_ID = 7000099; @@ -154,14 +146,6 @@ public static Version fromId(int id) { return V_7_0_0; case V_6_8_0_ID: return V_6_8_0; - case V_6_7_3_ID: - return V_6_7_3; - case V_6_7_1_ID: - return V_6_7_1; - case V_6_7_2_ID: - return V_6_7_2; - case V_6_7_0_ID: - return V_6_7_0; case V_6_6_2_ID: return V_6_6_2; case V_6_6_1_ID: diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index b981bdb8a8421..e7e0c4d927851 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -131,7 +131,7 @@ protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(final SeqNoStats seqNoStat // that guarantee that all operations have been flushed to Lucene. final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated(); if (indexVersionCreated.onOrAfter(Version.V_7_2_0) || - (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO && indexVersionCreated.onOrAfter(Version.V_6_7_0))) { + (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO)) { if (seqNoStats.getMaxSeqNo() != seqNoStats.getGlobalCheckpoint()) { throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo() + "] from last commit does not match global checkpoint [" + seqNoStats.getGlobalCheckpoint() + "]"); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index aad460b821e62..4e82798e34128 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -30,7 +30,6 @@ import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.StepListener; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; @@ -112,8 +111,7 @@ public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recov this.shardId = this.request.shardId().id(); this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName()); this.chunkSizeInBytes = fileChunkSizeInBytes; - // if the target is on an old version, it won't be able to handle out-of-order file chunks. - this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(Version.V_6_7_0) ? maxConcurrentFileChunks : 1; + this.maxConcurrentFileChunks = maxConcurrentFileChunks; } public StartRecoveryRequest getRequest() { diff --git a/server/src/test/java/org/elasticsearch/BuildTests.java b/server/src/test/java/org/elasticsearch/BuildTests.java index e0d8140c708d6..7a6f581bd7369 100644 --- a/server/src/test/java/org/elasticsearch/BuildTests.java +++ b/server/src/test/java/org/elasticsearch/BuildTests.java @@ -30,7 +30,6 @@ import java.io.InputStream; import java.net.URL; import java.util.Arrays; -import java.util.List; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -194,39 +193,6 @@ public void testSerialization() { }); } - public void testSerializationBWC() throws IOException { - final WriteableBuild dockerBuild = new WriteableBuild(new Build(randomFrom(Build.Flavor.values()), Build.Type.DOCKER, - randomAlphaOfLength(6), randomAlphaOfLength(6), randomBoolean(), randomAlphaOfLength(6))); - - final List versions = Version.getDeclaredVersions(Version.class); - final Version pre63Version = randomFrom(versions.stream().filter(v -> v.before(Version.V_6_3_0)).collect(Collectors.toList())); - final Version post63Pre67Version = randomFrom(versions.stream() - .filter(v -> v.onOrAfter(Version.V_6_3_0) && v.before(Version.V_6_7_0)).collect(Collectors.toList())); - final Version post67Pre70Version = randomFrom(versions.stream() - .filter(v -> v.onOrAfter(Version.V_6_7_0) && v.before(Version.V_7_0_0)).collect(Collectors.toList())); - final Version post70Version = randomFrom(versions.stream().filter(v -> v.onOrAfter(Version.V_7_0_0)).collect(Collectors.toList())); - - final WriteableBuild pre63 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, pre63Version); - final WriteableBuild post63pre67 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post63Pre67Version); - final WriteableBuild post67pre70 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post67Pre70Version); - final WriteableBuild post70 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post70Version); - - assertThat(pre63.build.flavor(), equalTo(Build.Flavor.OSS)); - assertThat(post63pre67.build.flavor(), equalTo(dockerBuild.build.flavor())); - assertThat(post67pre70.build.flavor(), equalTo(dockerBuild.build.flavor())); - assertThat(post70.build.flavor(), equalTo(dockerBuild.build.flavor())); - - assertThat(pre63.build.type(), equalTo(Build.Type.UNKNOWN)); - assertThat(post63pre67.build.type(), equalTo(Build.Type.TAR)); - assertThat(post67pre70.build.type(), equalTo(dockerBuild.build.type())); - assertThat(post70.build.type(), equalTo(dockerBuild.build.type())); - - assertThat(pre63.build.getQualifiedVersion(), equalTo(pre63Version.toString())); - assertThat(post63pre67.build.getQualifiedVersion(), equalTo(post63Pre67Version.toString())); - assertThat(post67pre70.build.getQualifiedVersion(), equalTo(post67Pre70Version.toString())); - assertThat(post70.build.getQualifiedVersion(), equalTo(dockerBuild.build.getQualifiedVersion())); - } - public void testFlavorParsing() { for (final Build.Flavor flavor : Build.Flavor.values()) { // strict or not should not impact parsing at all here diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 1fac56886de45..d6adc4d1ebadc 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -886,7 +886,7 @@ public void testShardLockObtainFailedException() throws IOException { public void testSnapshotInProgressException() throws IOException { SnapshotInProgressException orig = new SnapshotInProgressException("boom"); - Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_7_0, Version.CURRENT); + Version version = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, Version.CURRENT); SnapshotInProgressException ex = serialize(orig, version); assertEquals(orig.getMessage(), ex.getMessage()); } diff --git a/server/src/test/java/org/elasticsearch/VersionTests.java b/server/src/test/java/org/elasticsearch/VersionTests.java index 21a18e4a26ba5..66d7af0a4b20e 100644 --- a/server/src/test/java/org/elasticsearch/VersionTests.java +++ b/server/src/test/java/org/elasticsearch/VersionTests.java @@ -346,7 +346,6 @@ public static void assertUnknownVersion(Version version) { public void testIsCompatible() { assertTrue(isCompatible(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion())); assertFalse(isCompatible(Version.V_6_6_0, Version.V_7_0_0)); - assertFalse(isCompatible(Version.V_6_7_0, Version.V_7_0_0)); assertTrue(isCompatible(Version.V_6_8_0, Version.V_7_0_0)); assertFalse(isCompatible(Version.fromId(2000099), Version.V_7_0_0)); assertFalse(isCompatible(Version.fromId(2000099), Version.V_6_5_0)); diff --git a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java index bcd080e0b45fd..7eb995e04f3fa 100644 --- a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java @@ -519,11 +519,7 @@ public void testStartedShardEntrySerialization() throws Exception { final StartedShardEntry deserialized = new StartedShardEntry(in); assertThat(deserialized.shardId, equalTo(shardId)); assertThat(deserialized.allocationId, equalTo(allocationId)); - if (version.onOrAfter(Version.V_6_7_0)) { - assertThat(deserialized.primaryTerm, equalTo(primaryTerm)); - } else { - assertThat(deserialized.primaryTerm, equalTo(0L)); - } + assertThat(deserialized.primaryTerm, equalTo(primaryTerm)); assertThat(deserialized.message, equalTo(message)); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java b/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java index 4bd6c15853aa0..8d68684c9c368 100644 --- a/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java @@ -32,9 +32,7 @@ import java.util.List; import static java.util.EnumSet.copyOf; -import static org.elasticsearch.test.VersionUtils.getPreviousVersion; import static org.elasticsearch.test.VersionUtils.randomVersion; -import static org.elasticsearch.test.VersionUtils.randomVersionBetween; import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -62,45 +60,6 @@ public void testSerialization() throws Exception { } } - public void testBwcSerialization() throws Exception { - for (int runs = 0; runs < randomIntBetween(5, 20); runs++) { - // Generate a random cluster block in version < 7.0.0 - final Version version = randomVersionBetween(random(), Version.V_6_0_0, getPreviousVersion(Version.V_6_7_0)); - final ClusterBlock expected = randomClusterBlock(version); - assertNull(expected.uuid()); - - // Serialize to node in current version - final BytesStreamOutput out = new BytesStreamOutput(); - expected.writeTo(out); - - // Deserialize and check the cluster block - final ClusterBlock actual = new ClusterBlock(out.bytes().streamInput()); - assertClusterBlockEquals(expected, actual); - } - - for (int runs = 0; runs < randomIntBetween(5, 20); runs++) { - // Generate a random cluster block in current version - final ClusterBlock expected = randomClusterBlock(Version.CURRENT); - - // Serialize to node in version < 7.0.0 - final BytesStreamOutput out = new BytesStreamOutput(); - out.setVersion(randomVersionBetween(random(), Version.V_6_0_0, getPreviousVersion(Version.V_6_7_0))); - expected.writeTo(out); - - // Deserialize and check the cluster block - final StreamInput in = out.bytes().streamInput(); - in.setVersion(out.getVersion()); - final ClusterBlock actual = new ClusterBlock(in); - - assertThat(actual.id(), equalTo(expected.id())); - assertThat(actual.status(), equalTo(expected.status())); - assertThat(actual.description(), equalTo(expected.description())); - assertThat(actual.retryable(), equalTo(expected.retryable())); - assertThat(actual.disableStatePersistence(), equalTo(expected.disableStatePersistence())); - assertArrayEquals(actual.levels().toArray(), expected.levels().toArray()); - } - } - public void testToStringDanglingComma() { final ClusterBlock clusterBlock = randomClusterBlock(); assertThat(clusterBlock.toString(), not(endsWith(","))); @@ -171,7 +130,7 @@ private ClusterBlock randomClusterBlock() { } private ClusterBlock randomClusterBlock(final Version version) { - final String uuid = (version.onOrAfter(Version.V_6_7_0) && randomBoolean()) ? UUIDs.randomBase64UUID() : null; + final String uuid = randomBoolean() ? UUIDs.randomBase64UUID() : null; final List levels = Arrays.asList(ClusterBlockLevel.values()); return new ClusterBlock(randomInt(), uuid, "cluster block #" + randomInt(), randomBoolean(), randomBoolean(), randomBoolean(), randomFrom(RestStatus.values()), copyOf(randomSubsetOf(randomIntBetween(1, levels.size()), levels))); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowStats.java index 600bd5fced3ae..6c605ede85e24 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowStats.java @@ -116,15 +116,9 @@ public AutoFollowStats(StreamInput in) throws IOException { numberOfFailedFollowIndices = in.readVLong(); numberOfFailedRemoteClusterStateRequests = in.readVLong(); numberOfSuccessfulFollowIndices = in.readVLong(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - // note: the casts to the following Writeable.Reader instances are needed by some IDEs (e.g. Eclipse 4.8) as a compiler help - recentAutoFollowErrors = new TreeMap<>(in.readMap((Writeable.Reader) StreamInput::readString, - (Writeable.Reader>) in1 -> new Tuple<>(in1.readZLong(), in1.readException()))); - } else { - // note: the casts to the following Writeable.Reader instances are needed by some IDEs (e.g. Eclipse 4.8) as a compiler help - recentAutoFollowErrors = new TreeMap<>(in.readMap((Writeable.Reader) StreamInput::readString, - (Writeable.Reader>) in1 -> new Tuple<>(-1L, in1.readException()))); - } + // note: the casts to the following Writeable.Reader instances are needed by some IDEs (e.g. Eclipse 4.8) as a compiler help + recentAutoFollowErrors = new TreeMap<>(in.readMap((Writeable.Reader) StreamInput::readString, + (Writeable.Reader>) in1 -> new Tuple<>(in1.readZLong(), in1.readException()))); if (in.getVersion().onOrAfter(Version.V_6_6_0)) { autoFollowedClusters = new TreeMap<>(in.readMap(StreamInput::readString, AutoFollowedCluster::new)); } else { @@ -137,14 +131,11 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(numberOfFailedFollowIndices); out.writeVLong(numberOfFailedRemoteClusterStateRequests); out.writeVLong(numberOfSuccessfulFollowIndices); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeMap(recentAutoFollowErrors, StreamOutput::writeString, (out1, value) -> { - out1.writeZLong(value.v1()); - out1.writeException(value.v2()); - }); - } else { - out.writeMap(recentAutoFollowErrors, StreamOutput::writeString, (out1, value) -> out1.writeException(value.v2())); - } + out.writeMap(recentAutoFollowErrors, StreamOutput::writeString, (out1, value) -> { + out1.writeZLong(value.v1()); + out1.writeException(value.v2()); + }); + if (out.getVersion().onOrAfter(Version.V_6_6_0)) { out.writeMap(autoFollowedClusters, StreamOutput::writeString, (out1, value) -> value.writeTo(out1)); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java index ec946ce51e821..f26e8d7f82a51 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java @@ -5,14 +5,12 @@ */ package org.elasticsearch.xpack.core.ccr.action; -import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -153,21 +151,7 @@ public Request(StreamInput in) throws IOException { remoteCluster = in.readString(); leaderIndexPatterns = in.readStringList(); followIndexNamePattern = in.readOptionalString(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - parameters = new FollowParameters(in); - } else { - parameters = new FollowParameters(); - parameters.maxReadRequestOperationCount = in.readOptionalVInt(); - parameters.maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new); - parameters.maxOutstandingReadRequests = in.readOptionalVInt(); - parameters.maxWriteRequestOperationCount = in.readOptionalVInt(); - parameters.maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new); - parameters.maxOutstandingWriteRequests = in.readOptionalVInt(); - parameters.maxWriteBufferCount = in.readOptionalVInt(); - parameters.maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new); - parameters.maxRetryDelay = in.readOptionalTimeValue(); - parameters.readPollTimeout = in.readOptionalTimeValue(); - } + parameters = new FollowParameters(in); } @Override @@ -177,20 +161,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(remoteCluster); out.writeStringCollection(leaderIndexPatterns); out.writeOptionalString(followIndexNamePattern); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - parameters.writeTo(out); - } else { - out.writeOptionalVInt(parameters.maxReadRequestOperationCount); - out.writeOptionalWriteable(parameters.maxReadRequestSize); - out.writeOptionalVInt(parameters.maxOutstandingReadRequests); - out.writeOptionalVInt(parameters.maxWriteRequestOperationCount); - out.writeOptionalWriteable(parameters.maxWriteRequestSize); - out.writeOptionalVInt(parameters.maxOutstandingWriteRequests); - out.writeOptionalVInt(parameters.maxWriteBufferCount); - out.writeOptionalWriteable(parameters.maxWriteBufferSize); - out.writeOptionalTimeValue(parameters.maxRetryDelay); - out.writeOptionalTimeValue(parameters.readPollTimeout); - } + parameters.writeTo(out); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java index 89c18a9824ab4..4d20e6d820de2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java @@ -6,7 +6,6 @@ package org.elasticsearch.xpack.core.ccr.action; -import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -169,9 +168,7 @@ public Request(StreamInput in) throws IOException { this.leaderIndex = in.readString(); this.followerIndex = in.readString(); this.parameters = new FollowParameters(in); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - waitForActiveShards(ActiveShardCount.readFrom(in)); - } + waitForActiveShards(ActiveShardCount.readFrom(in)); } @Override @@ -181,9 +178,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(leaderIndex); out.writeString(followerIndex); parameters.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - waitForActiveShards.writeTo(out); - } + waitForActiveShards.writeTo(out); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java index 28aa09f6c1efb..54d260e32532f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.core.deprecation; -import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -29,7 +28,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -125,11 +123,7 @@ public void readFrom(StreamInput in) throws IOException { clusterSettingsIssues = in.readList(DeprecationIssue::new); nodeSettingsIssues = in.readList(DeprecationIssue::new); indexSettingsIssues = in.readMapOfLists(StreamInput::readString, DeprecationIssue::new); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - mlSettingsIssues = in.readList(DeprecationIssue::new); - } else { - mlSettingsIssues = Collections.emptyList(); - } + mlSettingsIssues = in.readList(DeprecationIssue::new); } @Override @@ -138,9 +132,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeList(clusterSettingsIssues); out.writeList(nodeSettingsIssues); out.writeMapOfLists(indexSettingsIssues, StreamOutput::writeString, (o, v) -> v.writeTo(o)); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeList(mlSettingsIssues); - } + out.writeList(mlSettingsIssues); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java index 0f502577195dd..dfe5560da3303 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java @@ -138,20 +138,14 @@ public MlMetadata(StreamInput in) throws IOException { } this.datafeeds = datafeeds; this.groupOrJobLookup = new GroupOrJobLookup(jobs.values()); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - this.upgradeMode = in.readBoolean(); - } else { - this.upgradeMode = false; - } + this.upgradeMode = in.readBoolean(); } @Override public void writeTo(StreamOutput out) throws IOException { writeMap(jobs, out); writeMap(datafeeds, out); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeBoolean(upgradeMode); - } + out.writeBoolean(upgradeMode); } private static void writeMap(Map map, StreamOutput out) throws IOException { @@ -202,11 +196,7 @@ public MlMetadataDiff(StreamInput in) throws IOException { MlMetadataDiff::readJobDiffFrom); this.datafeeds = DiffableUtils.readJdkMapDiff(in, DiffableUtils.getStringKeySerializer(), DatafeedConfig::new, MlMetadataDiff::readDatafeedDiffFrom); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - upgradeMode = in.readBoolean(); - } else { - upgradeMode = false; - } + upgradeMode = in.readBoolean(); } /** @@ -225,9 +215,7 @@ public MetaData.Custom apply(MetaData.Custom part) { public void writeTo(StreamOutput out) throws IOException { jobs.writeTo(out); datafeeds.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeBoolean(upgradeMode); - } + out.writeBoolean(upgradeMode); } @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/AggProvider.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/AggProvider.java index 8585e4122e673..1c39c6d985d45 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/AggProvider.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/AggProvider.java @@ -7,8 +7,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -68,14 +66,7 @@ static AggProvider fromParsedAggs(AggregatorFactories.Builder parsedAggs) throws } static AggProvider fromStream(StreamInput in) throws IOException { - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { // Has our bug fix for query/agg providers - return new AggProvider(in.readMap(), in.readOptionalWriteable(AggregatorFactories.Builder::new), in.readException()); - } else if (in.getVersion().onOrAfter(Version.V_6_6_0)) { // Has the bug, but supports lazy objects - return new AggProvider(in.readMap(), null, null); - } else { // only supports eagerly parsed objects - // Upstream, we have read the bool already and know for sure that we have parsed aggs in the stream - return AggProvider.fromParsedAggs(new AggregatorFactories.Builder(in)); - } + return new AggProvider(in.readMap(), in.readOptionalWriteable(AggregatorFactories.Builder::new), in.readException()); } AggProvider(Map aggs, AggregatorFactories.Builder parsedAggs, Exception parsingException) { @@ -92,29 +83,9 @@ static AggProvider fromStream(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { // Has our bug fix for query/agg providers - out.writeMap(aggs); - out.writeOptionalWriteable(parsedAggs); - out.writeException(parsingException); - } else if (out.getVersion().onOrAfter(Version.V_6_6_0)) { // Has the bug, but supports lazy objects - // We allow the lazy parsing nodes that have the bug throw any parsing errors themselves as - // they already have the ability to fully parse the passed Maps - out.writeMap(aggs); - } else { // only supports eagerly parsed objects - if (parsingException != null) { - if (parsingException instanceof IOException) { - throw (IOException) parsingException; - } else { - throw new ElasticsearchException(parsingException); - } - } else if (parsedAggs == null) { - // This is an admittedly rare case but we should fail early instead of writing null when there - // actually are aggregations defined - throw new ElasticsearchException("Unsupported operation: parsed aggregations are null"); - } - // Upstream we already verified that this calling object is not null, no need to write a second boolean to the stream - parsedAggs.writeTo(out); - } + out.writeMap(aggs); + out.writeOptionalWriteable(parsedAggs); + out.writeException(parsingException); } public Exception getParsingException() { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProvider.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProvider.java index ff6d2f595af81..755c5a3526d01 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProvider.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProvider.java @@ -7,8 +7,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -74,13 +72,7 @@ static QueryProvider fromParsedQuery(QueryBuilder parsedQuery) throws IOExceptio } static QueryProvider fromStream(StreamInput in) throws IOException { - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { // Has our bug fix for query/agg providers - return new QueryProvider(in.readMap(), in.readOptionalNamedWriteable(QueryBuilder.class), in.readException()); - } else if (in.getVersion().onOrAfter(Version.V_6_6_0)) { // Has the bug, but supports lazy objects - return new QueryProvider(in.readMap(), null, null); - } else { // only supports eagerly parsed objects - return QueryProvider.fromParsedQuery(in.readNamedWriteable(QueryBuilder.class)); - } + return new QueryProvider(in.readMap(), in.readOptionalNamedWriteable(QueryBuilder.class), in.readException()); } QueryProvider(Map query, QueryBuilder parsedQuery, Exception parsingException) { @@ -95,28 +87,9 @@ static QueryProvider fromStream(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { // Has our bug fix for query/agg providers - out.writeMap(query); - out.writeOptionalNamedWriteable(parsedQuery); - out.writeException(parsingException); - } else if (out.getVersion().onOrAfter(Version.V_6_6_0)) { // Has the bug, but supports lazy objects - // We allow the lazy parsing nodes that have the bug throw any parsing errors themselves as - // they already have the ability to fully parse the passed Maps - out.writeMap(query); - } else { // only supports eagerly parsed objects - if (parsingException != null) { // Do we have a parsing error? Throw it - if (parsingException instanceof IOException) { - throw (IOException) parsingException; - } else { - throw new ElasticsearchException(parsingException); - } - } else if (parsedQuery == null) { // Do we have a query defined but not parsed? - // This is an admittedly rare case but we should fail early instead of writing null when there - // actually is a query defined - throw new ElasticsearchException("Unsupported operation: parsed query is null"); - } - out.writeNamedWriteable(parsedQuery); - } + out.writeMap(query); + out.writeOptionalNamedWriteable(parsedQuery); + out.writeException(parsingException); } public Exception getParsingException() { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderTests.java index dc87cf744cb98..d544584376f47 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/AggProviderTests.java @@ -5,13 +5,8 @@ */ package org.elasticsearch.xpack.core.ml.datafeed; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.Version; -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.common.settings.Settings; import org.elasticsearch.common.xcontent.DeprecationHandler; @@ -29,9 +24,7 @@ import java.util.Collections; import java.util.Map; -import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; public class AggProviderTests extends AbstractSerializingTestCase { @@ -96,68 +89,6 @@ public void testEmptyAggMap() throws IOException { assertThat(e.getMessage(), equalTo("Datafeed aggregations are not parsable")); } - public void testSerializationBetweenBugVersion() throws IOException { - AggProvider tempAggProvider = createRandomValidAggProvider(); - AggProvider aggProviderWithEx = new AggProvider(tempAggProvider.getAggs(), tempAggProvider.getParsedAggs(), new IOException("ex")); - try (BytesStreamOutput output = new BytesStreamOutput()) { - output.setVersion(Version.V_6_6_2); - aggProviderWithEx.writeTo(output); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), writableRegistry())) { - in.setVersion(Version.V_6_6_2); - AggProvider streamedAggProvider = AggProvider.fromStream(in); - assertThat(streamedAggProvider.getAggs(), equalTo(aggProviderWithEx.getAggs())); - assertThat(streamedAggProvider.getParsingException(), is(nullValue())); - - AggregatorFactories.Builder streamedParsedAggs = XContentObjectTransformer.aggregatorTransformer(xContentRegistry()) - .fromMap(streamedAggProvider.getAggs()); - assertThat(streamedParsedAggs, equalTo(aggProviderWithEx.getParsedAggs())); - assertThat(streamedAggProvider.getParsedAggs(), is(nullValue())); - } - } - } - - public void testSerializationBetweenEagerVersion() throws IOException { - AggProvider validAggProvider = createRandomValidAggProvider(); - - try (BytesStreamOutput output = new BytesStreamOutput()) { - output.setVersion(Version.V_6_0_0); - validAggProvider.writeTo(output); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), writableRegistry())) { - in.setVersion(Version.V_6_0_0); - AggProvider streamedAggProvider = AggProvider.fromStream(in); - assertThat(streamedAggProvider.getAggs(), equalTo(validAggProvider.getAggs())); - assertThat(streamedAggProvider.getParsingException(), is(nullValue())); - assertThat(streamedAggProvider.getParsedAggs(), equalTo(validAggProvider.getParsedAggs())); - } - } - - try (BytesStreamOutput output = new BytesStreamOutput()) { - AggProvider aggProviderWithEx = new AggProvider(validAggProvider.getAggs(), - validAggProvider.getParsedAggs(), - new IOException("bad parsing")); - output.setVersion(Version.V_6_0_0); - IOException ex = expectThrows(IOException.class, () -> aggProviderWithEx.writeTo(output)); - assertThat(ex.getMessage(), equalTo("bad parsing")); - } - - try (BytesStreamOutput output = new BytesStreamOutput()) { - AggProvider aggProviderWithEx = new AggProvider(validAggProvider.getAggs(), - validAggProvider.getParsedAggs(), - new ElasticsearchException("bad parsing")); - output.setVersion(Version.V_6_0_0); - ElasticsearchException ex = expectThrows(ElasticsearchException.class, () -> aggProviderWithEx.writeTo(output)); - assertNotNull(ex.getCause()); - assertThat(ex.getCause().getMessage(), equalTo("bad parsing")); - } - - try (BytesStreamOutput output = new BytesStreamOutput()) { - AggProvider aggProviderWithOutParsed = new AggProvider(validAggProvider.getAggs(), null, null); - output.setVersion(Version.V_6_0_0); - ElasticsearchException ex = expectThrows(ElasticsearchException.class, () -> aggProviderWithOutParsed.writeTo(output)); - assertThat(ex.getMessage(), equalTo("Unsupported operation: parsed aggregations are null")); - } - } - @Override protected AggProvider mutateInstance(AggProvider instance) throws IOException { Exception parsingException = instance.getParsingException(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProviderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProviderTests.java index fb6c2e280d975..8d113aba33579 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProviderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/datafeed/QueryProviderTests.java @@ -5,13 +5,8 @@ */ package org.elasticsearch.xpack.core.ml.datafeed; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.Version; -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.common.settings.Settings; import org.elasticsearch.common.xcontent.DeprecationHandler; @@ -32,9 +27,7 @@ import java.util.Collections; import java.util.Map; -import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; public class QueryProviderTests extends AbstractSerializingTestCase { @@ -96,74 +89,6 @@ public void testEmptyQueryMap() throws IOException { assertThat(e.getMessage(), equalTo("Datafeed query is not parsable")); } - public void testSerializationBetweenBugVersion() throws IOException { - QueryProvider tempQueryProvider = createRandomValidQueryProvider(); - QueryProvider queryProviderWithEx = new QueryProvider(tempQueryProvider.getQuery(), - tempQueryProvider.getParsedQuery(), - new IOException("ex")); - try (BytesStreamOutput output = new BytesStreamOutput()) { - output.setVersion(Version.V_6_6_2); - queryProviderWithEx.writeTo(output); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), writableRegistry())) { - in.setVersion(Version.V_6_6_2); - QueryProvider streamedQueryProvider = QueryProvider.fromStream(in); - assertThat(streamedQueryProvider.getQuery(), equalTo(queryProviderWithEx.getQuery())); - assertThat(streamedQueryProvider.getParsingException(), is(nullValue())); - - QueryBuilder streamedParsedQuery = XContentObjectTransformer.queryBuilderTransformer(xContentRegistry()) - .fromMap(streamedQueryProvider.getQuery()); - assertThat(streamedParsedQuery, equalTo(queryProviderWithEx.getParsedQuery())); - assertThat(streamedQueryProvider.getParsedQuery(), is(nullValue())); - } - } - } - - public void testSerializationBetweenEagerVersion() throws IOException { - QueryProvider validQueryProvider = createRandomValidQueryProvider(); - - try (BytesStreamOutput output = new BytesStreamOutput()) { - output.setVersion(Version.V_6_0_0); - validQueryProvider.writeTo(output); - try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), writableRegistry())) { - in.setVersion(Version.V_6_0_0); - - QueryProvider streamedQueryProvider = QueryProvider.fromStream(in); - XContentObjectTransformer transformer = XContentObjectTransformer.queryBuilderTransformer(xContentRegistry()); - Map sourceQueryMapWithDefaults = transformer.toMap(transformer.fromMap(validQueryProvider.getQuery())); - - assertThat(streamedQueryProvider.getQuery(), equalTo(sourceQueryMapWithDefaults)); - assertThat(streamedQueryProvider.getParsingException(), is(nullValue())); - assertThat(streamedQueryProvider.getParsedQuery(), equalTo(validQueryProvider.getParsedQuery())); - } - } - - try (BytesStreamOutput output = new BytesStreamOutput()) { - QueryProvider queryProviderWithEx = new QueryProvider(validQueryProvider.getQuery(), - validQueryProvider.getParsedQuery(), - new IOException("bad parsing")); - output.setVersion(Version.V_6_0_0); - IOException ex = expectThrows(IOException.class, () -> queryProviderWithEx.writeTo(output)); - assertThat(ex.getMessage(), equalTo("bad parsing")); - } - - try (BytesStreamOutput output = new BytesStreamOutput()) { - QueryProvider queryProviderWithEx = new QueryProvider(validQueryProvider.getQuery(), - validQueryProvider.getParsedQuery(), - new ElasticsearchException("bad parsing")); - output.setVersion(Version.V_6_0_0); - ElasticsearchException ex = expectThrows(ElasticsearchException.class, () -> queryProviderWithEx.writeTo(output)); - assertNotNull(ex.getCause()); - assertThat(ex.getCause().getMessage(), equalTo("bad parsing")); - } - - try (BytesStreamOutput output = new BytesStreamOutput()) { - QueryProvider queryProviderWithOutParsed = new QueryProvider(validQueryProvider.getQuery(), null, null); - output.setVersion(Version.V_6_0_0); - ElasticsearchException ex = expectThrows(ElasticsearchException.class, () -> queryProviderWithOutParsed.writeTo(output)); - assertThat(ex.getMessage(), equalTo("Unsupported operation: parsed query is null")); - } - } - @Override protected QueryProvider mutateInstance(QueryProvider instance) throws IOException { Exception parsingException = instance.getParsingException(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java index 7f09444784c6d..8791f10467e00 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java @@ -143,7 +143,7 @@ public void setupClient() { // tokens docs on a separate index), let's test the TokenService works in a mixed cluster with nodes with versions prior to these // developments if (randomBoolean()) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_0_0)); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, Version.V_7_0_0); } } From d41c595ff7029dd14ecf32a268f13d1e4bf82065 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Mon, 20 May 2019 11:11:31 -0700 Subject: [PATCH 2/3] Improve some version checks in tests. --- .../java/org/elasticsearch/ExceptionSerializationTests.java | 2 +- .../elasticsearch/xpack/security/authc/TokenServiceTests.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index d6adc4d1ebadc..d1fde33223c4c 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -886,7 +886,7 @@ public void testShardLockObtainFailedException() throws IOException { public void testSnapshotInProgressException() throws IOException { SnapshotInProgressException orig = new SnapshotInProgressException("boom"); - Version version = VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, Version.CURRENT); + Version version = VersionUtils.randomIndexCompatibleVersion(random()); SnapshotInProgressException ex = serialize(orig, version); assertEquals(orig.getMessage(), ex.getMessage()); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java index 8791f10467e00..8bba4ff5ce0bf 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java @@ -143,7 +143,7 @@ public void setupClient() { // tokens docs on a separate index), let's test the TokenService works in a mixed cluster with nodes with versions prior to these // developments if (randomBoolean()) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, Version.V_7_0_0); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_7_0_0, Version.V_7_1_0)); } } From c6ffd76bb725584dcb3fec93852c02f0279a69d6 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Mon, 20 May 2019 17:14:20 -0700 Subject: [PATCH 3/3] Fix a compilation error from the merge. --- .../xpack/security/authc/TokenServiceTests.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java index 95c43172d73d7..49796333098ff 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java @@ -211,7 +211,7 @@ public void testRotateKey() throws Exception { TokenService tokenService = createTokenService(tokenServiceEnabledSettings, systemUTC()); // This test only makes sense in mixed clusters with pre v7.2.0 nodes where the Key is actually used if (null == oldNode) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_1_0)); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_7_0_0, Version.V_7_1_0)); } Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); @@ -272,7 +272,7 @@ public void testKeyExchange() throws Exception { TokenService tokenService = createTokenService(tokenServiceEnabledSettings, systemUTC()); // This test only makes sense in mixed clusters with pre v7.2.0 nodes where the Key is actually used if (null == oldNode) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_1_0)); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_7_0_0, Version.V_7_1_0)); } int numRotations = randomIntBetween(1, 5); for (int i = 0; i < numRotations; i++) { @@ -314,7 +314,7 @@ public void testPruneKeys() throws Exception { TokenService tokenService = createTokenService(tokenServiceEnabledSettings, systemUTC()); // This test only makes sense in mixed clusters with pre v7.2.0 nodes where the Key is actually used if (null == oldNode) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_1_0)); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_7_0_0, Version.V_7_1_0)); } Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); @@ -384,7 +384,7 @@ public void testPassphraseWorks() throws Exception { TokenService tokenService = createTokenService(tokenServiceEnabledSettings, systemUTC()); // This test only makes sense in mixed clusters with pre v7.1.0 nodes where the Key is actually used if (null == oldNode) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_1_0)); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_7_0_0, Version.V_7_1_0)); } Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null); PlainActionFuture> tokenFuture = new PlainActionFuture<>(); @@ -418,7 +418,7 @@ public void testGetTokenWhenKeyCacheHasExpired() throws Exception { TokenService tokenService = createTokenService(tokenServiceEnabledSettings, systemUTC()); // This test only makes sense in mixed clusters with pre v7.1.0 nodes where the Key is actually used if (null == oldNode) { - oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_6_7_0, Version.V_7_1_0)); + oldNode = addAnotherDataNodeWithVersion(this.clusterService, randomFrom(Version.V_7_0_0, Version.V_7_1_0)); } Authentication authentication = new Authentication(new User("joe", "admin"), new RealmRef("native_realm", "native", "node1"), null);