Remove LegacyESVersion.V_6_5_x constants (#1794)

* Remove v6.5.x constants

Signed-off-by: Rabi Panda <adnapibar@gmail.com>

* Address code review comments.

Signed-off-by: Rabi Panda <adnapibar@gmail.com>

* Add back version assumption for testRemoteClusterSettingsUpgraded

Signed-off-by: Rabi Panda <adnapibar@gmail.com>

* Remove unused import after rebase.

Signed-off-by: Rabi Panda <adnapibar@gmail.com>
This commit is contained in:
Rabi Panda 2021-12-28 09:52:51 -08:00 committed by GitHub
parent 7c73ed8d4c
commit 01e1b94e47
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 79 additions and 277 deletions

View File

@ -90,7 +90,6 @@ import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.hamcrest.Matchers.startsWith;
/**
* Tests to run before and after a full cluster restart. This is run twice,
@ -705,9 +704,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
// before timing out
.put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms")
.put(SETTING_ALLOCATION_MAX_RETRY.getKey(), "0"); // fail faster
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
}
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
if (randomBoolean()) {
settings.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1");
}
@ -977,7 +974,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
mappingsAndSettings.startObject("settings");
mappingsAndSettings.field("number_of_shards", 1);
mappingsAndSettings.field("number_of_replicas", 1);
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0) && randomBoolean()) {
if (randomBoolean()) {
mappingsAndSettings.field("soft_deletes.enabled", true);
}
mappingsAndSettings.endObject();
@ -1125,21 +1122,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
clearRoutingFromSettings.setJsonEntity("{\"persistent\":{\"cluster.routing.allocation.exclude.test_attr\": null}}");
client().performRequest(clearRoutingFromSettings);
} catch (WarningFailureException e) {
/*
* If this test is executed on the upgraded mode before testRemoteClusterSettingsUpgraded,
* we will hit a warning exception because we put some deprecated settings in that test.
*/
if (isRunningAgainstOldCluster() == false
&& getOldClusterVersion().before(LegacyESVersion.V_6_5_0)) {
for (String warning : e.getResponse().getWarnings()) {
assertThat(warning, containsString(
"setting was deprecated and will be removed in a future release! "
+ "See the breaking changes documentation for the next major version."));
assertThat(warning, startsWith("[search.remote."));
}
} else {
throw e;
}
throw e;
}
client().performRequest(new Request("DELETE", "/_template/test_template"));
@ -1335,7 +1318,6 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
}
public void testPeerRecoveryRetentionLeases() throws Exception {
assumeTrue(getOldClusterVersion() + " does not support soft deletes", getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0));
if (isRunningAgainstOldCluster()) {
XContentBuilder settings = jsonBuilder();
settings.startObject();
@ -1428,9 +1410,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
final Settings.Builder settings = Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1);
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
}
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
if (randomBoolean()) {
settings.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1");
}
@ -1572,9 +1552,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
final Settings.Builder settings = Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1);
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
}
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
createIndex(index, settings.build());
ensureGreen(index);
int numDocs = randomIntBetween(0, 100);
@ -1623,7 +1601,6 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
}
public void testForbidDisableSoftDeletesOnRestore() throws Exception {
assumeTrue("soft deletes is introduced in 6.5", getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0));
final String snapshot = "snapshot-" + index;
if (isRunningAgainstOldCluster()) {
final Settings.Builder settings = Settings.builder()

View File

@ -58,7 +58,7 @@ import static org.hamcrest.Matchers.equalTo;
public class FullClusterRestartSettingsUpgradeIT extends AbstractFullClusterRestartTestCase {
public void testRemoteClusterSettingsUpgraded() throws IOException {
assumeTrue("settings automatically upgraded since 6.5.0", getOldClusterVersion().before(LegacyESVersion.V_6_5_0));
assumeTrue("settings automatically upgraded since 6.5.0", getOldClusterVersion().before(LegacyESVersion.fromString("6.5.0")));
if (isRunningAgainstOldCluster()) {
final Request putSettingsRequest = new Request("PUT", "/_cluster/settings");
try (XContentBuilder builder = jsonBuilder()) {

View File

@ -103,9 +103,7 @@ public class TranslogPolicyIT extends AbstractFullClusterRestartTestCase {
final Settings.Builder settings = Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, between(0, 1));
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
}
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
if (randomBoolean()) {
settings.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1");
}
@ -121,9 +119,7 @@ public class TranslogPolicyIT extends AbstractFullClusterRestartTestCase {
final Settings.Builder settings = Settings.builder()
.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1);
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
}
settings.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean());
if (randomBoolean()) {
settings.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1");
}

View File

@ -31,7 +31,6 @@
package org.opensearch.index.seqno;
import org.opensearch.LegacyESVersion;
import org.opensearch.Version;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.UUIDs;
@ -75,12 +74,8 @@ public class PeerRecoveryRetentionLeaseCreationIT extends OpenSearchIntegTestCas
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true)
.put(
IndexMetadata.SETTING_VERSION_CREATED,
// simulate a version which supports soft deletes (v6.5.0-and-later) with which this node is compatible
VersionUtils.randomVersionBetween(
random(),
Version.max(Version.CURRENT.minimumIndexCompatibilityVersion(), LegacyESVersion.V_6_5_0),
Version.CURRENT
)
// simulate a version which supports soft deletes with which this node is compatible
VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumIndexCompatibilityVersion(), Version.CURRENT)
)
)
);

