Remove LegacyESVersion.V_6_7_x constants (#1807)

This commit removes all usages of the `LegacyESVersion.V_6_7_x` constants from the codebase.

Signed-off-by: Rabi Panda <adnapibar@gmail.com>
This commit is contained in:
Rabi Panda 2022-01-02 14:09:55 -08:00 committed by GitHub
parent 5ccb22fb4b
commit 06cbc47136
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 57 additions and 216 deletions

View File

@ -33,7 +33,6 @@
package org.opensearch.index.reindex;
import org.apache.logging.log4j.Logger;
import org.opensearch.LegacyESVersion;
import org.opensearch.action.ActionListener;
import org.opensearch.action.index.IndexRequest;
import org.opensearch.action.support.ActionFilters;
@ -113,8 +112,6 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
*/
static class AsyncIndexBySearchAction extends AbstractAsyncBulkByScrollAction<UpdateByQueryRequest, TransportUpdateByQueryAction> {
private final boolean useSeqNoForCAS;
AsyncIndexBySearchAction(
BulkByScrollTask task,
Logger logger,
@ -125,21 +122,7 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
ClusterState clusterState,
ActionListener<BulkByScrollResponse> listener
) {
super(
task,
// not all nodes support sequence number powered optimistic concurrency control, we fall back to version
clusterState.nodes().getMinNodeVersion().onOrAfter(LegacyESVersion.V_6_7_0) == false,
// all nodes support sequence number powered optimistic concurrency control and we can use it
clusterState.nodes().getMinNodeVersion().onOrAfter(LegacyESVersion.V_6_7_0),
logger,
client,
threadPool,
request,
listener,
scriptService,
null
);
useSeqNoForCAS = clusterState.nodes().getMinNodeVersion().onOrAfter(LegacyESVersion.V_6_7_0);
super(task, false, true, logger, client, threadPool, request, listener, scriptService, null);
}
@Override

View File

@ -51,7 +51,6 @@ import org.opensearch.common.xcontent.json.JsonXContent;
import org.opensearch.common.xcontent.support.XContentMapValues;
import org.opensearch.index.IndexSettings;
import org.opensearch.rest.action.document.RestBulkAction;
import org.opensearch.rest.action.document.RestGetAction;
import org.opensearch.rest.action.document.RestIndexAction;
import org.opensearch.rest.action.document.RestUpdateAction;
import org.opensearch.rest.action.search.RestExplainAction;
@ -109,7 +108,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
@Before
public void setType() {
type = getOldClusterVersion().before(LegacyESVersion.V_6_7_0) ? "doc" : "_doc";
type = "_doc";
}
public void testSearch() throws Exception {
@ -633,9 +632,6 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
client().performRequest(updateRequest);
Request getRequest = new Request("GET", "/" + index + "/" + typeName + "/" + docId);
if (getOldClusterVersion().before(LegacyESVersion.V_6_7_0)) {
getRequest.setOptions(expectWarnings(RestGetAction.TYPES_DEPRECATION_MESSAGE));
}
Map<String, Object> getRsp = entityAsMap(client().performRequest(getRequest));
Map<?, ?> source = (Map<?, ?>) getRsp.get("_source");
assertTrue("doc does not contain 'foo' key: " + source, source.containsKey("foo"));
@ -682,9 +678,6 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
Request request = new Request("GET", docLocation);
if (getOldClusterVersion().before(LegacyESVersion.V_6_7_0)) {
request.setOptions(expectWarnings(RestGetAction.TYPES_DEPRECATION_MESSAGE));
}
assertThat(toStr(client().performRequest(request)), containsString(doc));
}
@ -1269,9 +1262,6 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
private String loadInfoDocument(String type) throws IOException {
Request request = new Request("GET", "/info/" + this.type + "/" + index + "_" + type);
request.addParameter("filter_path", "_source");
if (getOldClusterVersion().before(LegacyESVersion.V_6_7_0)) {
request.setOptions(expectWarnings(RestGetAction.TYPES_DEPRECATION_MESSAGE));
}
String doc = toStr(client().performRequest(request));
Matcher m = Pattern.compile("\"value\":\"(.+)\"").matcher(doc);
assertTrue(doc, m.find());
@ -1352,9 +1342,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_7_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 committedDocs = randomIntBetween(100, 200);
@ -1383,7 +1371,6 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
* Verifies that once all shard copies on the new version, we should turn off the translog retention for indices with soft-deletes.
*/
public void testTurnOffTranslogRetentionAfterUpgraded() throws Exception {
assumeTrue("requires soft-deletes and retention leases", getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_7_0));
if (isRunningAgainstOldCluster()) {
createIndex(index, Settings.builder()
.put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1)
@ -1484,7 +1471,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
// make sure .tasks index exists
Request getTasksIndex = new Request("GET", "/.tasks");
getTasksIndex.addParameter("allow_no_indices", "false");
if (getOldClusterVersion().onOrAfter(LegacyESVersion.V_6_7_0) && getOldClusterVersion().before(LegacyESVersion.V_7_0_0)) {
if (getOldClusterVersion().before(LegacyESVersion.V_7_0_0)) {
getTasksIndex.addParameter("include_type_name", "false");
}

View File

@ -95,7 +95,7 @@ public class TranslogPolicyIT extends AbstractFullClusterRestartTestCase {
@Before
public void setType() {
type = getOldClusterVersion().before(LegacyESVersion.V_6_7_0) ? "doc" : "_doc";
type = "_doc";
}
public void testEmptyIndex() throws Exception {

View File

@ -246,12 +246,7 @@ public class Build {
if (out.getVersion().before(Version.V_2_0_0)) {
out.writeString("oss");
}
final Type buildType;
if (out.getVersion().before(LegacyESVersion.V_6_7_0) && build.type() == Type.DOCKER) {
buildType = Type.TAR;
} else {
buildType = build.type();
}
final Type buildType = build.type();
out.writeString(buildType.displayName());
out.writeString(build.hash());
out.writeString(build.date());

View File

@ -46,9 +46,6 @@ import java.lang.reflect.Field;
*/
public class LegacyESVersion extends Version {
public static final LegacyESVersion V_6_7_0 = new LegacyESVersion(6070099, org.apache.lucene.util.Version.LUCENE_7_7_0);
public static final LegacyESVersion V_6_7_1 = new LegacyESVersion(6070199, org.apache.lucene.util.Version.LUCENE_7_7_0);
public static final LegacyESVersion V_6_7_2 = new LegacyESVersion(6070299, org.apache.lucene.util.Version.LUCENE_7_7_0);
public static final LegacyESVersion V_6_8_0 = new LegacyESVersion(6080099, org.apache.lucene.util.Version.LUCENE_7_7_0);
public static final LegacyESVersion V_6_8_1 = new LegacyESVersion(6080199, org.apache.lucene.util.Version.LUCENE_7_7_0);
public static final LegacyESVersion V_6_8_2 = new LegacyESVersion(6080299, org.apache.lucene.util.Version.LUCENE_7_7_0);

View File

@ -1537,25 +1537,25 @@ public class OpenSearchException extends RuntimeException implements ToXContentF
org.opensearch.snapshots.SnapshotInProgressException.class,
org.opensearch.snapshots.SnapshotInProgressException::new,
151,
LegacyESVersion.V_6_7_0
UNKNOWN_VERSION_ADDED
),
NO_SUCH_REMOTE_CLUSTER_EXCEPTION(
org.opensearch.transport.NoSuchRemoteClusterException.class,
org.opensearch.transport.NoSuchRemoteClusterException::new,
152,
LegacyESVersion.V_6_7_0
UNKNOWN_VERSION_ADDED
),
RETENTION_LEASE_ALREADY_EXISTS_EXCEPTION(
org.opensearch.index.seqno.RetentionLeaseAlreadyExistsException.class,
org.opensearch.index.seqno.RetentionLeaseAlreadyExistsException::new,
153,
LegacyESVersion.V_6_7_0
UNKNOWN_VERSION_ADDED
),
RETENTION_LEASE_NOT_FOUND_EXCEPTION(
org.opensearch.index.seqno.RetentionLeaseNotFoundException.class,
org.opensearch.index.seqno.RetentionLeaseNotFoundException::new,
154,
LegacyESVersion.V_6_7_0
UNKNOWN_VERSION_ADDED
),
SHARD_NOT_IN_PRIMARY_MODE_EXCEPTION(
org.opensearch.index.shard.ShardNotInPrimaryModeException.class,

View File

@ -114,11 +114,7 @@ public class PutMappingRequest extends AcknowledgedRequest<PutMappingRequest> im
in.readBoolean(); // updateAllTypes
}
concreteIndex = in.readOptionalWriteable(Index::new);
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
origin = in.readOptionalString();
} else {
origin = null;
}
origin = in.readOptionalString();
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_9_0)) {
writeIndexOnly = in.readBoolean();
}
@ -376,9 +372,7 @@ public class PutMappingRequest extends AcknowledgedRequest<PutMappingRequest> im
out.writeBoolean(true); // updateAllTypes
}
out.writeOptionalWriteable(concreteIndex);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeOptionalString(origin);
}
out.writeOptionalString(origin);
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_9_0)) {
out.writeBoolean(writeIndexOnly);
}

View File

@ -32,7 +32,6 @@
package org.opensearch.action.admin.indices.stats;
import org.opensearch.LegacyESVersion;
import org.opensearch.cluster.routing.ShardRouting;
import org.opensearch.common.Nullable;
import org.opensearch.common.io.stream.StreamInput;
@ -80,9 +79,7 @@ public class ShardStats implements Writeable, ToXContentFragment {
dataPath = in.readString();
isCustomDataPath = in.readBoolean();
seqNoStats = in.readOptionalWriteable(SeqNoStats::new);
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
retentionLeaseStats = in.readOptionalWriteable(RetentionLeaseStats::new);
}
retentionLeaseStats = in.readOptionalWriteable(RetentionLeaseStats::new);
}
public ShardStats(
@ -145,9 +142,7 @@ public class ShardStats implements Writeable, ToXContentFragment {
out.writeString(dataPath);
out.writeBoolean(isCustomDataPath);
out.writeOptionalWriteable(seqNoStats);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeOptionalWriteable(retentionLeaseStats);
}
out.writeOptionalWriteable(retentionLeaseStats);
}
@Override

