Skip to content

Commit

Permalink
Remove include_relocations setting
Browse files Browse the repository at this point in the history
Setting `cluster.routing.allocation.disk.include_relocations` to `false` is a
bad idea since it will lead to the kinds of overshoot that were otherwise fixed
in elastic#46079. This setting was deprecated in elastic#47443. This commit removes it.
  • Loading branch information
DaveCTurner committed Oct 8, 2019
1 parent 9d67a02 commit ff20c4e
Show file tree
Hide file tree
Showing 5 changed files with 21 additions and 48 deletions.
25 changes: 13 additions & 12 deletions docs/reference/modules/cluster/disk_allocator.asciidoc
Original file line number Diff line number Diff line change
Expand Up @@ -67,18 +67,6 @@ PUT /twitter/_settings
How often Elasticsearch should check on disk usage for each node in the
cluster. Defaults to `30s`.

`cluster.routing.allocation.disk.include_relocations`::

deprecated[7.5, Future versions will always account for relocations.]
Defaults to +true+, which means that Elasticsearch will take into account
shards that are currently being relocated to the target node when computing
a node's disk usage. Taking relocating shards' sizes into account may,
however, mean that the disk usage for a node is incorrectly estimated on
the high side, since the relocation could be 90% complete and a recently
retrieved disk usage would include the total size of the relocating shard
as well as the space already used by the running relocation.


NOTE: Percentage values refer to used disk space, while byte values refer to
free disk space. This can be confusing, since it flips the meaning of high and
low. For example, it makes sense to set the low watermark to 10gb and the high
Expand All @@ -100,3 +88,16 @@ PUT _cluster/settings
}
}
--------------------------------------------------

{es} accounts for the future disk usage of ongoing shard relocations and
recoveries to help prevent these shard movements from breaching a watermark.
This mechanism may double-count some data that has already been relocated onto
a node. For instance, if a relocation of a 100GB shard is 90% complete then
{es} has copied 90GB of data onto the target node. This 90GB consumes disk
space and will be reflected in the node's disk usage statistics. However {es}
also treats the relocation as if it will consume another full 100GB in the
future, even though the shard may really only consume a further 10GB of space.
If the node's disks are close to a watermark then this may temporarily prevent
other shards from moving onto the same node. Eventually the relocation will
complete and then {es} will use the node's true disk usage statistics again.

Original file line number Diff line number Diff line change
Expand Up @@ -55,9 +55,6 @@ public class DiskThresholdSettings {
(s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.flood_stage"),
new FloodStageValidator(),
Setting.Property.Dynamic, Setting.Property.NodeScope);
public static final Setting<Boolean> CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING =
Setting.boolSetting("cluster.routing.allocation.disk.include_relocations", true,
Setting.Property.Dynamic, Setting.Property.NodeScope, Setting.Property.Deprecated);
public static final Setting<TimeValue> CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING =
Setting.positiveTimeSetting("cluster.routing.allocation.disk.reroute_interval", TimeValue.timeValueSeconds(60),
Setting.Property.Dynamic, Setting.Property.NodeScope);
Expand All @@ -68,7 +65,6 @@ public class DiskThresholdSettings {
private volatile Double freeDiskThresholdHigh;
private volatile ByteSizeValue freeBytesThresholdLow;
private volatile ByteSizeValue freeBytesThresholdHigh;
private volatile boolean includeRelocations;
private volatile boolean enabled;
private volatile TimeValue rerouteInterval;
private volatile Double freeDiskThresholdFloodStage;
Expand All @@ -90,13 +86,11 @@ public DiskThresholdSettings(Settings settings, ClusterSettings clusterSettings)
setHighWatermark(highWatermark);
setLowWatermark(lowWatermark);
setFloodStage(floodStage);
this.includeRelocations = CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.get(settings);
this.rerouteInterval = CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.get(settings);
this.enabled = CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, this::setLowWatermark);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, this::setHighWatermark);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING, this::setFloodStage);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, this::setIncludeRelocations);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, this::setRerouteInterval);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled);
}
Expand Down Expand Up @@ -227,10 +221,6 @@ private static void doValidateAsBytes(final String low, final String high, final
}
}