View File

@ -46,11 +46,6 @@ import java.lang.reflect.Field;
*/
public class LegacyESVersion extends Version {
public static final LegacyESVersion V_6_5_0 = new LegacyESVersion(6050099, org.apache.lucene.util.Version.LUCENE_7_5_0);
public static final LegacyESVersion V_6_5_1 = new LegacyESVersion(6050199, org.apache.lucene.util.Version.LUCENE_7_5_0);
public static final LegacyESVersion V_6_5_2 = new LegacyESVersion(6050299, org.apache.lucene.util.Version.LUCENE_7_5_0);
public static final LegacyESVersion V_6_5_3 = new LegacyESVersion(6050399, org.apache.lucene.util.Version.LUCENE_7_5_0);
public static final LegacyESVersion V_6_5_4 = new LegacyESVersion(6050499, org.apache.lucene.util.Version.LUCENE_7_5_0);
public static final LegacyESVersion V_6_6_0 = new LegacyESVersion(6060099, org.apache.lucene.util.Version.LUCENE_7_6_0);
public static final LegacyESVersion V_6_6_1 = new LegacyESVersion(6060199, org.apache.lucene.util.Version.LUCENE_7_6_0);
public static final LegacyESVersion V_6_6_2 = new LegacyESVersion(6060299, org.apache.lucene.util.Version.LUCENE_7_6_0);

View File

@ -111,14 +111,6 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
String source = in.readString();
mappings.put(type, source);
}
if (in.getVersion().before(LegacyESVersion.V_6_5_0)) {
// This used to be the size of custom metadata classes
int customSize = in.readVInt();
assert customSize == 0 : "unexpected custom metadata when none is supported";
if (customSize > 0) {
throw new IllegalStateException("unexpected custom metadata when none is supported");
}
}
int aliasesSize = in.readVInt();
for (int i = 0; i < aliasesSize; i++) {
aliases.add(new Alias(in));
@ -472,10 +464,6 @@ public class CreateIndexRequest extends AcknowledgedRequest<CreateIndexRequest>
out.writeString(entry.getKey());
out.writeString(entry.getValue());
}
if (out.getVersion().before(LegacyESVersion.V_6_5_0)) {
// Size of custom index metadata, which is removed
out.writeVInt(0);
}
out.writeVInt(aliases.size());
for (Alias alias : aliases) {
alias.writeTo(out);

View File

@ -31,7 +31,6 @@
package org.opensearch.action.admin.indices.template.put;
import org.opensearch.LegacyESVersion;
import org.opensearch.OpenSearchGenerationException;
import org.opensearch.OpenSearchParseException;
import org.opensearch.action.ActionRequestValidationException;
@ -114,14 +113,6 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
String mappingSource = in.readString();
mappings.put(type, mappingSource);
}
if (in.getVersion().before(LegacyESVersion.V_6_5_0)) {
// Used to be used for custom index metadata
int customSize = in.readVInt();
assert customSize == 0 : "expected not to have any custom metadata";
if (customSize > 0) {
throw new IllegalStateException("unexpected custom metadata when none is supported");
}
}
int aliasesSize = in.readVInt();
for (int i = 0; i < aliasesSize; i++) {
aliases.add(new Alias(in));
@ -502,9 +493,6 @@ public class PutIndexTemplateRequest extends MasterNodeRequest<PutIndexTemplateR
out.writeString(entry.getKey());
out.writeString(entry.getValue());
}
if (out.getVersion().before(LegacyESVersion.V_6_5_0)) {
out.writeVInt(0);
}
out.writeVInt(aliases.size());
for (Alias alias : aliases) {
alias.writeTo(out);

View File

@ -31,8 +31,6 @@
package org.opensearch.action.resync;
import org.opensearch.LegacyESVersion;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.action.support.replication.ReplicatedWriteRequest;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
@ -55,11 +53,7 @@ public final class ResyncReplicationRequest extends ReplicatedWriteRequest<Resyn
ResyncReplicationRequest(StreamInput in) throws IOException {
super(in);
trimAboveSeqNo = in.readZLong();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
} else {
maxSeenAutoIdTimestampOnPrimary = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP;
}
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
operations = in.readArray(Translog.Operation::readOperation, Translog.Operation[]::new);
}
@ -91,9 +85,7 @@ public final class ResyncReplicationRequest extends ReplicatedWriteRequest<Resyn
public void writeTo(final StreamOutput out) throws IOException {
super.writeTo(out);
out.writeZLong(trimAboveSeqNo);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
}
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
out.writeArray(Translog.Operation::writeOperation, operations);
}

