瀏覽代碼

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

Michael McCandless 10 年之前
父節點
當前提交
68d6427944

+ 10 - 2
dev-tools/create_bwc_index.py

@@ -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={

+ 103 - 7
src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java

@@ -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;
+    }
 }

+ 11 - 8
src/main/java/org/elasticsearch/cluster/settings/Validator.java

@@ -44,12 +44,14 @@ public interface Validator {
     public static final Validator TIME = new Validator() {
         @Override
         public String validate(String setting, String value) {
+            if (value == null) {
+                throw new NullPointerException("value must not be null");
+            }
             try {
-                if (TimeValue.parseTimeValue(value, null, setting) == null) {
-                    return "cannot parse value [" + value + "] as time";
-                }
+                // 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;
         }

+ 7 - 5
src/main/java/org/elasticsearch/common/unit/ByteSizeValue.java

@@ -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;
         }

+ 2 - 0
src/main/java/org/elasticsearch/common/unit/TimeValue.java

@@ -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;
         }

+ 1 - 1
src/main/java/org/elasticsearch/gateway/GatewayMetaState.java

@@ -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 {

+ 3 - 3
src/main/java/org/elasticsearch/index/engine/EngineConfig.java

@@ -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);
         }
     }
 

+ 8 - 0
src/test/java/org/elasticsearch/cluster/settings/ClusterSettingsTests.java

@@ -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();
+    }
 }

+ 11 - 0
src/test/java/org/elasticsearch/common/unit/TimeValueTests.java

@@ -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");
+    }
 }

+ 1 - 1
src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramTests.java

@@ -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()

+ 1 - 1
src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreTests.java

@@ -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");

+ 15 - 0
src/test/java/org/elasticsearch/versioning/SimpleVersioningTests.java

@@ -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");

二進制
src/test/resources/org/elasticsearch/bwcompat/index-1.3.4.zip


二進制
src/test/resources/org/elasticsearch/bwcompat/repo-1.3.4.zip