private void setIncludeRelocations(boolean includeRelocations) {
this.includeRelocations = includeRelocations;
}

private void setRerouteInterval(TimeValue rerouteInterval) {
this.rerouteInterval = rerouteInterval;
}
Expand Down Expand Up @@ -300,10 +290,6 @@ public ByteSizeValue getFreeBytesThresholdFloodStage() {
return freeBytesThresholdFloodStage;
}

public boolean includeRelocations() {
return includeRelocations;
}

public boolean isEnabled() {
return enabled;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -335,23 +335,16 @@ private DiskUsage getDiskUsage(RoutingNode node, RoutingAllocation allocation,
// If there is no usage, and we have other nodes in the cluster,
// use the average usage for all nodes as the usage for this node
usage = averageUsage(node, usages);
if (logger.isDebugEnabled()) {
logger.debug("unable to determine disk usage for {}, defaulting to average across nodes [{} total] [{} free] [{}% free]",
node.nodeId(), usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeDiskAsPercentage());
}
logger.debug("unable to determine disk usage for {}, defaulting to average across nodes [{} total] [{} free] [{}% free]",
node.nodeId(), usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeDiskAsPercentage());
}

if (diskThresholdSettings.includeRelocations()) {
long relocatingShardsSize = sizeOfRelocatingShards(node, allocation, subtractLeavingShards, usage.getPath());
DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().getName(), usage.getPath(),
usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize);
if (logger.isTraceEnabled()) {
logger.trace("usage without relocations: {}", usage);
logger.trace("usage with relocations: [{} bytes] {}", relocatingShardsSize, usageIncludingRelocations);
}
usage = usageIncludingRelocations;
}
return usage;
final long relocatingShardsSize = sizeOfRelocatingShards(node, allocation, subtractLeavingShards, usage.getPath());
final DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().getName(), usage.getPath(),
usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize);
logger.trace("getDiskUsage: usage [{}] with [{}] bytes relocating yields [{}]",
usage, relocatingShardsSize, usageIncludingRelocations);
return usageIncludingRelocations;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,6 @@ public void apply(Settings value, Settings current, Settings previous) {
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING,
SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@ public void testDefaults() {
assertEquals(15.0D, diskThresholdSettings.getFreeDiskThresholdLow(), 0.0D);
assertEquals(60L, diskThresholdSettings.getRerouteInterval().seconds());
assertTrue(diskThresholdSettings.isEnabled());
assertTrue(diskThresholdSettings.includeRelocations());
assertEquals(zeroBytes, diskThresholdSettings.getFreeBytesThresholdFloodStage());
assertEquals(5.0D, diskThresholdSettings.getFreeDiskThresholdFloodStage(), 0.0D);
}
Expand All @@ -55,7 +54,6 @@ public void testUpdate() {

Settings newSettings = Settings.builder()
.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), false)
.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), false)
.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "500mb")
.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "1000mb")
.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING.getKey(), "250mb")
Expand All @@ -71,10 +69,6 @@ public void testUpdate() {
assertEquals(0.0D, diskThresholdSettings.getFreeDiskThresholdFloodStage(), 0.0D);
assertEquals(30L, diskThresholdSettings.getRerouteInterval().seconds());
assertFalse(diskThresholdSettings.isEnabled());
assertFalse(diskThresholdSettings.includeRelocations());

assertWarnings("[cluster.routing.allocation.disk.include_relocations] setting was deprecated in Elasticsearch and " +
"will be removed in a future release! See the breaking changes documentation for the next major version.");
}

public void testInvalidConstruction() {
Expand Down

0 comments on commit ff20c4e

Please sign in to comment.