From 46a209c099576f817cace2bebe55dc0878caea36 Mon Sep 17 00:00:00 2001 From: Armin Date: Thu, 14 Mar 2019 08:59:14 +0100 Subject: [PATCH] Remove Obsolete BwC Serialization Code * Same as #39879 and #39883 <- no need for BwC logic that covers pre-7.0 versions in 8.0 --- .../cluster/RestoreInProgress.java | 17 +--- .../cluster/metadata/IndexMetaData.java | 96 +++++-------------- .../metadata/IndexTemplateMetaData.java | 24 +---- .../cluster/metadata/MappingMetaData.java | 25 ----- .../cluster/metadata/MetaData.java | 38 ++------ .../cluster/routing/RecoverySource.java | 21 +--- .../org/elasticsearch/common/Rounding.java | 13 +-- .../common/geo/builders/ShapeBuilder.java | 19 ++-- .../elasticsearch/common/lucene/Lucene.java | 39 +++----- .../common/settings/Settings.java | 51 +++------- ...ransportNodesListGatewayStartedShards.java | 9 -- .../elasticsearch/index/engine/Segment.java | 27 ++---- .../elasticsearch/index/flush/FlushStats.java | 9 +- .../elasticsearch/index/get/GetResult.java | 16 +--- .../index/query/GeoShapeQueryBuilder.java | 13 +-- .../index/query/InnerHitBuilder.java | 46 ++------- .../index/query/MatchPhraseQueryBuilder.java | 9 +- .../index/query/MatchQueryBuilder.java | 21 +--- .../index/query/MultiMatchQueryBuilder.java | 32 ++----- .../index/query/QueryStringQueryBuilder.java | 13 +-- .../index/query/SimpleQueryStringBuilder.java | 21 ++-- .../index/query/TypeQueryBuilder.java | 14 +-- .../RandomScoreFunctionBuilder.java | 9 +- .../index/reindex/RemoteInfo.java | 11 +-- .../index/seqno/ReplicationTracker.java | 16 +--- .../elasticsearch/index/shard/DocsStats.java | 7 +- .../elasticsearch/index/store/StoreStats.java | 7 -- .../index/translog/Translog.java | 9 +- .../index/translog/TranslogStats.java | 24 ++--- .../elasticsearch/indices/TermsLookup.java | 34 +------ .../RecoveryFinalizeRecoveryRequest.java | 12 +-- ...ryPrepareForTranslogOperationsRequest.java | 18 +--- .../RecoveryTranslogOperationsRequest.java | 34 ++----- .../RecoveryTranslogOperationsResponse.java | 15 +-- .../recovery/StartRecoveryRequest.java | 12 +-- .../org/elasticsearch/ingest/IngestStats.java | 35 +++---- .../org/elasticsearch/monitor/os/OsInfo.java | 11 +-- .../org/elasticsearch/monitor/os/OsStats.java | 21 ++-- .../PersistentTasksCustomMetaData.java | 12 +-- .../persistent/StartPersistentTaskAction.java | 13 +-- .../org/elasticsearch/plugins/PluginInfo.java | 41 ++------ .../org/elasticsearch/script/ScriptStats.java | 7 +- .../org/elasticsearch/search/SearchHit.java | 13 +-- .../org/elasticsearch/search/SearchHits.java | 21 ++-- .../search/SearchSortValues.java | 11 +-- .../search/aggregations/InternalOrder.java | 72 +------------- .../bucket/composite/InternalComposite.java | 27 ++---- .../DateHistogramAggregationBuilder.java | 4 +- .../HistogramAggregationBuilder.java | 4 +- .../histogram/InternalDateHistogram.java | 4 +- .../bucket/histogram/InternalHistogram.java | 4 +- .../bucket/range/InternalBinaryRange.java | 12 +-- .../metrics/TopHitsAggregationBuilder.java | 9 +- ...tilesBucketPipelineAggregationBuilder.java | 12 +-- .../PercentilesBucketPipelineAggregator.java | 11 +-- .../support/MultiValuesSourceFieldConfig.java | 14 +-- .../ValuesSourceAggregationBuilder.java | 14 +-- .../search/builder/SearchSourceBuilder.java | 53 ++-------- .../internal/ShardSearchLocalRequest.java | 30 ++---- .../search/query/QuerySearchResult.java | 16 +--- .../search/sort/GeoDistanceSortBuilder.java | 16 +--- .../search/sort/NestedSortBuilder.java | 11 +-- .../search/sort/ScriptSortBuilder.java | 8 +- .../elasticsearch/search/suggest/Suggest.java | 64 ++----------- .../search/suggest/term/TermSuggestion.java | 10 +- .../snapshots/RestoreService.java | 7 -- .../elasticsearch/snapshots/SnapshotInfo.java | 10 +- .../org/elasticsearch/tasks/TaskInfo.java | 11 +-- .../transport/InboundMessage.java | 7 +- .../transport/RemoteConnectionInfo.java | 65 +------------ .../transport/TransportLogger.java | 4 +- .../common/rounding/RoundingDuelTests.java | 25 ----- .../common/settings/SettingsTests.java | 56 +---------- .../indices/TermsLookupTests.java | 8 -- .../ingest/IngestStatsTests.java | 23 ----- .../elasticsearch/search/SearchHitsTests.java | 37 ------- .../search/SearchSortValuesTests.java | 22 ----- .../aggregations/InternalOrderTests.java | 26 ----- .../ShardSearchTransportRequestTests.java | 13 --- .../RemoteClusterConnectionTests.java | 28 ------ 80 files changed, 283 insertions(+), 1420 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java b/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java index d71a3f94d4063..53e66b1790b3b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/RestoreInProgress.java @@ -21,7 +21,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState.Custom; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -38,18 +37,12 @@ import java.util.Iterator; import java.util.List; import java.util.Objects; -import java.util.UUID; /** * Meta data about restore processes that are currently executing */ public class RestoreInProgress extends AbstractNamedDiffable implements Custom, Iterable { - /** - * Fallback UUID used for restore operations that were started before v6.6 and don't have a uuid in the cluster state. - */ - public static final String BWC_UUID = new UUID(0, 0).toString(); - public static final String TYPE = "restore"; private final ImmutableOpenMap entries; @@ -436,11 +429,7 @@ public RestoreInProgress(StreamInput in) throws IOException { final ImmutableOpenMap.Builder entriesBuilder = ImmutableOpenMap.builder(count); for (int i = 0; i < count; i++) { final String uuid; - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - uuid = in.readString(); - } else { - uuid = BWC_UUID; - } + uuid = in.readString(); Snapshot snapshot = new Snapshot(in); State state = State.fromValue(in.readByte()); int indices = in.readVInt(); @@ -468,9 +457,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(entries.size()); for (ObjectCursor v : entries.values()) { Entry entry = v.value; - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeString(entry.uuid); - } + out.writeString(entry.uuid); entry.snapshot().writeTo(out); out.writeByte(entry.state().value()); out.writeVInt(entry.indices().size()); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index e349bd6fb7383..6fc73ef5f7fa8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -682,16 +682,8 @@ private static class IndexMetaDataDiff implements Diff { index = in.readString(); routingNumShards = in.readInt(); version = in.readLong(); - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - mappingVersion = in.readVLong(); - } else { - mappingVersion = 1; - } - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - settingsVersion = in.readVLong(); - } else { - settingsVersion = 1; - } + mappingVersion = in.readVLong(); + settingsVersion = in.readVLong(); state = State.fromId(in.readByte()); settings = Settings.readSettingsFromStream(in); primaryTerms = in.readVLongArray(); @@ -703,13 +695,8 @@ private static class IndexMetaDataDiff implements Diff { DiffableStringMap::readDiffFrom); inSyncAllocationIds = DiffableUtils.readImmutableOpenIntMapDiff(in, DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance()); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - rolloverInfos = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), RolloverInfo::new, - RolloverInfo::readDiffFrom); - } else { - ImmutableOpenMap emptyMap = ImmutableOpenMap.of(); - rolloverInfos = DiffableUtils.diff(emptyMap, emptyMap, DiffableUtils.getStringKeySerializer()); - } + rolloverInfos = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), RolloverInfo::new, + RolloverInfo::readDiffFrom); } @Override @@ -717,12 +704,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(index); out.writeInt(routingNumShards); out.writeLong(version); - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeVLong(mappingVersion); - } - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeVLong(settingsVersion); - } + out.writeVLong(mappingVersion); + out.writeVLong(settingsVersion); out.writeByte(state.id); Settings.writeSettingsToStream(settings, out); out.writeVLongArray(primaryTerms); @@ -730,9 +713,7 @@ public void writeTo(StreamOutput out) throws IOException { aliases.writeTo(out); customData.writeTo(out); inSyncAllocationIds.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - rolloverInfos.writeTo(out); - } + rolloverInfos.writeTo(out); } @Override @@ -757,16 +738,8 @@ public IndexMetaData apply(IndexMetaData part) { public static IndexMetaData readFrom(StreamInput in) throws IOException { Builder builder = new Builder(in.readString()); builder.version(in.readLong()); - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - builder.mappingVersion(in.readVLong()); - } else { - builder.mappingVersion(1); - } - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - builder.settingsVersion(in.readVLong()); - } else { - builder.settingsVersion(1); - } + builder.mappingVersion(in.readVLong()); + builder.settingsVersion(in.readVLong()); builder.setRoutingNumShards(in.readInt()); builder.state(State.fromId(in.readByte())); builder.settings(readSettingsFromStream(in)); @@ -782,17 +755,10 @@ public static IndexMetaData readFrom(StreamInput in) throws IOException { builder.putAlias(aliasMd); } int customSize = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - for (int i = 0; i < customSize; i++) { - String key = in.readString(); - DiffableStringMap custom = new DiffableStringMap(in); - builder.putCustom(key, custom); - } - } else { - assert customSize == 0 : "expected no custom index metadata"; - if (customSize > 0) { - throw new IllegalStateException("unexpected custom metadata when none is supported"); - } + for (int i = 0; i < customSize; i++) { + String key = in.readString(); + DiffableStringMap custom = new DiffableStringMap(in); + builder.putCustom(key, custom); } int inSyncAllocationIdsSize = in.readVInt(); for (int i = 0; i < inSyncAllocationIdsSize; i++) { @@ -800,11 +766,9 @@ public static IndexMetaData readFrom(StreamInput in) throws IOException { Set allocationIds = DiffableUtils.StringSetValueSerializer.getInstance().read(in, key); builder.putInSyncAllocationIds(key, allocationIds); } - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - int rolloverAliasesSize = in.readVInt(); - for (int i = 0; i < rolloverAliasesSize; i++) { - builder.putRolloverInfo(new RolloverInfo(in)); - } + int rolloverAliasesSize = in.readVInt(); + for (int i = 0; i < rolloverAliasesSize; i++) { + builder.putRolloverInfo(new RolloverInfo(in)); } return builder.build(); } @@ -813,12 +777,8 @@ public static IndexMetaData readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeString(index.getName()); // uuid will come as part of settings out.writeLong(version); - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeVLong(mappingVersion); - } - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeVLong(settingsVersion); - } + out.writeVLong(mappingVersion); + out.writeVLong(settingsVersion); out.writeInt(routingNumShards); out.writeByte(state.id()); writeSettingsToStream(settings, out); @@ -831,25 +791,19 @@ public void writeTo(StreamOutput out) throws IOException { for (ObjectCursor cursor : aliases.values()) { cursor.value.writeTo(out); } - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeVInt(customData.size()); - for (final ObjectObjectCursor cursor : customData) { - out.writeString(cursor.key); - cursor.value.writeTo(out); - } - } else { - out.writeVInt(0); + out.writeVInt(customData.size()); + for (final ObjectObjectCursor cursor : customData) { + out.writeString(cursor.key); + cursor.value.writeTo(out); } out.writeVInt(inSyncAllocationIds.size()); for (IntObjectCursor> cursor : inSyncAllocationIds) { out.writeVInt(cursor.key); DiffableUtils.StringSetValueSerializer.getInstance().write(cursor.value, out); } - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeVInt(rolloverInfos.size()); - for (ObjectCursor cursor : rolloverInfos.values()) { - cursor.value.writeTo(out); - } + out.writeVInt(rolloverInfos.size()); + for (ObjectCursor cursor : rolloverInfos.values()) { + cursor.value.writeTo(out); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index f61b2fc208f83..1c84e8ce9fc6a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -22,7 +22,6 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.apache.logging.log4j.LogManager; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diff; import org.elasticsearch.common.Nullable; @@ -188,11 +187,7 @@ public int hashCode() { public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException { Builder builder = new Builder(in.readString()); builder.order(in.readInt()); - if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - builder.patterns(in.readStringList()); - } else { - builder.patterns(Collections.singletonList(in.readString())); - } + builder.patterns(in.readStringList()); builder.settings(Settings.readSettingsFromStream(in)); int mappingsSize = in.readVInt(); for (int i = 0; i < mappingsSize; i++) { @@ -203,14 +198,6 @@ public static IndexTemplateMetaData readFrom(StreamInput in) throws IOException AliasMetaData aliasMd = new AliasMetaData(in); builder.putAlias(aliasMd); } - if (in.getVersion().before(Version.V_6_5_0)) { - // Previously we allowed custom metadata - int customSize = in.readVInt(); - assert customSize == 0 : "expected no custom metadata"; - if (customSize > 0) { - throw new IllegalStateException("unexpected custom metadata when none is supported"); - } - } builder.version(in.readOptionalVInt()); return builder.build(); } @@ -223,11 +210,7 @@ public static Diff readDiffFrom(StreamInput in) throws IO public void writeTo(StreamOutput out) throws IOException { out.writeString(name); out.writeInt(order); - if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - out.writeStringCollection(patterns); - } else { - out.writeString(patterns.get(0)); - } + out.writeStringCollection(patterns); Settings.writeSettingsToStream(settings, out); out.writeVInt(mappings.size()); for (ObjectObjectCursor cursor : mappings) { @@ -238,9 +221,6 @@ public void writeTo(StreamOutput out) throws IOException { for (ObjectCursor cursor : aliases.values()) { cursor.value.writeTo(out); } - if (out.getVersion().before(Version.V_6_5_0)) { - out.writeVInt(0); - } out.writeOptionalVInt(version); } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java index 3734b9b599a02..f983647677f8f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MappingMetaData.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.metadata; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.Diff; import org.elasticsearch.common.bytes.BytesReference; @@ -30,7 +29,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.DocumentMapper; import java.io.IOException; @@ -171,16 +169,6 @@ public void writeTo(StreamOutput out) throws IOException { source().writeTo(out); // routing out.writeBoolean(routing().required()); - if (out.getVersion().before(Version.V_6_0_0_alpha1)) { - // timestamp - out.writeBoolean(false); // enabled - out.writeString(DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER.pattern()); - out.writeOptionalString("now"); // 5.x default - out.writeOptionalBoolean(null); - } - if (out.getVersion().before(Version.V_7_0_0)) { - out.writeBoolean(false); // hasParentField - } } @Override @@ -210,19 +198,6 @@ public MappingMetaData(StreamInput in) throws IOException { source = CompressedXContent.readCompressedString(in); // routing routing = new Routing(in.readBoolean()); - if (in.getVersion().before(Version.V_6_0_0_alpha1)) { - // timestamp - boolean enabled = in.readBoolean(); - if (enabled) { - throw new IllegalArgumentException("_timestamp may not be enabled"); - } - in.readString(); // format - in.readOptionalString(); // defaultTimestamp - in.readOptionalBoolean(); // ignoreMissing - } - if (in.getVersion().before(Version.V_7_0_0)) { - in.readBoolean(); // hasParentField - } } public static Diff readDiffFrom(StreamInput in) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index 54c3001d9036f..720c6f7d8ebae 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -22,11 +22,9 @@ import com.carrotsearch.hppc.ObjectHashSet; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.Version; import org.elasticsearch.action.AliasesRequest; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState.FeatureAware; @@ -837,15 +835,9 @@ private static class MetaDataDiff implements Diff { MetaDataDiff(StreamInput in) throws IOException { clusterUUID = in.readString(); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - clusterUUIDCommitted = in.readBoolean(); - } + clusterUUIDCommitted = in.readBoolean(); version = in.readLong(); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - coordinationMetaData = new CoordinationMetaData(in); - } else { - coordinationMetaData = CoordinationMetaData.EMPTY_META_DATA; - } + coordinationMetaData = new CoordinationMetaData(in); transientSettings = Settings.readSettingsFromStream(in); persistentSettings = Settings.readSettingsFromStream(in); indices = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), IndexMetaData::readFrom, @@ -858,13 +850,9 @@ private static class MetaDataDiff implements Diff { @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(clusterUUID); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeBoolean(clusterUUIDCommitted); - } + out.writeBoolean(clusterUUIDCommitted); out.writeLong(version); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - coordinationMetaData.writeTo(out); - } + coordinationMetaData.writeTo(out); Settings.writeSettingsToStream(transientSettings, out); Settings.writeSettingsToStream(persistentSettings, out); indices.writeTo(out); @@ -892,12 +880,8 @@ public static MetaData readFrom(StreamInput in) throws IOException { Builder builder = new Builder(); builder.version = in.readLong(); builder.clusterUUID = in.readString(); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - builder.clusterUUIDCommitted = in.readBoolean(); - } - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - builder.coordinationMetaData(new CoordinationMetaData(in)); - } + builder.clusterUUIDCommitted = in.readBoolean(); + builder.coordinationMetaData(new CoordinationMetaData(in)); builder.transientSettings(readSettingsFromStream(in)); builder.persistentSettings(readSettingsFromStream(in)); int size = in.readVInt(); @@ -920,12 +904,8 @@ public static MetaData readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeString(clusterUUID); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeBoolean(clusterUUIDCommitted); - } - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - coordinationMetaData.writeTo(out); - } + out.writeBoolean(clusterUUIDCommitted); + coordinationMetaData.writeTo(out); writeSettingsToStream(transientSettings, out); writeSettingsToStream(persistentSettings, out); out.writeVInt(indices.size()); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java b/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java index 25a605088ef66..94cd536f63805 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RecoverySource.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.routing; import org.elasticsearch.Version; -import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -148,11 +147,7 @@ private ExistingStoreRecoverySource(boolean bootstrapNewHistoryUUID) { } private ExistingStoreRecoverySource(StreamInput in) throws IOException { - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - bootstrapNewHistoryUUID = in.readBoolean(); - } else { - bootstrapNewHistoryUUID = false; - } + bootstrapNewHistoryUUID = in.readBoolean(); } @Override @@ -162,9 +157,7 @@ public void addAdditionalFields(XContentBuilder builder, Params params) throws I @Override protected void writeAdditionalFields(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeBoolean(bootstrapNewHistoryUUID); - } + out.writeBoolean(bootstrapNewHistoryUUID); } @Override @@ -222,11 +215,7 @@ public SnapshotRecoverySource(String restoreUUID, Snapshot snapshot, Version ver } SnapshotRecoverySource(StreamInput in) throws IOException { - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - restoreUUID = in.readString(); - } else { - restoreUUID = RestoreInProgress.BWC_UUID; - } + restoreUUID = in.readString(); snapshot = new Snapshot(in); version = Version.readVersion(in); index = in.readString(); @@ -250,9 +239,7 @@ public Version version() { @Override protected void writeAdditionalFields(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeString(restoreUUID); - } + out.writeString(restoreUUID); snapshot.writeTo(out); Version.writeVersion(version, out); out.writeString(index); diff --git a/server/src/main/java/org/elasticsearch/common/Rounding.java b/server/src/main/java/org/elasticsearch/common/Rounding.java index 3558b16aac1c8..3d5298c9c1501 100644 --- a/server/src/main/java/org/elasticsearch/common/Rounding.java +++ b/server/src/main/java/org/elasticsearch/common/Rounding.java @@ -19,7 +19,6 @@ package org.elasticsearch.common; 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; @@ -397,11 +396,7 @@ public long nextRoundingValue(long utcMillis) { @Override public void innerWriteTo(StreamOutput out) throws IOException { out.writeByte(unit.getId()); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeString(timeZone.getId()); - } else { - out.writeString(DateUtils.zoneIdToDateTimeZone(timeZone).getID()); - } + out.writeString(timeZone.getId()); } @Override @@ -522,11 +517,7 @@ public long nextRoundingValue(long time) { @Override public void innerWriteTo(StreamOutput out) throws IOException { out.writeVLong(interval); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeString(timeZone.getId()); - } else { - out.writeString(DateUtils.zoneIdToDateTimeZone(timeZone).getID()); - } + out.writeString(timeZone.getId()); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilder.java b/server/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilder.java index d6ba295be67e3..7d0517135266c 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/geo/builders/ShapeBuilder.java @@ -19,14 +19,9 @@ package org.elasticsearch.common.geo.builders; -import org.locationtech.jts.geom.Coordinate; -import org.locationtech.jts.geom.Geometry; -import org.locationtech.jts.geom.GeometryFactory; - -import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.Assertions; -import org.elasticsearch.Version; import org.elasticsearch.common.Strings; import org.elasticsearch.common.geo.GeoShapeType; import org.elasticsearch.common.geo.parsers.GeoWKTParser; @@ -35,6 +30,9 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.locationtech.jts.geom.Coordinate; +import org.locationtech.jts.geom.Geometry; +import org.locationtech.jts.geom.GeometryFactory; import org.locationtech.spatial4j.context.jts.JtsSpatialContext; import org.locationtech.spatial4j.exception.InvalidShapeException; import org.locationtech.spatial4j.shape.Shape; @@ -113,10 +111,7 @@ protected ShapeBuilder(StreamInput in) throws IOException { protected static Coordinate readFromStream(StreamInput in) throws IOException { double x = in.readDouble(); double y = in.readDouble(); - Double z = null; - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - z = in.readOptionalDouble(); - } + Double z = in.readOptionalDouble(); return z == null ? new Coordinate(x, y) : new Coordinate(x, y, z); } @@ -131,9 +126,7 @@ public void writeTo(StreamOutput out) throws IOException { protected static void writeCoordinateTo(Coordinate coordinate, StreamOutput out) throws IOException { out.writeDouble(coordinate.x); out.writeDouble(coordinate.y); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeOptionalDouble(Double.isNaN(coordinate.z) ? null : coordinate.z); - } + out.writeOptionalDouble(Double.isNaN(coordinate.z) ? null : coordinate.z); } @SuppressWarnings("unchecked") diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 4d4a2d838dbd3..88d7c57f5e424 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -303,10 +303,7 @@ public static boolean exists(IndexSearcher searcher, Query query) throws IOExcep public static TotalHits readTotalHits(StreamInput in) throws IOException { long totalHits = in.readVLong(); - TotalHits.Relation totalHitsRelation = TotalHits.Relation.EQUAL_TO; - if (in.getVersion().onOrAfter(org.elasticsearch.Version.V_7_0_0)) { - totalHitsRelation = in.readEnum(TotalHits.Relation.class); - } + TotalHits.Relation totalHitsRelation = in.readEnum(TotalHits.Relation.class); return new TotalHits(totalHits, totalHitsRelation); } @@ -415,11 +412,7 @@ public static ScoreDoc readScoreDoc(StreamInput in) throws IOException { public static void writeTotalHits(StreamOutput out, TotalHits totalHits) throws IOException { out.writeVLong(totalHits.value); - if (out.getVersion().onOrAfter(org.elasticsearch.Version.V_7_0_0)) { - out.writeEnum(totalHits.relation); - } else if (totalHits.value > 0 && totalHits.relation != TotalHits.Relation.EQUAL_TO) { - throw new IllegalArgumentException("Cannot serialize approximate total hit counts to nodes that are on a version < 7.0.0"); - } + out.writeEnum(totalHits.relation); } public static void writeTopDocs(StreamOutput out, TopDocsAndMaxScore topDocs) throws IOException { @@ -613,9 +606,8 @@ public static void writeSortField(StreamOutput out, SortField sortField) throws } private static Number readExplanationValue(StreamInput in) throws IOException { - if (in.getVersion().onOrAfter(org.elasticsearch.Version.V_7_0_0)) { - final int numberType = in.readByte(); - switch (numberType) { + final int numberType = in.readByte(); + switch (numberType) { case 0: return in.readFloat(); case 1: @@ -624,9 +616,6 @@ private static Number readExplanationValue(StreamInput in) throws IOException { return in.readZLong(); default: throw new IOException("Unexpected number type: " + numberType); - } - } else { - return in.readFloat(); } } @@ -645,19 +634,15 @@ public static Explanation readExplanation(StreamInput in) throws IOException { } private static void writeExplanationValue(StreamOutput out, Number value) throws IOException { - if (out.getVersion().onOrAfter(org.elasticsearch.Version.V_7_0_0)) { - if (value instanceof Float) { - out.writeByte((byte) 0); - out.writeFloat(value.floatValue()); - } else if (value instanceof Double) { - out.writeByte((byte) 1); - out.writeDouble(value.doubleValue()); - } else { - out.writeByte((byte) 2); - out.writeZLong(value.longValue()); - } - } else { + if (value instanceof Float) { + out.writeByte((byte) 0); out.writeFloat(value.floatValue()); + } else if (value instanceof Double) { + out.writeByte((byte) 1); + out.writeDouble(value.doubleValue()); + } else { + out.writeByte((byte) 2); + out.writeZLong(value.longValue()); } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/Settings.java b/server/src/main/java/org/elasticsearch/common/settings/Settings.java index 72f9406edac4c..3a7d719105afe 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Settings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Settings.java @@ -530,23 +530,15 @@ public int hashCode() { public static Settings readSettingsFromStream(StreamInput in) throws IOException { Builder builder = new Builder(); int numberOfSettings = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - for (int i = 0; i < numberOfSettings; i++) { - String key = in.readString(); - Object value = in.readGenericValue(); - if (value == null) { - builder.putNull(key); - } else if (value instanceof List) { - builder.putList(key, (List) value); - } else { - builder.put(key, value.toString()); - } - } - } else { - for (int i = 0; i < numberOfSettings; i++) { - String key = in.readString(); - String value = in.readOptionalString(); - builder.put(key, value); + for (int i = 0; i < numberOfSettings; i++) { + String key = in.readString(); + Object value = in.readGenericValue(); + if (value == null) { + builder.putNull(key); + } else if (value instanceof List) { + builder.putList(key, (List) value); + } else { + builder.put(key, value.toString()); } } return builder.build(); @@ -555,27 +547,10 @@ public static Settings readSettingsFromStream(StreamInput in) throws IOException public static void writeSettingsToStream(Settings settings, StreamOutput out) throws IOException { // pull settings to exclude secure settings in size() Set> entries = settings.settings.entrySet(); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeVInt(entries.size()); - for (Map.Entry entry : entries) { - out.writeString(entry.getKey()); - out.writeGenericValue(entry.getValue()); - } - } else { - int size = entries.stream().mapToInt(e -> e.getValue() instanceof List ? ((List)e.getValue()).size() : 1).sum(); - out.writeVInt(size); - for (Map.Entry entry : entries) { - if (entry.getValue() instanceof List) { - int idx = 0; - for (String value : (List)entry.getValue()) { - out.writeString(entry.getKey() + "." + idx++); - out.writeOptionalString(value); - } - } else { - out.writeString(entry.getKey()); - out.writeOptionalString(toString(entry.getValue())); - } - } + out.writeVInt(entries.size()); + for (Map.Entry entry : entries) { + out.writeString(entry.getKey()); + out.writeGenericValue(entry.getValue()); } } diff --git a/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java b/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java index b945d9827caaf..e2eb75458c0cc 100644 --- a/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java +++ b/server/src/main/java/org/elasticsearch/gateway/TransportNodesListGatewayStartedShards.java @@ -21,7 +21,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.support.ActionFilters; @@ -279,10 +278,6 @@ public Exception storeException() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - if (in.getVersion().before(Version.V_6_0_0_alpha1)) { - // legacy version - in.readLong(); - } allocationId = in.readOptionalString(); primary = in.readBoolean(); if (in.readBoolean()) { @@ -293,10 +288,6 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - if (out.getVersion().before(Version.V_6_0_0_alpha1)) { - // legacy version - out.writeLong(-1L); - } out.writeOptionalString(allocationId); out.writeBoolean(primary); if (storeException != null) { diff --git a/server/src/main/java/org/elasticsearch/index/engine/Segment.java b/server/src/main/java/org/elasticsearch/index/engine/Segment.java index fa15e7dc09e8b..945359eda1b17 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Segment.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Segment.java @@ -21,13 +21,12 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.SortField; -import org.apache.lucene.search.SortedSetSortField; +import org.apache.lucene.search.SortedNumericSelector; import org.apache.lucene.search.SortedNumericSortField; import org.apache.lucene.search.SortedSetSelector; -import org.apache.lucene.search.SortedNumericSelector; +import org.apache.lucene.search.SortedSetSortField; import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountables; -import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -178,12 +177,8 @@ public void readFrom(StreamInput in) throws IOException { // verbose mode ramTree = readRamTree(in); } - if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - segmentSort = readSegmentSort(in); - } else { - segmentSort = null; - } - if (in.getVersion().onOrAfter(Version.V_6_1_0) && in.readBoolean()) { + segmentSort = readSegmentSort(in); + if (in.readBoolean()) { attributes = in.readMap(StreamInput::readString, StreamInput::readString); } else { attributes = null; @@ -208,15 +203,11 @@ public void writeTo(StreamOutput out) throws IOException { if (verbose) { writeRamTree(out, ramTree); } - if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - writeSegmentSort(out, segmentSort); - } - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - boolean hasAttributes = attributes != null; - out.writeBoolean(hasAttributes); - if (hasAttributes) { - out.writeMap(attributes, StreamOutput::writeString, StreamOutput::writeString); - } + writeSegmentSort(out, segmentSort); + boolean hasAttributes = attributes != null; + out.writeBoolean(hasAttributes); + if (hasAttributes) { + out.writeMap(attributes, StreamOutput::writeString, StreamOutput::writeString); } } diff --git a/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java b/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java index 914a3bf87a258..92887f4fb2e69 100644 --- a/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java +++ b/server/src/main/java/org/elasticsearch/index/flush/FlushStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.flush; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -43,9 +42,7 @@ public FlushStats() { public FlushStats(StreamInput in) throws IOException { total = in.readVLong(); totalTimeInMillis = in.readVLong(); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - periodic = in.readVLong(); - } + periodic = in.readVLong(); } public FlushStats(long total, long periodic, long totalTimeInMillis) { @@ -128,8 +125,6 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeVLong(total); out.writeVLong(totalTimeInMillis); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeVLong(periodic); - } + out.writeVLong(periodic); } } diff --git a/server/src/main/java/org/elasticsearch/index/get/GetResult.java b/server/src/main/java/org/elasticsearch/index/get/GetResult.java index 869bc548f899f..5769b659e40b3 100644 --- a/server/src/main/java/org/elasticsearch/index/get/GetResult.java +++ b/server/src/main/java/org/elasticsearch/index/get/GetResult.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.get; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.Version; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; @@ -385,13 +384,8 @@ public void readFrom(StreamInput in) throws IOException { index = in.readString(); type = in.readOptionalString(); id = in.readString(); - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - seqNo = in.readZLong(); - primaryTerm = in.readVLong(); - } else { - seqNo = UNASSIGNED_SEQ_NO; - primaryTerm = UNASSIGNED_PRIMARY_TERM; - } + seqNo = in.readZLong(); + primaryTerm = in.readVLong(); version = in.readLong(); exists = in.readBoolean(); if (exists) { @@ -417,10 +411,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(index); out.writeOptionalString(type); out.writeString(id); - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeZLong(seqNo); - out.writeVLong(primaryTerm); - } + out.writeZLong(seqNo); + out.writeVLong(primaryTerm); out.writeLong(version); out.writeBoolean(exists); if (exists) { diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java index 7d082291069ab..b13666296b55d 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java @@ -33,7 +33,6 @@ import org.apache.lucene.spatial.query.SpatialArgs; import org.apache.lucene.spatial.query.SpatialOperation; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; @@ -205,11 +204,7 @@ public GeoShapeQueryBuilder(StreamInput in) throws IOException { indexedShapeType = in.readOptionalString(); indexedShapeIndex = in.readOptionalString(); indexedShapePath = in.readOptionalString(); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - indexedShapeRouting = in.readOptionalString(); - } else { - indexedShapeRouting = null; - } + indexedShapeRouting = in.readOptionalString(); } relation = ShapeRelation.readFromStream(in); strategy = in.readOptionalWriteable(SpatialStrategy::readFromStream); @@ -232,11 +227,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeOptionalString(indexedShapeType); out.writeOptionalString(indexedShapeIndex); out.writeOptionalString(indexedShapePath); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeOptionalString(indexedShapeRouting); - } else if (indexedShapeRouting != null) { - throw new IllegalStateException("indexed shape routing cannot be serialized to older nodes"); - } + out.writeOptionalString(indexedShapeRouting); } relation.writeTo(out); out.writeOptionalWriteable(strategy); diff --git a/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java b/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java index d72134198b8e8..b4631787af0f7 100644 --- a/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.index.query; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -47,7 +46,6 @@ import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; import static org.elasticsearch.common.xcontent.XContentParser.Token.END_OBJECT; @@ -156,26 +154,10 @@ public InnerHitBuilder(StreamInput in) throws IOException { size = in.readVInt(); explain = in.readBoolean(); version = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)){ - seqNoAndPrimaryTerm = in.readBoolean(); - } else { - seqNoAndPrimaryTerm = false; - } + seqNoAndPrimaryTerm = in.readBoolean(); trackScores = in.readBoolean(); storedFieldsContext = in.readOptionalWriteable(StoredFieldsContext::new); - if (in.getVersion().before(Version.V_6_4_0)) { - @SuppressWarnings("unchecked") - List fieldList = (List) in.readGenericValue(); - if (fieldList == null) { - docValueFields = null; - } else { - docValueFields = fieldList.stream() - .map(field -> new FieldAndFormat(field, null)) - .collect(Collectors.toList()); - } - } else { - docValueFields = in.readBoolean() ? in.readList(FieldAndFormat::new) : null; - } + docValueFields = in.readBoolean() ? in.readList(FieldAndFormat::new) : null; if (in.readBoolean()) { int size = in.readVInt(); scriptFields = new HashSet<>(size); @@ -192,9 +174,7 @@ public InnerHitBuilder(StreamInput in) throws IOException { } } highlightBuilder = in.readOptionalWriteable(HighlightBuilder::new); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - this.innerCollapseBuilder = in.readOptionalWriteable(CollapseBuilder::new); - } + this.innerCollapseBuilder = in.readOptionalWriteable(CollapseBuilder::new); } @Override @@ -205,20 +185,12 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(size); out.writeBoolean(explain); out.writeBoolean(version); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeBoolean(seqNoAndPrimaryTerm); - } + out.writeBoolean(seqNoAndPrimaryTerm); out.writeBoolean(trackScores); out.writeOptionalWriteable(storedFieldsContext); - if (out.getVersion().before(Version.V_6_4_0)) { - out.writeGenericValue(docValueFields == null - ? null - : docValueFields.stream().map(ff -> ff.field).collect(Collectors.toList())); - } else { - out.writeBoolean(docValueFields != null); - if (docValueFields != null) { - out.writeList(docValueFields); - } + out.writeBoolean(docValueFields != null); + if (docValueFields != null) { + out.writeList(docValueFields); } boolean hasScriptFields = scriptFields != null; out.writeBoolean(hasScriptFields); @@ -240,9 +212,7 @@ public void writeTo(StreamOutput out) throws IOException { } } out.writeOptionalWriteable(highlightBuilder); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeOptionalWriteable(innerCollapseBuilder); - } + out.writeOptionalWriteable(innerCollapseBuilder); } public String getName() { diff --git a/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java index 7dc01bb34503d..9cd7ae8d20677 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MatchPhraseQueryBuilder.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.query; import org.apache.lucene.search.Query; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -72,9 +71,7 @@ public MatchPhraseQueryBuilder(StreamInput in) throws IOException { fieldName = in.readString(); value = in.readGenericValue(); slop = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - zeroTermsQuery = ZeroTermsQuery.readFromStream(in); - } + zeroTermsQuery = ZeroTermsQuery.readFromStream(in); analyzer = in.readOptionalString(); } @@ -83,9 +80,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeString(fieldName); out.writeGenericValue(value); out.writeVInt(slop); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - zeroTermsQuery.writeTo(out); - } + zeroTermsQuery.writeTo(out); out.writeOptionalString(analyzer); } diff --git a/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java index 32d0fbbb37e9d..47ded82657d42 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MatchQueryBuilder.java @@ -21,7 +21,6 @@ import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -111,13 +110,7 @@ public MatchQueryBuilder(StreamInput in) throws IOException { super(in); fieldName = in.readString(); value = in.readGenericValue(); - if (in.getVersion().before(Version.V_6_0_0_rc1)) { - MatchQuery.Type.readFromStream(in); // deprecated type - } operator = Operator.readFromStream(in); - if (in.getVersion().before(Version.V_6_0_0_rc1)) { - in.readVInt(); // deprecated slop - } prefixLength = in.readVInt(); maxExpansions = in.readVInt(); fuzzyTranspositions = in.readBoolean(); @@ -129,22 +122,14 @@ public MatchQueryBuilder(StreamInput in) throws IOException { fuzzyRewrite = in.readOptionalString(); fuzziness = in.readOptionalWriteable(Fuzziness::new); cutoffFrequency = in.readOptionalFloat(); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - autoGenerateSynonymsPhraseQuery = in.readBoolean(); - } + autoGenerateSynonymsPhraseQuery = in.readBoolean(); } @Override protected void doWriteTo(StreamOutput out) throws IOException { out.writeString(fieldName); out.writeGenericValue(value); - if (out.getVersion().before(Version.V_6_0_0_rc1)) { - MatchQuery.Type.BOOLEAN.writeTo(out); // deprecated type - } operator.writeTo(out); - if (out.getVersion().before(Version.V_6_0_0_rc1)) { - out.writeVInt(MatchQuery.DEFAULT_PHRASE_SLOP); // deprecated slop - } out.writeVInt(prefixLength); out.writeVInt(maxExpansions); out.writeBoolean(fuzzyTranspositions); @@ -156,9 +141,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeOptionalString(fuzzyRewrite); out.writeOptionalWriteable(fuzziness); out.writeOptionalFloat(cutoffFrequency); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeBoolean(autoGenerateSynonymsPhraseQuery); - } + out.writeBoolean(autoGenerateSynonymsPhraseQuery); } /** Returns the field name used in this query. */ diff --git a/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java index 9f2c85106de08..0e9148e540102 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java @@ -22,7 +22,6 @@ import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -225,21 +224,12 @@ public MultiMatchQueryBuilder(StreamInput in) throws IOException { maxExpansions = in.readVInt(); minimumShouldMatch = in.readOptionalString(); fuzzyRewrite = in.readOptionalString(); - if (in.getVersion().before(Version.V_7_0_0)) { - in.readOptionalBoolean(); // unused use_dis_max flag - } tieBreaker = in.readOptionalFloat(); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - lenient = in.readOptionalBoolean(); - } else { - lenient = in.readBoolean(); - } + lenient = in.readOptionalBoolean(); cutoffFrequency = in.readOptionalFloat(); zeroTermsQuery = MatchQuery.ZeroTermsQuery.readFromStream(in); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - autoGenerateSynonymsPhraseQuery = in.readBoolean(); - fuzzyTranspositions = in.readBoolean(); - } + autoGenerateSynonymsPhraseQuery = in.readBoolean(); + fuzzyTranspositions = in.readBoolean(); } @Override @@ -259,21 +249,12 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeVInt(maxExpansions); out.writeOptionalString(minimumShouldMatch); out.writeOptionalString(fuzzyRewrite); - if (out.getVersion().before(Version.V_7_0_0)) { - out.writeOptionalBoolean(null); - } out.writeOptionalFloat(tieBreaker); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeOptionalBoolean(lenient); - } else { - out.writeBoolean(lenient == null ? MatchQuery.DEFAULT_LENIENCY : lenient); - } + out.writeOptionalBoolean(lenient); out.writeOptionalFloat(cutoffFrequency); zeroTermsQuery.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeBoolean(autoGenerateSynonymsPhraseQuery); - out.writeBoolean(fuzzyTranspositions); - } + out.writeBoolean(autoGenerateSynonymsPhraseQuery); + out.writeBoolean(fuzzyTranspositions); } public Object value() { @@ -616,7 +597,6 @@ public static MultiMatchQueryBuilder fromXContent(XContentParser parser) throws Operator operator = DEFAULT_OPERATOR; String minimumShouldMatch = null; String fuzzyRewrite = null; - Boolean useDisMax = null; Float tieBreaker = null; Float cutoffFrequency = null; Boolean lenient = null; diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java index 3f8a0acc91695..f129ccbec7254 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryStringQueryBuilder.java @@ -23,7 +23,6 @@ import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; import org.apache.lucene.util.automaton.Operations; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -195,10 +194,8 @@ public QueryStringQueryBuilder(StreamInput in) throws IOException { timeZone = in.readOptionalZoneId(); escape = in.readBoolean(); maxDeterminizedStates = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - autoGenerateSynonymsPhraseQuery = in.readBoolean(); - fuzzyTranspositions = in.readBoolean(); - } + autoGenerateSynonymsPhraseQuery = in.readBoolean(); + fuzzyTranspositions = in.readBoolean(); } @Override @@ -230,10 +227,8 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeOptionalZoneId(timeZone); out.writeBoolean(this.escape); out.writeVInt(this.maxDeterminizedStates); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeBoolean(autoGenerateSynonymsPhraseQuery); - out.writeBoolean(fuzzyTranspositions); - } + out.writeBoolean(autoGenerateSynonymsPhraseQuery); + out.writeBoolean(fuzzyTranspositions); } public String queryString() { diff --git a/server/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java b/server/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java index 2b2045266455b..bd74d34196345 100644 --- a/server/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/SimpleQueryStringBuilder.java @@ -22,7 +22,6 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -173,12 +172,10 @@ public SimpleQueryStringBuilder(StreamInput in) throws IOException { settings.analyzeWildcard(in.readBoolean()); minimumShouldMatch = in.readOptionalString(); settings.quoteFieldSuffix(in.readOptionalString()); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - settings.autoGenerateSynonymsPhraseQuery(in.readBoolean()); - settings.fuzzyPrefixLength(in.readVInt()); - settings.fuzzyMaxExpansions(in.readVInt()); - settings.fuzzyTranspositions(in.readBoolean()); - } + settings.autoGenerateSynonymsPhraseQuery(in.readBoolean()); + settings.fuzzyPrefixLength(in.readVInt()); + settings.fuzzyMaxExpansions(in.readVInt()); + settings.fuzzyTranspositions(in.readBoolean()); } @Override @@ -197,12 +194,10 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeBoolean(settings.analyzeWildcard()); out.writeOptionalString(minimumShouldMatch); out.writeOptionalString(settings.quoteFieldSuffix()); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeBoolean(settings.autoGenerateSynonymsPhraseQuery()); - out.writeVInt(settings.fuzzyPrefixLength()); - out.writeVInt(settings.fuzzyMaxExpansions()); - out.writeBoolean(settings.fuzzyTranspositions()); - } + out.writeBoolean(settings.autoGenerateSynonymsPhraseQuery()); + out.writeVInt(settings.fuzzyPrefixLength()); + out.writeVInt(settings.fuzzyMaxExpansions()); + out.writeBoolean(settings.fuzzyTranspositions()); } /** Returns the text to parse the query from. */ diff --git a/server/src/main/java/org/elasticsearch/index/query/TypeQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/TypeQueryBuilder.java index ceb473bce40d0..9ebcb9340d9dc 100644 --- a/server/src/main/java/org/elasticsearch/index/query/TypeQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/TypeQueryBuilder.java @@ -22,8 +22,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -59,20 +57,12 @@ public TypeQueryBuilder(String type) { */ public TypeQueryBuilder(StreamInput in) throws IOException { super(in); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - type = in.readString(); - } else { - type = in.readBytesRef().utf8ToString(); - } + type = in.readString(); } @Override protected void doWriteTo(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeString(type); - } else { - out.writeBytesRef(new BytesRef(type)); - } + out.writeString(type); } public String type() { diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java index f5bdd0316ce40..f4d95d057fb54 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.query.functionscore; import org.apache.logging.log4j.LogManager; -import org.elasticsearch.Version; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -58,9 +57,7 @@ public RandomScoreFunctionBuilder(StreamInput in) throws IOException { if (in.readBoolean()) { seed = in.readInt(); } - if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - field = in.readOptionalString(); - } + field = in.readOptionalString(); } @Override @@ -71,9 +68,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { } else { out.writeBoolean(false); } - if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - out.writeOptionalString(field); - } + out.writeOptionalString(field); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java b/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java index 91b0879cfdebf..2bca878df8873 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/RemoteInfo.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.reindex; -import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; @@ -96,11 +95,7 @@ public RemoteInfo(StreamInput in) throws IOException { this.headers = unmodifiableMap(headers); socketTimeout = in.readTimeValue(); connectTimeout = in.readTimeValue(); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - pathPrefix = in.readOptionalString(); - } else { - pathPrefix = null; - } + pathPrefix = in.readOptionalString(); } @Override @@ -118,9 +113,7 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeTimeValue(socketTimeout); out.writeTimeValue(connectTimeout); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeOptionalString(pathPrefix); - } + out.writeOptionalString(pathPrefix); } public String getScheme() { diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 566a81b3af4b0..e238e86fa7224 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -21,7 +21,6 @@ import com.carrotsearch.hppc.ObjectLongHashMap; import com.carrotsearch.hppc.ObjectLongMap; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.cluster.routing.AllocationId; @@ -388,16 +387,7 @@ public CheckpointState(StreamInput in) throws IOException { this.localCheckpoint = in.readZLong(); this.globalCheckpoint = in.readZLong(); this.inSync = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - this.tracked = in.readBoolean(); - } else { - // Every in-sync shard copy is also tracked (see invariant). This was the case even in earlier ES versions. - // Non in-sync shard copies might be tracked or not. As this information here is only serialized during relocation hand-off, - // after which replica recoveries cannot complete anymore (i.e. they cannot move from in-sync == false to in-sync == true), - // we can treat non in-sync replica shard copies as untracked. They will go through a fresh recovery against the new - // primary and will become tracked again under this primary before they are marked as in-sync. - this.tracked = inSync; - } + this.tracked = in.readBoolean(); } @Override @@ -405,9 +395,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeZLong(localCheckpoint); out.writeZLong(globalCheckpoint); out.writeBoolean(inSync); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeBoolean(tracked); - } + out.writeBoolean(tracked); } /** diff --git a/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java b/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java index b0461ece4b0d6..163738e5c9b3b 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java +++ b/server/src/main/java/org/elasticsearch/index/shard/DocsStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.shard; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -43,11 +42,7 @@ public DocsStats() { public DocsStats(StreamInput in) throws IOException { count = in.readVLong(); deleted = in.readVLong(); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - totalSizeInBytes = in.readVLong(); - } else { - totalSizeInBytes = -1; - } + totalSizeInBytes = in.readVLong(); } public DocsStats(long count, long deleted, long totalSizeInBytes) { diff --git a/server/src/main/java/org/elasticsearch/index/store/StoreStats.java b/server/src/main/java/org/elasticsearch/index/store/StoreStats.java index 3ccd5ef4993d4..3a9443636ea7e 100644 --- a/server/src/main/java/org/elasticsearch/index/store/StoreStats.java +++ b/server/src/main/java/org/elasticsearch/index/store/StoreStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.store; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -40,9 +39,6 @@ public StoreStats() { public StoreStats(StreamInput in) throws IOException { sizeInBytes = in.readVLong(); - if (in.getVersion().before(Version.V_6_0_0_alpha1)) { - in.readVLong(); // throttleTimeInNanos - } } public StoreStats(long sizeInBytes) { @@ -81,9 +77,6 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeVLong(sizeInBytes); - if (out.getVersion().before(Version.V_6_0_0_alpha1)) { - out.writeVLong(0L); // throttleTimeInNanos - } } @Override diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 63d21ffea158f..841201b321549 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1102,8 +1102,7 @@ public Source(BytesReference source, String routing) { public static class Index implements Operation { - public static final int FORMAT_6_0 = 8; // since 6.0.0 - public static final int FORMAT_NO_PARENT = FORMAT_6_0 + 1; // since 7.0 + public static final int FORMAT_NO_PARENT = 9; // since 7.0 public static final int FORMAT_NO_VERSION_TYPE = FORMAT_NO_PARENT + 1; public static final int SERIALIZATION_FORMAT = FORMAT_NO_VERSION_TYPE; @@ -1118,7 +1117,7 @@ public static class Index implements Operation { private Index(final StreamInput in) throws IOException { final int format = in.readVInt(); // SERIALIZATION_FORMAT - assert format >= FORMAT_6_0 : "format was: " + format; + assert format >= FORMAT_NO_PARENT : "format was: " + format; id = in.readString(); type = in.readString(); source = in.readBytesReference(); @@ -1208,7 +1207,7 @@ public Source getSource() { } private void write(final StreamOutput out) throws IOException { - final int format = out.getVersion().onOrAfter(Version.V_7_0_0) ? SERIALIZATION_FORMAT : FORMAT_6_0; + final int format = SERIALIZATION_FORMAT; out.writeVInt(format); out.writeString(id); out.writeString(type); @@ -1371,7 +1370,7 @@ public Source getSource() { } private void write(final StreamOutput out) throws IOException { - final int format = out.getVersion().onOrAfter(Version.V_7_0_0) ? SERIALIZATION_FORMAT : FORMAT_6_0; + final int format = SERIALIZATION_FORMAT; out.writeVInt(format); out.writeString(type); out.writeString(id); diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java index 18bb06fa59a46..e0b9817f12d16 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.index.translog; -import org.elasticsearch.Version; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -44,16 +43,9 @@ public TranslogStats() { public TranslogStats(StreamInput in) throws IOException { numberOfOperations = in.readVInt(); translogSizeInBytes = in.readVLong(); - if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - uncommittedOperations = in.readVInt(); - uncommittedSizeInBytes = in.readVLong(); - } else { - uncommittedOperations = numberOfOperations; - uncommittedSizeInBytes = translogSizeInBytes; - } - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - earliestLastModifiedAge = in.readVLong(); - } + uncommittedOperations = in.readVInt(); + uncommittedSizeInBytes = in.readVLong(); + earliestLastModifiedAge = in.readVLong(); } public TranslogStats(int numberOfOperations, long translogSizeInBytes, int uncommittedOperations, long uncommittedSizeInBytes, @@ -139,12 +131,8 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeVInt(numberOfOperations); out.writeVLong(translogSizeInBytes); - if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - out.writeVInt(uncommittedOperations); - out.writeVLong(uncommittedSizeInBytes); - } - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeVLong(earliestLastModifiedAge); - } + out.writeVInt(uncommittedOperations); + out.writeVLong(uncommittedSizeInBytes); + out.writeVLong(earliestLastModifiedAge); } } diff --git a/server/src/main/java/org/elasticsearch/indices/TermsLookup.java b/server/src/main/java/org/elasticsearch/indices/TermsLookup.java index 077116e2fd54e..d309df70b0a14 100644 --- a/server/src/main/java/org/elasticsearch/indices/TermsLookup.java +++ b/server/src/main/java/org/elasticsearch/indices/TermsLookup.java @@ -19,7 +19,6 @@ package org.elasticsearch.indices; -import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; @@ -72,44 +71,19 @@ public TermsLookup(String index, String type, String id, String path) { * Read from a stream. */ public TermsLookup(StreamInput in) throws IOException { - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - type = in.readOptionalString(); - } else { - // Before 7.0, the type parameter was always non-null and serialized as a (non-optional) string. - type = in.readString(); - } + type = in.readOptionalString(); id = in.readString(); path = in.readString(); - if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - index = in.readString(); - } else { - index = in.readOptionalString(); - if (index == null) { - throw new IllegalStateException("index must not be null in a terms lookup"); - } - } + index = in.readString(); routing = in.readOptionalString(); } @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeOptionalString(type); - } else { - if (type == null) { - throw new IllegalArgumentException("Typeless [terms] lookup queries are not supported if any " + - "node is running a version before 7.0."); - - } - out.writeString(type); - } + out.writeOptionalString(type); out.writeString(id); out.writeString(path); - if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - out.writeString(index); - } else { - out.writeOptionalString(index); - } + out.writeString(index); out.writeOptionalString(routing); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryFinalizeRecoveryRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryFinalizeRecoveryRequest.java index 6337467e78330..c21a112320c40 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryFinalizeRecoveryRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryFinalizeRecoveryRequest.java @@ -19,10 +19,8 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.transport.TransportRequest; @@ -60,11 +58,7 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); recoveryId = in.readLong(); shardId = ShardId.readShardId(in); - if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - globalCheckpoint = in.readZLong(); - } else { - globalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; - } + globalCheckpoint = in.readZLong(); } @Override @@ -72,9 +66,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeLong(recoveryId); shardId.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - out.writeZLong(globalCheckpoint); - } + out.writeZLong(globalCheckpoint); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java index 65ccb078c94a0..1d2be7bafda8a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java @@ -19,8 +19,6 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.Version; -import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.shard.ShardId; @@ -47,14 +45,7 @@ class RecoveryPrepareForTranslogOperationsRequest extends TransportRequest { recoveryId = in.readLong(); shardId = ShardId.readShardId(in); totalTranslogOps = in.readVInt(); - if (in.getVersion().before(Version.V_6_0_0_alpha1)) { - in.readLong(); // maxUnsafeAutoIdTimestamp - } - if (in.getVersion().onOrAfter(Version.V_6_2_0)) { - fileBasedRecovery = in.readBoolean(); - } else { - fileBasedRecovery = true; - } + fileBasedRecovery = in.readBoolean(); } public long recoveryId() { @@ -82,11 +73,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(recoveryId); shardId.writeTo(out); out.writeVInt(totalTranslogOps); - if (out.getVersion().before(Version.V_6_0_0_alpha1)) { - out.writeLong(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP); // maxUnsafeAutoIdTimestamp - } - if (out.getVersion().onOrAfter(Version.V_6_2_0)) { - out.writeBoolean(fileBasedRecovery); - } + out.writeBoolean(fileBasedRecovery); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java index 495d4cabd8bb7..baf5dc4d99d99 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsRequest.java @@ -19,12 +19,9 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.Version; -import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.seqno.RetentionLeases; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.transport.TransportRequest; @@ -97,22 +94,9 @@ public void readFrom(StreamInput in) throws IOException { shardId = ShardId.readShardId(in); operations = Translog.readOperations(in, "recovery"); totalTranslogOps = in.readVInt(); - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - maxSeenAutoIdTimestampOnPrimary = in.readZLong(); - } else { - maxSeenAutoIdTimestampOnPrimary = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP; - } - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - maxSeqNoOfUpdatesOrDeletesOnPrimary = in.readZLong(); - } else { - // UNASSIGNED_SEQ_NO means uninitialized and replica won't enable optimization using seq_no - maxSeqNoOfUpdatesOrDeletesOnPrimary = SequenceNumbers.UNASSIGNED_SEQ_NO; - } - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - retentionLeases = new RetentionLeases(in); - } else { - retentionLeases = RetentionLeases.EMPTY; - } + maxSeenAutoIdTimestampOnPrimary = in.readZLong(); + maxSeqNoOfUpdatesOrDeletesOnPrimary = in.readZLong(); + retentionLeases = new RetentionLeases(in); } @Override @@ -122,14 +106,8 @@ public void writeTo(StreamOutput out) throws IOException { shardId.writeTo(out); Translog.writeOperations(out, operations); out.writeVInt(totalTranslogOps); - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeZLong(maxSeenAutoIdTimestampOnPrimary); - } - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary); - } - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - retentionLeases.writeTo(out); - } + out.writeZLong(maxSeenAutoIdTimestampOnPrimary); + out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary); + retentionLeases.writeTo(out); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsResponse.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsResponse.java index 9f86feb573466..3d44f55a0996b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsResponse.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTranslogOperationsResponse.java @@ -19,10 +19,8 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.transport.TransportResponse; import java.io.IOException; @@ -36,21 +34,12 @@ final class RecoveryTranslogOperationsResponse extends TransportResponse { RecoveryTranslogOperationsResponse(final StreamInput in) throws IOException { super(in); - // before 6.0.0 we received an empty response so we have to maintain that - if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - localCheckpoint = in.readZLong(); - } - else { - localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; - } + localCheckpoint = in.readZLong(); } @Override public void writeTo(final StreamOutput out) throws IOException { super.writeTo(out); - // before 6.0.0 we responded with an empty response so we have to maintain that - if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - out.writeZLong(localCheckpoint); - } + out.writeZLong(localCheckpoint); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index cfdaddabdf15b..6c4e7b744b729 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -19,7 +19,6 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -121,11 +120,7 @@ public void readFrom(StreamInput in) throws IOException { targetNode = new DiscoveryNode(in); metadataSnapshot = new Store.MetadataSnapshot(in); primaryRelocation = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - startingSeqNo = in.readLong(); - } else { - startingSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; - } + startingSeqNo = in.readLong(); } @Override @@ -138,9 +133,6 @@ public void writeTo(StreamOutput out) throws IOException { targetNode.writeTo(out); metadataSnapshot.writeTo(out); out.writeBoolean(primaryRelocation); - if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1)) { - out.writeLong(startingSeqNo); - } + out.writeLong(startingSeqNo); } - } diff --git a/server/src/main/java/org/elasticsearch/ingest/IngestStats.java b/server/src/main/java/org/elasticsearch/ingest/IngestStats.java index 7a24cf3ee895e..f140c5f155563 100644 --- a/server/src/main/java/org/elasticsearch/ingest/IngestStats.java +++ b/server/src/main/java/org/elasticsearch/ingest/IngestStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.ingest; -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; @@ -64,16 +63,14 @@ public IngestStats(StreamInput in) throws IOException { String pipelineId = in.readString(); Stats pipelineStat = new Stats(in); this.pipelineStats.add(new PipelineStat(pipelineId, pipelineStat)); - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - int processorsSize = in.readVInt(); - List processorStatsPerPipeline = new ArrayList<>(processorsSize); - for (int j = 0; j < processorsSize; j++) { - String processorName = in.readString(); - Stats processorStat = new Stats(in); - processorStatsPerPipeline.add(new ProcessorStat(processorName, processorStat)); - } - this.processorStats.put(pipelineId, processorStatsPerPipeline); + int processorsSize = in.readVInt(); + List processorStatsPerPipeline = new ArrayList<>(processorsSize); + for (int j = 0; j < processorsSize; j++) { + String processorName = in.readString(); + Stats processorStat = new Stats(in); + processorStatsPerPipeline.add(new ProcessorStat(processorName, processorStat)); } + this.processorStats.put(pipelineId, processorStatsPerPipeline); } } @@ -84,16 +81,14 @@ public void writeTo(StreamOutput out) throws IOException { for (PipelineStat pipelineStat : pipelineStats) { out.writeString(pipelineStat.getPipelineId()); pipelineStat.getStats().writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - List processorStatsForPipeline = processorStats.get(pipelineStat.getPipelineId()); - if (processorStatsForPipeline == null) { - out.writeVInt(0); - } else { - out.writeVInt(processorStatsForPipeline.size()); - for (ProcessorStat processorStat : processorStatsForPipeline) { - out.writeString(processorStat.getName()); - processorStat.getStats().writeTo(out); - } + List processorStatsForPipeline = processorStats.get(pipelineStat.getPipelineId()); + if (processorStatsForPipeline == null) { + out.writeVInt(0); + } else { + out.writeVInt(processorStatsForPipeline.size()); + for (ProcessorStat processorStat : processorStatsForPipeline) { + out.writeString(processorStat.getName()); + processorStat.getStats().writeTo(out); } } } diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java b/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java index e7d02cb1a0902..3c3072987ebef 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsInfo.java @@ -19,7 +19,6 @@ package org.elasticsearch.monitor.os; -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; @@ -61,11 +60,7 @@ public OsInfo(StreamInput in) throws IOException { this.availableProcessors = in.readInt(); this.allocatedProcessors = in.readInt(); this.name = in.readOptionalString(); - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - this.prettyName = in.readOptionalString(); - } else { - this.prettyName = null; - } + this.prettyName = in.readOptionalString(); this.arch = in.readOptionalString(); this.version = in.readOptionalString(); } @@ -76,9 +71,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeInt(availableProcessors); out.writeInt(allocatedProcessors); out.writeOptionalString(name); - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeOptionalString(prettyName); - } + out.writeOptionalString(prettyName); out.writeOptionalString(arch); out.writeOptionalString(version); } diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java b/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java index 3bdfe95f1e2c6..86047281a22fb 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.monitor.os; -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; @@ -413,15 +412,9 @@ public Cgroup( cpuCfsPeriodMicros = in.readLong(); cpuCfsQuotaMicros = in.readLong(); cpuStat = new CpuStat(in); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - memoryControlGroup = in.readOptionalString(); - memoryLimitInBytes = in.readOptionalString(); - memoryUsageInBytes = in.readOptionalString(); - } else { - memoryControlGroup = null; - memoryLimitInBytes = null; - memoryUsageInBytes = null; - } + memoryControlGroup = in.readOptionalString(); + memoryLimitInBytes = in.readOptionalString(); + memoryUsageInBytes = in.readOptionalString(); } @Override @@ -432,11 +425,9 @@ public void writeTo(final StreamOutput out) throws IOException { out.writeLong(cpuCfsPeriodMicros); out.writeLong(cpuCfsQuotaMicros); cpuStat.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeOptionalString(memoryControlGroup); - out.writeOptionalString(memoryLimitInBytes); - out.writeOptionalString(memoryUsageInBytes); - } + out.writeOptionalString(memoryControlGroup); + out.writeOptionalString(memoryLimitInBytes); + out.writeOptionalString(memoryUsageInBytes); } @Override diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java index 9f31c6348a0a1..6c5aa741a797d 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java @@ -347,11 +347,7 @@ public PersistentTask(StreamInput in) throws IOException { id = in.readString(); allocationId = in.readLong(); taskName = in.readString(); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - params = (P) in.readNamedWriteable(PersistentTaskParams.class); - } else { - params = (P) in.readOptionalNamedWriteable(PersistentTaskParams.class); - } + params = (P) in.readNamedWriteable(PersistentTaskParams.class); state = in.readOptionalNamedWriteable(PersistentTaskState.class); assignment = new Assignment(in.readOptionalString(), in.readString()); allocationIdOnLastStatusUpdate = in.readOptionalLong(); @@ -362,11 +358,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(id); out.writeLong(allocationId); out.writeString(taskName); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeNamedWriteable(params); - } else { - out.writeOptionalNamedWriteable(params); - } + out.writeNamedWriteable(params); out.writeOptionalNamedWriteable(state); out.writeOptionalString(assignment.executorNode); out.writeString(assignment.explanation); diff --git a/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java b/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java index 2911bcb4f6376..72e7a43ad0497 100644 --- a/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java +++ b/server/src/main/java/org/elasticsearch/persistent/StartPersistentTaskAction.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.persistent; -import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; @@ -85,11 +84,7 @@ public void readFrom(StreamInput in) throws IOException { super.readFrom(in); taskId = in.readString(); taskName = in.readString(); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - params = in.readNamedWriteable(PersistentTaskParams.class); - } else { - params = in.readOptionalNamedWriteable(PersistentTaskParams.class); - } + params = in.readNamedWriteable(PersistentTaskParams.class); } @Override @@ -97,11 +92,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(taskId); out.writeString(taskName); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeNamedWriteable(params); - } else { - out.writeOptionalNamedWriteable(params); - } + out.writeNamedWriteable(params); } @Override diff --git a/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java b/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java index 3efd149be19c6..32add31ffd50e 100644 --- a/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java +++ b/server/src/main/java/org/elasticsearch/plugins/PluginInfo.java @@ -92,29 +92,11 @@ public PluginInfo(final StreamInput in) throws IOException { this.name = in.readString(); this.description = in.readString(); this.version = in.readString(); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - elasticsearchVersion = Version.readVersion(in); - javaVersion = in.readString(); - } else { - // the plugin must have the version of whichever node we are talking to, since this is enforced on startup - elasticsearchVersion = in.getVersion(); - // this might not be true, but it is not important, we just need something here for bwc that is a valid java version string - javaVersion = "1.8"; - } + elasticsearchVersion = Version.readVersion(in); + javaVersion = in.readString(); this.classname = in.readString(); - if (in.getVersion().onOrAfter(Version.V_6_2_0)) { - extendedPlugins = in.readStringList(); - } else { - extendedPlugins = Collections.emptyList(); - } + extendedPlugins = in.readStringList(); hasNativeController = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_6_0_0_beta2) && in.getVersion().before(Version.V_6_3_0)) { - /* - * Elasticsearch versions in [6.0.0-beta2, 6.3.0) allowed plugins to specify that they require the keystore and this was - * serialized into the plugin info. Therefore, we have to read and ignore this value from the stream. - */ - in.readBoolean(); - } } @Override @@ -122,22 +104,11 @@ public void writeTo(final StreamOutput out) throws IOException { out.writeString(name); out.writeString(description); out.writeString(version); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - Version.writeVersion(elasticsearchVersion, out); - out.writeString(javaVersion); - } + Version.writeVersion(elasticsearchVersion, out); + out.writeString(javaVersion); out.writeString(classname); - if (out.getVersion().onOrAfter(Version.V_6_2_0)) { - out.writeStringCollection(extendedPlugins); - } + out.writeStringCollection(extendedPlugins); out.writeBoolean(hasNativeController); - if (out.getVersion().onOrAfter(Version.V_6_0_0_beta2) && out.getVersion().before(Version.V_6_3_0)) { - /* - * Elasticsearch versions in [6.0.0-beta2, 6.3.0) allowed plugins to specify that they require the keystore and this was - * serialized into the plugin info. Therefore, we have to write out a value for this boolean. - */ - out.writeBoolean(false); - } } /** diff --git a/server/src/main/java/org/elasticsearch/script/ScriptStats.java b/server/src/main/java/org/elasticsearch/script/ScriptStats.java index 06eec72c0e0da..86a6a0d13569b 100644 --- a/server/src/main/java/org/elasticsearch/script/ScriptStats.java +++ b/server/src/main/java/org/elasticsearch/script/ScriptStats.java @@ -19,7 +19,6 @@ package org.elasticsearch.script; -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; @@ -42,16 +41,14 @@ public ScriptStats(long compilations, long cacheEvictions, long compilationLimit public ScriptStats(StreamInput in) throws IOException { compilations = in.readVLong(); cacheEvictions = in.readVLong(); - compilationLimitTriggered = in.getVersion().onOrAfter(Version.V_7_0_0) ? in.readVLong() : 0; + compilationLimitTriggered = in.readVLong(); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeVLong(compilations); out.writeVLong(cacheEvictions); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeVLong(compilationLimitTriggered); - } + out.writeVLong(compilationLimitTriggered); } public long getCompilations() { diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 4cf3bda83530b..27e735789f765 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -21,7 +21,6 @@ import org.apache.lucene.search.Explanation; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.Version; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; @@ -148,10 +147,8 @@ public SearchHit(StreamInput in) throws IOException { type = in.readOptionalText(); nestedIdentity = in.readOptionalWriteable(NestedIdentity::new); version = in.readLong(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - seqNo = in.readZLong(); - primaryTerm = in.readVLong(); - } + seqNo = in.readZLong(); + primaryTerm = in.readVLong(); source = in.readBytesReference(); if (source.length() == 0) { source = null; @@ -220,10 +217,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalText(type); out.writeOptionalWriteable(nestedIdentity); out.writeLong(version); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeZLong(seqNo); - out.writeVLong(primaryTerm); - } + out.writeZLong(seqNo); + out.writeVLong(primaryTerm); out.writeBytesReference(source); if (explanation == null) { out.writeBoolean(false); diff --git a/server/src/main/java/org/elasticsearch/search/SearchHits.java b/server/src/main/java/org/elasticsearch/search/SearchHits.java index 854e9caa79714..e5021b47a3e36 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHits.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHits.java @@ -22,7 +22,6 @@ import org.apache.lucene.search.SortField; import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.TotalHits.Relation; -import org.elasticsearch.Version; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -95,15 +94,9 @@ public SearchHits(StreamInput in) throws IOException { hits[i] = new SearchHit(in); } } - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - sortFields = in.readOptionalArray(Lucene::readSortField, SortField[]::new); - collapseField = in.readOptionalString(); - collapseValues = in.readOptionalArray(Lucene::readSortValue, Object[]::new); - } else { - sortFields = null; - collapseField = null; - collapseValues = null; - } + sortFields = in.readOptionalArray(Lucene::readSortField, SortField[]::new); + collapseField = in.readOptionalString(); + collapseValues = in.readOptionalArray(Lucene::readSortValue, Object[]::new); } @Override @@ -120,11 +113,9 @@ public void writeTo(StreamOutput out) throws IOException { hit.writeTo(out); } } - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeOptionalArray(Lucene::writeSortField, sortFields); - out.writeOptionalString(collapseField); - out.writeOptionalArray(Lucene::writeSortValue, collapseValues); - } + out.writeOptionalArray(Lucene::writeSortField, sortFields); + out.writeOptionalString(collapseField); + out.writeOptionalArray(Lucene::writeSortValue, collapseValues); } /** diff --git a/server/src/main/java/org/elasticsearch/search/SearchSortValues.java b/server/src/main/java/org/elasticsearch/search/SearchSortValues.java index bb4a0f6344736..d944e9bd89486 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchSortValues.java +++ b/server/src/main/java/org/elasticsearch/search/SearchSortValues.java @@ -20,7 +20,6 @@ package org.elasticsearch.search; import org.apache.lucene.util.BytesRef; -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; @@ -67,19 +66,13 @@ public SearchSortValues(Object[] rawSortValues, DocValueFormat[] sortValueFormat SearchSortValues(StreamInput in) throws IOException { this.formattedSortValues = in.readArray(Lucene::readSortValue, Object[]::new); - if (in.getVersion().onOrAfter(Version.V_6_6_0)) { - this.rawSortValues = in.readArray(Lucene::readSortValue, Object[]::new); - } else { - this.rawSortValues = EMPTY_ARRAY; - } + this.rawSortValues = in.readArray(Lucene::readSortValue, Object[]::new); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeArray(Lucene::writeSortValue, this.formattedSortValues); - if (out.getVersion().onOrAfter(Version.V_6_6_0)) { - out.writeArray(Lucene::writeSortValue, this.rawSortValues); - } + out.writeArray(Lucene::writeSortValue, this.rawSortValues); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java b/server/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java index a846c72fc8c01..7054365de2646 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/InternalOrder.java @@ -18,12 +18,9 @@ */ package org.elasticsearch.search.aggregations; -import org.apache.logging.log4j.LogManager; -import org.elasticsearch.Version; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.util.Comparators; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -423,34 +420,11 @@ public static BucketOrder readOrder(StreamInput in) throws IOException { * ONLY FOR HISTOGRAM ORDER: Backwards compatibility logic to read a {@link BucketOrder} from a {@link StreamInput}. * * @param in stream with order data to read. - * @param bwcOrderFlag {@code true} to check {@code in.readBoolean()} in the backwards compat logic before reading - * the order. {@code false} to skip this flag (order always present). * @return order read from the stream * @throws IOException on error reading from the stream. */ - public static BucketOrder readHistogramOrder(StreamInput in, boolean bwcOrderFlag) throws IOException { - if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha2)) { - return Streams.readOrder(in); - } else { // backwards compat logic - if (bwcOrderFlag == false || in.readBoolean()) { - // translate the old histogram order IDs to the new order objects - byte id = in.readByte(); - switch (id) { - case 1: return KEY_ASC; - case 2: return KEY_DESC; - case 3: return COUNT_ASC; - case 4: return COUNT_DESC; - case 0: // aggregation order stream logic is backwards compatible - boolean asc = in.readBoolean(); - String key = in.readString(); - return new Aggregation(key, asc); - default: // not expecting compound order ID - throw new RuntimeException("unknown histogram order id [" + id + "]"); - } - } else { // default to _key asc if no order specified - return KEY_ASC; - } - } + public static BucketOrder readHistogramOrder(StreamInput in) throws IOException { + return Streams.readOrder(in); } /** @@ -480,45 +454,10 @@ public static void writeOrder(BucketOrder order, StreamOutput out) throws IOExce * * @param order order to write to the stream. * @param out stream to write the order to. - * @param bwcOrderFlag {@code true} to always {@code out.writeBoolean(true)} for the backwards compat logic before - * writing the order. {@code false} to skip this flag. * @throws IOException on error writing to the stream. */ - public static void writeHistogramOrder(BucketOrder order, StreamOutput out, boolean bwcOrderFlag) throws IOException { - if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha2)) { - order.writeTo(out); - } else { // backwards compat logic - if(bwcOrderFlag) { // need to add flag that determines if order exists - out.writeBoolean(true); // order always exists - } - if (order instanceof CompoundOrder) { - // older versions do not support histogram compound order; the best we can do here is use the first order. - order = ((CompoundOrder) order).orderElements.get(0); - } - if (order instanceof Aggregation) { - // aggregation order stream logic is backwards compatible - order.writeTo(out); - } else { - // convert the new order IDs to the old histogram order IDs. - byte id; - switch (order.id()) { - case COUNT_DESC_ID: - id = 4; - break; - case COUNT_ASC_ID: - id = 3; - break; - case KEY_DESC_ID: - id = 2; - break; - case KEY_ASC_ID: - id = 1; - break; - default: throw new RuntimeException("unknown order id [" + order.id() + "]"); - } - out.writeByte(id); - } - } + public static void writeHistogramOrder(BucketOrder order, StreamOutput out) throws IOException { + order.writeTo(out); } } @@ -527,9 +466,6 @@ public static void writeHistogramOrder(BucketOrder order, StreamOutput out, bool */ public static class Parser { - private static final DeprecationLogger deprecationLogger = - new DeprecationLogger(LogManager.getLogger(Parser.class)); - /** * Parse a {@link BucketOrder} from {@link XContent}. * diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java index c5a1ab50b8230..715537597d906 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/InternalComposite.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.bucket.composite; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -72,37 +71,25 @@ public InternalComposite(StreamInput in) throws IOException { this.sourceNames = in.readStringList(); this.formats = new ArrayList<>(sourceNames.size()); for (int i = 0; i < sourceNames.size(); i++) { - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - formats.add(in.readNamedWriteable(DocValueFormat.class)); - } else { - formats.add(DocValueFormat.RAW); - } + formats.add(in.readNamedWriteable(DocValueFormat.class)); } this.reverseMuls = in.readIntArray(); this.buckets = in.readList((input) -> new InternalBucket(input, sourceNames, formats, reverseMuls)); - if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - this.afterKey = in.readBoolean() ? new CompositeKey(in) : null; - } else { - this.afterKey = buckets.size() > 0 ? buckets.get(buckets.size()-1).key : null; - } + this.afterKey = in.readBoolean() ? new CompositeKey(in) : null; } @Override protected void doWriteTo(StreamOutput out) throws IOException { out.writeVInt(size); out.writeStringCollection(sourceNames); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - for (DocValueFormat format : formats) { - out.writeNamedWriteable(format); - } + for (DocValueFormat format : formats) { + out.writeNamedWriteable(format); } out.writeIntArray(reverseMuls); out.writeList(buckets); - if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeBoolean(afterKey != null); - if (afterKey != null) { - afterKey.writeTo(out); - } + out.writeBoolean(afterKey != null); + if (afterKey != null) { + afterKey.writeTo(out); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index 6d7852a864453..d607f4bfaf204 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -173,7 +173,7 @@ protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map dvFields = (List) in.readGenericValue(); - if (dvFields == null) { - docValueFields = null; - } else { - docValueFields = dvFields.stream() - .map(field -> new FieldAndFormat(field, null)) - .collect(Collectors.toList()); - } + if (in.readBoolean()) { + docValueFields = in.readList(FieldAndFormat::new); } else { - if (in.readBoolean()) { - docValueFields = in.readList(FieldAndFormat::new); - } else { - docValueFields = null; - } + docValueFields = null; } storedFieldsContext = in.readOptionalWriteable(StoredFieldsContext::new); from = in.readVInt(); @@ -249,21 +236,13 @@ public SearchSourceBuilder(StreamInput in) throws IOException { timeout = in.readOptionalTimeValue(); trackScores = in.readBoolean(); version = in.readOptionalBoolean(); - if (in.getVersion().onOrAfter(Version.V_6_7_0)) { - seqNoAndPrimaryTerm = in.readOptionalBoolean(); - } else { - seqNoAndPrimaryTerm = null; - } + seqNoAndPrimaryTerm = in.readOptionalBoolean(); extBuilders = in.readNamedWriteableList(SearchExtBuilder.class); profile = in.readBoolean(); searchAfterBuilder = in.readOptionalWriteable(SearchAfterBuilder::new); sliceBuilder = in.readOptionalWriteable(SliceBuilder::new); collapse = in.readOptionalWriteable(CollapseBuilder::new); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - trackTotalHitsUpTo = in.readOptionalInt(); - } else { - trackTotalHitsUpTo = in.readBoolean() ? TRACK_TOTAL_HITS_ACCURATE : TRACK_TOTAL_HITS_DISABLED; - } + trackTotalHitsUpTo = in.readOptionalInt(); } @Override @@ -271,15 +250,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(aggregations); out.writeOptionalBoolean(explain); out.writeOptionalWriteable(fetchSourceContext); - if (out.getVersion().before(Version.V_6_4_0)) { - out.writeGenericValue(docValueFields == null - ? null - : docValueFields.stream().map(ff -> ff.field).collect(Collectors.toList())); - } else { - out.writeBoolean(docValueFields != null); - if (docValueFields != null) { - out.writeList(docValueFields); - } + out.writeBoolean(docValueFields != null); + if (docValueFields != null) { + out.writeList(docValueFields); } out.writeOptionalWriteable(storedFieldsContext); out.writeVInt(from); @@ -317,19 +290,13 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalTimeValue(timeout); out.writeBoolean(trackScores); out.writeOptionalBoolean(version); - if (out.getVersion().onOrAfter(Version.V_6_7_0)) { - out.writeOptionalBoolean(seqNoAndPrimaryTerm); - } + out.writeOptionalBoolean(seqNoAndPrimaryTerm); out.writeNamedWriteableList(extBuilders); out.writeBoolean(profile); out.writeOptionalWriteable(searchAfterBuilder); out.writeOptionalWriteable(sliceBuilder); out.writeOptionalWriteable(collapse); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeOptionalInt(trackTotalHitsUpTo); - } else { - out.writeBoolean(trackTotalHitsUpTo == null ? true : trackTotalHitsUpTo > SearchContext.TRACK_TOTAL_HITS_DISABLED); - } + out.writeOptionalInt(trackTotalHitsUpTo); } /** diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java index 33de72989a9dc..f39317712fe0c 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchLocalRequest.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.internal; -import org.elasticsearch.Version; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.common.Nullable; @@ -119,20 +118,9 @@ private ShardSearchLocalRequest(ShardId shardId, int numberOfShards, SearchType nowInMillis = in.readVLong(); requestCache = in.readOptionalBoolean(); clusterAlias = in.readOptionalString(); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - allowPartialSearchResults = in.readBoolean(); - } else if (in.getVersion().onOrAfter(Version.V_6_3_0)) { - allowPartialSearchResults = in.readOptionalBoolean(); - } else { - allowPartialSearchResults = false; - } - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - indexRoutings = in.readStringArray(); - preference = in.readOptionalString(); - } else { - indexRoutings = Strings.EMPTY_ARRAY; - preference = null; - } + allowPartialSearchResults = in.readBoolean(); + indexRoutings = in.readStringArray(); + preference = in.readOptionalString(); } protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOException { @@ -151,16 +139,10 @@ protected final void innerWriteTo(StreamOutput out, boolean asKey) throws IOExce } out.writeOptionalBoolean(requestCache); out.writeOptionalString(clusterAlias); - if (out.getVersion().onOrAfter(Version.V_7_0_0)) { - out.writeBoolean(allowPartialSearchResults); - } else if (out.getVersion().onOrAfter(Version.V_6_3_0)) { - out.writeOptionalBoolean(allowPartialSearchResults); - } + out.writeBoolean(allowPartialSearchResults); if (asKey == false) { - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeStringArray(indexRoutings); - out.writeOptionalString(preference); - } + out.writeStringArray(indexRoutings); + out.writeOptionalString(preference); } } diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index 43654823914b4..3e5f1f65692fa 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -21,7 +21,6 @@ import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; @@ -304,13 +303,8 @@ public void readFromWithId(long id, StreamInput in) throws IOException { terminatedEarly = in.readOptionalBoolean(); profileShardResults = in.readOptionalWriteable(ProfileShardResult::new); hasProfileResults = profileShardResults != null; - if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - serviceTimeEWMA = in.readZLong(); - nodeQueueSize = in.readInt(); - } else { - serviceTimeEWMA = -1; - nodeQueueSize = -1; - } + serviceTimeEWMA = in.readZLong(); + nodeQueueSize = in.readInt(); } @Override @@ -348,10 +342,8 @@ public void writeToNoId(StreamOutput out) throws IOException { out.writeBoolean(searchTimedOut); out.writeOptionalBoolean(terminatedEarly); out.writeOptionalWriteable(profileShardResults); - if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { - out.writeZLong(serviceTimeEWMA); - out.writeInt(nodeQueueSize); - } + out.writeZLong(serviceTimeEWMA); + out.writeInt(nodeQueueSize); } public TotalHits getTotalHits() { diff --git a/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java index 1bc6af2c9669e..116a54a71103b 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/GeoDistanceSortBuilder.java @@ -171,13 +171,9 @@ public GeoDistanceSortBuilder(StreamInput in) throws IOException { sortMode = in.readOptionalWriteable(SortMode::readFromStream); nestedFilter = in.readOptionalNamedWriteable(QueryBuilder.class); nestedPath = in.readOptionalString(); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); - } + nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); validation = GeoValidationMethod.readFromStream(in); - if (in.getVersion().onOrAfter(Version.V_6_4_0)) { - ignoreUnmapped = in.readBoolean(); - } + ignoreUnmapped = in.readBoolean(); } @Override @@ -190,13 +186,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(sortMode); out.writeOptionalNamedWriteable(nestedFilter); out.writeOptionalString(nestedPath); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeOptionalWriteable(nestedSort); - } + out.writeOptionalWriteable(nestedSort); validation.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_6_4_0)) { - out.writeBoolean(ignoreUnmapped); - } + out.writeBoolean(ignoreUnmapped); } /** diff --git a/server/src/main/java/org/elasticsearch/search/sort/NestedSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/NestedSortBuilder.java index eb39a8e0bc3bb..253b2f5ee024d 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/NestedSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/NestedSortBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.sort; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -54,11 +53,7 @@ public NestedSortBuilder(StreamInput in) throws IOException { path = in.readOptionalString(); filter = in.readOptionalNamedWriteable(QueryBuilder.class); nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); - if (in.getVersion().onOrAfter(Version.V_6_5_0)) { - maxChildren = in.readVInt(); - } else { - maxChildren = Integer.MAX_VALUE; - } + maxChildren = in.readVInt(); } public String getPath() { @@ -98,9 +93,7 @@ public void writeTo(final StreamOutput out) throws IOException { out.writeOptionalString(path); out.writeOptionalNamedWriteable(filter); out.writeOptionalWriteable(nestedSort); - if (out.getVersion().onOrAfter(Version.V_6_5_0)) { - out.writeVInt(maxChildren); - } + out.writeVInt(maxChildren); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java index 8d5690d85837c..d858f0f7c6ac3 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java @@ -121,9 +121,7 @@ public ScriptSortBuilder(StreamInput in) throws IOException { sortMode = in.readOptionalWriteable(SortMode::readFromStream); nestedPath = in.readOptionalString(); nestedFilter = in.readOptionalNamedWriteable(QueryBuilder.class); - if (in.getVersion().onOrAfter(Version.V_6_1_0)) { - nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); - } + nestedSort = in.readOptionalWriteable(NestedSortBuilder::new); } @Override @@ -134,9 +132,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalWriteable(sortMode); out.writeOptionalString(nestedPath); out.writeOptionalNamedWriteable(nestedFilter); - if (out.getVersion().onOrAfter(Version.V_6_1_0)) { - out.writeOptionalWriteable(nestedSort); - } + out.writeOptionalWriteable(nestedSort); } /** diff --git a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java index d17f00fdc9562..1fedfcf22dc33 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java @@ -20,7 +20,6 @@ import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.SetOnce; -import org.elasticsearch.Version; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; @@ -40,8 +39,6 @@ import org.elasticsearch.search.suggest.Suggest.Suggestion.Entry; import org.elasticsearch.search.suggest.Suggest.Suggestion.Entry.Option; import org.elasticsearch.search.suggest.completion.CompletionSuggestion; -import org.elasticsearch.search.suggest.phrase.PhraseSuggestion; -import org.elasticsearch.search.suggest.term.TermSuggestion; import java.io.IOException; import java.util.ArrayList; @@ -86,36 +83,11 @@ public Suggest(List>> suggestions) } public Suggest(StreamInput in) throws IOException { - // in older versions, Suggestion types were serialized as Streamable - if (in.getVersion().before(Version.V_7_0_0)) { - final int size = in.readVInt(); - suggestions = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - Suggestion> suggestion; - final int type = in.readVInt(); - switch (type) { - case TermSuggestion.TYPE: - suggestion = new TermSuggestion(in); - break; - case CompletionSuggestion.TYPE: - suggestion = new CompletionSuggestion(in); - break; - case PhraseSuggestion.TYPE: - suggestion = new PhraseSuggestion(in); - break; - default: - throw new IllegalArgumentException("Unknown suggestion type with ordinal " + type); - } - suggestions.add(suggestion); - } - } else { - int suggestionCount = in.readVInt(); - suggestions = new ArrayList<>(suggestionCount); - for (int i = 0; i < suggestionCount; i++) { - suggestions.add(in.readNamedWriteable(Suggestion.class)); - } + int suggestionCount = in.readVInt(); + suggestions = new ArrayList<>(suggestionCount); + for (int i = 0; i < suggestionCount; i++) { + suggestions.add(in.readNamedWriteable(Suggestion.class)); } - hasScoreDocs = filter(CompletionSuggestion.class).stream().anyMatch(CompletionSuggestion::hasScoreDocs); } @@ -154,18 +126,9 @@ public boolean hasScoreDocs() { @Override public void writeTo(StreamOutput out) throws IOException { - // in older versions, Suggestion types were serialized as Streamable - if (out.getVersion().before(Version.V_7_0_0)) { - out.writeVInt(suggestions.size()); - for (Suggestion command : suggestions) { - out.writeVInt(command.getWriteableType()); - command.writeTo(out); - } - } else { - out.writeVInt(suggestions.size()); - for (Suggestion> suggestion : suggestions) { - out.writeNamedWriteable(suggestion); - } + out.writeVInt(suggestions.size()); + for (Suggestion> suggestion : suggestions) { + out.writeNamedWriteable(suggestion); } } @@ -267,12 +230,6 @@ public Suggestion(StreamInput in) throws IOException { name = in.readString(); size = in.readVInt(); - // this is a hack to work around slightly different serialization order of earlier versions of TermSuggestion - if (in.getVersion().before(Version.V_7_0_0) && this instanceof TermSuggestion) { - TermSuggestion t = (TermSuggestion) this; - t.setSort(SortBy.readFromStream(in)); - } - int entriesCount = in.readVInt(); entries.clear(); for (int i = 0; i < entriesCount; i++) { @@ -375,13 +332,6 @@ public void trim() { public void writeTo(StreamOutput out) throws IOException { out.writeString(name); out.writeVInt(size); - - // this is a hack to work around slightly different serialization order in older versions of TermSuggestion - if (out.getVersion().before(Version.V_7_0_0) && this instanceof TermSuggestion) { - TermSuggestion termSuggestion = (TermSuggestion) this; - termSuggestion.getSort().writeTo(out); - } - out.writeVInt(entries.size()); for (Entry entry : entries) { entry.writeTo(out); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java index a1ffdd5670940..b6afaf4ba964b 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.suggest.term; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.Version; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -59,9 +58,7 @@ public TermSuggestion(String name, int size, SortBy sort) { public TermSuggestion(StreamInput in) throws IOException { super(in); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { - sort = SortBy.readFromStream(in); - } + sort = SortBy.readFromStream(in); } // Same behaviour as comparators in suggest module, but for SuggestedWord @@ -131,10 +128,7 @@ protected Comparator