View File

@ -36,7 +36,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.store.AlreadyClosedException;
import org.opensearch.Assertions;
import org.opensearch.ExceptionsHelper;
import org.opensearch.LegacyESVersion;
import org.opensearch.OpenSearchException;
import org.opensearch.action.ActionListener;
import org.opensearch.action.ActionListenerResponseHandler;
@ -1449,13 +1448,7 @@ public abstract class TransportReplicationAction<
public ConcreteReplicaRequest(Writeable.Reader<R> requestReader, StreamInput in) throws IOException {
super(requestReader, in);
globalCheckpoint = in.readZLong();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
maxSeqNoOfUpdatesOrDeletes = in.readZLong();
} else {
// UNASSIGNED_SEQ_NO (-2) means uninitialized, and replicas will disable
// optimization using seq_no if its max_seq_no_of_updates is still uninitialized
maxSeqNoOfUpdatesOrDeletes = SequenceNumbers.UNASSIGNED_SEQ_NO;
}
maxSeqNoOfUpdatesOrDeletes = in.readZLong();
}
public ConcreteReplicaRequest(
@ -1474,9 +1467,7 @@ public abstract class TransportReplicationAction<
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeZLong(globalCheckpoint);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeZLong(maxSeqNoOfUpdatesOrDeletes);
}
out.writeZLong(maxSeqNoOfUpdatesOrDeletes);
}
public long getGlobalCheckpoint() {

View File

@ -909,16 +909,8 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
index = in.readString();
routingNumShards = in.readInt();
version = in.readLong();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
mappingVersion = in.readVLong();
} else {
mappingVersion = 1;
}
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
settingsVersion = in.readVLong();
} else {
settingsVersion = 1;
}
mappingVersion = in.readVLong();
settingsVersion = in.readVLong();
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
aliasesVersion = in.readVLong();
} else {
@ -952,12 +944,8 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
out.writeString(index);
out.writeInt(routingNumShards);
out.writeLong(version);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeVLong(mappingVersion);
}
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeVLong(settingsVersion);
}
out.writeVLong(mappingVersion);
out.writeVLong(settingsVersion);
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
out.writeVLong(aliasesVersion);
}
@ -998,16 +986,8 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
public static IndexMetadata readFrom(StreamInput in) throws IOException {
Builder builder = new Builder(in.readString());
builder.version(in.readLong());
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
builder.mappingVersion(in.readVLong());
} else {
builder.mappingVersion(1);
}
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
builder.settingsVersion(in.readVLong());
} else {
builder.settingsVersion(1);
}
builder.mappingVersion(in.readVLong());
builder.settingsVersion(in.readVLong());
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
builder.aliasesVersion(in.readVLong());
} else {
@ -1053,12 +1033,8 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
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(LegacyESVersion.V_6_5_0)) {
out.writeVLong(mappingVersion);
}
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeVLong(settingsVersion);
}
out.writeVLong(mappingVersion);
out.writeVLong(settingsVersion);
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
out.writeVLong(aliasesVersion);
}
@ -1074,14 +1050,10 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
for (ObjectCursor<AliasMetadata> cursor : aliases.values()) {
cursor.value.writeTo(out);
}
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeVInt(customData.size());
for (final ObjectObjectCursor<String, DiffableStringMap> cursor : customData) {
out.writeString(cursor.key);
cursor.value.writeTo(out);
}
} else {
out.writeVInt(0);
out.writeVInt(customData.size());
for (final ObjectObjectCursor<String, DiffableStringMap> cursor : customData) {
out.writeString(cursor.key);
cursor.value.writeTo(out);
}
out.writeVInt(inSyncAllocationIds.size());
for (IntObjectCursor<Set<String>> cursor : inSyncAllocationIds) {
@ -1731,11 +1703,11 @@ public class IndexMetadata implements Diffable<IndexMetadata>, ToXContentFragmen
throw new IllegalArgumentException("Unexpected token " + token);
}
}
if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(LegacyESVersion.V_6_5_0)) {
assert mappingVersion : "mapping version should be present for indices created on or after 6.5.0";
if (Assertions.ENABLED) {
assert mappingVersion : "mapping version should be present for indices";
}
if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(LegacyESVersion.V_6_5_0)) {
assert settingsVersion : "settings version should be present for indices created on or after 6.5.0";
if (Assertions.ENABLED) {
assert settingsVersion : "settings version should be present for indices";
}
if (Assertions.ENABLED && Version.indexCreated(builder.settings).onOrAfter(LegacyESVersion.V_7_2_0)) {
assert aliasesVersion : "aliases version should be present for indices created on or after 7.2.0";

View File

@ -33,7 +33,6 @@ package org.opensearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.opensearch.LegacyESVersion;
import org.opensearch.OpenSearchParseException;
import org.opensearch.cluster.AbstractDiffable;
import org.opensearch.cluster.Diff;
@ -224,14 +223,6 @@ public class IndexTemplateMetadata extends AbstractDiffable<IndexTemplateMetadat
AliasMetadata aliasMd = new AliasMetadata(in);
builder.putAlias(aliasMd);
}
if (in.getVersion().before(LegacyESVersion.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();
}
@ -255,9 +246,6 @@ public class IndexTemplateMetadata extends AbstractDiffable<IndexTemplateMetadat
for (ObjectCursor<AliasMetadata> cursor : aliases.values()) {
cursor.value.writeTo(out);
}
if (out.getVersion().before(LegacyESVersion.V_6_5_0)) {
out.writeVInt(0);
}
out.writeOptionalVInt(version);
}

View File

@ -41,7 +41,6 @@ import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Accountable;
import org.opensearch.Assertions;
import org.opensearch.LegacyESVersion;
import org.opensearch.client.Client;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.cluster.metadata.IndexNameExpressionResolver;
@ -793,9 +792,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
public synchronized void updateMetadata(final IndexMetadata currentIndexMetadata, final IndexMetadata newIndexMetadata) {
final boolean updateIndexSettings = indexSettings.updateIndexMetadata(newIndexMetadata);
if (Assertions.ENABLED
&& currentIndexMetadata != null
&& currentIndexMetadata.getCreationVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
if (Assertions.ENABLED && currentIndexMetadata != null) {
final long currentSettingsVersion = currentIndexMetadata.getSettingsVersion();
final long newSettingsVersion = newIndexMetadata.getSettingsVersion();
if (currentSettingsVersion == newSettingsVersion) {

View File

@ -666,7 +666,7 @@ public final class IndexSettings {
flushAfterMergeThresholdSize = scopedSettings.get(INDEX_FLUSH_AFTER_MERGE_THRESHOLD_SIZE_SETTING);
mergeSchedulerConfig = new MergeSchedulerConfig(this);
gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis();
softDeleteEnabled = version.onOrAfter(LegacyESVersion.V_6_5_0) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING);
softDeleteEnabled = scopedSettings.get(INDEX_SOFT_DELETES_SETTING);
softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING);
retentionLeaseMillis = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING).millis();
warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING);

View File

@ -346,9 +346,7 @@ public class MapperService extends AbstractIndexComponent implements Closeable {
final IndexMetadata newIndexMetadata,
final Map<String, DocumentMapper> updatedEntries
) throws IOException {
if (Assertions.ENABLED
&& currentIndexMetadata != null
&& currentIndexMetadata.getCreationVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
if (Assertions.ENABLED && currentIndexMetadata != null) {
if (currentIndexMetadata.getMappingVersion() == newIndexMetadata.getMappingVersion()) {
// if the mapping version is unchanged, then there should not be any updates and all mappings should be the same
assert updatedEntries.isEmpty() : updatedEntries;

View File

@ -37,7 +37,6 @@ import com.carrotsearch.hppc.ObjectObjectMap;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.util.BytesRef;
import org.opensearch.LegacyESVersion;
import org.opensearch.common.xcontent.XContentParser;
import org.opensearch.index.IndexSettings;
@ -466,18 +465,10 @@ public abstract class ParseContext implements Iterable<ParseContext.Document> {
void postParse() {
if (documents.size() > 1) {
docsReversed = true;
if (indexSettings.getIndexVersionCreated().onOrAfter(LegacyESVersion.V_6_5_0)) {
/**
* For indices created on or after {@link LegacyESVersion#V_6_5_0} we preserve the order
* of the children while ensuring that parents appear after them.
*/
List<Document> newDocs = reorderParent(documents);
documents.clear();
documents.addAll(newDocs);
} else {
// reverse the order of docs for nested docs support, parent should be last
Collections.reverse(documents);
}
// Preserve the order of the children while ensuring that parents appear after them.
List<Document> newDocs = reorderParent(documents);
documents.clear();
documents.addAll(newDocs);
}
}

View File

@ -333,7 +333,7 @@ public final class SimilarityService extends AbstractIndexComponent {
private static void fail(Version indexCreatedVersion, String message) {
if (indexCreatedVersion.onOrAfter(LegacyESVersion.V_7_0_0)) {
throw new IllegalArgumentException(message);
} else if (indexCreatedVersion.onOrAfter(LegacyESVersion.V_6_5_0)) {
} else {
deprecationLogger.deprecate("similarity_failure", message);
}
}

View File

@ -33,11 +33,9 @@
package org.opensearch.indices.recovery;
import org.opensearch.LegacyESVersion;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
import org.opensearch.index.seqno.RetentionLeases;
import org.opensearch.index.seqno.SequenceNumbers;
import org.opensearch.index.shard.ShardId;
import org.opensearch.index.translog.Translog;
@ -120,17 +118,8 @@ public class RecoveryTranslogOperationsRequest extends RecoveryTransportRequest
shardId = new ShardId(in);
operations = Translog.readOperations(in, "recovery");
totalTranslogOps = in.readVInt();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
} else {
maxSeenAutoIdTimestampOnPrimary = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP;
}
if (in.getVersion().onOrAfter(LegacyESVersion.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;
}
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
maxSeqNoOfUpdatesOrDeletesOnPrimary = in.readZLong();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
retentionLeases = new RetentionLeases(in);
} else {
@ -150,12 +139,8 @@ public class RecoveryTranslogOperationsRequest extends RecoveryTransportRequest
shardId.writeTo(out);
Translog.writeOperations(out, operations);
out.writeVInt(totalTranslogOps);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
}
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary);
}
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
retentionLeases.writeTo(out);
}

View File

@ -78,20 +78,18 @@ public class IngestStats implements Writeable, ToXContentFragment {
String pipelineId = in.readString();
Stats pipelineStat = new Stats(in);
this.pipelineStats.add(new PipelineStat(pipelineId, pipelineStat));
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
int processorsSize = in.readVInt();
List<ProcessorStat> processorStatsPerPipeline = new ArrayList<>(processorsSize);
for (int j = 0; j < processorsSize; j++) {
String processorName = in.readString();
String processorType = "_NOT_AVAILABLE";
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_6_0)) {
processorType = in.readString();
}
Stats processorStat = new Stats(in);
processorStatsPerPipeline.add(new ProcessorStat(processorName, processorType, processorStat));
int processorsSize = in.readVInt();
List<ProcessorStat> processorStatsPerPipeline = new ArrayList<>(processorsSize);
for (int j = 0; j < processorsSize; j++) {
String processorName = in.readString();
String processorType = "_NOT_AVAILABLE";
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_6_0)) {
processorType = in.readString();
}
this.processorStats.put(pipelineId, processorStatsPerPipeline);
Stats processorStat = new Stats(in);
processorStatsPerPipeline.add(new ProcessorStat(processorName, processorType, processorStat));
}
this.processorStats.put(pipelineId, processorStatsPerPipeline);
}
}
@ -102,19 +100,17 @@ public class IngestStats implements Writeable, ToXContentFragment {
for (PipelineStat pipelineStat : pipelineStats) {
out.writeString(pipelineStat.getPipelineId());
pipelineStat.getStats().writeTo(out);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
List<ProcessorStat> processorStatsForPipeline = processorStats.get(pipelineStat.getPipelineId());
if (processorStatsForPipeline == null) {
out.writeVInt(0);
} else {
out.writeVInt(processorStatsForPipeline.size());
for (ProcessorStat processorStat : processorStatsForPipeline) {
out.writeString(processorStat.getName());
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_6_0)) {
out.writeString(processorStat.getType());
}
processorStat.getStats().writeTo(out);
List<ProcessorStat> processorStatsForPipeline = processorStats.get(pipelineStat.getPipelineId());
if (processorStatsForPipeline == null) {
out.writeVInt(0);
} else {
out.writeVInt(processorStatsForPipeline.size());
for (ProcessorStat processorStat : processorStatsForPipeline) {
out.writeString(processorStat.getName());
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_6_0)) {
out.writeString(processorStat.getType());
}
processorStat.getStats().writeTo(out);
}
}
}

