add missing units to index settings if index was created before 2.0

This commit is contained in:
Michael McCandless 2015-05-30 04:39:03 -04:00 committed by mikemccand
parent 6723c72716
commit 68d6427944
14 changed files with 173 additions and 28 deletions

View File

@ -240,7 +240,11 @@ def generate_index(client, version, index_name):
client.indices.create(index=index_name, body={
'settings': {
'number_of_shards': 1,
'number_of_replicas': 0
'number_of_replicas': 0,
# Same as ES default (60 seconds), but missing the units to make sure they are inserted on upgrade:
"gc_deletes": '60000',
# Same as ES default (5 GB), but missing the units to make sure they are inserted on upgrade:
"merge.policy.max_merged_segment": '5368709120'
},
'mappings': mappings
})
@ -261,7 +265,11 @@ def snapshot_index(client, version, repo_dir):
# Add bogus persistent settings to make sure they can be restored
client.cluster.put_settings(body={
'persistent': {
'cluster.routing.allocation.exclude.version_attr': version
'cluster.routing.allocation.exclude.version_attr': version,
# Same as ES default (30 seconds), but missing the units to make sure they are inserted on upgrade:
'discovery.zen.publish_timeout': '30000',
# Same as ES default (512 KB), but missing the units to make sure they are inserted on upgrade:
'indices.recovery.file_chunk_size': '524288',
}
})
client.indices.put_template(name='template_' + version.lower(), order=0, body={

View File

@ -24,7 +24,11 @@ import org.elasticsearch.cluster.routing.HashFunction;
import org.elasticsearch.cluster.routing.SimpleHashFunction;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import com.google.common.collect.ImmutableSet;
import java.util.Set;
/**
* This service is responsible for upgrading legacy index metadata to the current version
@ -73,28 +77,28 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
/**
* Checks that the index can be upgraded to the current version of the master node.
*
* If the index does need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index cannot be
* If the index does not need upgrade it returns the index metadata unchanged, otherwise it returns a modified index metadata. If index cannot be
* updated the method throws an exception.
*/
public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) throws Exception {
IndexMetaData newMetaData = indexMetaData;
newMetaData = checkSupportedVersion(newMetaData);
newMetaData = upgradeLegacyRoutingSettings(newMetaData);
// Throws an exception if there are too-old segments:
checkSupportedVersion(indexMetaData);
IndexMetaData newMetaData = upgradeLegacyRoutingSettings(indexMetaData);
newMetaData = addDefaultUnitsIfNeeded(newMetaData);
return newMetaData;
}
/**
* Elasticsearch 2.0 deprecated no longer supports indices with pre Lucene v4.0 segments. All indices
* Elasticsearch 2.0 no longer supports indices with pre Lucene v4.0 (Elasticsearch v 0.90.0) segments. All indices
* that were created before Elasticsearch v0.90.0 should be upgraded using upgrade plugin before they can
* be open by this version of elasticsearch.
*/
private IndexMetaData checkSupportedVersion(IndexMetaData indexMetaData) throws Exception {
private void checkSupportedVersion(IndexMetaData indexMetaData) throws Exception {
if (indexMetaData.getState() == IndexMetaData.State.OPEN && isSupportedVersion(indexMetaData) == false) {
throw new IllegalStateException("The index [" + indexMetaData.getIndex() + "] was created before v0.90.0 and wasn't upgraded."
+ " This index should be open using a version before " + Version.CURRENT.minimumCompatibilityVersion()
+ " and upgraded using the upgrade API.");
}
return indexMetaData;
}
/*
@ -131,4 +135,96 @@ public class MetaDataIndexUpgradeService extends AbstractComponent {
return indexMetaData;
}
/** All known byte-sized settings for an index. */
public static final Set<String> INDEX_BYTES_SIZE_SETTINGS = ImmutableSet.of(
"index.buffer_size",
"index.merge.policy.floor_segment",
"index.merge.policy.max_merged_segment",
"index.merge.policy.max_merge_size",
"index.merge.policy.min_merge_size",
"index.shard.recovery.file_chunk_size",
"index.shard.recovery.translog_size",
"index.store.throttle.max_bytes_per_sec",
"index.translog.flush_threshold_size",
"index.translog.fs.buffer_size",
"index.version_map_size");
/** All known time settings for an index. */
public static final Set<String> INDEX_TIME_SETTINGS = ImmutableSet.of(
"index.gateway.wait_for_mapping_update_post_recovery",
"index.gc_deletes",
"index.indexing.slowlog.threshold.index.debug",
"index.indexing.slowlog.threshold.index.info",
"index.indexing.slowlog.threshold.index.trace",
"index.indexing.slowlog.threshold.index.warn",
"index.refresh_interval",
"index.search.slowlog.threshold.fetch.debug",
"index.search.slowlog.threshold.fetch.info",
"index.search.slowlog.threshold.fetch.trace",
"index.search.slowlog.threshold.fetch.warn",
"index.search.slowlog.threshold.query.debug",
"index.search.slowlog.threshold.query.info",
"index.search.slowlog.threshold.query.trace",
"index.search.slowlog.threshold.query.warn",
"index.shadow.wait_for_initial_commit",
"index.store.stats_refresh_interval",
"index.translog.flush_threshold_period",
"index.translog.interval",
"index.translog.sync_interval");
/**
* Elasticsearch 2.0 requires units on byte/memory and time settings; this method adds the default unit to any such settings that are
* missing units.
*/
private IndexMetaData addDefaultUnitsIfNeeded(IndexMetaData indexMetaData) {
if (indexMetaData.getCreationVersion().before(Version.V_2_0_0)) {
// Created lazily if we find any settings that are missing units:
Settings settings = indexMetaData.settings();
Settings.Builder newSettings = null;
for(String byteSizeSetting : INDEX_BYTES_SIZE_SETTINGS) {
String value = settings.get(byteSizeSetting);
if (value != null) {
try {
Double.parseDouble(value);
} catch (NumberFormatException nfe) {
continue;
}
// It's a naked number; add default unit (b for bytes):
logger.warn("byte-sized setting [{}] with value [{}] is missing units; now adding default units (b)", byteSizeSetting, value);
if (newSettings == null) {
newSettings = Settings.builder();
newSettings.put(settings);
}
newSettings.put(byteSizeSetting, value + "b");
}
}
for(String timeSetting : INDEX_TIME_SETTINGS) {
String value = settings.get(timeSetting);
if (value != null) {
try {
Double.parseDouble(value);
} catch (NumberFormatException nfe) {
continue;
}
// It's a naked number; add default unit (ms for msec):
logger.warn("time setting [{}] with value [{}] is missing units; now adding default units (ms)", timeSetting, value);
if (newSettings == null) {
newSettings = Settings.builder();
newSettings.put(settings);
}
newSettings.put(timeSetting, value + "ms");
}
}
if (newSettings != null) {
// At least one setting was changed:
return IndexMetaData.builder(indexMetaData)
.version(indexMetaData.version())
.settings(newSettings.build())
.build();
}
}
// No changes:
return indexMetaData;
}
}

View File

@ -44,12 +44,14 @@ public interface Validator {
public static final Validator TIME = new Validator() {
@Override
public String validate(String setting, String value) {
try {
if (TimeValue.parseTimeValue(value, null, setting) == null) {
return "cannot parse value [" + value + "] as time";
if (value == null) {
throw new NullPointerException("value must not be null");
}
try {
// This never returns null:
TimeValue.parseTimeValue(value, null, setting);
} catch (ElasticsearchParseException ex) {
return "cannot parse value [" + value + "] as time";
return ex.getMessage();
}
return null;
}
@ -59,15 +61,16 @@ public interface Validator {
@Override
public String validate(String setting, String value) {
try {
TimeValue timeValue = TimeValue.parseTimeValue(value, null, setting);
if (timeValue == null) {
return "cannot parse value [" + value + "] as time";
if (value == null) {
throw new NullPointerException("value must not be null");
}
TimeValue timeValue = TimeValue.parseTimeValue(value, null, setting);
assert timeValue != null;
if (timeValue.millis() < 0) {
return "cannot parse value [" + value + "] as non negative time";
}
} catch (ElasticsearchParseException ex) {
return "cannot parse value [" + value + "] as time";
return ex.getMessage();
}
return null;
}

View File

@ -19,17 +19,18 @@
package org.elasticsearch.common.unit;
import java.io.IOException;
import java.io.Serializable;
import java.util.Locale;
import java.util.Objects;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import java.io.IOException;
import java.io.Serializable;
import java.util.Locale;
import java.util.Objects;
public class ByteSizeValue implements Serializable, Streamable {
private long size;
@ -175,6 +176,7 @@ public class ByteSizeValue implements Serializable, Streamable {
public static ByteSizeValue parseBytesSizeValue(String sValue, ByteSizeValue defaultValue, String settingName) throws ElasticsearchParseException {
settingName = Objects.requireNonNull(settingName);
assert settingName.startsWith("index.") == false || MetaDataIndexUpgradeService.INDEX_BYTES_SIZE_SETTINGS.contains(settingName);
if (sValue == null) {
return defaultValue;
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.common.unit;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -228,6 +229,7 @@ public class TimeValue implements Serializable, Streamable {
public static TimeValue parseTimeValue(String sValue, TimeValue defaultValue, String settingName) {
settingName = Objects.requireNonNull(settingName);
assert settingName.startsWith("index.") == false || MetaDataIndexUpgradeService.INDEX_TIME_SETTINGS.contains(settingName);
if (sValue == null) {
return defaultValue;
}

View File

@ -214,7 +214,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
/**
* Elasticsearch 2.0 removed several deprecated features and as well as support for Lucene 3.x. This method calls
* {@link MetaDataIndexUpgradeService} to makes sure that indices are compatible with the current version. The
* MetaDataIndexUpgradeService might also update updates obsolete settings if needed. When this happens we rewrite
* MetaDataIndexUpgradeService might also update obsolete settings if needed. When this happens we rewrite
* index metadata with new settings.
*/
private void pre20Upgrade() throws Exception {

View File

@ -128,7 +128,7 @@ public final class EngineConfig {
public static final TimeValue DEFAULT_REFRESH_INTERVAL = new TimeValue(1, TimeUnit.SECONDS);
public static final TimeValue DEFAULT_GC_DELETES = TimeValue.timeValueSeconds(60);
public static final ByteSizeValue DEFAUTL_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
public static final ByteSizeValue DEFAULT_INDEX_BUFFER_SIZE = new ByteSizeValue(64, ByteSizeUnit.MB);
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
public static final String DEFAULT_VERSION_MAP_SIZE = "25%";
@ -163,7 +163,7 @@ public final class EngineConfig {
this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
this.indexConcurrency = indexSettings.getAsInt(EngineConfig.INDEX_CONCURRENCY_SETTING, Math.max(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, (int) (EsExecutors.boundedNumberOfProcessors(indexSettings) * 0.65)));
codecName = indexSettings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
indexingBufferSize = indexSettings.getAsBytesSize(INDEX_BUFFER_SIZE_SETTING, DEFAUTL_INDEX_BUFFER_SIZE);
indexingBufferSize = indexSettings.getAsBytesSize(INDEX_BUFFER_SIZE_SETTING, DEFAULT_INDEX_BUFFER_SIZE);
gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis();
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize();
@ -180,7 +180,7 @@ public final class EngineConfig {
double percent = Double.parseDouble(versionMapSizeSetting.substring(0, versionMapSizeSetting.length() - 1));
versionMapSize = new ByteSizeValue((long) (((double) indexingBufferSize.bytes() * (percent / 100))));
} else {
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting, "VersionMapSize");
versionMapSize = ByteSizeValue.parseBytesSizeValue(versionMapSizeSetting, INDEX_VERSION_MAP_SIZE);
}
}

View File

@ -181,4 +181,12 @@ public class ClusterSettingsTests extends ElasticsearchIntegrationTest {
assertThat(response.getPersistentSettings().get(key1), nullValue());
assertThat(response.getPersistentSettings().get(key2), notNullValue());
}
@Test(expected = IllegalArgumentException.class)
public void testMissingUnits() {
assertAcked(prepareCreate("test"));
// Should fail (missing units for refresh_interval):
client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put("index.refresh_interval", "10")).execute().actionGet();
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.common.unit;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.test.ElasticsearchTestCase;
@ -86,4 +87,14 @@ public class TimeValueTests extends ElasticsearchTestCase {
assertEqualityAfterSerialize(new TimeValue(1, TimeUnit.NANOSECONDS));
}
@Test(expected = ElasticsearchParseException.class)
public void testFailOnUnknownUnits() {
TimeValue.parseTimeValue("23tw", null, "test");
}
@Test(expected = ElasticsearchParseException.class)
public void testFailOnMissingUnits() {
TimeValue.parseTimeValue("42", null, "test");
}
}

View File

@ -1307,7 +1307,7 @@ public class DateHistogramTests extends ElasticsearchIntegrationTest {
/**
* see issue #9634, negative interval in date_histogram should raise exception
*/
public void testExeptionOnNegativeInterval() {
public void testExceptionOnNegativeInterval() {
try {
client().prepareSearch("idx")
.addAggregation(dateHistogram("histo").field("date").interval(-TimeUnit.DAYS.toMillis(1)).minDocCount(0)).execute()

View File

@ -1479,7 +1479,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
// Update settings to make sure that relocation is slow so we can start snapshot before relocation is finished
assertAcked(client.admin().indices().prepareUpdateSettings("test-idx").setSettings(Settings.builder()
.put(IndexStore.INDEX_STORE_THROTTLE_TYPE, "all")
.put(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC, 100)
.put(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC, 100, ByteSizeUnit.BYTES)
));
logger.info("--> start relocations");

View File

@ -217,6 +217,21 @@ public class SimpleVersioningTests extends ElasticsearchIntegrationTest {
assertThat(indexResponse.getVersion(), equalTo(20l));
}
@Test
public void testRequireUnitsOnUpdateSettings() throws Exception {
createIndex("test");
ensureGreen();
HashMap<String,Object> newSettings = new HashMap<>();
newSettings.put("index.gc_deletes", "42");
try {
client().admin().indices().prepareUpdateSettings("test").setSettings(newSettings).execute().actionGet();
fail("did not hit expected exception");
} catch (IllegalArgumentException iae) {
// expected
assertTrue(iae.getMessage().contains("Failed to parse setting [index.gc_deletes] with value [42] as a time value: unit is missing or unrecognized"));
}
}
@Test
public void testInternalVersioningInitialDelete() throws Exception {
createIndex("test");