Skip to content

Commit e572214

Browse files
authored
Remove the 6.7 version constants. (#42039)
This PR removes all constants of the form `Version.V_6_7_*`, since master no longer needs to account for them. Relates to #41164.
1 parent 75be2a6 commit e572214

File tree

17 files changed

+37
-159
lines changed

17 files changed

+37
-159
lines changed

modules/reindex/src/main/java/org/elasticsearch/index/reindex/TransportUpdateByQueryAction.java

Lines changed: 2 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@
2020
package org.elasticsearch.index.reindex;
2121

2222
import org.apache.logging.log4j.Logger;
23-
import org.elasticsearch.Version;
2423
import org.elasticsearch.action.ActionListener;
2524
import org.elasticsearch.action.index.IndexRequest;
2625
import org.elasticsearch.action.support.ActionFilters;
@@ -82,18 +81,13 @@ protected void doExecute(Task task, UpdateByQueryRequest request, ActionListener
8281
*/
8382
static class AsyncIndexBySearchAction extends AbstractAsyncBulkByScrollAction<UpdateByQueryRequest, TransportUpdateByQueryAction> {
8483

85-
private final boolean useSeqNoForCAS;
86-
8784
AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
8885
ThreadPool threadPool, TransportUpdateByQueryAction action, UpdateByQueryRequest request, ClusterState clusterState,
8986
ActionListener<BulkByScrollResponse> listener) {
9087
super(task,
91-
// not all nodes support sequence number powered optimistic concurrency control, we fall back to version
92-
clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_6_7_0) == false,
93-
// all nodes support sequence number powered optimistic concurrency control and we can use it
94-
clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_6_7_0),
88+
// use sequence number powered optimistic concurrency control
89+
false, true,
9590
logger, client, threadPool, action, request, listener);
96-
useSeqNoForCAS = clusterState.nodes().getMinNodeVersion().onOrAfter(Version.V_6_7_0);
9791
}
9892

9993
@Override

server/src/main/java/org/elasticsearch/Build.java

Lines changed: 1 addition & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -254,12 +254,7 @@ public static void writeBuild(Build build, StreamOutput out) throws IOException
254254
out.writeString(build.flavor().displayName());
255255
}
256256
if (out.getVersion().onOrAfter(Version.V_6_3_0)) {
257-
final Type buildType;
258-
if (out.getVersion().before(Version.V_6_7_0) && build.type() == Type.DOCKER) {
259-
buildType = Type.TAR;
260-
} else {
261-
buildType = build.type();
262-
}
257+
final Type buildType = build.type();
263258
out.writeString(buildType.displayName());
264259
}
265260
out.writeString(build.shortHash());