View File

@ -236,17 +236,11 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla
preFilterShardSize = in.readVInt();
}
allowPartialSearchResults = in.readOptionalBoolean();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
localClusterAlias = in.readOptionalString();
if (localClusterAlias != null) {
absoluteStartMillis = in.readVLong();
finalReduce = in.readBoolean();
} else {
absoluteStartMillis = DEFAULT_ABSOLUTE_START_MILLIS;
finalReduce = true;
}
localClusterAlias = in.readOptionalString();
if (localClusterAlias != null) {
absoluteStartMillis = in.readVLong();
finalReduce = in.readBoolean();
} else {
localClusterAlias = null;
absoluteStartMillis = DEFAULT_ABSOLUTE_START_MILLIS;
finalReduce = true;
}
@ -279,12 +273,10 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla
out.writeVInt(preFilterShardSize == null ? DEFAULT_BATCHED_REDUCE_SIZE : preFilterShardSize);
}
out.writeOptionalBoolean(allowPartialSearchResults);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeOptionalString(localClusterAlias);
if (localClusterAlias != null) {
out.writeVLong(absoluteStartMillis);
out.writeBoolean(finalReduce);
}
out.writeOptionalString(localClusterAlias);
if (localClusterAlias != null) {
out.writeVLong(absoluteStartMillis);
out.writeBoolean(finalReduce);
}
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_0_0)) {
out.writeBoolean(ccsMinimizeRoundtrips);

View File

@ -34,7 +34,6 @@ package org.opensearch.cluster;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.opensearch.LegacyESVersion;
import org.opensearch.cluster.block.ClusterBlock;
import org.opensearch.cluster.block.ClusterBlocks;
import org.opensearch.cluster.coordination.CoordinationMetadata;
@ -736,7 +735,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
Custom customIndexMetadata = in.readNamedWriteable(Custom.class);
builder.putCustom(customIndexMetadata.getWriteableName(), customIndexMetadata);
}
builder.minimumMasterNodesOnPublishingMaster = in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0) ? in.readVInt() : -1;
builder.minimumMasterNodesOnPublishingMaster = in.readVInt();
return builder.build();
}
@ -762,9 +761,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
out.writeNamedWriteable(cursor.value);
}
}
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeVInt(minimumMasterNodesOnPublishingMaster);
}
out.writeVInt(minimumMasterNodesOnPublishingMaster);
}
private static class ClusterStateDiff implements Diff<ClusterState> {
@ -812,7 +809,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
metadata = Metadata.readDiffFrom(in);
blocks = ClusterBlocks.readDiffFrom(in);
customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), CUSTOM_VALUE_SERIALIZER);
minimumMasterNodesOnPublishingMaster = in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0) ? in.readVInt() : -1;
minimumMasterNodesOnPublishingMaster = in.readVInt();
}
@Override
@ -826,9 +823,7 @@ public class ClusterState implements ToXContentFragment, Diffable<ClusterState>
metadata.writeTo(out);
blocks.writeTo(out);
customs.writeTo(out);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeVInt(minimumMasterNodesOnPublishingMaster);
}
out.writeVInt(minimumMasterNodesOnPublishingMaster);
}
@Override

