Merge branch 'master' into feature/rank-eval
This commit is contained in:
commit
e58e25f338
|
@ -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);
|
||||
|
|
|
@ -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<IndexMetaData>, Diffable<MetaData>, 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(),
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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<String> 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<String, DiskUsage> 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<String> nodes = usages.keys();
|
||||
for (String node : nodeHasPassedWatermark) {
|
||||
if (nodes.contains(node) == false) {
|
||||
nodeHasPassedWatermark.remove(node);
|
||||
}
|
||||
}
|
||||
|
||||
for (ObjectObjectCursor<String, DiskUsage> 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<Boolean> 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<String> 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<String> 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<Boolean> 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<TimeValue> 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;
|
||||
}
|
||||
}
|
|
@ -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<Boolean> CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING =
|
||||
Setting.boolSetting("cluster.routing.allocation.disk.threshold_enabled", true, Property.Dynamic, Property.NodeScope);
|
||||
public static final Setting<String> 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<String> 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<Boolean> CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING =
|
||||
Setting.boolSetting("cluster.routing.allocation.disk.include_relocations", true,
|
||||
Property.Dynamic, Property.NodeScope);;
|
||||
public static final Setting<TimeValue> 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<String> 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<String, DiskUsage> 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<String> nodes = usages.keys();
|
||||
for (String node : nodeHasPassedWatermark) {
|
||||
if (nodes.contains(node) == false) {
|
||||
nodeHasPassedWatermark.remove(node);
|
||||
}
|
||||
}
|
||||
|
||||
for (ObjectObjectCursor<String, DiskUsage> 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 final 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<String, DiskUsage> usages) {
|
||||
DiskUsage averageUsage(RoutingNode node, ImmutableOpenMap<String, DiskUsage> 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<String, DiskUsage> 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) {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -368,7 +368,10 @@ public final class ShardGetService extends AbstractIndexShardComponent {
|
|||
}
|
||||
|
||||
List<Object> 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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, CopyTo copyTo);
|
||||
KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> 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<Boolean> 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<Boolean> 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<String, MappedFieldType> 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
|
||||
|
|
|
@ -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<Boolean> 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<Field> fields) throws IOException {
|
||||
String dateAsString;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -537,11 +537,7 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
|
|||
ImmutableOpenMap.Builder<String, FieldMapper> builder = new ImmutableOpenMap.Builder<>();
|
||||
// we disable the all in multi-field mappers
|
||||
for (ObjectObjectCursor<String, FieldMapper> 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);
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed,
|
||||
FieldMapper lonMapper, KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> 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<Boolean> ignoreMalformed, CopyTo copyTo) {
|
||||
KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed, CopyTo copyTo) {
|
||||
super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields,
|
||||
ignoreMalformed, copyTo);
|
||||
}
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed,
|
||||
FieldMapper lonMapper, KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> 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<Boolean> ignoreMalformed,
|
||||
KeywordFieldMapper geoHashMapper, MultiFields multiFields, Explicit<Boolean> ignoreMalformed,
|
||||
Explicit<Boolean> coerce, CopyTo copyTo) {
|
||||
super(simpleName, fieldType, defaultFieldType, indexSettings, latMapper, lonMapper, geoHashMapper, multiFields,
|
||||
ignoreMalformed, copyTo);
|
||||
|
|
|
@ -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<Boolean> 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<Field> fields) throws IOException {
|
||||
Object addressAsObject;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -161,26 +160,17 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap
|
|||
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;
|
||||
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 +190,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<Field> fields) throws IOException {
|
||||
final String value;
|
||||
|
@ -255,14 +212,12 @@ public final class KeywordFieldMapper extends FieldMapper implements AllFieldMap
|
|||
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)));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, Explicit<Boolean> coerce,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed,Explicit<Boolean> 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
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed,
|
||||
Explicit<Boolean> coerce, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
|
||||
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo);
|
||||
Explicit<Boolean> coerce, Boolean includeInAll, Settings indexSettings, MultiFields multiFields, CopyTo copyTo) {
|
||||
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, Explicit<Boolean> coerce,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, Explicit<Boolean> coerce,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, Explicit<Boolean> coerce,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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) {
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, Explicit<Boolean> coerce,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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
|
||||
|
|
|
@ -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<Boolean> COERCE_SETTING =
|
||||
Setting.boolSetting("index.mapping.coerce", true, Property.IndexScope);
|
||||
|
@ -158,12 +158,13 @@ public abstract class LegacyNumberFieldMapper extends FieldMapper implements All
|
|||
protected Explicit<Boolean> coerce;
|
||||
|
||||
protected LegacyNumberFieldMapper(String simpleName, MappedFieldType fieldType, MappedFieldType defaultFieldType,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> coerce, Settings indexSettings,
|
||||
MultiFields multiFields, CopyTo copyTo) {
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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<Field> fields) throws IOException {
|
||||
RuntimeException e = null;
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed, Explicit<Boolean> coerce,
|
||||
Explicit<Boolean> ignoreMalformed, Explicit<Boolean> 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
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed,
|
||||
Explicit<Boolean> coerce, Settings indexSettings, NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) {
|
||||
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, indexSettings, multiFields, copyTo);
|
||||
Explicit<Boolean> coerce, Boolean includeInAll, Settings indexSettings, NamedAnalyzer analyzer, MultiFields multiFields, CopyTo copyTo) {
|
||||
super(simpleName, fieldType, defaultFieldType, ignoreMalformed, coerce, includeInAll, indexSettings, multiFields, copyTo);
|
||||
this.analyzer = analyzer;
|
||||
}
|
||||
|
||||
|
|
|
@ -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<Boolean> 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<Boolean> ignoreMalformed,
|
||||
Explicit<Boolean> 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<Field> fields) throws IOException {
|
||||
XContentParser parser = context.parser();
|
||||
|
|
|
@ -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<String, Mapper> 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<String, Mapper> 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<String, Mapper> mappers;
|
||||
|
||||
ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map<String, Mapper> mappers) {
|
||||
ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic,
|
||||
Boolean includeInAll, Map<String, Mapper> 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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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 <tt>false</tt>. If its enabled, then will return <tt>true</tt> only if the specific flag is <tt>null</tt> 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();
|
||||
|
|
|
@ -100,7 +100,8 @@ public class RootObjectMapper extends ObjectMapper {
|
|||
|
||||
|
||||
@Override
|
||||
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, Map<String, Mapper> mappers, @Nullable Settings settings) {
|
||||
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic,
|
||||
Boolean includeInAll, Map<String, Mapper> 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<String, Mapper> mappers,
|
||||
RootObjectMapper(String name, boolean enabled, Dynamic dynamic, Boolean includeInAll, Map<String, Mapper> 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;
|
||||
|
|
|
@ -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<Boolean> ignoreMalformed,
|
||||
Explicit<Boolean> 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<Field> fields) throws IOException {
|
||||
XContentParser parser = context.parser();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
@ -68,11 +70,15 @@ 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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<String, DiskUsage> 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();
|
||||
|
|
|
@ -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<ShardRouting, String> shardRoutingMap = ImmutableOpenMap.builder();
|
||||
|
||||
DiscoveryNode node_0 = new DiscoveryNode("node_0", LocalTransportAddress.buildUnique(), Collections.emptyMap(),
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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<String> values = new HashSet<>();
|
||||
for (IndexableField f : doc.rootDoc().getFields("_all")) {
|
||||
values.add(f.stringValue());
|
||||
}
|
||||
assertEquals(new HashSet<>(Arrays.asList("b", "d")), values);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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"}'''
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -33,29 +33,34 @@ 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>
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
--------------------------------------------------
|
||||
// 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.
|
||||
|
||||
|
@ -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/]
|
||||
|
|
|
@ -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" : {
|
||||
|
@ -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/]
|
||||
|
|
|
@ -42,11 +42,13 @@ for more details) |Required |
|
|||
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
|
||||
|
|
|
@ -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/]
|
||||
|
|
|
@ -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,6 +112,9 @@ 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
|
||||
|
@ -120,7 +129,9 @@ 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,6 +214,10 @@ 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> 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
|
||||
|
||||
|
@ -208,7 +229,9 @@ of the total sales per month but ask for the derivative of the sales as in the u
|
|||
|
||||
[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'
|
||||
|
|
|
@ -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/]
|
||||
|
|
|
@ -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,6 +59,9 @@ The following snippet calculates the maximum of the total monthly `sales`:
|
|||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// 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.
|
||||
|
||||
|
@ -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
|
||||
|
||||
|
|
|
@ -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,6 +60,8 @@ 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
|
||||
`sales_per_month` date histogram.
|
||||
|
@ -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
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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
|
||||
<<buckets-path-syntax>> for a description of the syntax for `buckets_path`.
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
|
|
@ -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/]
|
||||
|
|
|
@ -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> `buckets_path` instructs this sum_bucket aggregation that we want the sum of 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,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/]
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 <<data-path,data path>> 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 <<data-path,data path>> 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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,6 +57,9 @@ public class RethrottleTests extends ReindexTestCase {
|
|||
request.source().setSize(1); // Make sure we use multiple batches
|
||||
ListenableActionFuture<? extends BulkIndexByScrollResponse> 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));
|
||||
|
|
|
@ -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": {
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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) {
|
||||
|
|
Loading…
Reference in New Issue