From b2c0f2d08fe4848698efc59a4b4c12fce42bef35 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Tue, 16 Aug 2016 23:56:38 -0700 Subject: [PATCH 01/13] Internal: Split disk threshold monitoring from decider In addition to be an allocation decider, DiskThresholdDecider also monitors the used disk in order to trigger a reroute when the thresholds are crossed. This change splits out the settings for disk thresholds into DiskThresholdSettings, and moves the monitoring to a new DiskThresholdMonitor. DiskThresholdDecider is then in line with other allocation deciders, needing only Settings and ClusterSettings for construction, which will allow deguicing allocation deciders. --- .../cluster/InternalClusterInfoService.java | 6 +- .../cluster/metadata/MetaData.java | 4 +- .../allocation/DiskThresholdMonitor.java | 144 ++++++++ .../allocation/DiskThresholdSettings.java | 174 +++++++++ .../decider/DiskThresholdDecider.java | 342 ++---------------- .../common/settings/ClusterSettings.java | 12 +- .../org/elasticsearch/node/NodeModule.java | 2 + .../DiskThresholdSettingsTests.java | 64 ++++ .../decider/DiskThresholdDeciderTests.java | 104 +++--- .../DiskThresholdDeciderUnitTests.java | 43 +-- .../allocation/decider/MockDiskUsagesIT.java | 7 +- .../elasticsearch/test/ESIntegTestCase.java | 6 +- .../test/InternalTestCluster.java | 6 +- 13 files changed, 498 insertions(+), 416 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java create mode 100644 core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java create mode 100644 core/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java diff --git a/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java b/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java index 6c22ff171dd..534f007e8b2 100644 --- a/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java +++ b/core/src/main/java/org/elasticsearch/cluster/InternalClusterInfoService.java @@ -34,7 +34,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.component.AbstractComponent; @@ -106,10 +106,10 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu this.threadPool = threadPool; this.updateFrequency = INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.get(settings); this.fetchTimeout = INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.get(settings); - this.enabled = DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings); + this.enabled = DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings); clusterSettings.addSettingsUpdateConsumer(INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, this::setFetchTimeout); clusterSettings.addSettingsUpdateConsumer(INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, this::setUpdateFrequency); - clusterSettings.addSettingsUpdateConsumer(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled); + clusterSettings.addSettingsUpdateConsumer(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled); // Add InternalClusterInfoService to listen for Master changes this.clusterService.add((LocalNodeMasterListener)this); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index 1bfd775ecce..29bb55f8107 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -29,7 +29,7 @@ import org.elasticsearch.cluster.DiffableUtils; import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; @@ -750,7 +750,7 @@ public class MetaData implements Iterable, Diffable, Fr RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING.getKey(), RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING.getKey(), RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING.getKey(), - DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(), InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.getKey(), DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java new file mode 100644 index 00000000000..103aa87dcd3 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdMonitor.java @@ -0,0 +1,144 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster.routing.allocation; + +import java.util.Set; + +import com.carrotsearch.hppc.ObjectLookupContainer; +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.ClusterInfoService; +import org.elasticsearch.cluster.DiskUsage; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.set.Sets; + +/** + * Listens for a node to go over the high watermark and kicks off an empty + * reroute if it does. Also responsible for logging about nodes that have + * passed the disk watermarks + */ +public class DiskThresholdMonitor extends AbstractComponent implements ClusterInfoService.Listener { + private final DiskThresholdSettings diskThresholdSettings; + private final Client client; + private final Set nodeHasPassedWatermark = Sets.newConcurrentHashSet(); + + private long lastRunNS; + + // TODO: remove injection when ClusterInfoService is not injected + @Inject + public DiskThresholdMonitor(Settings settings, ClusterSettings clusterSettings, + ClusterInfoService infoService, Client client) { + super(settings); + this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterSettings); + this.client = client; + infoService.addListener(this); + } + + /** + * Warn about the given disk usage if the low or high watermark has been passed + */ + private void warnAboutDiskIfNeeded(DiskUsage usage) { + // Check absolute disk values + if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) { + logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node", + diskThresholdSettings.getFreeBytesThresholdHigh(), usage); + } else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().bytes()) { + logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node", + diskThresholdSettings.getFreeBytesThresholdLow(), usage); + } + + // Check percentage disk values + if (usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) { + logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node", + Strings.format1Decimals(100.0 - diskThresholdSettings.getFreeDiskThresholdHigh(), "%"), usage); + } else if (usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdLow()) { + logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node", + Strings.format1Decimals(100.0 - diskThresholdSettings.getFreeDiskThresholdLow(), "%"), usage); + } + } + + @Override + public void onNewInfo(ClusterInfo info) { + ImmutableOpenMap usages = info.getNodeLeastAvailableDiskUsages(); + if (usages != null) { + boolean reroute = false; + String explanation = ""; + + // Garbage collect nodes that have been removed from the cluster + // from the map that tracks watermark crossing + ObjectLookupContainer nodes = usages.keys(); + for (String node : nodeHasPassedWatermark) { + if (nodes.contains(node) == false) { + nodeHasPassedWatermark.remove(node); + } + } + + for (ObjectObjectCursor entry : usages) { + String node = entry.key; + DiskUsage usage = entry.value; + warnAboutDiskIfNeeded(usage); + if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdHigh().bytes() || + usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdHigh()) { + if ((System.nanoTime() - lastRunNS) > diskThresholdSettings.getRerouteInterval().nanos()) { + lastRunNS = System.nanoTime(); + reroute = true; + explanation = "high disk watermark exceeded on one or more nodes"; + } else { + logger.debug("high disk watermark exceeded on {} but an automatic reroute has occurred " + + "in the last [{}], skipping reroute", + node, diskThresholdSettings.getRerouteInterval()); + } + nodeHasPassedWatermark.add(node); + } else if (usage.getFreeBytes() < diskThresholdSettings.getFreeBytesThresholdLow().bytes() || + usage.getFreeDiskAsPercentage() < diskThresholdSettings.getFreeDiskThresholdLow()) { + nodeHasPassedWatermark.add(node); + } else { + if (nodeHasPassedWatermark.contains(node)) { + // The node has previously been over the high or + // low watermark, but is no longer, so we should + // reroute so any unassigned shards can be allocated + // if they are able to be + if ((System.nanoTime() - lastRunNS) > diskThresholdSettings.getRerouteInterval().nanos()) { + lastRunNS = System.nanoTime(); + reroute = true; + explanation = "one or more nodes has gone under the high or low watermark"; + nodeHasPassedWatermark.remove(node); + } else { + logger.debug("{} has gone below a disk threshold, but an automatic reroute has occurred " + + "in the last [{}], skipping reroute", + node, diskThresholdSettings.getRerouteInterval()); + } + } + } + } + if (reroute) { + logger.info("rerouting shards: [{}]", explanation); + // Execute an empty reroute, but don't block on the response + client.admin().cluster().prepareReroute().execute(); + } + } + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java new file mode 100644 index 00000000000..81b9042fb33 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java @@ -0,0 +1,174 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster.routing.allocation; + +import org.elasticsearch.ElasticsearchParseException; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.RatioValue; +import org.elasticsearch.common.unit.TimeValue; + +/** + * A container to keep settings for disk thresholds up to date with cluster setting changes. + */ +public class DiskThresholdSettings { + public static final Setting CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING = + Setting.boolSetting("cluster.routing.allocation.disk.threshold_enabled", true, + Setting.Property.Dynamic, Setting.Property.NodeScope); + public static final Setting CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING = + new Setting<>("cluster.routing.allocation.disk.watermark.low", "85%", + (s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.low"), + Setting.Property.Dynamic, Setting.Property.NodeScope); + public static final Setting CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING = + new Setting<>("cluster.routing.allocation.disk.watermark.high", "90%", + (s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.high"), + Setting.Property.Dynamic, Setting.Property.NodeScope); + public static final Setting CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING = + Setting.boolSetting("cluster.routing.allocation.disk.include_relocations", true, + Setting.Property.Dynamic, Setting.Property.NodeScope);; + public static final Setting CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING = + Setting.positiveTimeSetting("cluster.routing.allocation.disk.reroute_interval", TimeValue.timeValueSeconds(60), + Setting.Property.Dynamic, Setting.Property.NodeScope); + + private volatile Double freeDiskThresholdLow; + private volatile Double freeDiskThresholdHigh; + private volatile ByteSizeValue freeBytesThresholdLow; + private volatile ByteSizeValue freeBytesThresholdHigh; + private volatile boolean includeRelocations; + private volatile boolean enabled; + private volatile TimeValue rerouteInterval; + + public DiskThresholdSettings(Settings settings, ClusterSettings clusterSettings) { + final String lowWatermark = CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.get(settings); + final String highWatermark = CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.get(settings); + setHighWatermark(highWatermark); + setLowWatermark(lowWatermark); + 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_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); + } + + private void setIncludeRelocations(boolean includeRelocations) { + this.includeRelocations = includeRelocations; + } + + private void setRerouteInterval(TimeValue rerouteInterval) { + this.rerouteInterval = rerouteInterval; + } + + private void setEnabled(boolean enabled) { + this.enabled = enabled; + } + + private void setLowWatermark(String lowWatermark) { + // Watermark is expressed in terms of used data, but we need "free" data watermark + this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark); + this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark, + CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey()); + } + + private void setHighWatermark(String highWatermark) { + // Watermark is expressed in terms of used data, but we need "free" data watermark + this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark); + this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark, + CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey()); + } + + public Double getFreeDiskThresholdLow() { + return freeDiskThresholdLow; + } + + public Double getFreeDiskThresholdHigh() { + return freeDiskThresholdHigh; + } + + public ByteSizeValue getFreeBytesThresholdLow() { + return freeBytesThresholdLow; + } + + public ByteSizeValue getFreeBytesThresholdHigh() { + return freeBytesThresholdHigh; + } + + public boolean includeRelocations() { + return includeRelocations; + } + + public boolean isEnabled() { + return enabled; + } + + public TimeValue getRerouteInterval() { + return rerouteInterval; + } + + /** + * Attempts to parse the watermark into a percentage, returning 100.0% if + * it cannot be parsed. + */ + private double thresholdPercentageFromWatermark(String watermark) { + try { + return RatioValue.parseRatioValue(watermark).getAsPercent(); + } catch (ElasticsearchParseException ex) { + // NOTE: this is not end-user leniency, since up above we check that it's a valid byte or percentage, and then store the two + // cases separately + return 100.0; + } + } + + /** + * Attempts to parse the watermark into a {@link ByteSizeValue}, returning + * a ByteSizeValue of 0 bytes if the value cannot be parsed. + */ + private ByteSizeValue thresholdBytesFromWatermark(String watermark, String settingName) { + try { + return ByteSizeValue.parseBytesSizeValue(watermark, settingName); + } catch (ElasticsearchParseException ex) { + // NOTE: this is not end-user leniency, since up above we check that it's a valid byte or percentage, and then store the two + // cases separately + return ByteSizeValue.parseBytesSizeValue("0b", settingName); + } + } + + /** + * Checks if a watermark string is a valid percentage or byte size value, + * @return the watermark value given + */ + private static String validWatermarkSetting(String watermark, String settingName) { + try { + RatioValue.parseRatioValue(watermark); + } catch (ElasticsearchParseException e) { + try { + ByteSizeValue.parseBytesSizeValue(watermark, settingName); + } catch (ElasticsearchParseException ex) { + ex.addSuppressed(e); + throw ex; + } + } + return watermark; + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index edeb609a9c7..26f66183efa 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -19,37 +19,27 @@ package org.elasticsearch.cluster.routing.allocation.decider; -import com.carrotsearch.hppc.ObjectLookupContainer; +import java.util.Set; + import com.carrotsearch.hppc.cursors.ObjectCursor; -import com.carrotsearch.hppc.cursors.ObjectObjectCursor; -import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterInfo; -import org.elasticsearch.cluster.ClusterInfoService; import org.elasticsearch.cluster.DiskUsage; -import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; -import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.RatioValue; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; -import java.util.Set; - /** * The {@link DiskThresholdDecider} checks that the node a shard is potentially * being allocated to has enough disk space. @@ -77,226 +67,12 @@ public class DiskThresholdDecider extends AllocationDecider { public static final String NAME = "disk_threshold"; - private volatile Double freeDiskThresholdLow; - private volatile Double freeDiskThresholdHigh; - private volatile ByteSizeValue freeBytesThresholdLow; - private volatile ByteSizeValue freeBytesThresholdHigh; - private volatile boolean includeRelocations; - private volatile boolean enabled; - private volatile TimeValue rerouteInterval; - - public static final Setting CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING = - Setting.boolSetting("cluster.routing.allocation.disk.threshold_enabled", true, Property.Dynamic, Property.NodeScope); - public static final Setting CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING = - new Setting<>("cluster.routing.allocation.disk.watermark.low", "85%", - (s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.low"), - Property.Dynamic, Property.NodeScope); - public static final Setting CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING = - new Setting<>("cluster.routing.allocation.disk.watermark.high", "90%", - (s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.high"), - Property.Dynamic, Property.NodeScope); - public static final Setting CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING = - Setting.boolSetting("cluster.routing.allocation.disk.include_relocations", true, - Property.Dynamic, Property.NodeScope);; - public static final Setting CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING = - Setting.positiveTimeSetting("cluster.routing.allocation.disk.reroute_interval", TimeValue.timeValueSeconds(60), - Property.Dynamic, Property.NodeScope); - - /** - * Listens for a node to go over the high watermark and kicks off an empty - * reroute if it does. Also responsible for logging about nodes that have - * passed the disk watermarks - */ - class DiskListener implements ClusterInfoService.Listener { - private final Client client; - private final Set nodeHasPassedWatermark = Sets.newConcurrentHashSet(); - - private long lastRunNS; - - DiskListener(Client client) { - this.client = client; - } - - /** - * Warn about the given disk usage if the low or high watermark has been passed - */ - private void warnAboutDiskIfNeeded(DiskUsage usage) { - // Check absolute disk values - if (usage.getFreeBytes() < DiskThresholdDecider.this.freeBytesThresholdHigh.bytes()) { - logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node", - DiskThresholdDecider.this.freeBytesThresholdHigh, usage); - } else if (usage.getFreeBytes() < DiskThresholdDecider.this.freeBytesThresholdLow.bytes()) { - logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node", - DiskThresholdDecider.this.freeBytesThresholdLow, usage); - } - - // Check percentage disk values - if (usage.getFreeDiskAsPercentage() < DiskThresholdDecider.this.freeDiskThresholdHigh) { - logger.warn("high disk watermark [{}] exceeded on {}, shards will be relocated away from this node", - Strings.format1Decimals(100.0 - DiskThresholdDecider.this.freeDiskThresholdHigh, "%"), usage); - } else if (usage.getFreeDiskAsPercentage() < DiskThresholdDecider.this.freeDiskThresholdLow) { - logger.info("low disk watermark [{}] exceeded on {}, replicas will not be assigned to this node", - Strings.format1Decimals(100.0 - DiskThresholdDecider.this.freeDiskThresholdLow, "%"), usage); - } - } - - @Override - public void onNewInfo(ClusterInfo info) { - ImmutableOpenMap usages = info.getNodeLeastAvailableDiskUsages(); - if (usages != null) { - boolean reroute = false; - String explanation = ""; - - // Garbage collect nodes that have been removed from the cluster - // from the map that tracks watermark crossing - ObjectLookupContainer nodes = usages.keys(); - for (String node : nodeHasPassedWatermark) { - if (nodes.contains(node) == false) { - nodeHasPassedWatermark.remove(node); - } - } - - for (ObjectObjectCursor entry : usages) { - String node = entry.key; - DiskUsage usage = entry.value; - warnAboutDiskIfNeeded(usage); - if (usage.getFreeBytes() < DiskThresholdDecider.this.freeBytesThresholdHigh.bytes() || - usage.getFreeDiskAsPercentage() < DiskThresholdDecider.this.freeDiskThresholdHigh) { - if ((System.nanoTime() - lastRunNS) > DiskThresholdDecider.this.rerouteInterval.nanos()) { - lastRunNS = System.nanoTime(); - reroute = true; - explanation = "high disk watermark exceeded on one or more nodes"; - } else { - logger.debug("high disk watermark exceeded on {} but an automatic reroute has occurred " + - "in the last [{}], skipping reroute", - node, DiskThresholdDecider.this.rerouteInterval); - } - nodeHasPassedWatermark.add(node); - } else if (usage.getFreeBytes() < DiskThresholdDecider.this.freeBytesThresholdLow.bytes() || - usage.getFreeDiskAsPercentage() < DiskThresholdDecider.this.freeDiskThresholdLow) { - nodeHasPassedWatermark.add(node); - } else { - if (nodeHasPassedWatermark.contains(node)) { - // The node has previously been over the high or - // low watermark, but is no longer, so we should - // reroute so any unassigned shards can be allocated - // if they are able to be - if ((System.nanoTime() - lastRunNS) > DiskThresholdDecider.this.rerouteInterval.nanos()) { - lastRunNS = System.nanoTime(); - reroute = true; - explanation = "one or more nodes has gone under the high or low watermark"; - nodeHasPassedWatermark.remove(node); - } else { - logger.debug("{} has gone below a disk threshold, but an automatic reroute has occurred " + - "in the last [{}], skipping reroute", - node, DiskThresholdDecider.this.rerouteInterval); - } - } - } - } - if (reroute) { - logger.info("rerouting shards: [{}]", explanation); - // Execute an empty reroute, but don't block on the response - client.admin().cluster().prepareReroute().execute(); - } - } - } - } - - public DiskThresholdDecider(Settings settings) { - // It's okay the Client is null here, because the empty cluster info - // service will never actually call the listener where the client is - // needed. Also this constructor is only used for tests - this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), EmptyClusterInfoService.INSTANCE, null); - } + private DiskThresholdSettings diskThresholdSettings; @Inject - public DiskThresholdDecider(Settings settings, ClusterSettings clusterSettings, ClusterInfoService infoService, Client client) { + public DiskThresholdDecider(Settings settings, ClusterSettings clusterSettings) { super(settings); - final String lowWatermark = CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.get(settings); - final String highWatermark = CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.get(settings); - setHighWatermark(highWatermark); - setLowWatermark(lowWatermark); - 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_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); - infoService.addListener(new DiskListener(client)); - } - - private void setIncludeRelocations(boolean includeRelocations) { - this.includeRelocations = includeRelocations; - } - - private void setRerouteInterval(TimeValue rerouteInterval) { - this.rerouteInterval = rerouteInterval; - } - - private void setEnabled(boolean enabled) { - this.enabled = enabled; - } - - private void setLowWatermark(String lowWatermark) { - // Watermark is expressed in terms of used data, but we need "free" data watermark - this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark); - this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark, - CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey()); - } - - private void setHighWatermark(String highWatermark) { - // Watermark is expressed in terms of used data, but we need "free" data watermark - this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark); - this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark, - CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey()); - } - - // For Testing - public Double getFreeDiskThresholdLow() { - return freeDiskThresholdLow; - } - - // For Testing - public Double getFreeDiskThresholdHigh() { - return freeDiskThresholdHigh; - } - - // For Testing - public Double getUsedDiskThresholdLow() { - return 100.0 - freeDiskThresholdLow; - } - - // For Testing - public Double getUsedDiskThresholdHigh() { - return 100.0 - freeDiskThresholdHigh; - } - - // For Testing - public ByteSizeValue getFreeBytesThresholdLow() { - return freeBytesThresholdLow; - } - - // For Testing - public ByteSizeValue getFreeBytesThresholdHigh() { - return freeBytesThresholdHigh; - } - - // For Testing - public boolean isIncludeRelocations() { - return includeRelocations; - } - - // For Testing - public boolean isEnabled() { - return enabled; - } - - // For Testing - public TimeValue getRerouteInterval() { - return rerouteInterval; + this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterSettings); } /** @@ -306,7 +82,7 @@ public class DiskThresholdDecider extends AllocationDecider { * If subtractShardsMovingAway is set then the size of shards moving away is subtracted from the total size * of all shards */ - public static long sizeOfRelocatingShards(RoutingNode node, RoutingAllocation allocation, + static long sizeOfRelocatingShards(RoutingNode node, RoutingAllocation allocation, boolean subtractShardsMovingAway, String dataPath) { ClusterInfo clusterInfo = allocation.clusterInfo(); long totalSize = 0; @@ -333,8 +109,8 @@ public class DiskThresholdDecider extends AllocationDecider { return decision; } - final double usedDiskThresholdLow = 100.0 - DiskThresholdDecider.this.freeDiskThresholdLow; - final double usedDiskThresholdHigh = 100.0 - DiskThresholdDecider.this.freeDiskThresholdHigh; + final double usedDiskThresholdLow = 100.0 - diskThresholdSettings.getFreeDiskThresholdLow(); + final double usedDiskThresholdHigh = 100.0 - diskThresholdSettings.getFreeDiskThresholdHigh(); DiskUsage usage = getDiskUsage(node, allocation, usages); // First, check that the node currently over the low watermark @@ -351,23 +127,23 @@ public class DiskThresholdDecider extends AllocationDecider { boolean primaryHasBeenAllocated = shardRouting.primary() && shardRouting.allocatedPostIndexCreate(indexMetaData); // checks for exact byte comparisons - if (freeBytes < freeBytesThresholdLow.bytes()) { + if (freeBytes < diskThresholdSettings.getFreeBytesThresholdLow().bytes()) { // If the shard is a replica or has a primary that has already been allocated before, check the low threshold if (!shardRouting.primary() || (shardRouting.primary() && primaryHasBeenAllocated)) { if (logger.isDebugEnabled()) { logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, preventing allocation", - freeBytesThresholdLow, freeBytes, node.nodeId()); + diskThresholdSettings.getFreeBytesThresholdLow(), freeBytes, node.nodeId()); } return allocation.decision(Decision.NO, NAME, "the node is above the low watermark and has less than required [%s] free, free: [%s]", - freeBytesThresholdLow, new ByteSizeValue(freeBytes)); - } else if (freeBytes > freeBytesThresholdHigh.bytes()) { + diskThresholdSettings.getFreeBytesThresholdLow(), new ByteSizeValue(freeBytes)); + } else if (freeBytes > diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) { // Allow the shard to be allocated because it is primary that // has never been allocated if it's under the high watermark if (logger.isDebugEnabled()) { logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, " + "but allowing allocation because primary has never been allocated", - freeBytesThresholdLow, freeBytes, node.nodeId()); + diskThresholdSettings.getFreeBytesThresholdLow(), freeBytes, node.nodeId()); } return allocation.decision(Decision.YES, NAME, "the node is above the low watermark, but this primary shard has never been allocated before"); @@ -377,17 +153,17 @@ public class DiskThresholdDecider extends AllocationDecider { if (logger.isDebugEnabled()) { logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, " + "preventing allocation even though primary has never been allocated", - freeBytesThresholdHigh, freeBytes, node.nodeId()); + diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytes, node.nodeId()); } return allocation.decision(Decision.NO, NAME, "the node is above the high watermark even though this shard has never been allocated " + "and has less than required [%s] free on node, free: [%s]", - freeBytesThresholdHigh, new ByteSizeValue(freeBytes)); + diskThresholdSettings.getFreeBytesThresholdHigh(), new ByteSizeValue(freeBytes)); } } // checks for percentage comparisons - if (freeDiskPercentage < freeDiskThresholdLow) { + if (freeDiskPercentage < diskThresholdSettings.getFreeDiskThresholdLow()) { // If the shard is a replica or has a primary that has already been allocated before, check the low threshold if (!shardRouting.primary() || (shardRouting.primary() && primaryHasBeenAllocated)) { if (logger.isDebugEnabled()) { @@ -398,7 +174,7 @@ public class DiskThresholdDecider extends AllocationDecider { return allocation.decision(Decision.NO, NAME, "the node is above the low watermark and has more than allowed [%s%%] used disk, free: [%s%%]", usedDiskThresholdLow, freeDiskPercentage); - } else if (freeDiskPercentage > freeDiskThresholdHigh) { + } else if (freeDiskPercentage > diskThresholdSettings.getFreeDiskThresholdHigh()) { // Allow the shard to be allocated because it is primary that // has never been allocated if it's under the high watermark if (logger.isDebugEnabled()) { @@ -415,7 +191,7 @@ public class DiskThresholdDecider extends AllocationDecider { if (logger.isDebugEnabled()) { logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, " + "preventing allocation even though primary has never been allocated", - Strings.format1Decimals(freeDiskThresholdHigh, "%"), + Strings.format1Decimals(diskThresholdSettings.getFreeDiskThresholdHigh(), "%"), Strings.format1Decimals(freeDiskPercentage, "%"), node.nodeId()); } return allocation.decision(Decision.NO, NAME, @@ -429,19 +205,20 @@ public class DiskThresholdDecider extends AllocationDecider { final long shardSize = getExpectedShardSize(shardRouting, allocation, 0); double freeSpaceAfterShard = freeDiskPercentageAfterShardAssigned(usage, shardSize); long freeBytesAfterShard = freeBytes - shardSize; - if (freeBytesAfterShard < freeBytesThresholdHigh.bytes()) { + if (freeBytesAfterShard < diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) { logger.warn("after allocating, node [{}] would have less than the required " + "{} free bytes threshold ({} bytes free), preventing allocation", - node.nodeId(), freeBytesThresholdHigh, freeBytesAfterShard); + node.nodeId(), diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytesAfterShard); return allocation.decision(Decision.NO, NAME, "after allocating the shard to this node, it would be above the high watermark " + "and have less than required [%s] free, free: [%s]", - freeBytesThresholdLow, new ByteSizeValue(freeBytesAfterShard)); + diskThresholdSettings.getFreeBytesThresholdLow(), new ByteSizeValue(freeBytesAfterShard)); } - if (freeSpaceAfterShard < freeDiskThresholdHigh) { + if (freeSpaceAfterShard < diskThresholdSettings.getFreeDiskThresholdHigh()) { logger.warn("after allocating, node [{}] would have more than the allowed " + "{} free disk threshold ({} free), preventing allocation", - node.nodeId(), Strings.format1Decimals(freeDiskThresholdHigh, "%"), Strings.format1Decimals(freeSpaceAfterShard, "%")); + node.nodeId(), Strings.format1Decimals(diskThresholdSettings.getFreeDiskThresholdHigh(), "%"), + Strings.format1Decimals(freeSpaceAfterShard, "%")); return allocation.decision(Decision.NO, NAME, "after allocating the shard to this node, it would be above the high watermark " + "and have more than allowed [%s%%] used disk, free: [%s%%]", @@ -479,25 +256,25 @@ public class DiskThresholdDecider extends AllocationDecider { return allocation.decision(Decision.YES, NAME, "this shard is not allocated on the most utilized disk and can remain"); } - if (freeBytes < freeBytesThresholdHigh.bytes()) { + if (freeBytes < diskThresholdSettings.getFreeBytesThresholdHigh().bytes()) { if (logger.isDebugEnabled()) { logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, shard cannot remain", - freeBytesThresholdHigh, freeBytes, node.nodeId()); + diskThresholdSettings.getFreeBytesThresholdHigh(), freeBytes, node.nodeId()); } return allocation.decision(Decision.NO, NAME, "after allocating this shard this node would be above the high watermark " + "and there would be less than required [%s] free on node, free: [%s]", - freeBytesThresholdHigh, new ByteSizeValue(freeBytes)); + diskThresholdSettings.getFreeBytesThresholdHigh(), new ByteSizeValue(freeBytes)); } - if (freeDiskPercentage < freeDiskThresholdHigh) { + if (freeDiskPercentage < diskThresholdSettings.getFreeDiskThresholdHigh()) { if (logger.isDebugEnabled()) { logger.debug("less than the required {}% free disk threshold ({}% free) on node {}, shard cannot remain", - freeDiskThresholdHigh, freeDiskPercentage, node.nodeId()); + diskThresholdSettings.getFreeDiskThresholdHigh(), freeDiskPercentage, node.nodeId()); } return allocation.decision(Decision.NO, NAME, "after allocating this shard this node would be above the high watermark " + "and there would be less than required [%s%%] free disk on node, free: [%s%%]", - freeDiskThresholdHigh, freeDiskPercentage); + diskThresholdSettings.getFreeDiskThresholdHigh(), freeDiskPercentage); } return allocation.decision(Decision.YES, NAME, @@ -516,7 +293,7 @@ public class DiskThresholdDecider extends AllocationDecider { } } - if (includeRelocations) { + if (diskThresholdSettings.includeRelocations()) { long relocatingShardsSize = sizeOfRelocatingShards(node, allocation, true, usage.getPath()); DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().getName(), usage.getPath(), usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize); @@ -536,7 +313,7 @@ public class DiskThresholdDecider extends AllocationDecider { * @param usages Map of nodeId to DiskUsage for all known nodes * @return DiskUsage representing given node using the average disk usage */ - public DiskUsage averageUsage(RoutingNode node, ImmutableOpenMap usages) { + DiskUsage averageUsage(RoutingNode node, ImmutableOpenMap usages) { if (usages.size() == 0) { return new DiskUsage(node.nodeId(), node.node().getName(), "_na_", 0, 0); } @@ -556,63 +333,16 @@ public class DiskThresholdDecider extends AllocationDecider { * @param shardSize Size in bytes of the shard * @return Percentage of free space after the shard is assigned to the node */ - public double freeDiskPercentageAfterShardAssigned(DiskUsage usage, Long shardSize) { + double freeDiskPercentageAfterShardAssigned(DiskUsage usage, Long shardSize) { shardSize = (shardSize == null) ? 0 : shardSize; DiskUsage newUsage = new DiskUsage(usage.getNodeId(), usage.getNodeName(), usage.getPath(), usage.getTotalBytes(), usage.getFreeBytes() - shardSize); return newUsage.getFreeDiskAsPercentage(); } - /** - * Attempts to parse the watermark into a percentage, returning 100.0% if - * it cannot be parsed. - */ - public double thresholdPercentageFromWatermark(String watermark) { - try { - return RatioValue.parseRatioValue(watermark).getAsPercent(); - } catch (ElasticsearchParseException ex) { - // NOTE: this is not end-user leniency, since up above we check that it's a valid byte or percentage, and then store the two - // cases separately - return 100.0; - } - } - - /** - * Attempts to parse the watermark into a {@link ByteSizeValue}, returning - * a ByteSizeValue of 0 bytes if the value cannot be parsed. - */ - public ByteSizeValue thresholdBytesFromWatermark(String watermark, String settingName) { - try { - return ByteSizeValue.parseBytesSizeValue(watermark, settingName); - } catch (ElasticsearchParseException ex) { - // NOTE: this is not end-user leniency, since up above we check that it's a valid byte or percentage, and then store the two - // cases separately - return ByteSizeValue.parseBytesSizeValue("0b", settingName); - } - } - - /** - * Checks if a watermark string is a valid percentage or byte size value, - * @return the watermark value given - */ - public static String validWatermarkSetting(String watermark, String settingName) { - try { - RatioValue.parseRatioValue(watermark); - } catch (ElasticsearchParseException e) { - try { - ByteSizeValue.parseBytesSizeValue(watermark, settingName); - } catch (ElasticsearchParseException ex) { - ex.addSuppressed(e); - throw ex; - } - } - return watermark; - - } - private Decision earlyTerminate(RoutingAllocation allocation, ImmutableOpenMap usages) { // Always allow allocation if the decider is disabled - if (!enabled) { + if (diskThresholdSettings.isEnabled() == false) { return allocation.decision(Decision.YES, NAME, "the disk threshold decider is disabled"); } @@ -647,7 +377,7 @@ public class DiskThresholdDecider extends AllocationDecider { * Returns the expected shard size for the given shard or the default value provided if not enough information are available * to estimate the shards size. */ - public static final long getExpectedShardSize(ShardRouting shard, RoutingAllocation allocation, long defaultValue) { + public static long getExpectedShardSize(ShardRouting shard, RoutingAllocation allocation, long defaultValue) { final IndexMetaData metaData = allocation.metaData().getIndexSafe(shard.index()); final ClusterInfo info = allocation.clusterInfo(); if (metaData.getMergeSourceIndex() != null && shard.allocatedPostIndexCreate(metaData) == false) { diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 507d0d9c410..3ce483d0fa6 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -32,11 +32,11 @@ import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider; @@ -193,11 +193,11 @@ public final class ClusterSettings extends AbstractScopedSettings { ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING, - DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, - DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, - DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, - DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, - DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, + DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, SnapshotInProgressAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED_SETTING, diff --git a/core/src/main/java/org/elasticsearch/node/NodeModule.java b/core/src/main/java/org/elasticsearch/node/NodeModule.java index 8299834435b..6a8f8b90681 100644 --- a/core/src/main/java/org/elasticsearch/node/NodeModule.java +++ b/core/src/main/java/org/elasticsearch/node/NodeModule.java @@ -19,6 +19,7 @@ package org.elasticsearch.node; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdMonitor; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.monitor.MonitorService; import org.elasticsearch.node.service.NodeService; @@ -38,5 +39,6 @@ public class NodeModule extends AbstractModule { bind(Node.class).toInstance(node); bind(MonitorService.class).toInstance(monitorService); bind(NodeService.class).asEagerSingleton(); + bind(DiskThresholdMonitor.class).asEagerSingleton(); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java new file mode 100644 index 00000000000..fa4518218c9 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettingsTests.java @@ -0,0 +1,64 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.cluster.routing.allocation; + +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.test.ESTestCase; + +public class DiskThresholdSettingsTests extends ESTestCase { + + public void testDefaults() { + ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + DiskThresholdSettings diskThresholdSettings = new DiskThresholdSettings(Settings.EMPTY, nss); + + ByteSizeValue zeroBytes = ByteSizeValue.parseBytesSizeValue("0b", "test"); + assertEquals(zeroBytes, diskThresholdSettings.getFreeBytesThresholdHigh()); + assertEquals(10.0D, diskThresholdSettings.getFreeDiskThresholdHigh(), 0.0D); + assertEquals(zeroBytes, diskThresholdSettings.getFreeBytesThresholdLow()); + assertEquals(15.0D, diskThresholdSettings.getFreeDiskThresholdLow(), 0.0D); + assertEquals(60L, diskThresholdSettings.getRerouteInterval().seconds()); + assertTrue(diskThresholdSettings.isEnabled()); + assertTrue(diskThresholdSettings.includeRelocations()); + } + + public void testUpdate() { + ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + DiskThresholdSettings diskThresholdSettings = new DiskThresholdSettings(Settings.EMPTY, nss); + + 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(), "70%") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "500mb") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "30s") + .build(); + nss.applySettings(newSettings); + + assertEquals(ByteSizeValue.parseBytesSizeValue("0b", "test"), diskThresholdSettings.getFreeBytesThresholdHigh()); + assertEquals(30.0D, diskThresholdSettings.getFreeDiskThresholdHigh(), 0.0D); + assertEquals(ByteSizeValue.parseBytesSizeValue("500mb", "test"), diskThresholdSettings.getFreeBytesThresholdLow()); + assertEquals(0.0D, diskThresholdSettings.getFreeDiskThresholdLow(), 0.0D); + assertEquals(30L, diskThresholdSettings.getRerouteInterval().seconds()); + assertFalse(diskThresholdSettings.isEnabled()); + assertFalse(diskThresholdSettings.includeRelocations()); + } +} diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 02a603fed63..e9c82e7e180 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -39,12 +39,14 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESAllocationTestCase; @@ -67,12 +69,16 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; public class DiskThresholdDeciderTests extends ESAllocationTestCase { + + DiskThresholdDecider makeDecider(Settings settings) { + return new DiskThresholdDecider(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); + } public void testDiskThreshold() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.8).build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.8).build(); ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); usagesBuilder.put("node1", new DiskUsage("node1", "node1", "/dev/null", 100, 10)); // 90% used @@ -90,7 +96,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); ClusterInfoService cis = new ClusterInfoService() { @Override @@ -184,14 +190,14 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { // Set the high threshold to 70 instead of 80 // node2 now should not have new shards allocated to it, but shards can remain diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.7).build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.7).build(); deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); strategy = new AllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 10) @@ -215,14 +221,14 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { // Set the high threshold to 60 instead of 70 // node2 now should not have new shards allocated to it, and shards cannot remain diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.5) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.6).build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.5) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.6).build(); deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); strategy = new AllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 10) @@ -269,9 +275,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testDiskThresholdWithAbsoluteSizes() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "30b") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "9b").build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "30b") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "9b").build(); ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 10)); // 90% used @@ -290,7 +296,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); ClusterInfoService cis = new ClusterInfoService() { @Override @@ -423,14 +429,14 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { // Set the high threshold to 70 instead of 80 // node2 now should not have new shards allocated to it, but shards can remain diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "40b") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "30b").build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "40b") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "30b").build(); deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); strategy = new AllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 10) @@ -454,14 +460,14 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { // Set the high threshold to 60 instead of 70 // node2 now should not have new shards allocated to it, and shards cannot remain diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "50b") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "40b").build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "50b") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "40b").build(); deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); strategy = new AllocationService(Settings.builder() .put("cluster.routing.allocation.node_concurrent_recoveries", 10) @@ -542,9 +548,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testDiskThresholdWithShardSizes() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "71%").build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "71%").build(); ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 31)); // 69% used @@ -559,7 +565,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); ClusterInfoService cis = new ClusterInfoService() { @Override @@ -611,9 +617,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testUnknownDiskUsage() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.85).build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.85).build(); ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); usagesBuilder.put("node2", new DiskUsage("node2", "node2", "/dev/null", 100, 50)); // 50% used @@ -629,7 +635,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); ClusterInfoService cis = new ClusterInfoService() { @Override @@ -687,7 +693,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testAverageUsage() { RoutingNode rn = new RoutingNode("node1", newNode("node1")); - DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY); + DiskThresholdDecider decider = makeDecider(Settings.EMPTY); ImmutableOpenMap.Builder usages = ImmutableOpenMap.builder(); usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 50)); // 50% used @@ -700,7 +706,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testFreeDiskPercentageAfterShardAssigned() { RoutingNode rn = new RoutingNode("node1", newNode("node1")); - DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY); + DiskThresholdDecider decider = makeDecider(Settings.EMPTY); Map usages = new HashMap<>(); usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 50)); // 50% used @@ -712,10 +718,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testShardRelocationsTakenIntoAccount() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.8).build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.8).build(); ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 40)); // 60% used @@ -734,7 +740,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( new SameShardAllocationDecider(Settings.EMPTY), - new DiskThresholdDecider(diskSettings)))); + makeDecider(diskSettings)))); ClusterInfoService cis = new ClusterInfoService() { @Override @@ -821,10 +827,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testCanRemainWithShardRelocatingAway() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%").build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%").build(); // We have an index with 2 primary shards each taking 40 bytes. Each node has 100 bytes available ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); @@ -839,7 +845,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { final ClusterInfo clusterInfo = new DevNullClusterInfo(usages, usages, shardSizes); - DiskThresholdDecider diskThresholdDecider = new DiskThresholdDecider(diskSettings); + DiskThresholdDecider diskThresholdDecider = makeDecider(diskSettings); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) .build(); @@ -937,10 +943,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { public void testForSingleDataNode() { Settings diskSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%").build(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%").build(); ImmutableOpenMap.Builder usagesBuilder = ImmutableOpenMap.builder(); usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 100)); // 0% used @@ -954,7 +960,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase { shardSizes.put("[test][1][p]", 40L); final ClusterInfo clusterInfo = new DevNullClusterInfo(usages, usages, shardSizes.build()); - DiskThresholdDecider diskThresholdDecider = new DiskThresholdDecider(diskSettings); + DiskThresholdDecider diskThresholdDecider = makeDecider(diskSettings); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) .build(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 80309004fff..56c7d69c59b 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -59,48 +59,10 @@ import static org.hamcrest.CoreMatchers.equalTo; * Unit tests for the DiskThresholdDecider */ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { - public void testDynamicSettings() { - ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - - ClusterInfoService cis = EmptyClusterInfoService.INSTANCE; - DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null); - - assertThat(decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test"))); - assertThat(decider.getFreeDiskThresholdHigh(), equalTo(10.0d)); - assertThat(decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test"))); - assertThat(decider.getFreeDiskThresholdLow(), equalTo(15.0d)); - assertThat(decider.getUsedDiskThresholdLow(), equalTo(85.0d)); - assertThat(decider.getRerouteInterval().seconds(), equalTo(60L)); - assertTrue(decider.isEnabled()); - assertTrue(decider.isIncludeRelocations()); - - Settings newSettings = Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), false) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), false) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "500mb") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "30s") - .build(); - - nss.applySettings(newSettings); - assertThat("high threshold bytes should be unset", - decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test"))); - assertThat("high threshold percentage should be changed", - decider.getFreeDiskThresholdHigh(), equalTo(30.0d)); - assertThat("low threshold bytes should be set to 500mb", - decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb", "test"))); - assertThat("low threshold bytes should be unset", - decider.getFreeDiskThresholdLow(), equalTo(0.0d)); - assertThat("reroute interval should be changed to 30 seconds", - decider.getRerouteInterval().seconds(), equalTo(30L)); - assertFalse("disk threshold decider should now be disabled", decider.isEnabled()); - assertFalse("relocations should now be disabled", decider.isIncludeRelocations()); - } public void testCanAllocateUsesMaxAvailableSpace() { ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - ClusterInfoService cis = EmptyClusterInfoService.INSTANCE; - DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null); + DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss); MetaData metaData = MetaData.builder() .put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1)) @@ -144,8 +106,7 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { public void testCanRemainUsesLeastAvailableSpace() { ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - ClusterInfoService cis = EmptyClusterInfoService.INSTANCE; - DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null); + DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss); ImmutableOpenMap.Builder shardRoutingMap = ImmutableOpenMap.builder(); DiscoveryNode node_0 = new DiscoveryNode("node_0", LocalTransportAddress.buildUnique(), Collections.emptyMap(), diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java index 3bd7094796f..68e8fc9c94e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/MockDiskUsagesIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.cluster.ClusterInfoService; import org.elasticsearch.cluster.DiskUsage; import org.elasticsearch.cluster.MockInternalClusterInfoService; import org.elasticsearch.cluster.routing.RoutingNode; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.plugins.Plugin; @@ -74,9 +75,9 @@ public class MockDiskUsagesIT extends ESIntegTestCase { cis.setN3Usage(nodes.get(2), new DiskUsage(nodes.get(2), "n3", "/dev/null", 100, 50)); client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder() - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), randomFrom("20b", "80%")) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), randomFrom("10b", "90%")) - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "1ms")).get(); + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), randomFrom("20b", "80%")) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), randomFrom("10b", "90%")) + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "1ms")).get(); // Create an index with 10 shards so we can check allocation for it prepareCreate("test").setSettings(Settings.builder() .put("number_of_shards", 10) diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index dffa285101f..e723f970bbd 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -29,6 +29,7 @@ import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.env.NodeEnvironment; @@ -72,7 +73,6 @@ import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; @@ -1634,8 +1634,8 @@ public abstract class ESIntegTestCase extends ESTestCase { .put(NodeEnvironment.MAX_LOCAL_STORAGE_NODES_SETTING.getKey(), Integer.MAX_VALUE) // Default the watermarks to absurdly low to prevent the tests // from failing on nodes without enough disk space - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "1b") - .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "1b") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "1b") + .put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "1b") .put(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey(), 1000) .put("script.stored", "true") .put("script.inline", "true") diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index d76fea08911..1dd1c5d9b64 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -43,7 +43,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode.Role; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.OperationRouting; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; +import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings; import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; @@ -316,8 +316,8 @@ public final class InternalTestCluster extends TestCluster { } // Default the watermarks to absurdly low to prevent the tests // from failing on nodes without enough disk space - builder.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "1b"); - builder.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "1b"); + builder.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "1b"); + builder.put(DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "1b"); // Some tests make use of scripting quite a bit, so increase the limit for integration tests builder.put(ScriptService.SCRIPT_MAX_COMPILATIONS_PER_MINUTE.getKey(), 1000); if (TEST_NIGHTLY) { From f556424ab946acec6444f9505c951d1d53396c66 Mon Sep 17 00:00:00 2001 From: markwalkom Date: Wed, 17 Aug 2016 21:38:56 +1000 Subject: [PATCH 02/13] Update synonym-tokenfilter.asciidoc (#19988) * Update synonym-tokenfilter.asciidoc * Update synonym-tokenfilter.asciidoc --- .../tokenfilters/synonym-tokenfilter.asciidoc | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc index 0d57b5bd2d1..12da352b51c 100644 --- a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc @@ -45,20 +45,20 @@ The following is a sample format of the file: [source,js] -------------------------------------------------- -# blank lines and lines starting with pound are comments. +# Blank lines and lines starting with pound are comments. -#Explicit mappings match any token sequence on the LHS of "=>" -#and replace with all alternatives on the RHS. These types of mappings -#ignore the expand parameter in the schema. -#Examples: +# Explicit mappings match any token sequence on the LHS of "=>" +# and replace with all alternatives on the RHS. These types of mappings +# ignore the expand parameter in the schema. +# Examples: i-pod, i pod => ipod, sea biscuit, sea biscit => seabiscuit -#Equivalent synonyms may be separated with commas and give -#no explicit mapping. In this case the mapping behavior will -#be taken from the expand parameter in the schema. This allows -#the same synonym file to be used in different synonym handling strategies. -#Examples: +# Equivalent synonyms may be separated with commas and give +# no explicit mapping. In this case the mapping behavior will +# be taken from the expand parameter in the schema. This allows +# the same synonym file to be used in different synonym handling strategies. +# Examples: ipod, i-pod, i pod foozball , foosball universe , cosmos @@ -70,10 +70,10 @@ ipod, i-pod, i pod => ipod, i-pod, i pod # to the explicit mapping: ipod, i-pod, i pod => ipod -#multiple synonym mapping entries are merged. +# Multiple synonym mapping entries are merged. foo => foo bar foo => baz -#is equivalent to +# is equivalent to foo => foo bar, baz -------------------------------------------------- @@ -96,7 +96,7 @@ configuration file (note use of `synonyms` instead of `synonyms_path`): -------------------------------------------------- However, it is recommended to define large synonyms set in a file using -`synonyms_path`. +`synonyms_path`, because specifying them inline increases cluster size unnecessarily. [float] ==== WordNet synonyms From 7da9d826ff00f6c4fdee8c2acbf27210cf532236 Mon Sep 17 00:00:00 2001 From: Clinton Gormley Date: Wed, 17 Aug 2016 14:40:59 +0200 Subject: [PATCH 03/13] Update ingest.get_pipeline.json The `id` parameter is not required Closes #20010 --- .../main/resources/rest-api-spec/api/ingest.get_pipeline.json | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json index 5574ddbf5b7..31725087423 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/ingest.get_pipeline.json @@ -8,8 +8,7 @@ "parts": { "id": { "type" : "string", - "description" : "Comma separated list of pipeline ids. Wildcards supported", - "required" : true + "description" : "Comma separated list of pipeline ids. Wildcards supported" } }, "params": { From c66db9a81eefb03e43ce69a34dcfcc68cd7e1cbd Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 12 Aug 2016 18:42:19 -0400 Subject: [PATCH 04/13] Add `// CONSOLE` to much of pipeline agg docs Most of the examples in the pipeline aggregation docs use a small "sales" test data set and I converted all of the examples that use it to `// CONSOLE`. There are still a bunch of snippets in the pipeline aggregation docs that aren't `// CONSOLE` so they aren't tested. Most of them are "this is the most basic form of this aggregation" so they are more immune to errors and bit rot then the examples that I converted. I'd like to do something with them as well but I'm not sure what. Also, the moving average docs and serial diff docs didn't get a lot of love from this pass because they don't use the test data set or follow the same general layout. Relates to #18160 --- docs/build.gradle | 35 +++++++ docs/reference/aggregations/pipeline.asciidoc | 50 ++++++---- .../pipeline/avg-bucket-aggregation.asciidoc | 59 +++++++----- .../bucket-script-aggregation.asciidoc | 44 +++++---- .../bucket-selector-aggregation.asciidoc | 35 ++++--- .../cumulative-sum-aggregation.asciidoc | 25 +++-- .../pipeline/derivative-aggregation.asciidoc | 91 +++++++++++++------ ...extended-stats-bucket-aggregation.asciidoc | 39 +++++--- .../pipeline/max-bucket-aggregation.asciidoc | 25 +++-- .../pipeline/min-bucket-aggregation.asciidoc | 26 ++++-- .../pipeline/movavg-aggregation.asciidoc | 26 ++++-- .../percentiles-bucket-aggregation.asciidoc | 29 ++++-- .../pipeline/serial-diff-aggregation.asciidoc | 12 +-- .../stats-bucket-aggregation.asciidoc | 27 ++++-- .../pipeline/sum-bucket-aggregation.asciidoc | 25 +++-- 15 files changed, 369 insertions(+), 179 deletions(-) diff --git a/docs/build.gradle b/docs/build.gradle index 9c9cf9922c5..caf7cfea01e 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -105,3 +105,38 @@ buildRestTests.setups['host'] = ''' - is_true: nodes.$master.http.publish_address - set: {nodes.$master.http.publish_address: host} ''' + +// Used by pipeline aggregation docs +buildRestTests.setups['sales'] = ''' + - do: + indices.create: + index: sales + body: + settings: + number_of_shards: 2 + number_of_replicas: 1 + mappings: + sale: + properties: + type: + type: keyword + - do: + bulk: + index: sales + type: sale + refresh: true + body: | + {"index":{}} + {"date": "2015/01/01 00:00:00", "price": 200, "type": "hat"} + {"index":{}} + {"date": "2015/01/01 00:00:00", "price": 200, "type": "t-shirt"} + {"index":{}} + {"date": "2015/01/01 00:00:00", "price": 150, "type": "bag"} + {"index":{}} + {"date": "2015/02/01 00:00:00", "price": 50, "type": "hat"} + {"index":{}} + {"date": "2015/02/01 00:00:00", "price": 10, "type": "t-shirt"} + {"index":{}} + {"date": "2015/03/01 00:00:00", "price": 200, "type": "hat"} + {"index":{}} + {"date": "2015/03/01 00:00:00", "price": 175, "type": "t-shirt"}''' diff --git a/docs/reference/aggregations/pipeline.asciidoc b/docs/reference/aggregations/pipeline.asciidoc index 4ca249d31db..da78e188559 100644 --- a/docs/reference/aggregations/pipeline.asciidoc +++ b/docs/reference/aggregations/pipeline.asciidoc @@ -51,23 +51,27 @@ metric `"the_sum"`: [source,js] -------------------------------------------------- +POST /_search { - "my_date_histo":{ - "date_histogram":{ - "field":"timestamp", - "interval":"day" - }, - "aggs":{ - "the_sum":{ - "sum":{ "field": "lemmings" } <1> + "aggs": { + "my_date_histo":{ + "date_histogram":{ + "field":"timestamp", + "interval":"day" }, - "the_movavg":{ - "moving_avg":{ "buckets_path": "the_sum" } <2> + "aggs":{ + "the_sum":{ + "sum":{ "field": "lemmings" } <1> + }, + "the_movavg":{ + "moving_avg":{ "buckets_path": "the_sum" } <2> + } } } } } -------------------------------------------------- +// CONSOLE <1> The metric is called `"the_sum"` <2> The `buckets_path` refers to the metric via a relative path `"the_sum"` @@ -77,6 +81,7 @@ a metric embedded inside a sibling aggregation: [source,js] -------------------------------------------------- +POST /_search { "aggs" : { "sales_per_month" : { @@ -100,6 +105,8 @@ a metric embedded inside a sibling aggregation: } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] <1> `buckets_path` instructs this max_bucket aggregation that we want the maximum value of the `sales` aggregation in the `sales_per_month` date histogram. @@ -111,20 +118,24 @@ the pipeline aggregation to use the document count as it's input. For example, [source,js] -------------------------------------------------- +POST /_search { - "my_date_histo":{ - "date_histogram":{ - "field":"timestamp", - "interval":"day" - }, - "aggs":{ - "the_movavg":{ - "moving_avg":{ "buckets_path": "_count" } <1> + "aggs": { + "my_date_histo": { + "date_histogram": { + "field":"timestamp", + "interval":"day" + }, + "aggs": { + "the_movavg": { + "moving_avg": { "buckets_path": "_count" } <1> + } } } } } -------------------------------------------------- +// CONSOLE <1> By using `_count` instead of a metric name, we can calculate the moving average of document counts in the histogram The `buckets_path` can also use `"_bucket_count"` and path to a multi-bucket aggregation to use the number of buckets @@ -133,6 +144,7 @@ used here to filter out buckets which contain no buckets for an inner terms aggr [source,js] -------------------------------------------------- +POST /sales/_search { "size": 0, "aggs": { @@ -162,6 +174,8 @@ used here to filter out buckets which contain no buckets for an inner terms aggr } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] <1> By using `_bucket_count` instead of a metric name, we can filter out `histo` buckets where they contain no buckets for the `categories` aggregation diff --git a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc index 541ffecbf95..dc1aa78baea 100644 --- a/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/avg-bucket-aggregation.asciidoc @@ -3,7 +3,7 @@ experimental[] -A sibling pipeline aggregation which calculates the (mean) average value of a specified metric in a sibling aggregation. +A sibling pipeline aggregation which calculates the (mean) average value of a specified metric in a sibling aggregation. The specified metric must be numeric and the sibling aggregation must be a multi-bucket aggregation. ==== Syntax @@ -33,30 +33,35 @@ The following snippet calculates the average of the total monthly `sales`: [source,js] -------------------------------------------------- +POST /_search { - "aggs" : { - "sales_per_month" : { - "date_histogram" : { - "field" : "date", - "interval" : "month" - }, - "aggs": { - "sales": { - "sum": { - "field": "price" - } - } - } - }, - "avg_monthly_sales": { - "avg_bucket": { - "buckets_path": "sales_per_month>sales" <1> - } + "size": 0, + "aggs": { + "sales_per_month": { + "date_histogram": { + "field": "date", + "interval": "month" + }, + "aggs": { + "sales": { + "sum": { + "field": "price" + } } + } + }, + "avg_monthly_sales": { + "avg_bucket": { + "buckets_path": "sales_per_month>sales" <1> + } } + } } + -------------------------------------------------- -<1> `buckets_path` instructs this avg_bucket aggregation that we want the (mean) average value of the `sales` aggregation in the +// CONSOLE +// TEST[setup:sales] +<1> `buckets_path` instructs this avg_bucket aggregation that we want the (mean) average value of the `sales` aggregation in the `sales_per_month` date histogram. And the following may be the response: @@ -64,6 +69,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -72,7 +81,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -80,7 +89,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -88,7 +97,7 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] @@ -99,4 +108,6 @@ And the following may be the response: } } -------------------------------------------------- - +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] diff --git a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc index b1bbfcd7bc6..1bfd080090a 100644 --- a/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-script-aggregation.asciidoc @@ -3,7 +3,7 @@ experimental[] -A parent pipeline aggregation which executes a script which can perform per bucket computations on specified metrics +A parent pipeline aggregation which executes a script which can perform per bucket computations on specified metrics in the parent multi-bucket aggregation. The specified metric must be numeric and the script must return a numeric value. ==== Syntax @@ -22,16 +22,16 @@ A `bucket_script` aggregation looks like this in isolation: } } -------------------------------------------------- -<1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to +<1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to the metrics to use for that variable. .`bucket_script` Parameters |=== |Parameter Name |Description |Required |Default Value -|`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> +|`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> for more details) |Required | -|`buckets_path` |A map of script variables and their associated path to the buckets we wish to use for the variable +|`buckets_path` |A map of script variables and their associated path to the buckets we wish to use for the variable (see <> for more details) |Required | |`gap_policy` |The policy to apply when gaps are found in the data (see <> for more details)|Optional, defaults to `skip` | @@ -42,7 +42,9 @@ The following snippet calculates the ratio percentage of t-shirt sales compared [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -71,7 +73,7 @@ The following snippet calculates the ratio percentage of t-shirt sales compared }, "t-shirt-percentage": { "bucket_script": { - "buckets_path": { + "buckets_path": { "tShirtSales": "t-shirts>sales", "totalSales": "total_sales" }, @@ -83,12 +85,18 @@ The following snippet calculates the ratio percentage of t-shirt sales compared } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -97,33 +105,33 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "total_sales": { - "value": 50 + "value": 550.0 }, "t-shirts": { - "doc_count": 2, + "doc_count": 1, "sales": { - "value": 10 + "value": 200.0 } }, "t-shirt-percentage": { - "value": 20 + "value": 36.36363636363637 } }, { "key_as_string": "2015/02/01 00:00:00", "key": 1422748800000, - "doc_count": 2 + "doc_count": 2, "total_sales": { - "value": 60 + "value": 60.0 }, "t-shirts": { "doc_count": 1, "sales": { - "value": 15 + "value": 10.0 } }, "t-shirt-percentage": { - "value": 25 + "value": 16.666666666666664 } }, { @@ -131,16 +139,16 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "total_sales": { - "value": 40 + "value": 375.0 }, "t-shirts": { "doc_count": 1, "sales": { - "value": 20 + "value": 175.0 } }, "t-shirt-percentage": { - "value": 50 + "value": 46.666666666666664 } } ] @@ -148,4 +156,6 @@ And the following may be the response: } } -------------------------------------------------- - +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] diff --git a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc index 4aacb748ae7..86a56a77aa2 100644 --- a/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/bucket-selector-aggregation.asciidoc @@ -3,12 +3,12 @@ experimental[] -A parent pipeline aggregation which executes a script which determines whether the current bucket will be retained -in the parent multi-bucket aggregation. The specified metric must be numeric and the script must return a boolean value. -If the script language is `expression` then a numeric return value is permitted. In this case 0.0 will be evaluated as `false` +A parent pipeline aggregation which executes a script which determines whether the current bucket will be retained +in the parent multi-bucket aggregation. The specified metric must be numeric and the script must return a boolean value. +If the script language is `expression` then a numeric return value is permitted. In this case 0.0 will be evaluated as `false` and all other values will evaluate to true. -Note: The bucket_selector aggregation, like all pipeline aggregations, executions after all other sibling aggregations. This means that +Note: The bucket_selector aggregation, like all pipeline aggregations, executions after all other sibling aggregations. This means that using the bucket_selector aggregation to filter the returned buckets in the response does not save on execution time running the aggregations. ==== Syntax @@ -27,26 +27,28 @@ A `bucket_selector` aggregation looks like this in isolation: } } -------------------------------------------------- -<1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to +<1> Here, `my_var1` is the name of the variable for this buckets path to use in the script, `the_sum` is the path to the metrics to use for that variable. .`bucket_selector` Parameters |=== |Parameter Name |Description |Required |Default Value -|`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> +|`script` |The script to run for this aggregation. The script can be inline, file or indexed. (see <> for more details) |Required | -|`buckets_path` |A map of script variables and their associated path to the buckets we wish to use for the variable +|`buckets_path` |A map of script variables and their associated path to the buckets we wish to use for the variable (see <> for more details) |Required | |`gap_policy` |The policy to apply when gaps are found in the data (see <> for more details)|Optional, defaults to `skip` | |=== -The following snippet only retains buckets where the total sales for the month is less than or equal to 50: +The following snippet only retains buckets where the total sales for the month is more than 400: [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -64,7 +66,7 @@ The following snippet only retains buckets where the total sales for the month i "buckets_path": { "totalSales": "total_sales" }, - "script": "totalSales <= 50" + "script": "totalSales > 200" } } } @@ -72,12 +74,18 @@ The following snippet only retains buckets where the total sales for the month i } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -86,7 +94,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "total_sales": { - "value": 50 + "value": 550.0 } },<1> { @@ -94,7 +102,7 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "total_sales": { - "value": 40 + "value": 375.0 }, } ] @@ -102,4 +110,7 @@ And the following may be the response: } } -------------------------------------------------- -<1> Bucket for `2015/02/01 00:00:00` has been removed as its total sales exceeded 50 +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] +<1> Bucket for `2015/02/01 00:00:00` has been removed as its total sales was less than 200 diff --git a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc index e29dbbe7ee9..6039813351c 100644 --- a/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/cumulative-sum-aggregation.asciidoc @@ -3,7 +3,7 @@ experimental[] -A parent pipeline aggregation which calculates the cumulative sum of a specified metric in a parent histogram (or date_histogram) +A parent pipeline aggregation which calculates the cumulative sum of a specified metric in a parent histogram (or date_histogram) aggregation. The specified metric must be numeric and the enclosing histogram must have `min_doc_count` set to `0` (default for `histogram` aggregations). @@ -32,7 +32,9 @@ The following snippet calculates the cumulative sum of the total monthly `sales` [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -55,6 +57,8 @@ The following snippet calculates the cumulative sum of the total monthly `sales` } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] <1> `buckets_path` instructs this cumulative sum aggregation to use the output of the `sales` aggregation for the cumulative sum @@ -63,6 +67,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -71,10 +79,10 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 }, "cumulative_sales": { - "value": 550 + "value": 550.0 } }, { @@ -82,10 +90,10 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 }, "cumulative_sales": { - "value": 610 + "value": 610.0 } }, { @@ -93,10 +101,10 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 }, "cumulative_sales": { - "value": 985 + "value": 985.0 } } ] @@ -104,3 +112,6 @@ And the following may be the response: } } -------------------------------------------------- +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] diff --git a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc index f68a8118a34..d7844cd6a3e 100644 --- a/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/derivative-aggregation.asciidoc @@ -3,7 +3,7 @@ experimental[] -A parent pipeline aggregation which calculates the derivative of a specified metric in a parent histogram (or date_histogram) +A parent pipeline aggregation which calculates the derivative of a specified metric in a parent histogram (or date_histogram) aggregation. The specified metric must be numeric and the enclosing histogram must have `min_doc_count` set to `0` (default for `histogram` aggregations). @@ -13,10 +13,8 @@ A `derivative` aggregation looks like this in isolation: [source,js] -------------------------------------------------- -{ - "derivative": { - "buckets_path": "the_sum" - } +"derivative": { + "buckets_path": "the_sum" } -------------------------------------------------- @@ -37,7 +35,9 @@ The following snippet calculates the derivative of the total monthly `sales`: [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -60,6 +60,8 @@ The following snippet calculates the derivative of the total monthly `sales`: } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] <1> `buckets_path` instructs this derivative aggregation to use the output of the `sales` aggregation for the derivative @@ -68,6 +70,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -76,7 +82,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } <1> }, { @@ -84,10 +90,10 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 }, "sales_deriv": { - "value": -490 <2> + "value": -490.0 <2> } }, { @@ -95,10 +101,10 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, <3> "sales": { - "value": 375 + "value": 375.0 }, "sales_deriv": { - "value": 315 + "value": 315.0 } } ] @@ -106,21 +112,26 @@ And the following may be the response: } } -------------------------------------------------- +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] <1> No derivative for the first bucket since we need at least 2 data points to calculate the derivative -<2> Derivative value units are implicitly defined by the `sales` aggregation and the parent histogram so in this case the units +<2> Derivative value units are implicitly defined by the `sales` aggregation and the parent histogram so in this case the units would be $/month assuming the `price` field has units of $. <3> The number of documents in the bucket are represented by the `doc_count` f ==== Second Order Derivative -A second order derivative can be calculated by chaining the derivative pipeline aggregation onto the result of another derivative -pipeline aggregation as in the following example which will calculate both the first and the second order derivative of the total +A second order derivative can be calculated by chaining the derivative pipeline aggregation onto the result of another derivative +pipeline aggregation as in the following example which will calculate both the first and the second order derivative of the total monthly sales: [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -148,6 +159,8 @@ monthly sales: } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] <1> `buckets_path` for the second derivative points to the name of the first derivative @@ -156,6 +169,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 50, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -164,7 +181,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } <1> }, { @@ -172,10 +189,10 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 }, "sales_deriv": { - "value": -490 + "value": -490.0 } <1> }, { @@ -183,13 +200,13 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 }, "sales_deriv": { - "value": 315 + "value": 315.0 }, "sales_2nd_deriv": { - "value": 805 + "value": 805.0 } } ] @@ -197,18 +214,24 @@ And the following may be the response: } } -------------------------------------------------- -<1> No second derivative for the first two buckets since we need at least 2 data points from the first derivative to calculate the +// TESTRESPONSE[s/"took": 50/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] + +<1> No second derivative for the first two buckets since we need at least 2 data points from the first derivative to calculate the second derivative ==== Units -The derivative aggregation allows the units of the derivative values to be specified. This returns an extra field in the response -`normalized_value` which reports the derivative value in the desired x-axis units. In the below example we calculate the derivative +The derivative aggregation allows the units of the derivative values to be specified. This returns an extra field in the response +`normalized_value` which reports the derivative value in the desired x-axis units. In the below example we calculate the derivative of the total sales per month but ask for the derivative of the sales as in the units of sales per day: [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -232,7 +255,8 @@ of the total sales per month but ask for the derivative of the sales as in the u } } -------------------------------------------------- - +// CONSOLE +// TEST[setup:sales] <1> `unit` specifies what unit to use for the x-axis of the derivative calculation And the following may be the response: @@ -240,6 +264,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 50, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -248,7 +276,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } <1> }, { @@ -256,11 +284,11 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 }, "sales_deriv": { - "value": -490, <1> - "normalized_value": -17.5 <2> + "value": -490.0, <1> + "normalized_value": -15.806451612903226 <2> } }, { @@ -268,11 +296,11 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 }, "sales_deriv": { - "value": 315, - "normalized_value": 10.16129032258065 + "value": 315.0, + "normalized_value": 11.25 } } ] @@ -280,5 +308,8 @@ And the following may be the response: } } -------------------------------------------------- +// TESTRESPONSE[s/"took": 50/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] <1> `value` is reported in the original units of 'per month' <2> `normalized_value` is reported in the desired units of 'per day' diff --git a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc index 0a44685ba1c..11d9c2906b9 100644 --- a/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/extended-stats-bucket-aggregation.asciidoc @@ -36,7 +36,9 @@ The following snippet calculates the sum of all the total monthly `sales` bucket [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -53,12 +55,15 @@ The following snippet calculates the sum of all the total monthly `sales` bucket }, "stats_monthly_sales": { "extended_stats_bucket": { - "buckets_paths": "sales_per_month>sales" <1> + "buckets_path": "sales_per_month>sales" <1> } } } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] + <1> `bucket_paths` instructs this `extended_stats_bucket` aggregation that we want the calculate stats for the `sales` aggregation in the `sales_per_month` date histogram. @@ -67,6 +72,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -75,7 +84,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -83,7 +92,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -91,26 +100,28 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] }, "stats_monthly_sales": { "count": 3, - "min": 60, - "max": 550, - "avg": 328.333333333, - "sum": 985, - "sum_of_squares": 446725, - "variance": 41105.5555556, - "std_deviation": 117.054909559, + "min": 60.0, + "max": 550.0, + "avg": 328.3333333333333, + "sum": 985.0, + "sum_of_squares": 446725.0, + "variance": 41105.55555555556, + "std_deviation": 202.74505063146563, "std_deviation_bounds": { - "upper": 562.443152451, - "lower": 94.2235142151 + "upper": 733.8234345962646, + "lower": -77.15676792959795 } } } } -------------------------------------------------- - +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] diff --git a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc index 96094d04562..4c11d7afc41 100644 --- a/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/max-bucket-aggregation.asciidoc @@ -4,7 +4,7 @@ experimental[] A sibling pipeline aggregation which identifies the bucket(s) with the maximum value of a specified metric in a sibling aggregation -and outputs both the value and the key(s) of the bucket(s). The specified metric must be numeric and the sibling aggregation must +and outputs both the value and the key(s) of the bucket(s). The specified metric must be numeric and the sibling aggregation must be a multi-bucket aggregation. ==== Syntax @@ -34,7 +34,9 @@ The following snippet calculates the maximum of the total monthly `sales`: [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -57,7 +59,10 @@ The following snippet calculates the maximum of the total monthly `sales`: } } -------------------------------------------------- -<1> `buckets_path` instructs this max_bucket aggregation that we want the maximum value of the `sales` aggregation in the +// CONSOLE +// TEST[setup:sales] + +<1> `buckets_path` instructs this max_bucket aggregation that we want the maximum value of the `sales` aggregation in the `sales_per_month` date histogram. And the following may be the response: @@ -65,6 +70,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -73,7 +82,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -81,7 +90,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -89,18 +98,20 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] }, "max_monthly_sales": { "keys": ["2015/01/01 00:00:00"], <1> - "value": 550 + "value": 550.0 } } } -------------------------------------------------- +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] <1> `keys` is an array of strings since the maximum value may be present in multiple buckets - diff --git a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc index c970384ad64..9056707ee46 100644 --- a/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/min-bucket-aggregation.asciidoc @@ -3,8 +3,8 @@ experimental[] -A sibling pipeline aggregation which identifies the bucket(s) with the minimum value of a specified metric in a sibling aggregation -and outputs both the value and the key(s) of the bucket(s). The specified metric must be numeric and the sibling aggregation must +A sibling pipeline aggregation which identifies the bucket(s) with the minimum value of a specified metric in a sibling aggregation +and outputs both the value and the key(s) of the bucket(s). The specified metric must be numeric and the sibling aggregation must be a multi-bucket aggregation. ==== Syntax @@ -35,7 +35,9 @@ The following snippet calculates the minimum of the total monthly `sales`: [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -58,8 +60,10 @@ The following snippet calculates the minimum of the total monthly `sales`: } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] -<1> `buckets_path` instructs this max_bucket aggregation that we want the minimum value of the `sales` aggregation in the +<1> `buckets_path` instructs this max_bucket aggregation that we want the minimum value of the `sales` aggregation in the `sales_per_month` date histogram. And the following may be the response: @@ -67,6 +71,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -75,7 +83,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -83,7 +91,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -91,18 +99,20 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] }, "min_monthly_sales": { "keys": ["2015/02/01 00:00:00"], <1> - "value": 60 + "value": 60.0 } } } -------------------------------------------------- +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] <1> `keys` is an array of strings since the minimum value may be present in multiple buckets - diff --git a/docs/reference/aggregations/pipeline/movavg-aggregation.asciidoc b/docs/reference/aggregations/pipeline/movavg-aggregation.asciidoc index 510cd7caae5..bbe4170e2e7 100644 --- a/docs/reference/aggregations/pipeline/movavg-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/movavg-aggregation.asciidoc @@ -52,23 +52,29 @@ embedded like any other metric aggregation: [source,js] -------------------------------------------------- +POST /_search { - "my_date_histo":{ <1> - "date_histogram":{ - "field":"timestamp", - "interval":"day" - }, - "aggs":{ - "the_sum":{ - "sum":{ "field": "lemmings" } <2> + "size": 0, + "aggs": { + "my_date_histo":{ <1> + "date_histogram":{ + "field":"timestamp", + "interval":"day" }, - "the_movavg":{ - "moving_avg":{ "buckets_path": "the_sum" } <3> + "aggs":{ + "the_sum":{ + "sum":{ "field": "lemmings" } <2> + }, + "the_movavg":{ + "moving_avg":{ "buckets_path": "the_sum" } <3> + } } } } } -------------------------------------------------- +// CONSOLE + <1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-day intervals <2> A `sum` metric is used to calculate the sum of a field. This could be any metric (sum, min, max, etc) <3> Finally, we specify a `moving_avg` aggregation which uses "the_sum" metric as its input. diff --git a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc index aa84c954197..fcb29104d67 100644 --- a/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/percentiles-bucket-aggregation.asciidoc @@ -34,7 +34,9 @@ The following snippet calculates the sum of all the total monthly `sales` bucket [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -49,7 +51,7 @@ The following snippet calculates the sum of all the total monthly `sales` bucket } } }, - "sum_monthly_sales": { + "percentiles_monthly_sales": { "percentiles_bucket": { "buckets_path": "sales_per_month>sales", <1> "percents": [ 25.0, 50.0, 75.0 ] <2> @@ -58,6 +60,9 @@ The following snippet calculates the sum of all the total monthly `sales` bucket } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] + <1> `buckets_path` instructs this percentiles_bucket aggregation that we want to calculate percentiles for the `sales` aggregation in the `sales_per_month` date histogram. <2> `percents` specifies which percentiles we wish to calculate, in this case, the 25th, 50th and 75th percentil @@ -67,6 +72,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -75,7 +84,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -83,7 +92,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -91,22 +100,24 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] }, "percentiles_monthly_sales": { "values" : { - "25.0": 60, - "50.0": 375", - "75.0": 550 + "25.0": 60.0, + "50.0": 375.0, + "75.0": 550.0 } } } } -------------------------------------------------- - +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] ==== Percentiles_bucket implementation @@ -116,4 +127,4 @@ interpolate between data points. The percentiles are calculated exactly and is not an approximation (unlike the Percentiles Metric). This means the implementation maintains an in-memory, sorted list of your data to compute the percentiles, before discarding the data. You may run into memory pressure issues if you attempt to calculate percentiles over many millions of -data-points in a single `percentiles_bucket`. \ No newline at end of file +data-points in a single `percentiles_bucket`. diff --git a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc index 31e715c3e96..fadc07d54f4 100644 --- a/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/serial-diff-aggregation.asciidoc @@ -61,7 +61,9 @@ A `serial_diff` aggregation looks like this in isolation: [source,js] -------------------------------------------------- +POST /_search { + "size": 0, "aggs": { "my_date_histo": { <1> "date_histogram": { @@ -85,6 +87,8 @@ A `serial_diff` aggregation looks like this in isolation: } } -------------------------------------------------- +// CONSOLE + <1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-day intervals <2> A `sum` metric is used to calculate the sum of a field. This could be any metric (sum, min, max, etc) <3> Finally, we specify a `serial_diff` aggregation which uses "the_sum" metric as its input. @@ -93,11 +97,3 @@ Serial differences are built by first specifying a `histogram` or `date_histogra add normal metrics, such as a `sum`, inside of that histogram. Finally, the `serial_diff` is embedded inside the histogram. The `buckets_path` parameter is then used to "point" at one of the sibling metrics inside of the histogram (see <> for a description of the syntax for `buckets_path`. - - - - - - - - diff --git a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc index cc863fe6a05..6c08fe5cd2f 100644 --- a/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/stats-bucket-aggregation.asciidoc @@ -33,7 +33,9 @@ The following snippet calculates the sum of all the total monthly `sales` bucket [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -56,6 +58,9 @@ The following snippet calculates the sum of all the total monthly `sales` bucket } } -------------------------------------------------- +// CONSOLE +// TEST[setup:sales] + <1> `bucket_paths` instructs this `stats_bucket` aggregation that we want the calculate stats for the `sales` aggregation in the `sales_per_month` date histogram. @@ -64,6 +69,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -72,7 +81,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -80,7 +89,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -88,19 +97,21 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] }, "stats_monthly_sales": { "count": 3, - "min": 60, - "max": 550, - "avg": 328.333333333, - "sum": 985 + "min": 60.0, + "max": 550.0, + "avg": 328.3333333333333, + "sum": 985.0 } } } -------------------------------------------------- - +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] diff --git a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc index 52022b376f3..142b6b6f777 100644 --- a/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/sum-bucket-aggregation.asciidoc @@ -3,7 +3,7 @@ experimental[] -A sibling pipeline aggregation which calculates the sum across all bucket of a specified metric in a sibling aggregation. +A sibling pipeline aggregation which calculates the sum across all bucket of a specified metric in a sibling aggregation. The specified metric must be numeric and the sibling aggregation must be a multi-bucket aggregation. ==== Syntax @@ -33,7 +33,9 @@ The following snippet calculates the sum of all the total monthly `sales` bucket [source,js] -------------------------------------------------- +POST /sales/_search { + "size": 0, "aggs" : { "sales_per_month" : { "date_histogram" : { @@ -56,7 +58,10 @@ The following snippet calculates the sum of all the total monthly `sales` bucket } } -------------------------------------------------- -<1> `buckets_path` instructs this sum_bucket aggregation that we want the sum of the `sales` aggregation in the +// CONSOLE +// TEST[setup:sales] + +<1> `buckets_path` instructs this sum_bucket aggregation that we want the sum of the `sales` aggregation in the `sales_per_month` date histogram. And the following may be the response: @@ -64,6 +69,10 @@ And the following may be the response: [source,js] -------------------------------------------------- { + "took": 11, + "timed_out": false, + "_shards": ..., + "hits": ..., "aggregations": { "sales_per_month": { "buckets": [ @@ -72,7 +81,7 @@ And the following may be the response: "key": 1420070400000, "doc_count": 3, "sales": { - "value": 550 + "value": 550.0 } }, { @@ -80,7 +89,7 @@ And the following may be the response: "key": 1422748800000, "doc_count": 2, "sales": { - "value": 60 + "value": 60.0 } }, { @@ -88,15 +97,17 @@ And the following may be the response: "key": 1425168000000, "doc_count": 2, "sales": { - "value": 375 + "value": 375.0 } } ] }, "sum_monthly_sales": { - "value": 985 + "value": 985.0 } } } -------------------------------------------------- - +// TESTRESPONSE[s/"took": 11/"took": $body.took/] +// TESTRESPONSE[s/"_shards": \.\.\./"_shards": $body._shards/] +// TESTRESPONSE[s/"hits": \.\.\./"hits": $body.hits/] From f27a7f4c1a63d068fdd075270d714a24440ee510 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 17 Aug 2016 09:56:00 -0400 Subject: [PATCH 05/13] Test response snippets in kuromoji docs Relates to #18160 --- docs/plugins/analysis-kuromoji.asciidoc | 44 ++++++++++++++----------- 1 file changed, 24 insertions(+), 20 deletions(-) diff --git a/docs/plugins/analysis-kuromoji.asciidoc b/docs/plugins/analysis-kuromoji.asciidoc index 454ba3d2de2..cba44a4d645 100644 --- a/docs/plugins/analysis-kuromoji.asciidoc +++ b/docs/plugins/analysis-kuromoji.asciidoc @@ -189,22 +189,23 @@ The above `analyze` request returns the following: "start_offset" : 0, "end_offset" : 2, "type" : "word", - "position" : 1 + "position" : 0 }, { "token" : "スカイツリー", "start_offset" : 2, "end_offset" : 8, "type" : "word", - "position" : 2 + "position" : 1 } ] } -------------------------------------------------- +// TESTRESPONSE [[analysis-kuromoji-baseform]] ==== `kuromoji_baseform` token filter The `kuromoji_baseform` token filter replaces terms with their -BaseFormAttribute. This acts as a lemmatizer for verbs and adjectives. +BaseFormAttribute. This acts as a lemmatizer for verbs and adjectives. Example: [source,js] -------------------------------------------------- @@ -230,19 +231,21 @@ POST kuromoji_sample/_analyze?analyzer=my_analyzer&text=飲み -------------------------------------------------- // CONSOLE -[source,text] +which responds with: + +[source,js] -------------------------------------------------- -# Result { "tokens" : [ { "token" : "飲む", "start_offset" : 0, "end_offset" : 2, "type" : "word", - "position" : 1 + "position" : 0 } ] } -------------------------------------------------- +// TESTRESPONSE [[analysis-kuromoji-speech]] ==== `kuromoji_part_of_speech` token filter @@ -255,6 +258,8 @@ part-of-speech tags. It accepts the following setting: An array of part-of-speech tags that should be removed. It defaults to the `stoptags.txt` file embedded in the `lucene-analyzer-kuromoji.jar`. +For example: + [source,js] -------------------------------------------------- PUT kuromoji_sample @@ -285,29 +290,30 @@ PUT kuromoji_sample } POST kuromoji_sample/_analyze?analyzer=my_analyzer&text=寿司がおいしいね - -------------------------------------------------- // CONSOLE -[source,text] +Which responds with: + +[source,js] -------------------------------------------------- -# Result { "tokens" : [ { "token" : "寿司", "start_offset" : 0, "end_offset" : 2, "type" : "word", - "position" : 1 + "position" : 0 }, { "token" : "おいしい", "start_offset" : 3, "end_offset" : 7, "type" : "word", - "position" : 3 + "position" : 2 } ] } -------------------------------------------------- +// TESTRESPONSE [[analysis-kuromoji-readingform]] ==== `kuromoji_readingform` token filter @@ -359,7 +365,6 @@ PUT kuromoji_sample POST kuromoji_sample/_analyze?analyzer=katakana_analyzer&text=寿司 <1> POST kuromoji_sample/_analyze?analyzer=romaji_analyzer&text=寿司 <2> - -------------------------------------------------- // CONSOLE @@ -410,7 +415,6 @@ PUT kuromoji_sample POST kuromoji_sample/_analyze?analyzer=my_analyzer&text=コピー <1> POST kuromoji_sample/_analyze?analyzer=my_analyzer&text=サーバー <2> - -------------------------------------------------- // CONSOLE @@ -462,9 +466,8 @@ POST kuromoji_sample/_analyze?analyzer=analyzer_with_ja_stop&text=ストップ The above request returns: -[source,text] +[source,js] -------------------------------------------------- -# Result { "tokens" : [ { "token" : "消える", @@ -480,7 +483,7 @@ The above request returns: ===== `kuromoji_number` token filter The `kuromoji_number` token filter normalizes Japanese numbers (kansūji) -to regular Arabic decimal numbers in half-width characters. +to regular Arabic decimal numbers in half-width characters. For example: [source,js] -------------------------------------------------- @@ -503,20 +506,21 @@ PUT kuromoji_sample } POST kuromoji_sample/_analyze?analyzer=my_analyzer&text=一〇〇〇 - -------------------------------------------------- // CONSOLE -[source,text] +Which results in: + +[source,js] -------------------------------------------------- -# Result { "tokens" : [ { "token" : "1000", "start_offset" : 0, "end_offset" : 4, "type" : "word", - "position" : 1 + "position" : 0 } ] } -------------------------------------------------- +// TESTRESPONSE From 066afcf3c36910210d50850f8880d38cd40e5dbe Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 17 Aug 2016 10:13:06 -0400 Subject: [PATCH 06/13] Add NOTCONSOLE to a few of the docs These are docs for the java client and console doesn't make much sense there. --- docs/java-api/docs/update.asciidoc | 6 ++---- docs/java-api/query-dsl/template-query.asciidoc | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/docs/java-api/docs/update.asciidoc b/docs/java-api/docs/update.asciidoc index 1f63cd7001b..c768cfdb397 100644 --- a/docs/java-api/docs/update.asciidoc +++ b/docs/java-api/docs/update.asciidoc @@ -101,6 +101,7 @@ If the document `index/type/1` already exists, we will have after this operation "gender": "male" <1> } -------------------------------------------------- +// NOTCONSOLE <1> This field is added by the update request If it does not exist, we will have a new document: @@ -112,7 +113,4 @@ If it does not exist, we will have a new document: "gender": "male" } -------------------------------------------------- - - - - +// NOTCONSOLE diff --git a/docs/java-api/query-dsl/template-query.asciidoc b/docs/java-api/query-dsl/template-query.asciidoc index dc31d9087e8..dfba0d63a63 100644 --- a/docs/java-api/query-dsl/template-query.asciidoc +++ b/docs/java-api/query-dsl/template-query.asciidoc @@ -41,6 +41,7 @@ For example, if you have a file named `config/scripts/template_gender.mustache` } } -------------------------------------------------- +// NOTCONSOLE Define your template query: @@ -86,4 +87,3 @@ QueryBuilder qb = new TemplateQueryBuilder( <1> template name <2> template stored in the cluster state <3> parameters - From 39d8f5f123ba9b9d3c5b3b32b97a7ab8343189a3 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 17 Aug 2016 10:25:38 -0400 Subject: [PATCH 07/13] Reindex tests should expect the right failure Reindex intentionally tries to fail the search operation to make sure that the exception flows back. The exception message changed so we should catch the appropriate exception. --- .../elasticsearch/index/reindex/ReindexFailureTests.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java index 45ac5640221..b81be4a1bb2 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ReindexFailureTests.java @@ -108,7 +108,11 @@ public class ReindexFailureTests extends ReindexTestCase { attempt++; } catch (ExecutionException e) { logger.info("Triggered a reindex failure on the {} attempt", attempt); - assertThat(e.getMessage(), either(containsString("all shards failed")).or(containsString("No search context found"))); + assertThat(e.getMessage(), + either(containsString("all shards failed")) + .or(containsString("No search context found")) + .or(containsString("no such index")) + ); return; } } From efd8d837e8387d2145b8d706f271a84d1e86a718 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Wed, 17 Aug 2016 07:58:27 -0700 Subject: [PATCH 08/13] Make disk threshold settings final --- .../routing/allocation/decider/DiskThresholdDecider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index 26f66183efa..2aacac0cdd0 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -67,7 +67,7 @@ public class DiskThresholdDecider extends AllocationDecider { public static final String NAME = "disk_threshold"; - private DiskThresholdSettings diskThresholdSettings; + private final DiskThresholdSettings diskThresholdSettings; @Inject public DiskThresholdDecider(Settings settings, ClusterSettings clusterSettings) { From 312a7d45ba4662d11dbb5981cd3b826c9c924ca0 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Wed, 17 Aug 2016 12:07:04 -0400 Subject: [PATCH 09/13] Wait for task to start in reindex test `RethrottleTests#testReindex` fail in CI: https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+multijob-intake/1274/console I was unable to reproduce it locally but it *looks* like a race to start the task. So I've added a wait for it to start just in case. --- .../java/org/elasticsearch/index/reindex/RethrottleTests.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java index 38f9dbc6d97..7abd1212f6c 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java @@ -57,6 +57,9 @@ public class RethrottleTests extends ReindexTestCase { request.source().setSize(1); // Make sure we use multiple batches ListenableActionFuture responseListener = request.execute(); + // Wait for the task to start + assertBusy(() -> assertEquals(1, client().admin().cluster().prepareListTasks().setActions(actionName).get().getTasks().size())); + // Now rethrottle it so it'll finish ListTasksResponse rethrottleResponse = rethrottle().setActions(actionName).setRequestsPerSecond(Float.POSITIVE_INFINITY).get(); assertThat(rethrottleResponse.getTasks(), hasSize(1)); From d4dec26aa00ced5ead648881301a035272210288 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 17 Aug 2016 12:13:09 -0400 Subject: [PATCH 10/13] Update max local storage nodes docs This commit updates the max local storage nodes docs to reflect that the default is now one after 1f0673c9bd7e26ddf5ce55c6a184db2dbf4c778a. Relates #20029 --- docs/reference/modules/node.asciidoc | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/docs/reference/modules/node.asciidoc b/docs/reference/modules/node.asciidoc index 29379a2d99f..e1cc3e0b866 100644 --- a/docs/reference/modules/node.asciidoc +++ b/docs/reference/modules/node.asciidoc @@ -281,21 +281,16 @@ your data! The RPM and Debian distributions do this for you already. [[max-local-storage-nodes]] === `node.max_local_storage_nodes` -The <> can be shared by multiple nodes, even by nodes -from different clusters. This is very useful for testing failover and -different configurations on your development machine. In production, however, -it is recommended to run only one node of Elasticsearch per server. +The <> can be shared by multiple nodes, even by nodes from different +clusters. This is very useful for testing failover and different configurations on your development +machine. In production, however, it is recommended to run only one node of Elasticsearch per server. -To prevent more than one node from sharing the same data path, add this -setting to the `elasticsearch.yml` config file: +By default, Elasticsearch is configured to prevent more than one node from sharing the same data +path. To allow for more than one node (e.g., on your development machine), use the setting +`node.max_local_storage_nodes` and set this to a positve integer larger than one. -[source,yaml] ------------------------------- -node.max_local_storage_nodes: 1 ------------------------------- - -WARNING: Never run different node types (i.e. master, data) from the -same data directory. This can lead to unexpected data loss. +WARNING: Never run different node types (i.e. master, data) from the same data directory. This can +lead to unexpected data loss. [float] == Other node settings From 6030acb43b5ccc5cf051052b7241f3330c62a820 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 17 Aug 2016 14:58:28 -0600 Subject: [PATCH 11/13] Disallow creating indices starting with '-' or '+' Previously this was possible, which was problematic when issuing a request like `DELETE /-myindex`, which was interpretted as "delete everything except for myindex". Resolves #19800 --- .../cluster/metadata/MetaDataCreateIndexService.java | 6 +++--- .../java/org/elasticsearch/snapshots/RestoreService.java | 2 +- .../cluster/metadata/MetaDataCreateIndexServiceTests.java | 6 ++++-- docs/reference/migration/migrate_5_0/index-apis.asciidoc | 8 ++++++++ 4 files changed, 16 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 04316cbc638..dfb5474ca59 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -144,7 +144,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { this.activeShardsObserver = new ActiveShardsObserver(settings, clusterService, threadPool); } - public void validateIndexName(String index, ClusterState state) { + public static void validateIndexName(String index, ClusterState state) { if (state.routingTable().hasIndex(index)) { throw new IndexAlreadyExistsException(state.routingTable().index(index).getIndex()); } @@ -157,8 +157,8 @@ public class MetaDataCreateIndexService extends AbstractComponent { if (index.contains("#")) { throw new InvalidIndexNameException(index, "must not contain '#'"); } - if (index.charAt(0) == '_') { - throw new InvalidIndexNameException(index, "must not start with '_'"); + if (index.charAt(0) == '_' || index.charAt(0) == '-' || index.charAt(0) == '+') { + throw new InvalidIndexNameException(index, "must not start with '_', '-', or '+'"); } if (!index.toLowerCase(Locale.ROOT).equals(index)) { throw new InvalidIndexNameException(index, "must be lowercase"); diff --git a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 7ab579aa455..16700f3221a 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -257,7 +257,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis if (currentIndexMetaData == null) { // Index doesn't exist - create it and start recovery // Make sure that the index we are about to create has a validate name - createIndexService.validateIndexName(renamedIndexName, currentState); + MetaDataCreateIndexService.validateIndexName(renamedIndexName, currentState); createIndexService.validateIndexSettings(renamedIndexName, snapshotIndexMetaData.getSettings()); IndexMetaData.Builder indexMdBuilder = IndexMetaData.builder(snapshotIndexMetaData).state(IndexMetaData.State.OPEN).index(renamedIndexName); indexMdBuilder.settings(Settings.builder().put(snapshotIndexMetaData.getSettings()).put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID())); diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java index f1f20511fcc..adf34859d4e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java @@ -191,7 +191,9 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { validateIndexName("index#name", "must not contain '#'"); - validateIndexName("_indexname", "must not start with '_'"); + validateIndexName("_indexname", "must not start with '_', '-', or '+'"); + validateIndexName("-indexname", "must not start with '_', '-', or '+'"); + validateIndexName("+indexname", "must not start with '_', '-', or '+'"); validateIndexName("INDEXNAME", "must be lowercase"); @@ -201,7 +203,7 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { private void validateIndexName(String indexName, String errorMessage) { InvalidIndexNameException e = expectThrows(InvalidIndexNameException.class, - () -> getCreateIndexService().validateIndexName(indexName, ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING + () -> MetaDataCreateIndexService.validateIndexName(indexName, ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING .getDefault(Settings.EMPTY)).build())); assertThat(e.getMessage(), endsWith(errorMessage)); } diff --git a/docs/reference/migration/migrate_5_0/index-apis.asciidoc b/docs/reference/migration/migrate_5_0/index-apis.asciidoc index 599b1681ab5..612572577c9 100644 --- a/docs/reference/migration/migrate_5_0/index-apis.asciidoc +++ b/docs/reference/migration/migrate_5_0/index-apis.asciidoc @@ -49,3 +49,11 @@ CPU usage can be obtained from `OsStats.Cpu#getPercent`. Suggest stats exposed through `suggest` in indices stats has been merged with `search` stats. `suggest` stats is exposed as part of `search` stats. + +==== Creating indices starting with '-' or '+' + +Elasticsearch no longer allows indices to be created started with '-' or '+', so +that the multi-index matching and expansion is not confused. It was previously +possible (but a really bad idea) to create indices starting with a hyphen or +plus sign. Any index already existing with these preceding characters will +continue to work normally. From a7a7123d74948d2c932db4723047a5409c3d2812 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Wed, 17 Aug 2016 16:47:03 +0200 Subject: [PATCH 12/13] Simplify inclusion in `_all`. #20028 Currently, when you set `include_in_all` on an object, it will propagate the information to its sub mappers immediately. This is annoying because this is done using a different mechanism than regular mapping updates. This PR changes object fields to propagate the information at document parsing time rather than when `include_an_all` is updated. While moving this cost to document parsing time rather than mapping update time is probably a bad trade-off, I am confident that this cost is very low and think this new way makes things simpler. --- .../index/mapper/AllFieldMapper.java | 22 ------ .../index/mapper/DateFieldMapper.java | 43 ++--------- .../index/mapper/DocumentParser.java | 6 ++ .../index/mapper/FieldMapper.java | 10 +-- .../index/mapper/IpFieldMapper.java | 42 ++--------- .../index/mapper/KeywordFieldMapper.java | 43 ++--------- .../index/mapper/LegacyByteFieldMapper.java | 9 +-- .../index/mapper/LegacyDateFieldMapper.java | 10 +-- .../index/mapper/LegacyDoubleFieldMapper.java | 9 +-- .../index/mapper/LegacyFloatFieldMapper.java | 9 +-- .../mapper/LegacyIntegerFieldMapper.java | 9 +-- .../index/mapper/LegacyIpFieldMapper.java | 9 +-- .../index/mapper/LegacyLongFieldMapper.java | 10 +-- .../index/mapper/LegacyNumberFieldMapper.java | 40 +--------- .../index/mapper/LegacyShortFieldMapper.java | 10 +-- .../mapper/LegacyTokenCountFieldMapper.java | 9 +-- .../index/mapper/NumberFieldMapper.java | 43 ++--------- .../index/mapper/ObjectMapper.java | 73 ++++--------------- .../index/mapper/ParseContext.java | 25 ++++++- .../index/mapper/RootObjectMapper.java | 9 ++- .../index/mapper/ScaledFloatFieldMapper.java | 43 ++--------- .../index/mapper/StringFieldMapper.java | 43 ++--------- .../index/mapper/TextFieldMapper.java | 43 ++--------- .../index/mapper/DocumentParserTests.java | 50 +++++++++++++ 24 files changed, 173 insertions(+), 446 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java index 05d94ff919d..f52123784d1 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/AllFieldMapper.java @@ -47,28 +47,6 @@ import static org.elasticsearch.index.mapper.TypeParsers.parseTextField; */ public class AllFieldMapper extends MetadataFieldMapper { - public interface IncludeInAll { - - /** - * If {@code includeInAll} is not null then return a copy of this mapper - * that will include values in the _all field according to {@code includeInAll}. - */ - Mapper includeInAll(Boolean includeInAll); - - /** - * If {@code includeInAll} is not null and not set on this mapper yet, then - * return a copy of this mapper that will include values in the _all field - * according to {@code includeInAll}. - */ - Mapper includeInAllIfNotSet(Boolean includeInAll); - - /** - * If {@code includeInAll} was already set on this mapper then return a copy - * of this mapper that has {@code includeInAll} not set. - */ - Mapper unsetIncludeInAll(); - } - public static final String NAME = "_all"; public static final String CONTENT_TYPE = "_all"; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java index f3d1fbd53f3..717f0361552 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java @@ -37,7 +37,6 @@ import org.elasticsearch.common.joda.DateMathParser; import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.util.LocaleUtils; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.fielddata.IndexFieldData; @@ -60,7 +59,7 @@ import java.util.concurrent.Callable; import static org.elasticsearch.index.mapper.TypeParsers.parseDateTimeFormatter; /** A {@link FieldMapper} for ip addresses. */ -public class DateFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public class DateFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "date"; public static final FormatDateTimeFormatter DEFAULT_DATE_TIME_FORMATTER = Joda.forPattern( @@ -119,9 +118,8 @@ public class DateFieldMapper extends FieldMapper implements AllFieldMapper.Inclu @Override public DateFieldMapper build(BuilderContext context) { setupFieldType(context); - DateFieldMapper fieldMapper = new DateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (DateFieldMapper) fieldMapper.includeInAll(includeInAll); + return new DateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } } @@ -473,11 +471,13 @@ public class DateFieldMapper extends FieldMapper implements AllFieldMapper.Inclu MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, + Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); this.ignoreMalformed = ignoreMalformed; + this.includeInAll = includeInAll; } @Override @@ -495,39 +495,6 @@ public class DateFieldMapper extends FieldMapper implements AllFieldMapper.Inclu return (DateFieldMapper) super.clone(); } - @Override - public Mapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - DateFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - DateFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper unsetIncludeInAll() { - if (includeInAll != null) { - DateFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { String dateAsString; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 80d59ec39f6..60ca0c22cca 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -343,6 +343,12 @@ final class DocumentParser { context = nestedContext(context, mapper); } + // update the default value of include_in_all if necessary + Boolean includeInAll = mapper.includeInAll(); + if (includeInAll != null) { + context = context.setIncludeInAllDefault(includeInAll); + } + // if we are at the end of the previous object, advance if (token == XContentParser.Token.END_OBJECT) { token = parser.nextToken(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java index e3f88b2231a..ee54a473998 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/FieldMapper.java @@ -537,11 +537,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable { ImmutableOpenMap.Builder builder = new ImmutableOpenMap.Builder<>(); // we disable the all in multi-field mappers for (ObjectObjectCursor cursor : mappers) { - FieldMapper mapper = cursor.value; - if (mapper instanceof AllFieldMapper.IncludeInAll) { - mapper = (FieldMapper) ((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll(); - } - builder.put(cursor.key, mapper); + builder.put(cursor.key, cursor.value); } this.mappers = builder.build(); } @@ -568,10 +564,6 @@ public abstract class FieldMapper extends Mapper implements Cloneable { FieldMapper mergeWithMapper = cursor.value; FieldMapper mergeIntoMapper = mappers.get(mergeWithMapper.simpleName()); if (mergeIntoMapper == null) { - // we disable the all in multi-field mappers - if (mergeWithMapper instanceof AllFieldMapper.IncludeInAll) { - mergeWithMapper = (FieldMapper) ((AllFieldMapper.IncludeInAll) mergeWithMapper).unsetIncludeInAll(); - } newMappersBuilder.put(mergeWithMapper.simpleName(), mergeWithMapper); } else { FieldMapper merged = mergeIntoMapper.merge(mergeWithMapper, false); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java index c5e83657608..69a8e06f859 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java @@ -50,7 +50,7 @@ import java.util.List; import java.util.Map; /** A {@link FieldMapper} for ip addresses. */ -public class IpFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public class IpFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "ip"; @@ -81,9 +81,8 @@ public class IpFieldMapper extends FieldMapper implements AllFieldMapper.Include @Override public IpFieldMapper build(BuilderContext context) { setupFieldType(context); - IpFieldMapper fieldMapper = new IpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (IpFieldMapper) fieldMapper.includeInAll(includeInAll); + return new IpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } } @@ -261,11 +260,13 @@ public class IpFieldMapper extends FieldMapper implements AllFieldMapper.Include MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, + Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); this.ignoreMalformed = ignoreMalformed; + this.includeInAll = includeInAll; } @Override @@ -283,39 +284,6 @@ public class IpFieldMapper extends FieldMapper implements AllFieldMapper.Include return (IpFieldMapper) super.clone(); } - @Override - public Mapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - IpFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - IpFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper unsetIncludeInAll() { - if (includeInAll != null) { - IpFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { Object addressAsObject; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java index 1874c86ac77..56c55e1475b 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java @@ -41,7 +41,7 @@ import static org.elasticsearch.index.mapper.TypeParsers.parseField; /** * A field mapper for keywords. This mapper accepts strings and indexes them as-is. */ -public final class KeywordFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public final class KeywordFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "keyword"; @@ -94,10 +94,9 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap @Override public KeywordFieldMapper build(BuilderContext context) { setupFieldType(context); - KeywordFieldMapper fieldMapper = new KeywordFieldMapper( - name, fieldType, defaultFieldType, ignoreAbove, + return new KeywordFieldMapper( + name, fieldType, defaultFieldType, ignoreAbove, includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return fieldMapper.includeInAll(includeInAll); } } @@ -177,10 +176,11 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap private int ignoreAbove; protected KeywordFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - int ignoreAbove, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + int ignoreAbove, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); assert fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) <= 0; this.ignoreAbove = ignoreAbove; + this.includeInAll = includeInAll; } /** Values that have more chars than the return value of this method will @@ -200,39 +200,6 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap return includeInAll; } - @Override - public KeywordFieldMapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - KeywordFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public KeywordFieldMapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - KeywordFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public KeywordFieldMapper unsetIncludeInAll() { - if (includeInAll != null) { - KeywordFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { final String value; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyByteFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyByteFieldMapper.java index a7f8f85f8f5..2c63806ebbe 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyByteFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyByteFieldMapper.java @@ -75,9 +75,8 @@ public class LegacyByteFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyByteFieldMapper fieldMapper = new LegacyByteFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyByteFieldMapper) fieldMapper.includeInAll(includeInAll); + return new LegacyByteFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } @Override @@ -176,9 +175,9 @@ public class LegacyByteFieldMapper extends LegacyNumberFieldMapper { } protected LegacyByteFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyDateFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyDateFieldMapper.java index 58fbe2895dd..29689d06dff 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyDateFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyDateFieldMapper.java @@ -36,7 +36,6 @@ import org.elasticsearch.common.joda.DateMathParser; import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.util.LocaleUtils; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -116,9 +115,8 @@ public class LegacyDateFieldMapper extends LegacyNumberFieldMapper { } setupFieldType(context); fieldType.setNullValue(nullValue); - LegacyDateFieldMapper fieldMapper = new LegacyDateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyDateFieldMapper) fieldMapper.includeInAll(includeInAll); + return new LegacyDateFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } @Override @@ -467,8 +465,8 @@ public class LegacyDateFieldMapper extends LegacyNumberFieldMapper { } protected LegacyDateFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed,Explicit coerce, - Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyDoubleFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyDoubleFieldMapper.java index 0e727161191..07e459e8ea9 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyDoubleFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyDoubleFieldMapper.java @@ -78,9 +78,8 @@ public class LegacyDoubleFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyDoubleFieldMapper fieldMapper = new LegacyDoubleFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), - context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyDoubleFieldMapper) fieldMapper.includeInAll(includeInAll); + return new LegacyDoubleFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), + includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } @Override @@ -187,8 +186,8 @@ public class LegacyDoubleFieldMapper extends LegacyNumberFieldMapper { } protected LegacyDoubleFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, - Explicit coerce, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyFloatFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyFloatFieldMapper.java index 831ac973dee..3fbc639ea67 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyFloatFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyFloatFieldMapper.java @@ -77,9 +77,8 @@ public class LegacyFloatFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyFloatFieldMapper fieldMapper = new LegacyFloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), - context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyFloatFieldMapper) fieldMapper.includeInAll(includeInAll); + return new LegacyFloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), coerce(context), + includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } @Override @@ -171,9 +170,9 @@ public class LegacyFloatFieldMapper extends LegacyNumberFieldMapper { } protected LegacyFloatFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyIntegerFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyIntegerFieldMapper.java index 6a840813abe..65b9b65eaf9 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyIntegerFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyIntegerFieldMapper.java @@ -81,10 +81,9 @@ public class LegacyIntegerFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyIntegerFieldMapper fieldMapper = new LegacyIntegerFieldMapper(name, fieldType, defaultFieldType, - ignoreMalformed(context), coerce(context), + return new LegacyIntegerFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyIntegerFieldMapper) fieldMapper.includeInAll(includeInAll); } @Override protected int maxPrecisionStep() { @@ -175,9 +174,9 @@ public class LegacyIntegerFieldMapper extends LegacyNumberFieldMapper { } protected LegacyIntegerFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyIpFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyIpFieldMapper.java index 4c76e3ec82a..699124a4c05 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyIpFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyIpFieldMapper.java @@ -117,9 +117,8 @@ public class LegacyIpFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyIpFieldMapper fieldMapper = new LegacyIpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyIpFieldMapper) fieldMapper.includeInAll(includeInAll); + return new LegacyIpFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } @Override @@ -267,9 +266,9 @@ public class LegacyIpFieldMapper extends LegacyNumberFieldMapper { } protected LegacyIpFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } private static long parseValue(Object value) { diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyLongFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyLongFieldMapper.java index b1e156e263e..4661d1cd365 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyLongFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyLongFieldMapper.java @@ -81,9 +81,9 @@ public class LegacyLongFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyLongFieldMapper fieldMapper = new LegacyLongFieldMapper(name, fieldType, defaultFieldType, - ignoreMalformed(context), coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyLongFieldMapper) fieldMapper.includeInAll(includeInAll); + return new LegacyLongFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), includeInAll, context.indexSettings(), + multiFieldsBuilder.build(this, context), copyTo); } @Override @@ -175,9 +175,9 @@ public class LegacyLongFieldMapper extends LegacyNumberFieldMapper { } protected LegacyLongFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyNumberFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyNumberFieldMapper.java index b1d3ead8464..f377883aa24 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyNumberFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyNumberFieldMapper.java @@ -40,7 +40,7 @@ import org.joda.time.DateTimeZone; /** * */ -public abstract class LegacyNumberFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public abstract class LegacyNumberFieldMapper extends FieldMapper { // this is private since it has a different default private static final Setting COERCE_SETTING = Setting.boolSetting("index.mapping.coerce", true, Property.IndexScope); @@ -158,12 +158,13 @@ public abstract class LegacyNumberFieldMapper extends FieldMapper implements All protected Explicit coerce; protected LegacyNumberFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, Settings indexSettings, - MultiFields multiFields, CopyTo copyTo) { + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, + Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); assert fieldType.tokenized() == false; this.ignoreMalformed = ignoreMalformed; this.coerce = coerce; + this.includeInAll = includeInAll; } @Override @@ -171,39 +172,6 @@ public abstract class LegacyNumberFieldMapper extends FieldMapper implements All return (LegacyNumberFieldMapper) super.clone(); } - @Override - public Mapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - LegacyNumberFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - LegacyNumberFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper unsetIncludeInAll() { - if (includeInAll != null) { - LegacyNumberFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { RuntimeException e = null; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyShortFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyShortFieldMapper.java index 39e00801789..b42ec620aea 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyShortFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyShortFieldMapper.java @@ -34,7 +34,6 @@ import org.elasticsearch.Version; import org.elasticsearch.action.fieldstats.FieldStats; import org.elasticsearch.common.Explicit; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.fielddata.IndexFieldData; @@ -78,10 +77,9 @@ public class LegacyShortFieldMapper extends LegacyNumberFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyShortFieldMapper fieldMapper = new LegacyShortFieldMapper(name, fieldType, defaultFieldType, - ignoreMalformed(context), coerce(context), + return new LegacyShortFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (LegacyShortFieldMapper) fieldMapper.includeInAll(includeInAll); } @Override @@ -181,9 +179,9 @@ public class LegacyShortFieldMapper extends LegacyNumberFieldMapper { } protected LegacyShortFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - Explicit ignoreMalformed, Explicit coerce, + Explicit ignoreMalformed, Explicit coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java index 48244ffb377..fd9f6632fe0 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/LegacyTokenCountFieldMapper.java @@ -73,10 +73,9 @@ public class LegacyTokenCountFieldMapper extends LegacyIntegerFieldMapper { throw new IllegalStateException("Cannot use legacy numeric types after 5.0"); } setupFieldType(context); - LegacyTokenCountFieldMapper fieldMapper = new LegacyTokenCountFieldMapper(name, fieldType, defaultFieldType, - ignoreMalformed(context), coerce(context), context.indexSettings(), + return new LegacyTokenCountFieldMapper(name, fieldType, defaultFieldType, + ignoreMalformed(context), coerce(context), includeInAll, context.indexSettings(), analyzer, multiFieldsBuilder.build(this, context), copyTo); - return (LegacyTokenCountFieldMapper) fieldMapper.includeInAll(includeInAll); } @Override @@ -117,8 +116,8 @@ public class LegacyTokenCountFieldMapper extends LegacyIntegerFieldMapper { private NamedAnalyzer analyzer; protected LegacyTokenCountFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Explicit ignoreMalformed, - Explicit coerce, Settings indexSettings, NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) { - super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo); + Explicit coerce, Boolean includeInAll, Settings indexSettings, NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) { + super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo); this.analyzer = analyzer; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java index 6eddf07b979..be6ed23c508 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java @@ -60,7 +60,7 @@ import java.util.Map; import java.util.Objects; /** A {@link FieldMapper} for numeric types: byte, short, int, long, float and double. */ -public class NumberFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public class NumberFieldMapper extends FieldMapper { // this is private since it has a different default static final Setting COERCE_SETTING = @@ -114,10 +114,8 @@ public class NumberFieldMapper extends FieldMapper implements AllFieldMapper.Inc @Override public NumberFieldMapper build(BuilderContext context) { setupFieldType(context); - NumberFieldMapper fieldMapper = - new NumberFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (NumberFieldMapper) fieldMapper.includeInAll(includeInAll); + return new NumberFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } } @@ -871,12 +869,14 @@ public class NumberFieldMapper extends FieldMapper implements AllFieldMapper.Inc MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, + Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); this.ignoreMalformed = ignoreMalformed; this.coerce = coerce; + this.includeInAll = includeInAll; } @Override @@ -894,39 +894,6 @@ public class NumberFieldMapper extends FieldMapper implements AllFieldMapper.Inc return (NumberFieldMapper) super.clone(); } - @Override - public Mapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - NumberFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - NumberFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper unsetIncludeInAll() { - if (includeInAll != null) { - NumberFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { XContentParser parser = context.parser(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java index 9d6dcafc0c0..f3941f9b25f 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java @@ -25,7 +25,6 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.CopyOnWriteHashMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; @@ -47,7 +46,7 @@ import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenien /** * */ -public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll, Cloneable { +public class ObjectMapper extends Mapper implements Cloneable { public static final String CONTENT_TYPE = "object"; public static final String NESTED_CONTENT_TYPE = "nested"; @@ -154,14 +153,15 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll, } context.path().remove(); - ObjectMapper objectMapper = createMapper(name, context.path().pathAsText(name), enabled, nested, dynamic, mappers, context.indexSettings()); - objectMapper = objectMapper.includeInAllIfNotSet(includeInAll); + ObjectMapper objectMapper = createMapper(name, context.path().pathAsText(name), enabled, nested, dynamic, + includeInAll, mappers, context.indexSettings()); return (Y) objectMapper; } - protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map mappers, @Nullable Settings settings) { - return new ObjectMapper(name, fullPath, enabled, nested, dynamic, mappers); + protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, + Boolean includeInAll, Map mappers, @Nullable Settings settings) { + return new ObjectMapper(name, fullPath, enabled, nested, dynamic, includeInAll, mappers); } } @@ -322,12 +322,14 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll, private volatile CopyOnWriteHashMap mappers; - ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map mappers) { + ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, + Boolean includeInAll, Map mappers) { super(name); this.fullPath = fullPath; this.enabled = enabled; this.nested = nested; this.dynamic = dynamic; + this.includeInAll = includeInAll; if (mappers == null) { this.mappers = new CopyOnWriteHashMap<>(); } else { @@ -373,68 +375,19 @@ public class ObjectMapper extends Mapper implements AllFieldMapper.IncludeInAll, return mappers.get(field); } - @Override - public ObjectMapper includeInAll(Boolean includeInAll) { - if (includeInAll == null) { - return this; - } - - ObjectMapper clone = clone(); - clone.includeInAll = includeInAll; - // when called from outside, apply this on all the inner mappers - for (Mapper mapper : clone.mappers.values()) { - if (mapper instanceof AllFieldMapper.IncludeInAll) { - clone.putMapper(((AllFieldMapper.IncludeInAll) mapper).includeInAll(includeInAll)); - } - } - return clone; - } - - @Override - public ObjectMapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll == null || this.includeInAll != null) { - return this; - } - - ObjectMapper clone = clone(); - clone.includeInAll = includeInAll; - // when called from outside, apply this on all the inner mappers - for (Mapper mapper : clone.mappers.values()) { - if (mapper instanceof AllFieldMapper.IncludeInAll) { - clone.putMapper(((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll)); - } - } - return clone; - } - - @Override - public ObjectMapper unsetIncludeInAll() { - if (includeInAll == null) { - return this; - } - ObjectMapper clone = clone(); - clone.includeInAll = null; - // when called from outside, apply this on all the inner mappers - for (Mapper mapper : mappers.values()) { - if (mapper instanceof AllFieldMapper.IncludeInAll) { - clone.putMapper(((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll()); - } - } - return clone; - } - public Nested nested() { return this.nested; } + public Boolean includeInAll() { + return includeInAll; + } + public Query nestedTypeFilter() { return this.nestedTypeFilter; } protected void putMapper(Mapper mapper) { - if (mapper instanceof AllFieldMapper.IncludeInAll) { - mapper = ((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll); - } mappers = mappers.copyAndPut(mapper.simpleName(), mapper); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index 477f88cc53c..8a2aca97e68 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -423,6 +423,22 @@ public abstract class ParseContext { public abstract DocumentMapperParser docMapperParser(); + /** Return a view of this {@link ParseContext} that changes the return + * value of {@link #getIncludeInAllDefault()}. */ + public final ParseContext setIncludeInAllDefault(boolean includeInAll) { + return new FilterParseContext(this) { + @Override + public Boolean getIncludeInAllDefault() { + return includeInAll; + } + }; + } + + /** Whether field values should be added to the _all field by default. */ + public Boolean getIncludeInAllDefault() { + return null; + } + /** * Return a new context that will be within a copy-to operation. */ @@ -526,7 +542,7 @@ public abstract class ParseContext { * is false. If its enabled, then will return true only if the specific flag is null or * its actual value (so, if not set, defaults to "true") and the field is indexed. */ - private boolean includeInAll(Boolean specificIncludeInAll, boolean indexed) { + private boolean includeInAll(Boolean includeInAll, boolean indexed) { if (isWithinCopyTo()) { return false; } @@ -536,11 +552,14 @@ public abstract class ParseContext { if (!docMapper().allFieldMapper().enabled()) { return false; } + if (includeInAll == null) { + includeInAll = getIncludeInAllDefault(); + } // not explicitly set - if (specificIncludeInAll == null) { + if (includeInAll == null) { return indexed; } - return specificIncludeInAll; + return includeInAll; } public abstract AllEntries allEntries(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java index f419d525e1a..eed1ac0e8d1 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/RootObjectMapper.java @@ -100,7 +100,8 @@ public class RootObjectMapper extends ObjectMapper { @Override - protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map mappers, @Nullable Settings settings) { + protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, + Boolean includeInAll, Map mappers, @Nullable Settings settings) { assert !nested.isNested(); FormatDateTimeFormatter[] dates = null; if (dynamicDateTimeFormatters == null) { @@ -111,7 +112,7 @@ public class RootObjectMapper extends ObjectMapper { } else { dates = dynamicDateTimeFormatters.toArray(new FormatDateTimeFormatter[dynamicDateTimeFormatters.size()]); } - return new RootObjectMapper(name, enabled, dynamic, mappers, + return new RootObjectMapper(name, enabled, dynamic, includeInAll, mappers, dates, dynamicTemplates.toArray(new DynamicTemplate[dynamicTemplates.size()]), dateDetection, numericDetection); @@ -207,9 +208,9 @@ public class RootObjectMapper extends ObjectMapper { private volatile DynamicTemplate dynamicTemplates[]; - RootObjectMapper(String name, boolean enabled, Dynamic dynamic, Map mappers, + RootObjectMapper(String name, boolean enabled, Dynamic dynamic, Boolean includeInAll, Map mappers, FormatDateTimeFormatter[] dynamicDateTimeFormatters, DynamicTemplate dynamicTemplates[], boolean dateDetection, boolean numericDetection) { - super(name, name, enabled, Nested.NO, dynamic, mappers); + super(name, name, enabled, Nested.NO, dynamic, includeInAll, mappers); this.dynamicTemplates = dynamicTemplates; this.dynamicDateTimeFormatters = dynamicDateTimeFormatters; this.dateDetection = dateDetection; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java index 8c50e868020..b7276e4e114 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java @@ -65,7 +65,7 @@ import java.util.Map; /** A {@link FieldMapper} for scaled floats. Values are internally multiplied * by a scaling factor and rounded to the closest long. */ -public class ScaledFloatFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public class ScaledFloatFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "scaled_float"; // use the same default as numbers @@ -124,10 +124,8 @@ public class ScaledFloatFieldMapper extends FieldMapper implements AllFieldMappe throw new IllegalArgumentException("Field [" + name + "] misses required parameter [scaling_factor]"); } setupFieldType(context); - ScaledFloatFieldMapper fieldMapper = - new ScaledFloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), - coerce(context), context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return (ScaledFloatFieldMapper) fieldMapper.includeInAll(includeInAll); + return new ScaledFloatFieldMapper(name, fieldType, defaultFieldType, ignoreMalformed(context), + coerce(context), includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); } } @@ -336,6 +334,7 @@ public class ScaledFloatFieldMapper extends FieldMapper implements AllFieldMappe MappedFieldType defaultFieldType, Explicit ignoreMalformed, Explicit coerce, + Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { @@ -346,6 +345,7 @@ public class ScaledFloatFieldMapper extends FieldMapper implements AllFieldMappe } this.ignoreMalformed = ignoreMalformed; this.coerce = coerce; + this.includeInAll = includeInAll; } @Override @@ -363,39 +363,6 @@ public class ScaledFloatFieldMapper extends FieldMapper implements AllFieldMappe return (ScaledFloatFieldMapper) super.clone(); } - @Override - public Mapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - ScaledFloatFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - ScaledFloatFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public Mapper unsetIncludeInAll() { - if (includeInAll != null) { - ScaledFloatFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected void parseCreateField(ParseContext context, List fields) throws IOException { XContentParser parser = context.parser(); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java index 887a40fe70b..5dc6efb28f7 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java @@ -49,7 +49,7 @@ import java.util.Set; import static org.apache.lucene.index.IndexOptions.NONE; import static org.elasticsearch.index.mapper.TypeParsers.parseTextField; -public class StringFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public class StringFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "string"; private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1; @@ -177,10 +177,9 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc fieldType.setSearchQuoteAnalyzer(new NamedAnalyzer(fieldType.searchQuoteAnalyzer(), positionIncrementGap)); } setupFieldType(context); - StringFieldMapper fieldMapper = new StringFieldMapper( - name, fieldType(), defaultFieldType, positionIncrementGap, ignoreAbove, + return new StringFieldMapper( + name, fieldType(), defaultFieldType, positionIncrementGap, ignoreAbove, includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return fieldMapper.includeInAll(includeInAll); } } @@ -488,7 +487,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc private int ignoreAbove; protected StringFieldMapper(String simpleName, StringFieldType fieldType, MappedFieldType defaultFieldType, - int positionIncrementGap, int ignoreAbove, + int positionIncrementGap, int ignoreAbove, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); if (Version.indexCreated(indexSettings).onOrAfter(Version.V_5_0_0_alpha1)) { @@ -506,6 +505,7 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc } this.positionIncrementGap = positionIncrementGap; this.ignoreAbove = ignoreAbove; + this.includeInAll = includeInAll; } @Override @@ -513,39 +513,6 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc return (StringFieldMapper) super.clone(); } - @Override - public StringFieldMapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - StringFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public StringFieldMapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - StringFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public StringFieldMapper unsetIncludeInAll() { - if (includeInAll != null) { - StringFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - @Override protected boolean customBoost() { return true; diff --git a/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java index 987586db721..6d677d50786 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java @@ -38,7 +38,7 @@ import java.util.Objects; import static org.elasticsearch.index.mapper.TypeParsers.parseTextField; /** A {@link FieldMapper} for full-text fields. */ -public class TextFieldMapper extends FieldMapper implements AllFieldMapper.IncludeInAll { +public class TextFieldMapper extends FieldMapper { public static final String CONTENT_TYPE = "text"; private static final int POSITION_INCREMENT_GAP_USE_ANALYZER = -1; @@ -120,10 +120,9 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu fieldType.setSearchQuoteAnalyzer(new NamedAnalyzer(fieldType.searchQuoteAnalyzer(), positionIncrementGap)); } setupFieldType(context); - TextFieldMapper fieldMapper = new TextFieldMapper( - name, fieldType, defaultFieldType, positionIncrementGap, + return new TextFieldMapper( + name, fieldType, defaultFieldType, positionIncrementGap, includeInAll, context.indexSettings(), multiFieldsBuilder.build(this, context), copyTo); - return fieldMapper.includeInAll(includeInAll); } } @@ -297,7 +296,7 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu private int positionIncrementGap; protected TextFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, - int positionIncrementGap, + int positionIncrementGap, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); assert fieldType.tokenized(); @@ -306,6 +305,7 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu throw new IllegalArgumentException("Cannot enable fielddata on a [text] field that is not indexed: [" + name() + "]"); } this.positionIncrementGap = positionIncrementGap; + this.includeInAll = includeInAll; } @Override @@ -318,39 +318,6 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu return includeInAll; } - @Override - public TextFieldMapper includeInAll(Boolean includeInAll) { - if (includeInAll != null) { - TextFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public TextFieldMapper includeInAllIfNotSet(Boolean includeInAll) { - if (includeInAll != null && this.includeInAll == null) { - TextFieldMapper clone = clone(); - clone.includeInAll = includeInAll; - return clone; - } else { - return this; - } - } - - @Override - public TextFieldMapper unsetIncludeInAll() { - if (includeInAll != null) { - TextFieldMapper clone = clone(); - clone.includeInAll = null; - return clone; - } else { - return this; - } - } - public int getPositionIncrementGap() { return this.positionIncrementGap; } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java index 3320b90a991..d650692e8ea 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DocumentParserTests.java @@ -19,14 +19,20 @@ package org.elasticsearch.index.mapper; +import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import org.apache.lucene.index.IndexableField; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; +import org.elasticsearch.common.lucene.all.AllField; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.IndexService; @@ -1154,4 +1160,48 @@ public class DocumentParserTests extends ESSingleNodeTestCase { assertThat(doc.rootDoc().get("type.test2"), equalTo("value2")); assertThat(doc.rootDoc().get("type.inner.inner_field"), equalTo("inner_value")); } + + public void testIncludeInAllPropagation() throws IOException { + String defaultMapping = XContentFactory.jsonBuilder().startObject() + .startObject("type") + .field("dynamic", "strict") + .startObject("properties") + .startObject("a") + .field("type", "keyword") + .endObject() + .startObject("o") + .field("include_in_all", false) + .startObject("properties") + .startObject("a") + .field("type", "keyword") + .endObject() + .startObject("o") + .field("include_in_all", true) + .startObject("properties") + .startObject("a") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject().endObject().string(); + DocumentMapper defaultMapper = createIndex("test").mapperService().documentMapperParser().parse("type", new CompressedXContent(defaultMapping)); + ParsedDocument doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder() + .startObject() + .field("a", "b") + .startObject("o") + .field("a", "c") + .startObject("o") + .field("a", "d") + .endObject() + .endObject() + .endObject().bytes()); + Set values = new HashSet<>(); + for (IndexableField f : doc.rootDoc().getFields("_all")) { + values.add(f.stringValue()); + } + assertEquals(new HashSet<>(Arrays.asList("b", "d")), values); + } } From d805266d94adcf3643b77194a7895de6200f2914 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 18 Aug 2016 08:17:28 +0200 Subject: [PATCH 13/13] Revert "Save one utf8 conversion in KeywordFieldMapper. #19867" This reverts commit c44679d952d96a0f660cb698108f0a7101c23b55. Conflicts: core/src/main/java/org/elasticsearch/index/mapper/BaseGeoPointFieldMapper.java core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapperLegacy.java core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java --- .../index/get/ShardGetService.java | 5 ++++- .../index/mapper/BaseGeoPointFieldMapper.java | 20 ++++++------------- .../index/mapper/GeoPointFieldMapper.java | 4 ++-- .../mapper/GeoPointFieldMapperLegacy.java | 5 +++-- .../index/mapper/KeywordFieldMapper.java | 16 ++------------- .../index/mapper/DynamicTemplatesTests.java | 13 ++++++------ .../mapper/ExternalFieldMapperTests.java | 3 +-- .../mapper/GeoPointFieldMapperTests.java | 19 +++--------------- .../mapper/GeohashMappingGeoPointTests.java | 3 +-- .../index/mapper/KeywordFieldMapperTests.java | 4 ++-- .../index/mapper/MultiFieldTests.java | 3 +-- 11 files changed, 31 insertions(+), 64 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java b/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java index b3676b984c2..bbe4ffdb9b3 100644 --- a/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java +++ b/core/src/main/java/org/elasticsearch/index/get/ShardGetService.java @@ -368,7 +368,10 @@ public final class ShardGetService extends AbstractIndexShardComponent { } List values = searchLookup.source().extractRawValues(field); - if (values.isEmpty() == false) { + if (!values.isEmpty()) { + for (int i = 0; i < values.size(); i++) { + values.set(i, fieldMapper.fieldType().valueForSearch(values.get(i))); + } value = values; } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/BaseGeoPointFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/BaseGeoPointFieldMapper.java index 99dd39ef2e6..2cdc146b274 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/BaseGeoPointFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/BaseGeoPointFieldMapper.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.document.Field; -import org.apache.lucene.index.IndexOptions; import org.apache.lucene.search.Query; import org.elasticsearch.common.geo.GeoHashUtils; import org.apache.lucene.util.LegacyNumericUtils; @@ -140,7 +139,7 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr public abstract Y build(BuilderContext context, String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper, FieldMapper lonMapper, - FieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo); + KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo); public Y build(Mapper.BuilderContext context) { GeoPointFieldType geoPointFieldType = (GeoPointFieldType)fieldType; @@ -167,17 +166,10 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr } geoPointFieldType.setLatLonEnabled(latMapper.fieldType(), lonMapper.fieldType()); } - FieldMapper geoHashMapper = null; + KeywordFieldMapper geoHashMapper = null; if (enableGeoHash || enableGeoHashPrefix) { // TODO: possible also implicitly enable geohash if geohash precision is set - if (context.indexCreatedVersion().onOrAfter(Version.V_5_0_0_alpha1)) { - geoHashMapper = new KeywordFieldMapper.Builder(Names.GEOHASH) - .index(true).includeInAll(false).store(fieldType.stored()).build(context); - } else { - geoHashMapper = new StringFieldMapper.Builder(Names.GEOHASH) - .tokenized(false).index(true).omitNorms(true).indexOptions(IndexOptions.DOCS) - .includeInAll(false).store(fieldType.stored()).build(context); - } + geoHashMapper = new KeywordFieldMapper.Builder(Names.GEOHASH).index(true).includeInAll(false).store(fieldType.stored()).build(context); geoPointFieldType.setGeoHashEnabled(geoHashMapper.fieldType(), geoHashPrecision, enableGeoHashPrefix); } context.path().remove(); @@ -378,12 +370,12 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr protected FieldMapper lonMapper; - protected FieldMapper geoHashMapper; + protected KeywordFieldMapper geoHashMapper; protected Explicit ignoreMalformed; protected BaseGeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, - FieldMapper latMapper, FieldMapper lonMapper, FieldMapper geoHashMapper, + FieldMapper latMapper, FieldMapper lonMapper, KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, multiFields, copyTo); this.latMapper = latMapper; @@ -554,7 +546,7 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr @Override public FieldMapper updateFieldType(Map fullNameToFieldType) { BaseGeoPointFieldMapper updated = (BaseGeoPointFieldMapper) super.updateFieldType(fullNameToFieldType); - FieldMapper geoUpdated = geoHashMapper == null ? null : geoHashMapper.updateFieldType(fullNameToFieldType); + KeywordFieldMapper geoUpdated = geoHashMapper == null ? null : (KeywordFieldMapper) geoHashMapper.updateFieldType(fullNameToFieldType); FieldMapper latUpdated = latMapper == null ? null : latMapper.updateFieldType(fullNameToFieldType); FieldMapper lonUpdated = lonMapper == null ? null : lonMapper.updateFieldType(fullNameToFieldType); if (updated == this diff --git a/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java index c27ddc1811b..590679cb989 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapper.java @@ -73,7 +73,7 @@ public class GeoPointFieldMapper extends BaseGeoPointFieldMapper { @Override public GeoPointFieldMapper build(BuilderContext context, String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper, - FieldMapper lonMapper, FieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, + FieldMapper lonMapper, KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo) { fieldType.setTokenized(false); if (context.indexCreatedVersion().before(Version.V_2_3_0)) { @@ -104,7 +104,7 @@ public class GeoPointFieldMapper extends BaseGeoPointFieldMapper { public GeoPointFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper, FieldMapper lonMapper, - FieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo) { + KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields, ignoreMalformed, copyTo); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapperLegacy.java b/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapperLegacy.java index 4d09e6f9833..319e0bde7f4 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapperLegacy.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/GeoPointFieldMapperLegacy.java @@ -25,6 +25,7 @@ import org.apache.lucene.document.Field; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.Explicit; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.geo.GeoDistance; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; @@ -100,7 +101,7 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement @Override public GeoPointFieldMapperLegacy build(BuilderContext context, String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper, - FieldMapper lonMapper, FieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, + FieldMapper lonMapper, KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, CopyTo copyTo) { fieldType.setTokenized(false); setupFieldType(context); @@ -258,7 +259,7 @@ public class GeoPointFieldMapperLegacy extends BaseGeoPointFieldMapper implement public GeoPointFieldMapperLegacy(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType, Settings indexSettings, FieldMapper latMapper, FieldMapper lonMapper, - FieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, + KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit ignoreMalformed, Explicit coerce, CopyTo copyTo) { super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields, ignoreMalformed, copyTo); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java index 56c55e1475b..b0468e8f83d 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java @@ -160,16 +160,6 @@ public final class KeywordFieldMapper extends FieldMapper { failIfNoDocValues(); return new DocValuesIndexFieldData.Builder(); } - - @Override - public Object valueForSearch(Object value) { - if (value == null) { - return null; - } - // keywords are internally stored as utf8 bytes - BytesRef binaryValue = (BytesRef) value; - return binaryValue.utf8ToString(); - } } private Boolean includeInAll; @@ -222,14 +212,12 @@ public final class KeywordFieldMapper extends FieldMapper { context.allEntries().addText(fieldType().name(), value, fieldType().boost()); } - // convert to utf8 only once before feeding postings/dv/stored fields - final BytesRef binaryValue = new BytesRef(value); if (fieldType().indexOptions() != IndexOptions.NONE || fieldType().stored()) { - Field field = new Field(fieldType().name(), binaryValue, fieldType()); + Field field = new Field(fieldType().name(), value, fieldType()); fields.add(field); } if (fieldType().hasDocValues()) { - fields.add(new SortedSetDocValuesField(fieldType().name(), binaryValue)); + fields.add(new SortedSetDocValuesField(fieldType().name(), new BytesRef(value))); } } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DynamicTemplatesTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DynamicTemplatesTests.java index 9e4eb14e378..845125849c1 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DynamicTemplatesTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DynamicTemplatesTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; @@ -81,7 +80,7 @@ public class DynamicTemplatesTests extends ESSingleNodeTestCase { IndexableField f = doc.getField("name"); assertThat(f.name(), equalTo("name")); - assertThat(f.binaryValue(), equalTo(new BytesRef("some name"))); + assertThat(f.stringValue(), equalTo("some name")); assertNotSame(IndexOptions.NONE, f.fieldType().indexOptions()); assertThat(f.fieldType().tokenized(), equalTo(false)); @@ -99,7 +98,7 @@ public class DynamicTemplatesTests extends ESSingleNodeTestCase { f = doc.getField("multi1.org"); assertThat(f.name(), equalTo("multi1.org")); - assertThat(f.binaryValue(), equalTo(new BytesRef("multi 1"))); + assertThat(f.stringValue(), equalTo("multi 1")); assertNotSame(IndexOptions.NONE, f.fieldType().indexOptions()); assertThat(f.fieldType().tokenized(), equalTo(false)); @@ -117,7 +116,7 @@ public class DynamicTemplatesTests extends ESSingleNodeTestCase { f = doc.getField("multi2.org"); assertThat(f.name(), equalTo("multi2.org")); - assertThat(f.binaryValue(), equalTo(new BytesRef("multi 2"))); + assertThat(f.stringValue(), equalTo("multi 2")); assertNotSame(IndexOptions.NONE, f.fieldType().indexOptions()); assertThat(f.fieldType().tokenized(), equalTo(false)); @@ -138,7 +137,7 @@ public class DynamicTemplatesTests extends ESSingleNodeTestCase { IndexableField f = doc.getField("name"); assertThat(f.name(), equalTo("name")); - assertThat(f.binaryValue(), equalTo(new BytesRef("some name"))); + assertThat(f.stringValue(), equalTo("some name")); assertNotSame(IndexOptions.NONE, f.fieldType().indexOptions()); assertThat(f.fieldType().tokenized(), equalTo(false)); @@ -156,7 +155,7 @@ public class DynamicTemplatesTests extends ESSingleNodeTestCase { f = doc.getField("multi1.org"); assertThat(f.name(), equalTo("multi1.org")); - assertThat(f.binaryValue(), equalTo(new BytesRef("multi 1"))); + assertThat(f.stringValue(), equalTo("multi 1")); assertNotSame(IndexOptions.NONE, f.fieldType().indexOptions()); assertThat(f.fieldType().tokenized(), equalTo(false)); @@ -174,7 +173,7 @@ public class DynamicTemplatesTests extends ESSingleNodeTestCase { f = doc.getField("multi2.org"); assertThat(f.name(), equalTo("multi2.org")); - assertThat(f.binaryValue(), equalTo(new BytesRef("multi 2"))); + assertThat(f.stringValue(), equalTo("multi 2")); assertNotSame(IndexOptions.NONE, f.fieldType().indexOptions()); assertThat(f.fieldType().tokenized(), equalTo(false)); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java index f1541ed02b9..f523f15820e 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/ExternalFieldMapperTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.spatial.geopoint.document.GeoPointField; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.compress.CompressedXContent; @@ -156,7 +155,7 @@ public class ExternalFieldMapperTests extends ESSingleNodeTestCase { assertThat(doc.rootDoc().getField("field.field").stringValue(), is("foo")); assertThat(doc.rootDoc().getField("field.field.raw"), notNullValue()); - assertThat(doc.rootDoc().getField("field.field.raw").binaryValue(), is(new BytesRef("foo"))); + assertThat(doc.rootDoc().getField("field.field.raw").stringValue(), is("foo")); } public void testExternalValuesWithMultifieldTwoLevels() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java index 9262c6d0d6d..ce30fd2f8b7 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/GeoPointFieldMapperTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.index.IndexableField; import org.apache.lucene.spatial.geopoint.document.GeoPointField; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; @@ -113,11 +112,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { assertThat(doc.rootDoc().getField("point.lat"), notNullValue()); assertThat(doc.rootDoc().getField("point.lon"), notNullValue()); - if (version.onOrAfter(Version.V_5_0_0_alpha1)) { - assertThat(doc.rootDoc().getBinaryValue("point.geohash"), equalTo(new BytesRef(stringEncode(1.3, 1.2)))); - } else { - assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); - } + assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); } public void testLatLonInOneValueWithGeohash() throws Exception { @@ -137,11 +132,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { assertThat(doc.rootDoc().getField("point.lat"), notNullValue()); assertThat(doc.rootDoc().getField("point.lon"), notNullValue()); - if (version.onOrAfter(Version.V_5_0_0_alpha1)) { - assertThat(doc.rootDoc().getBinaryValue("point.geohash"), equalTo(new BytesRef(stringEncode(1.3, 1.2)))); - } else { - assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); - } + assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); } public void testGeoHashIndexValue() throws Exception { @@ -161,11 +152,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { assertThat(doc.rootDoc().getField("point.lat"), notNullValue()); assertThat(doc.rootDoc().getField("point.lon"), notNullValue()); - if (version.onOrAfter(Version.V_5_0_0_alpha1)) { - assertThat(doc.rootDoc().getBinaryValue("point.geohash"), equalTo(new BytesRef(stringEncode(1.3, 1.2)))); - } else { - assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); - } + assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); } public void testGeoHashValue() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/GeohashMappingGeoPointTests.java b/core/src/test/java/org/elasticsearch/index/mapper/GeohashMappingGeoPointTests.java index 05581e79021..b3fd8ae2c39 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/GeohashMappingGeoPointTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/GeohashMappingGeoPointTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.spatial.geopoint.document.GeoPointField; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.compress.CompressedXContent; @@ -119,7 +118,7 @@ public class GeohashMappingGeoPointTests extends ESSingleNodeTestCase { assertThat(doc.rootDoc().getField("point.lat"), nullValue()); assertThat(doc.rootDoc().getField("point.lon"), nullValue()); - assertThat(doc.rootDoc().getBinaryValue("point.geohash"), equalTo(new BytesRef(stringEncode(1.3, 1.2)))); + assertThat(doc.rootDoc().get("point.geohash"), equalTo(stringEncode(1.3, 1.2))); assertThat(doc.rootDoc().get("point"), notNullValue()); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java index 9c4eed15bc9..a32e1702490 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java @@ -78,7 +78,7 @@ public class KeywordFieldMapperTests extends ESSingleNodeTestCase { IndexableField[] fields = doc.rootDoc().getFields("field"); assertEquals(2, fields.length); - assertEquals(new BytesRef("1234"), fields[0].binaryValue()); + assertEquals("1234", fields[0].stringValue()); IndexableFieldType fieldType = fields[0].fieldType(); assertThat(fieldType.omitNorms(), equalTo(true)); assertFalse(fieldType.tokenized()); @@ -163,7 +163,7 @@ public class KeywordFieldMapperTests extends ESSingleNodeTestCase { fields = doc.rootDoc().getFields("field"); assertEquals(2, fields.length); - assertEquals(new BytesRef("uri"), fields[0].binaryValue()); + assertEquals("uri", fields[0].stringValue()); } public void testEnableStore() throws IOException { diff --git a/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldTests.java b/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldTests.java index 4bf1995722b..871bd20844d 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/MultiFieldTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressedXContent; @@ -90,7 +89,7 @@ public class MultiFieldTests extends ESSingleNodeTestCase { f = doc.getField("object1.multi1.string"); assertThat(f.name(), equalTo("object1.multi1.string")); - assertThat(f.binaryValue(), equalTo(new BytesRef("2010-01-01"))); + assertThat(f.stringValue(), equalTo("2010-01-01")); assertThat(docMapper.mappers().getMapper("name"), notNullValue()); assertThat(docMapper.mappers().getMapper("name"), instanceOf(TextFieldMapper.class));