View File

@ -35,7 +35,6 @@ package org.opensearch.cluster.action.shard;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.opensearch.LegacyESVersion;
import org.opensearch.OpenSearchException;
import org.opensearch.ExceptionsHelper;
import org.opensearch.action.ActionListener;
@ -88,8 +87,6 @@ import java.util.Set;
import java.util.function.Predicate;
import java.util.function.Supplier;
import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM;
public class ShardStateAction {
private static final Logger logger = LogManager.getLogger(ShardStateAction.class);
@ -819,11 +816,7 @@ public class ShardStateAction {
super(in);
shardId = new ShardId(in);
allocationId = in.readString();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
primaryTerm = in.readVLong();
} else {
primaryTerm = UNASSIGNED_PRIMARY_TERM;
}
primaryTerm = in.readVLong();
this.message = in.readString();
}
@ -839,9 +832,7 @@ public class ShardStateAction {
super.writeTo(out);
shardId.writeTo(out);
out.writeString(allocationId);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeVLong(primaryTerm);
}
out.writeVLong(primaryTerm);
out.writeString(message);
}

View File

@ -32,7 +32,6 @@
package org.opensearch.cluster.block;
import org.opensearch.LegacyESVersion;
import org.opensearch.common.Nullable;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.io.stream.StreamOutput;
@ -60,11 +59,7 @@ public class ClusterBlock implements Writeable, ToXContentFragment {
public ClusterBlock(StreamInput in) throws IOException {
id = in.readVInt();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
uuid = in.readOptionalString();
} else {
uuid = null;
}
uuid = in.readOptionalString();
description = in.readString();
this.levels = in.readEnumSet(ClusterBlockLevel.class);
retryable = in.readBoolean();
@ -173,9 +168,7 @@ public class ClusterBlock implements Writeable, ToXContentFragment {
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(id);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeOptionalString(uuid);
}
out.writeOptionalString(uuid);
out.writeString(description);
out.writeEnumSet(levels);
out.writeBoolean(retryable);

View File

@ -167,8 +167,7 @@ public class ReadOnlyEngine extends Engine {
// that guarantee that all operations have been flushed to Lucene.
final Version indexVersionCreated = engineConfig.getIndexSettings().getIndexVersionCreated();
if (indexVersionCreated.onOrAfter(LegacyESVersion.V_7_2_0)
|| (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO
&& indexVersionCreated.onOrAfter(LegacyESVersion.V_6_7_0))) {
|| (seqNoStats.getGlobalCheckpoint() != SequenceNumbers.UNASSIGNED_SEQ_NO)) {
assert assertMaxSeqNoEqualsToGlobalCheckpoint(seqNoStats.getMaxSeqNo(), seqNoStats.getGlobalCheckpoint());
if (seqNoStats.getMaxSeqNo() != seqNoStats.getGlobalCheckpoint()) {
throw new IllegalStateException(

View File

@ -183,11 +183,7 @@ public final class InnerHitBuilder implements Writeable, ToXContentObject {
size = in.readVInt();
explain = in.readBoolean();
version = in.readBoolean();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
seqNoAndPrimaryTerm = in.readBoolean();
} else {
seqNoAndPrimaryTerm = false;
}
seqNoAndPrimaryTerm = in.readBoolean();
trackScores = in.readBoolean();
storedFieldsContext = in.readOptionalWriteable(StoredFieldsContext::new);
docValueFields = in.readBoolean() ? in.readList(FieldAndFormat::new) : null;
@ -224,9 +220,7 @@ public final class InnerHitBuilder implements Writeable, ToXContentObject {
out.writeVInt(size);
out.writeBoolean(explain);
out.writeBoolean(version);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeBoolean(seqNoAndPrimaryTerm);
}
out.writeBoolean(seqNoAndPrimaryTerm);
out.writeBoolean(trackScores);
out.writeOptionalWriteable(storedFieldsContext);

View File

@ -150,7 +150,7 @@ public class RecoverySourceHandler {
this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName());
this.chunkSizeInBytes = fileChunkSizeInBytes;
// if the target is on an old version, it won't be able to handle out-of-order file chunks.
this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(LegacyESVersion.V_6_7_0) ? maxConcurrentFileChunks : 1;
this.maxConcurrentFileChunks = maxConcurrentFileChunks;
this.maxConcurrentOperations = maxConcurrentOperations;
}

View File

@ -120,16 +120,8 @@ public class RecoveryTranslogOperationsRequest extends RecoveryTransportRequest
totalTranslogOps = in.readVInt();
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
maxSeqNoOfUpdatesOrDeletesOnPrimary = in.readZLong();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
retentionLeases = new RetentionLeases(in);
} else {
retentionLeases = RetentionLeases.EMPTY;
}
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
mappingVersionOnPrimary = in.readVLong();
} else {
mappingVersionOnPrimary = Long.MAX_VALUE;
}
retentionLeases = new RetentionLeases(in);
mappingVersionOnPrimary = in.readVLong();
}
@Override
@ -141,9 +133,7 @@ public class RecoveryTranslogOperationsRequest extends RecoveryTransportRequest
out.writeVInt(totalTranslogOps);
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
out.writeZLong(maxSeqNoOfUpdatesOrDeletesOnPrimary);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
retentionLeases.writeTo(out);
}
retentionLeases.writeTo(out);
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_2_0)) {
out.writeVLong(mappingVersionOnPrimary);
}