View File

@ -43,7 +43,6 @@ import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.TotalHits;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.BitSet;
import org.opensearch.LegacyESVersion;
import org.opensearch.common.CheckedBiConsumer;
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.document.DocumentField;
@ -531,32 +530,20 @@ public class FetchPhase {
BitSet parentBits = context.bitsetFilterCache().getBitSetProducer(parentFilter).getBitSet(subReaderContext);
int offset = 0;
if (indexSettings.getIndexVersionCreated().onOrAfter(LegacyESVersion.V_6_5_0)) {
/**
* Starts from the previous parent and finds the offset of the
* <code>nestedSubDocID</code> within the nested children. Nested documents
* are indexed in the same order than in the source array so the offset
* of the nested child is the number of nested document with the same parent
* that appear before him.
*/
int previousParent = parentBits.prevSetBit(currentParent);
for (int docId = childIter.advance(previousParent + 1); docId < nestedSubDocId
&& docId != DocIdSetIterator.NO_MORE_DOCS; docId = childIter.nextDoc()) {
offset++;
}
currentParent = nestedSubDocId;
} else {
/**
* Nested documents are in reverse order in this version so we start from the current nested document
* and find the number of documents with the same parent that appear after it.
*/
int nextParent = parentBits.nextSetBit(currentParent);
for (int docId = childIter.advance(currentParent + 1); docId < nextParent && docId != DocIdSetIterator.NO_MORE_DOCS; docId =
childIter.nextDoc()) {
offset++;
}
currentParent = nextParent;
/*
* Starts from the previous parent and finds the offset of the
* <code>nestedSubDocID</code> within the nested children. Nested documents
* are indexed in the same order than in the source array so the offset
* of the nested child is the number of nested document with the same parent
* that appear before him.
*/
int previousParent = parentBits.prevSetBit(currentParent);
for (int docId = childIter.advance(previousParent + 1); docId < nestedSubDocId
&& docId != DocIdSetIterator.NO_MORE_DOCS; docId = childIter.nextDoc()) {
offset++;
}
currentParent = nestedSubDocId;
current = nestedObjectMapper = nestedParentObjectMapper;
int currentPrefix = current == null ? 0 : current.name().length() + 1;
nestedIdentity = new SearchHit.NestedIdentity(originalName.substring(currentPrefix), offset, nestedIdentity);

View File

@ -579,9 +579,6 @@ public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
if (nestedSort == null) {
return resolveNested(context, nestedPath, nestedFilter);
}
if (context.indexVersionCreated().before(LegacyESVersion.V_6_5_0) && nestedSort.getMaxChildren() != Integer.MAX_VALUE) {
throw new QueryShardException(context, "max_children is only supported on v6.5.0 or higher");
}
validateMaxChildrenExistOnlyInTopLevelNestedSort(context, nestedSort);
return resolveNested(context, nestedSort);
}

View File

@ -41,7 +41,6 @@ import org.apache.lucene.search.LeafFieldComparator;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.comparators.DoubleComparator;
import org.apache.lucene.util.BitSet;
import org.opensearch.LegacyESVersion;
import org.opensearch.OpenSearchParseException;
import org.opensearch.common.ParseField;
import org.opensearch.common.ParsingException;
@ -69,7 +68,6 @@ import org.opensearch.index.query.GeoValidationMethod;
import org.opensearch.index.query.QueryBuilder;
import org.opensearch.index.query.QueryRewriteContext;
import org.opensearch.index.query.QueryShardContext;
import org.opensearch.index.query.QueryShardException;
import org.opensearch.search.DocValueFormat;
import org.opensearch.search.MultiValueMode;
@ -696,9 +694,6 @@ public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
if (nestedSort == null) {
return resolveNested(context, nestedPath, nestedFilter);
}
if (context.indexVersionCreated().before(LegacyESVersion.V_6_5_0) && nestedSort.getMaxChildren() != Integer.MAX_VALUE) {
throw new QueryShardException(context, "max_children is only supported on v6.5.0 or higher");
}
validateMaxChildrenExistOnlyInTopLevelNestedSort(context, nestedSort);
return resolveNested(context, nestedSort);
}