server/src/main/java/org/elasticsearch/ElasticsearchException.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1010,19 +1010,19 @@ private enum ElasticsearchExceptionHandle {
10101010
COORDINATION_STATE_REJECTED_EXCEPTION(org.elasticsearch.cluster.coordination.CoordinationStateRejectedException.class,
10111011
org.elasticsearch.cluster.coordination.CoordinationStateRejectedException::new, 150, Version.V_7_0_0),
10121012
SNAPSHOT_IN_PROGRESS_EXCEPTION(org.elasticsearch.snapshots.SnapshotInProgressException.class,
1013-
org.elasticsearch.snapshots.SnapshotInProgressException::new, 151, Version.V_6_7_0),
1013+
org.elasticsearch.snapshots.SnapshotInProgressException::new, 151, UNKNOWN_VERSION_ADDED),
10141014
NO_SUCH_REMOTE_CLUSTER_EXCEPTION(org.elasticsearch.transport.NoSuchRemoteClusterException.class,
1015-
org.elasticsearch.transport.NoSuchRemoteClusterException::new, 152, Version.V_6_7_0),
1015+
org.elasticsearch.transport.NoSuchRemoteClusterException::new, 152, UNKNOWN_VERSION_ADDED),
10161016
RETENTION_LEASE_ALREADY_EXISTS_EXCEPTION(
10171017
org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException.class,
10181018
org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException::new,
10191019
153,
1020-
Version.V_6_7_0),
1020+
UNKNOWN_VERSION_ADDED),
10211021
RETENTION_LEASE_NOT_FOUND_EXCEPTION(
10221022
org.elasticsearch.index.seqno.RetentionLeaseNotFoundException.class,
10231023
org.elasticsearch.index.seqno.RetentionLeaseNotFoundException::new,
10241024
154,
1025-
Version.V_6_7_0),
1025+
UNKNOWN_VERSION_ADDED),
10261026
SHARD_NOT_IN_PRIMARY_MODE_EXCEPTION(
10271027
org.elasticsearch.index.shard.ShardNotInPrimaryModeException.class,
10281028
org.elasticsearch.index.shard.ShardNotInPrimaryModeException::new,

server/src/main/java/org/elasticsearch/Version.java

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -78,12 +78,6 @@ public class Version implements Comparable<Version>, ToXContentFragment {
7878
public static final Version V_6_6_1 = new Version(V_6_6_1_ID, org.apache.lucene.util.Version.LUCENE_7_6_0);
7979
public static final int V_6_6_2_ID = 6060299;
8080
public static final Version V_6_6_2 = new Version(V_6_6_2_ID, org.apache.lucene.util.Version.LUCENE_7_6_0);
81-
public static final int V_6_7_0_ID = 6070099;
82-
public static final Version V_6_7_0 = new Version(V_6_7_0_ID, org.apache.lucene.util.Version.LUCENE_7_7_0);
83-
public static final int V_6_7_1_ID = 6070199;
84-
public static final Version V_6_7_1 = new Version(V_6_7_1_ID, org.apache.lucene.util.Version.LUCENE_7_7_0);
85-
public static final int V_6_7_2_ID = 6070299;
86-
public static final Version V_6_7_2 = new Version(V_6_7_2_ID, org.apache.lucene.util.Version.LUCENE_7_7_0);
8781
public static final int V_6_8_0_ID = 6080099;
8882
public static final Version V_6_8_0 = new Version(V_6_8_0_ID, org.apache.lucene.util.Version.LUCENE_7_7_0);
8983
public static final int V_6_8_1_ID = 6080199;
@@ -130,12 +124,6 @@ public static Version fromId(int id) {
130124
return V_6_8_1;
131125
case V_6_8_0_ID:
132126
return V_6_8_0;
133-
case V_6_7_1_ID:
134-
return V_6_7_1;
135-
case V_6_7_2_ID:
136-
return V_6_7_2;
137-
case V_6_7_0_ID:
138-
return V_6_7_0;
139127
case V_6_6_2_ID:
140128
return V_6_6_2;
141129
case V_6_6_1_ID:

server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -131,7 +131,7 @@ protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(final SeqNoStats seqNoStat
131131
// that guarantee that all operations have been flushed to Lucene.
132132
final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated();
133133
if (indexVersionCreated.onOrAfter(Version.V_7_2_0) ||
134-
(seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO && indexVersionCreated.onOrAfter(Version.V_6_7_0))) {
134+
(seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO)) {
135135
if (seqNoStats.getMaxSeqNo() != seqNoStats.getGlobalCheckpoint()) {
136136
throw new IllegalStateException("Maximum sequence number [" + seqNoStats.getMaxSeqNo()
137137
+ "] from last commit does not match global checkpoint [" + seqNoStats.getGlobalCheckpoint() + "]");

server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@
3030
import org.apache.lucene.store.RateLimiter;
3131
import org.apache.lucene.util.ArrayUtil;
3232
import org.elasticsearch.ExceptionsHelper;
33-
import org.elasticsearch.Version;
3433
import org.elasticsearch.action.ActionListener;
3534
import org.elasticsearch.action.StepListener;
3635
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
@@ -112,8 +111,7 @@ public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recov
112111
this.shardId = this.request.shardId().id();
113112
this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName());
114113
this.chunkSizeInBytes = fileChunkSizeInBytes;
115-
// if the target is on an old version, it won't be able to handle out-of-order file chunks.
116-
this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(Version.V_6_7_0) ? maxConcurrentFileChunks : 1;
114+
this.maxConcurrentFileChunks = maxConcurrentFileChunks;
117115
}
118116

119117
public StartRecoveryRequest getRequest() {

server/src/test/java/org/elasticsearch/BuildTests.java

Lines changed: 1 addition & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,6 @@
3030
import java.io.InputStream;
3131
import java.net.URL;
3232
import java.util.Arrays;
33-
import java.util.List;
3433
import java.util.Objects;
3534
import java.util.Set;
3635
import java.util.stream.Collectors;
@@ -193,35 +192,7 @@ public void testSerialization() {
193192
throw new AssertionError();
194193
});
195194
}
196-
197-
public void testSerializationBWC() throws IOException {
198-
final WriteableBuild dockerBuild = new WriteableBuild(new Build(randomFrom(Build.Flavor.values()), Build.Type.DOCKER,
199-
randomAlphaOfLength(6), randomAlphaOfLength(6), randomBoolean(), randomAlphaOfLength(6)));
200-
201-
final List<Version> versions = Version.getDeclaredVersions(Version.class);
202-
final Version post63Pre67Version = randomFrom(versions.stream()
203-
.filter(v -> v.onOrAfter(Version.V_6_3_0) && v.before(Version.V_6_7_0)).collect(Collectors.toList()));
204-
final Version post67Pre70Version = randomFrom(versions.stream()
205-
.filter(v -> v.onOrAfter(Version.V_6_7_0) && v.before(Version.V_7_0_0)).collect(Collectors.toList()));
206-
final Version post70Version = randomFrom(versions.stream().filter(v -> v.onOrAfter(Version.V_7_0_0)).collect(Collectors.toList()));
207-
208-
final WriteableBuild post63pre67 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post63Pre67Version);
209-
final WriteableBuild post67pre70 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post67Pre70Version);
210-
final WriteableBuild post70 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post70Version);
211-
212-
assertThat(post63pre67.build.flavor(), equalTo(dockerBuild.build.flavor()));
213-
assertThat(post67pre70.build.flavor(), equalTo(dockerBuild.build.flavor()));
214-
assertThat(post70.build.flavor(), equalTo(dockerBuild.build.flavor()));
215-
216-
assertThat(post63pre67.build.type(), equalTo(Build.Type.TAR));
217-
assertThat(post67pre70.build.type(), equalTo(dockerBuild.build.type()));
218-
assertThat(post70.build.type(), equalTo(dockerBuild.build.type()));
219-
220-
assertThat(post63pre67.build.getQualifiedVersion(), equalTo(post63Pre67Version.toString()));
221-
assertThat(post67pre70.build.getQualifiedVersion(), equalTo(post67Pre70Version.toString()));
222-
assertThat(post70.build.getQualifiedVersion(), equalTo(dockerBuild.build.getQualifiedVersion()));
223-
}
224-
195+
225196
public void testFlavorParsing() {
226197
for (final Build.Flavor flavor : Build.Flavor.values()) {
227198
// strict or not should not impact parsing at all here

server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -888,7 +888,7 @@ public void testShardLockObtainFailedException() throws IOException {
888888

889889
public void testSnapshotInProgressException() throws IOException {
890890
SnapshotInProgressException orig = new SnapshotInProgressException("boom");
891-
Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_7_0, Version.CURRENT);
891+
Version version = VersionUtils.randomIndexCompatibleVersion(random());
892892
SnapshotInProgressException ex = serialize(orig, version);
893893
assertEquals(orig.getMessage(), ex.getMessage());
894894
}

server/src/test/java/org/elasticsearch/VersionTests.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -346,7 +346,6 @@ public static void assertUnknownVersion(Version version) {
346346
public void testIsCompatible() {
347347
assertTrue(isCompatible(Version.CURRENT, Version.CURRENT.minimumCompatibilityVersion()));
348348
assertFalse(isCompatible(Version.V_6_6_0, Version.V_7_0_0));
349-
assertFalse(isCompatible(Version.V_6_7_0, Version.V_7_0_0));
350349
assertTrue(isCompatible(Version.V_6_8_0, Version.V_7_0_0));
351350
assertFalse(isCompatible(Version.fromId(2000099), Version.V_7_0_0));
352351
assertFalse(isCompatible(Version.fromId(2000099), Version.V_6_5_0));

server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -519,11 +519,7 @@ public void testStartedShardEntrySerialization() throws Exception {
519519
final StartedShardEntry deserialized = new StartedShardEntry(in);
520520
assertThat(deserialized.shardId, equalTo(shardId));
521521
assertThat(deserialized.allocationId, equalTo(allocationId));
522-
if (version.onOrAfter(Version.V_6_7_0)) {
523-
assertThat(deserialized.primaryTerm, equalTo(primaryTerm));
524-
} else {
525-
assertThat(deserialized.primaryTerm, equalTo(0L));
526-
}
522+
assertThat(deserialized.primaryTerm, equalTo(primaryTerm));
527523
assertThat(deserialized.message, equalTo(message));
528524
}
529525
}

server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -130,7 +130,7 @@ private ClusterBlock randomClusterBlock() {
130130
}
131131

132132
private ClusterBlock randomClusterBlock(final Version version) {
133-
final String uuid = (version.onOrAfter(Version.V_6_7_0) && randomBoolean()) ? UUIDs.randomBase64UUID() : null;
133+
final String uuid = randomBoolean() ? UUIDs.randomBase64UUID() : null;
134134
final List<ClusterBlockLevel> levels = Arrays.asList(ClusterBlockLevel.values());
135135
return new ClusterBlock(randomInt(), uuid, "cluster block #" + randomInt(), randomBoolean(), randomBoolean(), randomBoolean(),
136136
randomFrom(RestStatus.values()), copyOf(randomSubsetOf(randomIntBetween(1, levels.size()), levels)));

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/AutoFollowStats.java

Lines changed: 8 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -116,15 +116,9 @@ public AutoFollowStats(StreamInput in) throws IOException {
116116
numberOfFailedFollowIndices = in.readVLong();
117117
numberOfFailedRemoteClusterStateRequests = in.readVLong();
118118
numberOfSuccessfulFollowIndices = in.readVLong();
119-
if (in.getVersion().onOrAfter(Version.V_6_7_0)) {
120-
// note: the casts to the following Writeable.Reader<T> instances are needed by some IDEs (e.g. Eclipse 4.8) as a compiler help
121-
recentAutoFollowErrors = new TreeMap<>(in.readMap((Writeable.Reader<String>) StreamInput::readString,
122-
(Writeable.Reader<Tuple<Long, ElasticsearchException>>) in1 -> new Tuple<>(in1.readZLong(), in1.readException())));
123-
} else {
124-
// note: the casts to the following Writeable.Reader<T> instances are needed by some IDEs (e.g. Eclipse 4.8) as a compiler help
125-
recentAutoFollowErrors = new TreeMap<>(in.readMap((Writeable.Reader<String>) StreamInput::readString,
126-
(Writeable.Reader<Tuple<Long, ElasticsearchException>>) in1 -> new Tuple<>(-1L, in1.readException())));
127-
}
119+
// note: the casts to the following Writeable.Reader<T> instances are needed by some IDEs (e.g. Eclipse 4.8) as a compiler help
120+
recentAutoFollowErrors = new TreeMap<>(in.readMap((Writeable.Reader<String>) StreamInput::readString,
121+
(Writeable.Reader<Tuple<Long, ElasticsearchException>>) in1 -> new Tuple<>(in1.readZLong(), in1.readException())));
128122
if (in.getVersion().onOrAfter(Version.V_6_6_0)) {
129123
autoFollowedClusters = new TreeMap<>(in.readMap(StreamInput::readString, AutoFollowedCluster::new));
130124
} else {
@@ -137,14 +131,11 @@ public void writeTo(StreamOutput out) throws IOException {
137131
out.writeVLong(numberOfFailedFollowIndices);
138132
out.writeVLong(numberOfFailedRemoteClusterStateRequests);
139133
out.writeVLong(numberOfSuccessfulFollowIndices);
140-
if (out.getVersion().onOrAfter(Version.V_6_7_0)) {
141-
out.writeMap(recentAutoFollowErrors, StreamOutput::writeString, (out1, value) -> {
142-
out1.writeZLong(value.v1());
143-
out1.writeException(value.v2());
144-
});
145-
} else {
146-
out.writeMap(recentAutoFollowErrors, StreamOutput::writeString, (out1, value) -> out1.writeException(value.v2()));
147-
}
134+
out.writeMap(recentAutoFollowErrors, StreamOutput::writeString, (out1, value) -> {
135+
out1.writeZLong(value.v1());
136+
out1.writeException(value.v2());
137+
});
138+
148139
if (out.getVersion().onOrAfter(Version.V_6_6_0)) {
149140
out.writeMap(autoFollowedClusters, StreamOutput::writeString, (out1, value) -> value.writeTo(out1));
150141
}

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutAutoFollowPatternAction.java

Lines changed: 2 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -5,14 +5,12 @@
55
*/
66
package org.elasticsearch.xpack.core.ccr.action;
77

8-
import org.elasticsearch.Version;
98
import org.elasticsearch.action.Action;
109
import org.elasticsearch.action.ActionRequestValidationException;
1110
import org.elasticsearch.action.support.master.AcknowledgedRequest;
1211
import org.elasticsearch.action.support.master.AcknowledgedResponse;
1312
import org.elasticsearch.common.io.stream.StreamInput;
1413
import org.elasticsearch.common.io.stream.StreamOutput;
15-
import org.elasticsearch.common.unit.ByteSizeValue;
1614
import org.elasticsearch.common.xcontent.ObjectParser;
1715
import org.elasticsearch.common.xcontent.ToXContentObject;
1816
import org.elasticsearch.common.xcontent.XContentBuilder;
@@ -153,21 +151,7 @@ public Request(StreamInput in) throws IOException {
153151
remoteCluster = in.readString();
154152
leaderIndexPatterns = in.readStringList();
155153
followIndexNamePattern = in.readOptionalString();
156-
if (in.getVersion().onOrAfter(Version.V_6_7_0)) {
157-
parameters = new FollowParameters(in);
158-
} else {
159-
parameters = new FollowParameters();
160-
parameters.maxReadRequestOperationCount = in.readOptionalVInt();
161-
parameters.maxReadRequestSize = in.readOptionalWriteable(ByteSizeValue::new);
162-
parameters.maxOutstandingReadRequests = in.readOptionalVInt();
163-
parameters.maxWriteRequestOperationCount = in.readOptionalVInt();
164-
parameters.maxWriteRequestSize = in.readOptionalWriteable(ByteSizeValue::new);
165-
parameters.maxOutstandingWriteRequests = in.readOptionalVInt();
166-
parameters.maxWriteBufferCount = in.readOptionalVInt();
167-
parameters.maxWriteBufferSize = in.readOptionalWriteable(ByteSizeValue::new);
168-
parameters.maxRetryDelay = in.readOptionalTimeValue();
169-
parameters.readPollTimeout = in.readOptionalTimeValue();
170-
}
154+
parameters = new FollowParameters(in);
171155
}
172156

173157
@Override
@@ -177,20 +161,7 @@ public void writeTo(StreamOutput out) throws IOException {
177161
out.writeString(remoteCluster);
178162
out.writeStringCollection(leaderIndexPatterns);
179163
out.writeOptionalString(followIndexNamePattern);
180-
if (out.getVersion().onOrAfter(Version.V_6_7_0)) {
181-
parameters.writeTo(out);
182-
} else {
183-
out.writeOptionalVInt(parameters.maxReadRequestOperationCount);
184-
out.writeOptionalWriteable(parameters.maxReadRequestSize);
185-
out.writeOptionalVInt(parameters.maxOutstandingReadRequests);
186-
out.writeOptionalVInt(parameters.maxWriteRequestOperationCount);
187-
out.writeOptionalWriteable(parameters.maxWriteRequestSize);
188-
out.writeOptionalVInt(parameters.maxOutstandingWriteRequests);
189-
out.writeOptionalVInt(parameters.maxWriteBufferCount);
190-
out.writeOptionalWriteable(parameters.maxWriteBufferSize);
191-
out.writeOptionalTimeValue(parameters.maxRetryDelay);
192-
out.writeOptionalTimeValue(parameters.readPollTimeout);
193-
}
164+
parameters.writeTo(out);
194165
}
195166

196167
@Override

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ccr/action/PutFollowAction.java

Lines changed: 2 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@
66

77
package org.elasticsearch.xpack.core.ccr.action;
88

9-
import org.elasticsearch.Version;
109
import org.elasticsearch.action.Action;
1110
import org.elasticsearch.action.ActionRequestValidationException;
1211
import org.elasticsearch.action.ActionResponse;
@@ -169,9 +168,7 @@ public Request(StreamInput in) throws IOException {
169168
this.leaderIndex = in.readString();
170169
this.followerIndex = in.readString();
171170
this.parameters = new FollowParameters(in);
172-
if (in.getVersion().onOrAfter(Version.V_6_7_0)) {
173-
waitForActiveShards(ActiveShardCount.readFrom(in));
174-
}
171+
waitForActiveShards(ActiveShardCount.readFrom(in));
175172
}
176173

177174
@Override
@@ -181,9 +178,7 @@ public void writeTo(StreamOutput out) throws IOException {
181178
out.writeString(leaderIndex);
182179
out.writeString(followerIndex);
183180
parameters.writeTo(out);
184-
if (out.getVersion().onOrAfter(Version.V_6_7_0)) {
185-
waitForActiveShards.writeTo(out);
186-
}
181+
waitForActiveShards.writeTo(out);
187182
}
188183

189184
@Override

0 commit comments

Comments
 (0)