View File

@ -168,10 +168,8 @@ public final class SearchHit implements Writeable, ToXContentObject, Iterable<Do
type = in.readOptionalText();
nestedIdentity = in.readOptionalWriteable(NestedIdentity::new);
version = in.readLong();
if (in.getVersion().onOrAfter(LegacyESVersion.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;
@ -270,10 +268,8 @@ public final class SearchHit implements Writeable, ToXContentObject, Iterable<Do
out.writeOptionalText(type);
out.writeOptionalWriteable(nestedIdentity);
out.writeLong(version);
if (out.getVersion().onOrAfter(LegacyESVersion.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);

View File

@ -108,7 +108,7 @@ public final class InternalAggregations extends Aggregations implements Writeabl
public static InternalAggregations readFrom(StreamInput in) throws IOException {
final InternalAggregations res = from(in.readList(stream -> in.readNamedWriteable(InternalAggregation.class)));
if (in.getVersion().before(LegacyESVersion.V_7_8_0) && in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
if (in.getVersion().before(LegacyESVersion.V_7_8_0)) {
/*
* Setting the pipeline tree source to null is here is correct but
* only because we don't immediately pass the InternalAggregations
@ -127,16 +127,12 @@ public final class InternalAggregations extends Aggregations implements Writeabl
if (pipelineTreeForBwcSerialization == null) {
mergePipelineTreeForBWCSerialization(PipelineTree.EMPTY);
out.writeNamedWriteableList(getInternalAggregations());
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeNamedWriteableList(emptyList());
}
out.writeNamedWriteableList(emptyList());
} else {
PipelineAggregator.PipelineTree pipelineTree = pipelineTreeForBwcSerialization.get();
mergePipelineTreeForBWCSerialization(pipelineTree);
out.writeNamedWriteableList(getInternalAggregations());
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeNamedWriteableList(pipelineTree.aggregators());
}
out.writeNamedWriteableList(pipelineTree.aggregators());
}
} else {
out.writeNamedWriteableList(getInternalAggregations());

View File

@ -157,9 +157,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
}
trackScores = in.readBoolean();
version = in.readBoolean();
if (in.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
seqNoAndPrimaryTerm = in.readBoolean();
}
seqNoAndPrimaryTerm = in.readBoolean();
if (in.getVersion().onOrAfter(LegacyESVersion.V_7_10_0)) {
if (in.readBoolean()) {
@ -193,9 +191,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
}
out.writeBoolean(trackScores);
out.writeBoolean(version);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeBoolean(seqNoAndPrimaryTerm);
}
out.writeBoolean(seqNoAndPrimaryTerm);
if (out.getVersion().onOrAfter(LegacyESVersion.V_7_10_0)) {
out.writeBoolean(fetchFields != null);

View File

@ -252,11 +252,7 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
timeout = in.readOptionalTimeValue();
trackScores = in.readBoolean();
version = in.readOptionalBoolean();
if (in.getVersion().onOrAfter(LegacyESVersion.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);
@ -322,9 +318,7 @@ public final class SearchSourceBuilder implements Writeable, ToXContentObject, R
out.writeOptionalTimeValue(timeout);
out.writeBoolean(trackScores);
out.writeOptionalBoolean(version);
if (out.getVersion().onOrAfter(LegacyESVersion.V_6_7_0)) {
out.writeOptionalBoolean(seqNoAndPrimaryTerm);
}
out.writeOptionalBoolean(seqNoAndPrimaryTerm);
out.writeNamedWriteableList(extBuilders);
out.writeBoolean(profile);
out.writeOptionalWriteable(searchAfterBuilder);

View File

@ -300,12 +300,6 @@ public class BuildTests extends OpenSearchTestCase {
);
final List<Version> versions = Version.getDeclaredVersions(LegacyESVersion.class);
final Version post67Pre70Version = randomFrom(
versions.stream()
.filter(v -> v.onOrAfter(LegacyESVersion.V_6_7_0) && v.before(LegacyESVersion.V_7_0_0))
.collect(Collectors.toList())
);
final Version post70Version = randomFrom(
versions.stream().filter(v -> v.onOrAfter(LegacyESVersion.V_7_0_0)).collect(Collectors.toList())
);
@ -313,7 +307,6 @@ public class BuildTests extends OpenSearchTestCase {
versions.stream().filter(v -> v.onOrAfter(Version.V_1_0_0)).collect(Collectors.toList())
);
final WriteableBuild post67pre70 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post67Pre70Version);
final WriteableBuild post70 = copyWriteable(dockerBuild, writableRegistry(), WriteableBuild::new, post70Version);
final WriteableBuild post10OpenSearch = copyWriteable(
dockerBuild,
@ -322,10 +315,8 @@ public class BuildTests extends OpenSearchTestCase {
post10OpenSearchVersion
);
assertThat(post67pre70.build.type(), equalTo(dockerBuild.build.type()));
assertThat(post70.build.type(), equalTo(dockerBuild.build.type()));
assertThat(post67pre70.build.getQualifiedVersion(), equalTo(post67Pre70Version.toString()));
assertThat(post70.build.getQualifiedVersion(), equalTo(dockerBuild.build.getQualifiedVersion()));
assertThat(post70.build.getDistribution(), equalTo(dockerBuild.build.getDistribution()));
assertThat(post10OpenSearch.build.getQualifiedVersion(), equalTo(dockerBuild.build.getQualifiedVersion()));

View File

@ -921,7 +921,7 @@ public class ExceptionSerializationTests extends OpenSearchTestCase {
public void testSnapshotInProgressException() throws IOException {
SnapshotInProgressException orig = new SnapshotInProgressException("boom");
Version version = VersionUtils.randomVersionBetween(random(), LegacyESVersion.V_6_7_0, Version.CURRENT);
Version version = VersionUtils.randomIndexCompatibleVersion(random());
SnapshotInProgressException ex = serialize(orig, version);
assertEquals(orig.getMessage(), ex.getMessage());
}

View File

@ -282,7 +282,7 @@ public class LegacyESVersionTests extends OpenSearchTestCase {
public void testIsCompatible() {
assertTrue(isCompatible(LegacyESVersion.V_6_8_0, LegacyESVersion.V_7_0_0));
assertFalse(isCompatible(LegacyESVersion.fromString("6.6.0"), LegacyESVersion.V_7_0_0));
assertFalse(isCompatible(LegacyESVersion.V_6_7_0, LegacyESVersion.V_7_0_0));
assertFalse(isCompatible(LegacyESVersion.fromString("6.7.0"), LegacyESVersion.V_7_0_0));
assertFalse(isCompatible(LegacyESVersion.fromId(5000099), LegacyESVersion.fromString("6.0.0")));
assertFalse(isCompatible(LegacyESVersion.fromId(5000099), LegacyESVersion.fromString("7.0.0")));

View File

@ -55,10 +55,7 @@ import java.util.List;
import static java.util.Collections.emptyMap;
import static org.opensearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
public class SearchRequestTests extends AbstractSearchTestCase {
@ -115,15 +112,9 @@ public class SearchRequestTests extends AbstractSearchTestCase {
} else {
assertEquals(searchRequest.isCcsMinimizeRoundtrips(), deserializedRequest.isCcsMinimizeRoundtrips());
}
if (version.before(LegacyESVersion.V_6_7_0)) {
assertNull(deserializedRequest.getLocalClusterAlias());
assertAbsoluteStartMillisIsCurrentTime(deserializedRequest);
assertTrue(deserializedRequest.isFinalReduce());
} else {
assertEquals(searchRequest.getLocalClusterAlias(), deserializedRequest.getLocalClusterAlias());
assertEquals(searchRequest.getAbsoluteStartMillis(), deserializedRequest.getAbsoluteStartMillis());
assertEquals(searchRequest.isFinalReduce(), deserializedRequest.isFinalReduce());
}
assertEquals(searchRequest.getLocalClusterAlias(), deserializedRequest.getLocalClusterAlias());
assertEquals(searchRequest.getAbsoluteStartMillis(), deserializedRequest.getAbsoluteStartMillis());
assertEquals(searchRequest.isFinalReduce(), deserializedRequest.isFinalReduce());
if (version.onOrAfter(Version.V_1_1_0)) {
assertEquals(searchRequest.getCancelAfterTimeInterval(), deserializedRequest.getCancelAfterTimeInterval());
@ -132,13 +123,6 @@ public class SearchRequestTests extends AbstractSearchTestCase {
}
}
private static void assertAbsoluteStartMillisIsCurrentTime(SearchRequest searchRequest) {
long before = System.currentTimeMillis();
long absoluteStartMillis = searchRequest.getOrCreateAbsoluteStartMillis();
long after = System.currentTimeMillis();
assertThat(absoluteStartMillis, allOf(greaterThanOrEqualTo(before), lessThanOrEqualTo(after)));
}
public void testIllegalArguments() {
SearchRequest searchRequest = new SearchRequest();
assertNotNull(searchRequest.indices());

View File

@ -34,7 +34,6 @@ package org.opensearch.cluster.action.shard;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.util.SetOnce;
import org.opensearch.LegacyESVersion;
import org.opensearch.Version;
import org.opensearch.action.ActionListener;
import org.opensearch.action.support.replication.ClusterStateCreationUtils;
@ -570,11 +569,7 @@ public class ShardStateActionTests extends OpenSearchTestCase {
final StartedShardEntry deserialized = new StartedShardEntry(in);
assertThat(deserialized.shardId, equalTo(shardId));
assertThat(deserialized.allocationId, equalTo(allocationId));
if (version.onOrAfter(LegacyESVersion.V_6_7_0)) {
assertThat(deserialized.primaryTerm, equalTo(primaryTerm));
} else {
assertThat(deserialized.primaryTerm, equalTo(0L));
}
assertThat(deserialized.primaryTerm, equalTo(primaryTerm));
assertThat(deserialized.message, equalTo(message));
}
}

View File

@ -32,7 +32,6 @@
package org.opensearch.cluster.block;
import org.opensearch.LegacyESVersion;
import org.opensearch.Version;
import org.opensearch.common.UUIDs;
import org.opensearch.common.collect.ImmutableOpenMap;
@ -60,7 +59,7 @@ public class ClusterBlockTests extends OpenSearchTestCase {
int iterations = randomIntBetween(5, 20);
for (int i = 0; i < iterations; i++) {
Version version = randomVersion(random());
ClusterBlock clusterBlock = randomClusterBlock(version);
ClusterBlock clusterBlock = randomClusterBlock();
BytesStreamOutput out = new BytesStreamOutput();
out.setVersion(version);
@ -138,11 +137,7 @@ public class ClusterBlockTests extends OpenSearchTestCase {
}
private ClusterBlock randomClusterBlock() {
return randomClusterBlock(randomVersion(random()));
}
private ClusterBlock randomClusterBlock(final Version version) {
final String uuid = (version.onOrAfter(LegacyESVersion.V_6_7_0) && randomBoolean()) ? UUIDs.randomBase64UUID() : null;
final String uuid = randomBoolean() ? UUIDs.randomBase64UUID() : null;
final List<ClusterBlockLevel> levels = Arrays.asList(ClusterBlockLevel.values());
return new ClusterBlock(
randomInt(),

View File

@ -54,7 +54,6 @@ import java.util.HashSet;
import java.util.Map;
import static org.mockito.Matchers.anyBoolean;
import static org.opensearch.test.VersionUtils.getPreviousVersion;
import static org.opensearch.test.VersionUtils.maxCompatibleVersion;
import static org.opensearch.test.VersionUtils.randomCompatibleVersion;
import static org.opensearch.test.VersionUtils.randomVersion;
@ -108,7 +107,7 @@ public class JoinTaskExecutorTests extends OpenSearchTestCase {
final Version maxNodeVersion = nodes.getMaxNodeVersion();
final Version minNodeVersion = nodes.getMinNodeVersion();
if (maxNodeVersion.onOrAfter(LegacyESVersion.V_7_0_0)) {
final Version tooLow = getPreviousVersion(maxNodeVersion.minimumCompatibilityVersion());
final Version tooLow = LegacyESVersion.fromString("6.7.0");
expectThrows(IllegalStateException.class, () -> {
if (randomBoolean()) {
JoinTaskExecutor.ensureNodesCompatibility(tooLow, nodes);

View File

@ -32,7 +32,6 @@
package org.opensearch.indices;
import org.opensearch.LegacyESVersion;
import org.opensearch.common.io.stream.BytesStreamOutput;
import org.opensearch.common.io.stream.StreamInput;
import org.opensearch.common.xcontent.XContentParser;
@ -98,15 +97,6 @@ public class TermsLookupTests extends OpenSearchTestCase {
assertNotSame(deserializedLookup, termsLookup);
}
}
try (BytesStreamOutput output = new BytesStreamOutput()) {
output.setVersion(LegacyESVersion.V_6_7_0);
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> termsLookup.writeTo(output));
assertEquals(
"Typeless [terms] lookup queries are not supported if any " + "node is running a version before 7.0.",
e.getMessage()
);
}
}
public void testSerializationWithTypes() throws IOException {