View File

@ -32,7 +32,6 @@
package org.opensearch.search.sort;
import org.opensearch.LegacyESVersion;
import org.opensearch.common.ParseField;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
@ -67,11 +66,7 @@ public class NestedSortBuilder implements Writeable, ToXContentObject {
path = in.readOptionalString();
filter = in.readOptionalNamedWriteable(QueryBuilder.class);
nestedSort = in.readOptionalWriteable(NestedSortBuilder::new);
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
maxChildren = in.readVInt();
} else {
maxChildren = Integer.MAX_VALUE;
}
maxChildren = in.readVInt();
}
public String getPath() {
@ -113,9 +108,7 @@ public class NestedSortBuilder implements Writeable, ToXContentObject {
out.writeOptionalString(path);
out.writeOptionalNamedWriteable(filter);
out.writeOptionalWriteable(nestedSort);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_5_0)) {
out.writeVInt(maxChildren);
}
out.writeVInt(maxChildren);
}
@Override

View File

@ -38,7 +38,6 @@ import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.opensearch.LegacyESVersion;
import org.opensearch.common.ParseField;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
@ -341,9 +340,6 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
final Nested nested;
if (nestedSort != null) {
if (context.indexVersionCreated().before(LegacyESVersion.V_6_5_0) && nestedSort.getMaxChildren() != Integer.MAX_VALUE) {
throw new QueryShardException(context, "max_children is only supported on v6.5.0 or higher");
}
// new nested sorts takes priority
validateMaxChildrenExistOnlyInTopLevelNestedSort(context, nestedSort);
nested = resolveNested(context, nestedSort);

View File

@ -142,7 +142,7 @@ public class LegacyESVersionTests extends OpenSearchTestCase {
assertThat(LegacyESVersion.fromString("5.3.0").minimumCompatibilityVersion(), equalTo(major5x));
Version major56x = LegacyESVersion.fromString("5.6.0");
assertThat(LegacyESVersion.V_6_5_0.minimumCompatibilityVersion(), equalTo(major56x));
assertThat(LegacyESVersion.fromString("6.5.0").minimumCompatibilityVersion(), equalTo(major56x));
assertThat(LegacyESVersion.fromString("6.3.1").minimumCompatibilityVersion(), equalTo(major56x));
// from 7.0 on we are supporting the latest minor of the previous major... this might fail once we add a new version ie. 5.x is

View File

@ -728,16 +728,6 @@ public class IndexSettingsTests extends OpenSearchTestCase {
Settings settings = Settings.builder().put(IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(), createdVersion).build();
assertTrue(IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings));
}
// disabled by default on the previous versions
{
Version prevVersion = VersionUtils.randomVersionBetween(
random(),
LegacyESVersion.V_6_5_0,
VersionUtils.getPreviousVersion(LegacyESVersion.V_7_0_0)
);
Settings settings = Settings.builder().put(IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(), prevVersion).build();
assertFalse(IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings));
}
}
public void testIgnoreTranslogRetentionSettingsIfSoftDeletesEnabled() {

View File

@ -32,8 +32,6 @@
package org.opensearch.index.replication;
import org.opensearch.LegacyESVersion;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.action.support.replication.ReplicationResponse;
@ -172,10 +170,7 @@ public class RetentionLeasesReplicationTests extends OpenSearchIndexLevelReplica
public void testTurnOffTranslogRetentionAfterAllShardStarted() throws Exception {
final Settings.Builder settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true);
if (randomBoolean()) {
settings.put(
IndexMetadata.SETTING_VERSION_CREATED,
VersionUtils.randomVersionBetween(random(), LegacyESVersion.V_6_5_0, Version.CURRENT)
);
settings.put(IndexMetadata.SETTING_VERSION_CREATED, VersionUtils.randomIndexCompatibleVersion(random()));
}
try (ReplicationGroup group = createGroup(between(1, 2), settings.build())) {
group.startAll();