Skip to content

Commit

Permalink
Remove Version 6 pre-release constants (elastic#41517)
Browse files Browse the repository at this point in the history
Now that master is 8.0, we can remove uses of these constants and the
backcompat code that uses them, since 8 will always walk to 7.0+ nodes.
This PR starts by removing the pre-6 release constants, remove obsolete code
and replacing its occurances in tests where needed.

Relates to elastic#41164
  • Loading branch information
Christoph Büscher authored and Gurkan Kaymak committed May 27, 2019
1 parent 6414c0c commit 8004ac5
Show file tree
Hide file tree
Showing 36 changed files with 114 additions and 251 deletions.
34 changes: 1 addition & 33 deletions server/src/main/java/org/elasticsearch/Version.java
Original file line number Diff line number Diff line change
Expand Up @@ -46,24 +46,6 @@ public class Version implements Comparable<Version>, ToXContentFragment {
*/
public static final int V_EMPTY_ID = 0;
public static final Version V_EMPTY = new Version(V_EMPTY_ID, org.apache.lucene.util.Version.LATEST);
public static final int V_6_0_0_alpha1_ID = 6000001;
public static final Version V_6_0_0_alpha1 =
new Version(V_6_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_7_0_0);
public static final int V_6_0_0_alpha2_ID = 6000002;
public static final Version V_6_0_0_alpha2 =
new Version(V_6_0_0_alpha2_ID, org.apache.lucene.util.Version.LUCENE_7_0_0);
public static final int V_6_0_0_beta1_ID = 6000026;
public static final Version V_6_0_0_beta1 =
new Version(V_6_0_0_beta1_ID, org.apache.lucene.util.Version.LUCENE_7_0_0);
public static final int V_6_0_0_beta2_ID = 6000027;
public static final Version V_6_0_0_beta2 =
new Version(V_6_0_0_beta2_ID, org.apache.lucene.util.Version.LUCENE_7_0_0);
public static final int V_6_0_0_rc1_ID = 6000051;
public static final Version V_6_0_0_rc1 =
new Version(V_6_0_0_rc1_ID, org.apache.lucene.util.Version.LUCENE_7_0_0);
public static final int V_6_0_0_rc2_ID = 6000052;
public static final Version V_6_0_0_rc2 =
new Version(V_6_0_0_rc2_ID, org.apache.lucene.util.Version.LUCENE_7_0_1);
public static final int V_6_0_0_ID = 6000099;
public static final Version V_6_0_0 =
new Version(V_6_0_0_ID, org.apache.lucene.util.Version.LUCENE_7_0_1);
Expand Down Expand Up @@ -234,18 +216,6 @@ public static Version fromId(int id) {
return V_6_0_1;
case V_6_0_0_ID:
return V_6_0_0;
case V_6_0_0_rc2_ID:
return V_6_0_0_rc2;
case V_6_0_0_beta2_ID:
return V_6_0_0_beta2;
case V_6_0_0_rc1_ID:
return V_6_0_0_rc1;
case V_6_0_0_beta1_ID:
return V_6_0_0_beta1;
case V_6_0_0_alpha2_ID:
return V_6_0_0_alpha2;
case V_6_0_0_alpha1_ID:
return V_6_0_0_alpha1;
case V_EMPTY_ID:
return V_EMPTY;
default:
Expand Down Expand Up @@ -445,7 +415,7 @@ public Version minimumCompatibilityVersion() {
return bwcVersion == null ? this : bwcVersion;
}

return Version.min(this, fromId((int) major * 1000000 + 0 * 10000 + 99));
return Version.min(this, fromId(major * 1000000 + 0 * 10000 + 99));
}

/**
Expand All @@ -457,8 +427,6 @@ public Version minimumIndexCompatibilityVersion() {
final int bwcMajor;
if (major == 5) {
bwcMajor = 2; // we jumped from 2 to 5
} else if (major == 7) {
return V_6_0_0_beta1;
} else {
bwcMajor = major - 1;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,9 @@
package org.elasticsearch.cluster.metadata;

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.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ResourceAlreadyExistsException;
Expand Down Expand Up @@ -714,13 +715,6 @@ static void validateSplitIndex(ClusterState state, String sourceIndex,
Settings targetIndexSettings) {
IndexMetaData sourceMetaData = validateResize(state, sourceIndex, targetIndexMappingsTypes, targetIndexName, targetIndexSettings);
IndexMetaData.selectSplitShard(0, sourceMetaData, IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.get(targetIndexSettings));
if (sourceMetaData.getCreationVersion().before(Version.V_6_0_0_alpha1)) {
// ensure we have a single type since this would make the splitting code considerably more complex
// and a 5.x index would not be splittable unless it has been shrunk before so rather opt out of the complexity
// since in 5.x we don't have a setting to artificially set the number of routing shards
throw new IllegalStateException("source index created version is too old to apply a split operation");
}

}

static IndexMetaData validateResize(ClusterState state, String sourceIndex,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

package org.elasticsearch.cluster.routing;

import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
Expand Down Expand Up @@ -194,16 +193,7 @@ private ShardIterator preferenceActiveShardIterator(IndexShardRoutingTable index
}
}
// if not, then use it as the index
int routingHash = Murmur3HashFunction.hash(preference);
if (nodes.getMinNodeVersion().onOrAfter(Version.V_6_0_0_alpha1)) {
// The AllocationService lists shards in a fixed order based on nodes
// so earlier versions of this class would have a tendency to
// select the same node across different shardIds.
// Better overall balancing can be achieved if each shardId opts
// for a different element in the list by also incorporating the
// shard ID into the hash of the user-supplied preference key.
routingHash = 31 * routingHash + indexShard.shardId.hashCode();
}
int routingHash = 31 * Murmur3HashFunction.hash(preference) + indexShard.shardId.hashCode();
if (awarenessAttributes.isEmpty()) {
return indexShard.activeInitializingShardsIt(routingHash);
} else {
Expand Down
10 changes: 0 additions & 10 deletions server/src/main/java/org/elasticsearch/index/IndexSortConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.SortedNumericSortField;
import org.apache.lucene.search.SortedSetSortField;
import org.elasticsearch.Version;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.fielddata.IndexFieldData;
Expand Down Expand Up @@ -120,15 +119,6 @@ public IndexSortConfig(IndexSettings indexSettings) {
.map((name) -> new FieldSortSpec(name))
.toArray(FieldSortSpec[]::new);

if (sortSpecs.length > 0 && indexSettings.getIndexVersionCreated().before(Version.V_6_0_0_alpha1)) {
/**
* This index might be assigned to a node where the index sorting feature is not available
* (ie. versions prior to {@link Version.V_6_0_0_alpha1_UNRELEASED}) so we must fail here rather than later.
*/
throw new IllegalArgumentException("unsupported index.version.created:" + indexSettings.getIndexVersionCreated() +
", can't set index.sort on versions prior to " + Version.V_6_0_0_alpha1);
}

if (INDEX_SORT_ORDER_SETTING.exists(settings)) {
List<SortOrder> orders = INDEX_SORT_ORDER_SETTING.get(settings);
if (orders.size() != sortSpecs.length) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,19 +202,7 @@ public static DynamicTemplate parse(String name, Map<String, Object> conf,

XContentFieldType xcontentFieldType = null;
if (matchMappingType != null && matchMappingType.equals("*") == false) {
try {
xcontentFieldType = XContentFieldType.fromString(matchMappingType);
} catch (IllegalArgumentException e) {
if (indexVersionCreated.onOrAfter(Version.V_6_0_0_alpha1)) {
throw e;
} else {
deprecationLogger.deprecated("match_mapping_type [" + matchMappingType + "] is invalid and will be ignored: "
+ e.getMessage());
// this template is on an unknown type so it will never match anything
// null indicates that the template should be ignored
return null;
}
}
xcontentFieldType = XContentFieldType.fromString(matchMappingType);
}

final MatchType matchType = MatchType.fromString(matchPattern);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

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;
Expand Down Expand Up @@ -758,10 +759,9 @@ public synchronized void activatePrimaryMode(final long localCheckpoint) {
* @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master
* @param inSyncAllocationIds the allocation IDs of the currently in-sync shard copies
* @param routingTable the shard routing table
* @param pre60AllocationIds the allocation IDs of shards that are allocated to pre-6.0 nodes
*/
public synchronized void updateFromMaster(final long applyingClusterStateVersion, final Set<String> inSyncAllocationIds,
final IndexShardRoutingTable routingTable, final Set<String> pre60AllocationIds) {
final IndexShardRoutingTable routingTable) {
assert invariant();
if (applyingClusterStateVersion > appliedClusterStateVersion) {
// check that the master does not fabricate new in-sync entries out of thin air once we are in primary mode
Expand All @@ -782,8 +782,7 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion
final boolean inSync = inSyncAllocationIds.contains(initializingId);
assert inSync == false : "update from master in primary mode has " + initializingId +
" as in-sync but it does not exist locally";
final long localCheckpoint = pre60AllocationIds.contains(initializingId) ?
SequenceNumbers.PRE_60_NODE_CHECKPOINT : SequenceNumbers.UNASSIGNED_SEQ_NO;
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
final long globalCheckpoint = localCheckpoint;
checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, inSync, inSync));
}
Expand All @@ -794,8 +793,7 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion
} else {
for (String initializingId : initializingAllocationIds) {
if (shardAllocationId.equals(initializingId) == false) {
final long localCheckpoint = pre60AllocationIds.contains(initializingId) ?
SequenceNumbers.PRE_60_NODE_CHECKPOINT : SequenceNumbers.UNASSIGNED_SEQ_NO;
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
final long globalCheckpoint = localCheckpoint;
checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false));
}
Expand All @@ -807,8 +805,7 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion
checkpointState.inSync = true;
checkpointState.tracked = true;
} else {
final long localCheckpoint = pre60AllocationIds.contains(inSyncId) ?
SequenceNumbers.PRE_60_NODE_CHECKPOINT : SequenceNumbers.UNASSIGNED_SEQ_NO;
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
final long globalCheckpoint = localCheckpoint;
checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true));
}
Expand Down Expand Up @@ -1082,17 +1079,13 @@ private Runnable getMasterUpdateOperationFromCurrentState() {
assert primaryMode == false;
final long lastAppliedClusterStateVersion = appliedClusterStateVersion;
final Set<String> inSyncAllocationIds = new HashSet<>();
final Set<String> pre60AllocationIds = new HashSet<>();
checkpoints.entrySet().forEach(entry -> {
if (entry.getValue().inSync) {
inSyncAllocationIds.add(entry.getKey());
}
if (entry.getValue().getLocalCheckpoint() == SequenceNumbers.PRE_60_NODE_CHECKPOINT) {
pre60AllocationIds.add(entry.getKey());
}
});
final IndexShardRoutingTable lastAppliedRoutingTable = routingTable;
return () -> updateFromMaster(lastAppliedClusterStateVersion, inSyncAllocationIds, lastAppliedRoutingTable, pre60AllocationIds);
return () -> updateFromMaster(lastAppliedClusterStateVersion, inSyncAllocationIds, lastAppliedRoutingTable);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.elasticsearch.index.shard;

import com.carrotsearch.hppc.ObjectLongMap;

import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.index.CheckIndex;
Expand Down Expand Up @@ -433,8 +434,7 @@ public void updateShardState(final ShardRouting newRouting,
final BiConsumer<IndexShard, ActionListener<ResyncTask>> primaryReplicaSyncer,
final long applyingClusterStateVersion,
final Set<String> inSyncAllocationIds,
final IndexShardRoutingTable routingTable,
final Set<String> pre60AllocationIds) throws IOException {
final IndexShardRoutingTable routingTable) throws IOException {
final ShardRouting currentRouting;
synchronized (mutex) {
currentRouting = this.shardRouting;
Expand All @@ -453,7 +453,7 @@ public void updateShardState(final ShardRouting newRouting,
}

if (newRouting.primary()) {
replicationTracker.updateFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable, pre60AllocationIds);
replicationTracker.updateFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable);
}

if (state == IndexShardState.POST_RECOVERY && newRouting.active()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,10 +65,6 @@ final class ShardSplittingQuery extends Query {
private final BitSetProducer nestedParentBitSetProducer;

ShardSplittingQuery(IndexMetaData indexMetaData, int shardId, boolean hasNested) {
if (indexMetaData.getCreationVersion().before(Version.V_6_0_0_rc2)) {
throw new IllegalArgumentException("Splitting query can only be executed on an index created with version "
+ Version.V_6_0_0_rc2 + " or higher");
}
this.indexMetaData = indexMetaData;
this.shardId = shardId;
this.nestedParentBitSetProducer = hasNested ? newParentDocBitSetProducer(indexMetaData.getCreationVersion()) : null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.elasticsearch.index.shard;

import com.carrotsearch.hppc.cursors.ObjectObjectCursor;

import org.apache.logging.log4j.Logger;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
Expand All @@ -31,7 +32,6 @@
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.routing.RecoverySource;
Expand Down Expand Up @@ -119,8 +119,6 @@ boolean recoverFromLocalShards(BiConsumer<String, MappingMetaData> mappingUpdate
Sort indexSort = indexShard.getIndexSort();
final boolean hasNested = indexShard.mapperService().hasNested();
final boolean isSplit = sourceMetaData.getNumberOfShards() < indexShard.indexSettings().getNumberOfShards();
assert isSplit == false || sourceMetaData.getCreationVersion().onOrAfter(Version.V_6_0_0_alpha1) : "for split we require a " +
"single type but the index is created before 6.0.0";
return executeRecovery(indexShard, () -> {
logger.debug("starting recovery from local shards {}", shards);
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.Version;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
Expand Down Expand Up @@ -216,15 +215,8 @@ private ArrayList<TranslogReader> recoverFromFiles(Checkpoint checkpoint) throws
try (ReleasableLock lock = writeLock.acquire()) {
logger.debug("open uncommitted translog checkpoint {}", checkpoint);

final long minGenerationToRecoverFrom;
if (checkpoint.minTranslogGeneration < 0) {
final Version indexVersionCreated = indexSettings().getIndexVersionCreated();
assert indexVersionCreated.before(Version.V_6_0_0_beta1) :
"no minTranslogGeneration in checkpoint, but index was created with version [" + indexVersionCreated + "]";
minGenerationToRecoverFrom = deletionPolicy.getMinTranslogGenerationForRecovery();
} else {
minGenerationToRecoverFrom = checkpoint.minTranslogGeneration;
}
final long minGenerationToRecoverFrom = checkpoint.minTranslogGeneration;
assert minGenerationToRecoverFrom >= 0 : "minTranslogGeneration should be non-negative";

final String checkpointTranslogFile = getFilename(checkpoint.generation);
// we open files in reverse order in order to validate tranlsog uuid before we start traversing the translog based on
Expand Down Expand Up @@ -882,6 +874,7 @@ public Location(long generation, long translogLocation, int size) {
this.size = size;
}

@Override
public String toString() {
return "[generation: " + generation + ", location: " + translogLocation + ", size: " + size + "]";
}
Expand Down
Loading

0 comments on commit 8004ac5

Please sign in to comment.