Skip to content

Commit

Permalink
Merge branch 'main' into add_node_reduction
Browse files Browse the repository at this point in the history
  • Loading branch information
elasticmachine authored Apr 25, 2024
2 parents 3f33b73 + b6b20a5 commit bf37966
Show file tree
Hide file tree
Showing 23 changed files with 83 additions and 47 deletions.
6 changes: 6 additions & 0 deletions docs/changelog/107828.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,6 @@
pr: 107828
summary: Update several references to `IndexVersion.toString` to use `toReleaseVersion`
area: Infra/Core
type: bug
issues:
- 107821
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@
import org.elasticsearch.xcontent.XContentParser;
import org.elasticsearch.xcontent.XContentType;
import org.elasticsearch.xcontent.json.JsonXContent;
import org.hamcrest.Matchers;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.rules.RuleChain;
Expand Down Expand Up @@ -81,8 +80,8 @@
import static org.elasticsearch.test.MapMatcher.matchesMap;
import static org.elasticsearch.transport.RemoteClusterService.REMOTE_CLUSTER_COMPRESS;
import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.either;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
Expand Down Expand Up @@ -1289,7 +1288,11 @@ private void checkSnapshot(String snapshotName, int count, String tookOnVersion,
// the format can change depending on the ES node version running & this test code running
assertThat(
XContentMapValues.extractValue("snapshots.version", snapResponse),
either(Matchers.<Object>equalTo(List.of(tookOnVersion))).or(equalTo(List.of(tookOnIndexVersion.toString())))
anyOf(
equalTo(List.of(tookOnVersion)),
equalTo(List.of(tookOnIndexVersion.toString())),
equalTo(List.of(tookOnIndexVersion.toReleaseVersion()))
)
);

// Remove the routing setting and template so we can test restoring them.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ public void testGetFeatureUpgradeStatus() throws Exception {
.orElse(Collections.emptyMap());

assertThat(feature, aMapWithSize(4));
assertThat(feature.get("minimum_index_version"), equalTo(getOldClusterIndexVersion().toString()));
assertThat(feature.get("minimum_index_version"), equalTo(getOldClusterIndexVersion().toReleaseVersion()));

// Feature migration happens only across major versions; also, we usually begin to require migrations once we start testing
// for the next major version upgrade (see e.g. #93666). Trying to express this with features may be problematic, so we
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ public void testGetFeatureUpgradedStatuses() throws Exception {
.orElse(Collections.emptyMap());

assertThat(testFeature.size(), equalTo(4));
assertThat(testFeature.get("minimum_index_version"), equalTo(IndexVersion.current().toString()));
assertThat(testFeature.get("minimum_index_version"), equalTo(IndexVersion.current().toReleaseVersion()));
assertThat(testFeature.get("migration_status"), equalTo("NO_MIGRATION_NEEDED"));
assertThat(testFeature.get("indices"), instanceOf(List.class));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -890,9 +890,9 @@ public void testFailOnAncientVersion() throws Exception {
snapshotRestoreException.getMessage(),
containsString(
"the snapshot was created with Elasticsearch version ["
+ oldVersion
+ oldVersion.toReleaseVersion()
+ "] which is below the current versions minimum index compatibility version ["
+ IndexVersions.MINIMUM_COMPATIBLE
+ IndexVersions.MINIMUM_COMPATIBLE.toReleaseVersion()
+ "]"
)
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,7 +185,7 @@ public void writeTo(StreamOutput out) throws IOException {
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("feature_name", this.featureName);
builder.field("minimum_index_version", this.minimumIndexVersion.toString());
builder.field("minimum_index_version", this.minimumIndexVersion.toReleaseVersion());
builder.field("migration_status", this.upgradeStatus);
builder.startArray("indices");
for (IndexInfo version : this.indexInfos) {
Expand Down Expand Up @@ -300,7 +300,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws

builder.startObject();
builder.field("index", this.indexName);
builder.field("version", this.version.toString());
builder.field("version", this.version.toReleaseVersion());
if (exception != null) {
builder.startObject("failure_cause");
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -168,7 +168,7 @@ static class SingleVersionStats implements ToXContentObject, Writeable, Comparab
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("version", version.toString());
builder.field("version", version.toReleaseVersion());
builder.field("index_count", indexCount);
builder.field("primary_shard_count", primaryShardCount);
builder.humanReadableField("total_primary_bytes", "total_primary_size", ByteSizeValue.ofBytes(totalPrimaryByteCount));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -362,19 +362,19 @@ public static void ensureIndexCompatibility(IndexVersion minSupportedVersion, In
"index "
+ idxMetadata.getIndex()
+ " version not supported: "
+ idxMetadata.getCompatibilityVersion()
+ idxMetadata.getCompatibilityVersion().toReleaseVersion()
+ " maximum compatible index version is: "
+ maxSupportedVersion
+ maxSupportedVersion.toReleaseVersion()
);
}
if (idxMetadata.getCompatibilityVersion().before(minSupportedVersion)) {
throw new IllegalStateException(
"index "
+ idxMetadata.getIndex()
+ " version not supported: "
+ idxMetadata.getCompatibilityVersion()
+ idxMetadata.getCompatibilityVersion().toReleaseVersion()
+ " minimum compatible index version is: "
+ minSupportedVersion
+ minSupportedVersion.toReleaseVersion()
);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -383,11 +383,11 @@ public void validate(final IndexVersion compatibilityVersion, final Map<Setting<
throw new IllegalArgumentException(
SETTING_VERSION_COMPATIBILITY
+ " ["
+ compatibilityVersion
+ compatibilityVersion.toReleaseVersion()
+ "] must be >= "
+ SETTING_VERSION_CREATED
+ " ["
+ createdVersion
+ createdVersion.toReleaseVersion()
+ "]"
);
}
Expand Down Expand Up @@ -2629,7 +2629,7 @@ public static IndexMetadata legacyFromXContent(XContentParser parser) throws IOE
throw new IllegalStateException(
"this method should only be used to parse older incompatible index metadata versions "
+ "but got "
+ SETTING_INDEX_VERSION_COMPATIBILITY.get(settings)
+ SETTING_INDEX_VERSION_COMPATIBILITY.get(settings).toReleaseVersion()
);
}
builder.settings(settings);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,9 @@ private static void checkSupportedVersion(IndexMetadata indexMetadata, IndexVers
"The index "
+ indexMetadata.getIndex()
+ " has current compatibility version ["
+ indexMetadata.getCompatibilityVersion()
+ indexMetadata.getCompatibilityVersion().toReleaseVersion()
+ "] but the minimum compatible version is ["
+ minimumIndexCompatibilityVersion
+ minimumIndexCompatibilityVersion.toReleaseVersion()
+ "]. It should be re-indexed in Elasticsearch "
+ (Version.CURRENT.major - 1)
+ ".x before upgrading to "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ public Type getType() {
public void addAdditionalFields(XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.field("repository", snapshot.getRepository())
.field("snapshot", snapshot.getSnapshotId().getName())
.field("version", version.toString())
.field("version", version.toReleaseVersion())
.field("index", index.getName())
.field("restoreUUID", restoreUUID);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ public ByteArrayIndexInput(String resourceDesc, byte[] bytes, int offset, int le
super(resourceDesc);
this.bytes = bytes;
this.offset = offset;
this.pos = offset;
this.length = length;
}

Expand All @@ -43,7 +44,7 @@ public void close() throws IOException {}

@Override
public long getFilePointer() {
return pos;
return pos - offset;
}

@Override
Expand All @@ -57,7 +58,7 @@ private int position(long p) throws EOFException {
} else if (p > length) {
throw new EOFException("seek past EOF");
}
return (int) p;
return (int) p + offset;
}

@Override
Expand Down Expand Up @@ -108,22 +109,22 @@ public byte readByte() throws IOException {
if (pos >= offset + length) {
throw new EOFException("seek past EOF");
}
return bytes[offset + pos++];
return bytes[pos++];
}

@Override
public void readBytes(final byte[] b, final int offset, int len) throws IOException {
if (pos + len > this.offset + length) {
throw new EOFException("seek past EOF");
}
System.arraycopy(bytes, this.offset + pos, b, offset, len);
System.arraycopy(bytes, pos, b, offset, len);
pos += len;
}

@Override
public short readShort() throws IOException {
try {
return (short) BitUtil.VH_LE_SHORT.get(bytes, pos + offset);
return (short) BitUtil.VH_LE_SHORT.get(bytes, pos);
} finally {
pos += Short.BYTES;
}
Expand All @@ -132,7 +133,7 @@ public short readShort() throws IOException {
@Override
public int readInt() throws IOException {
try {
return (int) BitUtil.VH_LE_INT.get(bytes, pos + offset);
return (int) BitUtil.VH_LE_INT.get(bytes, pos);
} finally {
pos += Integer.BYTES;
}
Expand All @@ -141,7 +142,7 @@ public int readInt() throws IOException {
@Override
public long readLong() throws IOException {
try {
return (long) BitUtil.VH_LE_LONG.get(bytes, pos + offset);
return (long) BitUtil.VH_LE_LONG.get(bytes, pos);
} finally {
pos += Long.BYTES;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -528,13 +528,13 @@ static void checkForIndexCompatibility(Logger logger, DataPath... dataPaths) thr
String bestDowngradeVersion = getBestDowngradeVersion(metadata.previousNodeVersion().toString());
throw new IllegalStateException(
"Cannot start this node because it holds metadata for indices with version ["
+ metadata.oldestIndexVersion()
+ metadata.oldestIndexVersion().toReleaseVersion()
+ "] with which this node of version ["
+ Build.current().version()
+ "] is incompatible. Revert this node to version ["
+ bestDowngradeVersion
+ "] and delete any indices with versions earlier than ["
+ IndexVersions.MINIMUM_COMPATIBLE
+ IndexVersions.MINIMUM_COMPATIBLE.toReleaseVersion()
+ "] before upgrading to version ["
+ Build.current().version()
+ "]. If all such indices have already been deleted, revert this node to version ["
Expand Down
11 changes: 8 additions & 3 deletions server/src/main/java/org/elasticsearch/index/IndexSettings.java
Original file line number Diff line number Diff line change
Expand Up @@ -1114,16 +1114,21 @@ public synchronized boolean updateIndexMetadata(IndexMetadata indexMetadata) {
final Settings newSettings = indexMetadata.getSettings();
IndexVersion newIndexVersion = SETTING_INDEX_VERSION_CREATED.get(newSettings);
if (version.equals(newIndexVersion) == false) {
throw new IllegalArgumentException("version mismatch on settings update expected: " + version + " but was: " + newIndexVersion);
throw new IllegalArgumentException(
"version mismatch on settings update expected: "
+ version.toReleaseVersion()
+ " but was: "
+ newIndexVersion.toReleaseVersion()
);
}
IndexVersion newCompatibilityVersion = IndexMetadata.SETTING_INDEX_VERSION_COMPATIBILITY.get(newSettings);
IndexVersion compatibilityVersion = IndexMetadata.SETTING_INDEX_VERSION_COMPATIBILITY.get(settings);
if (compatibilityVersion.equals(newCompatibilityVersion) == false) {
throw new IllegalArgumentException(
"compatibility version mismatch on settings update expected: "
+ compatibilityVersion
+ compatibilityVersion.toReleaseVersion()
+ " but was: "
+ newCompatibilityVersion
+ newCompatibilityVersion.toReleaseVersion()
);
}
final String newUUID = newSettings.get(IndexMetadata.SETTING_INDEX_UUID, IndexMetadata.INDEX_UUID_NA_VALUE);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,9 +103,9 @@ public RepositoriesModule(
throw new SnapshotRestoreException(
snapshot,
"the snapshot was created with Elasticsearch version ["
+ version
+ version.toReleaseVersion()
+ "] which is below the current versions minimum index compatibility version ["
+ IndexVersions.MINIMUM_COMPATIBLE
+ IndexVersions.MINIMUM_COMPATIBLE.toReleaseVersion()
+ "]"
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2346,7 +2346,7 @@ private void cacheRepositoryData(RepositoryData repositoryData, IndexVersion ver
toCache = repositoryData.withoutShardGenerations();
assert repositoryData.indexMetaDataGenerations().equals(IndexMetaDataGenerations.EMPTY)
: "repository data should not contain index generations at version ["
+ version
+ version.toReleaseVersion()
+ "] but saw ["
+ repositoryData.indexMetaDataGenerations()
+ "]";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -567,7 +567,7 @@ public XContentBuilder toXContentExternal(final XContentBuilder builder, final T

if (version != null) {
builder.field(VERSION_ID, version.id());
builder.field(VERSION, version.toString());
builder.field(VERSION, version.toReleaseVersion());
}

if (params.paramAsBoolean(INDEX_NAMES_XCONTENT_PARAM, true)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,10 +116,10 @@ public void testIncompatibleVersion() {
"The index [foo/"
+ metadata.getIndexUUID()
+ "] has current compatibility version ["
+ indexCreated
+ indexCreated.toReleaseVersion()
+ "] "
+ "but the minimum compatible version is ["
+ minCompat
+ minCompat.toReleaseVersion()
+ "]. It should be re-indexed in Elasticsearch "
+ (Version.CURRENT.major - 1)
+ ".x before upgrading to "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -601,7 +601,7 @@ public void testIndexCompatibilityChecks() throws IOException {
ex.getMessage(),
allOf(
containsString("Cannot start this node"),
containsString("it holds metadata for indices with version [" + oldIndexVersion + "]"),
containsString("it holds metadata for indices with version [" + oldIndexVersion.toReleaseVersion() + "]"),
containsString(
"Revert this node to version ["
+ (previousNodeVersion.major == Version.V_8_0_0.major ? Version.V_7_17_0 : previousNodeVersion)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
package org.elasticsearch.common.lucene.store;

import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.RandomAccessInput;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.common.settings.Settings;
Expand Down Expand Up @@ -73,16 +74,36 @@ protected byte[] randomReadAndSlice(IndexInput indexInput, int length) throws IO
switch (readStrategy) {
case 0, 1, 2, 3:
if (length - readPos >= Long.BYTES && readStrategy <= 0) {
ByteBuffer.wrap(output, readPos, Long.BYTES).order(ByteOrder.LITTLE_ENDIAN).putLong(indexInput.readLong());
long read = indexInput.readLong();
ByteBuffer.wrap(output, readPos, Long.BYTES).order(ByteOrder.LITTLE_ENDIAN).putLong(read);
readPos += Long.BYTES;
if (indexInput instanceof RandomAccessInput randomAccessInput) {
assertEquals(read, randomAccessInput.readLong(indexInput.getFilePointer() - Long.BYTES));
indexInput.seek(readPos);
}
} else if (length - readPos >= Integer.BYTES && readStrategy <= 1) {
ByteBuffer.wrap(output, readPos, Integer.BYTES).order(ByteOrder.LITTLE_ENDIAN).putInt(indexInput.readInt());
int read = indexInput.readInt();
ByteBuffer.wrap(output, readPos, Integer.BYTES).order(ByteOrder.LITTLE_ENDIAN).putInt(read);
readPos += Integer.BYTES;
if (indexInput instanceof RandomAccessInput randomAccessInput) {
assertEquals(read, randomAccessInput.readInt(indexInput.getFilePointer() - Integer.BYTES));
indexInput.seek(readPos);
}
} else if (length - readPos >= Short.BYTES && readStrategy <= 2) {
ByteBuffer.wrap(output, readPos, Short.BYTES).order(ByteOrder.LITTLE_ENDIAN).putShort(indexInput.readShort());
short read = indexInput.readShort();
ByteBuffer.wrap(output, readPos, Short.BYTES).order(ByteOrder.LITTLE_ENDIAN).putShort(read);
readPos += Short.BYTES;
if (indexInput instanceof RandomAccessInput randomAccessInput) {
assertEquals(read, randomAccessInput.readShort(indexInput.getFilePointer() - Short.BYTES));
indexInput.seek(readPos);
}
} else {
output[readPos++] = indexInput.readByte();
byte read = indexInput.readByte();
output[readPos++] = read;
if (indexInput instanceof RandomAccessInput randomAccessInput) {
assertEquals(read, randomAccessInput.readByte(indexInput.getFilePointer() - 1));
indexInput.seek(readPos);
}
}
break;
case 4:
Expand Down
Loading

0 comments on commit bf37966

Please sign in to comment.