mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-26 09:58:28 +00:00
Merge branch 'master' into feature/seq_no
* master: (22 commits) Add proper toString() method to UpdateTask (#21582) Fix `InternalEngine#isThrottled` to not always return `false`. (#21592) add `ignore_missing` option to SplitProcessor (#20982) fix trace_match behavior for when there is only one grok pattern (#21413) Remove dead code from GetResponse.java Fixes date range query using epoch with timezone (#21542) Do not cache term queries. (#21566) Updated dynamic mapper section Docs: Clarify date_histogram bucket sizes for DST time zones Handle release of 5.0.1 Fix skip reason for stats API parameters test Reduce skip version for stats API parameter tests Strict level parsing for indices stats Remove cluster update task when task times out (#21578) [DOCS] Mention "all-fields" mode doesn't search across nested documents InternalTestCluster: when restarting a node we should validate the cluster is formed via the node we just restarted Fixed bad asciidoc in boolean mapping docs Fixed bad asciidoc ID in node stats Be strict when parsing values searching for booleans (#21555) Fix time zone rounding edge case for DST overlaps ...
This commit is contained in:
commit
d06a8903fd
@ -19,7 +19,6 @@
|
||||
|
||||
package org.elasticsearch;
|
||||
|
||||
import org.apache.lucene.util.MathUtil;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
@ -92,6 +91,8 @@ public class Version {
|
||||
public static final Version V_5_0_0_rc1 = new Version(V_5_0_0_rc1_ID, org.apache.lucene.util.Version.LUCENE_6_2_0);
|
||||
public static final int V_5_0_0_ID = 5000099;
|
||||
public static final Version V_5_0_0 = new Version(V_5_0_0_ID, org.apache.lucene.util.Version.LUCENE_6_2_0);
|
||||
public static final int V_5_0_1_ID = 5000199;
|
||||
public static final Version V_5_0_1 = new Version(V_5_0_1_ID, org.apache.lucene.util.Version.LUCENE_6_2_1);
|
||||
public static final int V_6_0_0_alpha1_ID = 6000001;
|
||||
public static final Version V_6_0_0_alpha1 = new Version(V_6_0_0_alpha1_ID, org.apache.lucene.util.Version.LUCENE_6_3_0);
|
||||
public static final Version CURRENT = V_6_0_0_alpha1;
|
||||
@ -118,6 +119,8 @@ public class Version {
|
||||
switch (id) {
|
||||
case V_6_0_0_alpha1_ID:
|
||||
return V_6_0_0_alpha1;
|
||||
case V_5_0_1_ID:
|
||||
return V_5_0_1;
|
||||
case V_5_0_0_ID:
|
||||
return V_5_0_0;
|
||||
case V_5_0_0_rc1_ID:
|
||||
|
@ -152,12 +152,14 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
String level = params.param("level", "indices");
|
||||
boolean isLevelValid = "indices".equalsIgnoreCase(level) || "shards".equalsIgnoreCase(level) || "cluster".equalsIgnoreCase(level);
|
||||
final String level = params.param("level", "indices");
|
||||
final boolean isLevelValid =
|
||||
"cluster".equalsIgnoreCase(level) || "indices".equalsIgnoreCase(level) || "shards".equalsIgnoreCase(level);
|
||||
if (!isLevelValid) {
|
||||
return builder;
|
||||
throw new IllegalArgumentException("level parameter must be one of [cluster] or [indices] or [shards] but was [" + level + "]");
|
||||
}
|
||||
|
||||
|
||||
builder.startObject("_all");
|
||||
|
||||
builder.startObject("primaries");
|
||||
|
@ -164,12 +164,6 @@ public class GetResponse extends ActionResponse implements Iterable<GetField>, T
|
||||
return getResult.toXContent(builder, params);
|
||||
}
|
||||
|
||||
public static GetResponse readGetResponse(StreamInput in) throws IOException {
|
||||
GetResponse result = new GetResponse();
|
||||
result.readFrom(in);
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
|
@ -70,6 +70,7 @@ import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
@ -114,7 +115,7 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
private final Collection<ClusterStateListener> priorityClusterStateListeners = new CopyOnWriteArrayList<>();
|
||||
private final Collection<ClusterStateListener> clusterStateListeners = new CopyOnWriteArrayList<>();
|
||||
private final Collection<ClusterStateListener> lastClusterStateListeners = new CopyOnWriteArrayList<>();
|
||||
private final Map<ClusterStateTaskExecutor, List<UpdateTask>> updateTasksPerExecutor = new HashMap<>();
|
||||
final Map<ClusterStateTaskExecutor, LinkedHashSet<UpdateTask>> updateTasksPerExecutor = new HashMap<>();
|
||||
// TODO this is rather frequently changing I guess a Synced Set would be better here and a dedicated remove API
|
||||
private final Collection<ClusterStateListener> postAppliedListeners = new CopyOnWriteArrayList<>();
|
||||
private final Iterable<ClusterStateListener> preAppliedListeners = Iterables.concat(priorityClusterStateListeners,
|
||||
@ -450,11 +451,12 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
// convert to an identity map to check for dups based on update tasks semantics of using identity instead of equal
|
||||
final IdentityHashMap<T, ClusterStateTaskListener> tasksIdentity = new IdentityHashMap<>(tasks);
|
||||
final List<UpdateTask<T>> updateTasks = tasksIdentity.entrySet().stream().map(
|
||||
entry -> new UpdateTask<>(source, entry.getKey(), config, executor, safe(entry.getValue(), logger))
|
||||
entry -> new UpdateTask<>(source, entry.getKey(), config.priority(), executor, safe(entry.getValue(), logger))
|
||||
).collect(Collectors.toList());
|
||||
|
||||
synchronized (updateTasksPerExecutor) {
|
||||
List<UpdateTask> existingTasks = updateTasksPerExecutor.computeIfAbsent(executor, k -> new ArrayList<>());
|
||||
LinkedHashSet<UpdateTask> existingTasks = updateTasksPerExecutor.computeIfAbsent(executor,
|
||||
k -> new LinkedHashSet<>(updateTasks.size()));
|
||||
for (@SuppressWarnings("unchecked") UpdateTask<T> existing : existingTasks) {
|
||||
if (tasksIdentity.containsKey(existing.task)) {
|
||||
throw new IllegalStateException("task [" + executor.describeTasks(Collections.singletonList(existing.task)) +
|
||||
@ -466,12 +468,29 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
|
||||
final UpdateTask<T> firstTask = updateTasks.get(0);
|
||||
|
||||
if (config.timeout() != null) {
|
||||
updateTasksExecutor.execute(firstTask, threadPool.scheduler(), config.timeout(), () -> threadPool.generic().execute(() -> {
|
||||
final TimeValue timeout = config.timeout();
|
||||
if (timeout != null) {
|
||||
updateTasksExecutor.execute(firstTask, threadPool.scheduler(), timeout, () -> threadPool.generic().execute(() -> {
|
||||
final ArrayList<UpdateTask<T>> toRemove = new ArrayList<>();
|
||||
for (UpdateTask<T> task : updateTasks) {
|
||||
if (task.processed.getAndSet(true) == false) {
|
||||
logger.debug("cluster state update task [{}] timed out after [{}]", source, config.timeout());
|
||||
task.listener.onFailure(source, new ProcessClusterEventTimeoutException(config.timeout(), source));
|
||||
logger.debug("cluster state update task [{}] timed out after [{}]", source, timeout);
|
||||
toRemove.add(task);
|
||||
}
|
||||
}
|
||||
if (toRemove.isEmpty() == false) {
|
||||
ClusterStateTaskExecutor<T> clusterStateTaskExecutor = toRemove.get(0).executor;
|
||||
synchronized (updateTasksPerExecutor) {
|
||||
LinkedHashSet<UpdateTask> existingTasks = updateTasksPerExecutor.get(clusterStateTaskExecutor);
|
||||
if (existingTasks != null) {
|
||||
existingTasks.removeAll(toRemove);
|
||||
if (existingTasks.isEmpty()) {
|
||||
updateTasksPerExecutor.remove(clusterStateTaskExecutor);
|
||||
}
|
||||
}
|
||||
}
|
||||
for (UpdateTask<T> task : toRemove) {
|
||||
task.listener.onFailure(source, new ProcessClusterEventTimeoutException(timeout, source));
|
||||
}
|
||||
}
|
||||
}));
|
||||
@ -567,15 +586,15 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
final ArrayList<UpdateTask<T>> toExecute = new ArrayList<>();
|
||||
final Map<String, ArrayList<T>> processTasksBySource = new HashMap<>();
|
||||
synchronized (updateTasksPerExecutor) {
|
||||
List<UpdateTask> pending = updateTasksPerExecutor.remove(executor);
|
||||
LinkedHashSet<UpdateTask> pending = updateTasksPerExecutor.remove(executor);
|
||||
if (pending != null) {
|
||||
for (UpdateTask<T> task : pending) {
|
||||
if (task.processed.getAndSet(true) == false) {
|
||||
logger.trace("will process {}", task.toString(executor));
|
||||
logger.trace("will process {}", task);
|
||||
toExecute.add(task);
|
||||
processTasksBySource.computeIfAbsent(task.source, s -> new ArrayList<>()).add(task.task);
|
||||
} else {
|
||||
logger.trace("skipping {}, already processed", task.toString(executor));
|
||||
logger.trace("skipping {}, already processed", task);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -633,7 +652,7 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
if (assertsEnabled) {
|
||||
for (UpdateTask<T> updateTask : toExecute) {
|
||||
assert batchResult.executionResults.containsKey(updateTask.task) :
|
||||
"missing task result for " + updateTask.toString(executor);
|
||||
"missing task result for " + updateTask;
|
||||
}
|
||||
}
|
||||
|
||||
@ -641,7 +660,7 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
final ArrayList<UpdateTask<T>> proccessedListeners = new ArrayList<>();
|
||||
// fail all tasks that have failed and extract those that are waiting for results
|
||||
for (UpdateTask<T> updateTask : toExecute) {
|
||||
assert batchResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask.toString(executor);
|
||||
assert batchResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask;
|
||||
final ClusterStateTaskExecutor.TaskResult executionResult =
|
||||
batchResult.executionResults.get(updateTask.task);
|
||||
executionResult.handle(
|
||||
@ -649,7 +668,7 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
ex -> {
|
||||
logger.debug(
|
||||
(Supplier<?>)
|
||||
() -> new ParameterizedMessage("cluster state update task {} failed", updateTask.toString(executor)), ex);
|
||||
() -> new ParameterizedMessage("cluster state update task {} failed", updateTask), ex);
|
||||
updateTask.listener.onFailure(updateTask.source, ex);
|
||||
}
|
||||
);
|
||||
@ -927,16 +946,13 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
class UpdateTask<T> extends SourcePrioritizedRunnable {
|
||||
|
||||
public final T task;
|
||||
public final ClusterStateTaskConfig config;
|
||||
public final ClusterStateTaskExecutor<T> executor;
|
||||
public final ClusterStateTaskListener listener;
|
||||
private final ClusterStateTaskExecutor<T> executor;
|
||||
public final AtomicBoolean processed = new AtomicBoolean();
|
||||
|
||||
UpdateTask(String source, T task, ClusterStateTaskConfig config, ClusterStateTaskExecutor<T> executor,
|
||||
ClusterStateTaskListener listener) {
|
||||
super(config.priority(), source);
|
||||
UpdateTask(String source, T task, Priority priority, ClusterStateTaskExecutor<T> executor, ClusterStateTaskListener listener) {
|
||||
super(priority, source);
|
||||
this.task = task;
|
||||
this.config = config;
|
||||
this.executor = executor;
|
||||
this.listener = listener;
|
||||
}
|
||||
@ -950,7 +966,8 @@ public class ClusterService extends AbstractLifecycleComponent {
|
||||
}
|
||||
}
|
||||
|
||||
public String toString(ClusterStateTaskExecutor<T> executor) {
|
||||
@Override
|
||||
public String toString() {
|
||||
String taskDescription = executor.describeTasks(Collections.singletonList(task));
|
||||
if (taskDescription.isEmpty()) {
|
||||
return "[" + source + "]";
|
||||
|
@ -128,15 +128,38 @@ public abstract class Rounding implements Streamable {
|
||||
@Override
|
||||
public long round(long utcMillis) {
|
||||
long rounded = field.roundFloor(utcMillis);
|
||||
if (timeZone.isFixed() == false && timeZone.getOffset(utcMillis) != timeZone.getOffset(rounded)) {
|
||||
// in this case, we crossed a time zone transition. In some edge
|
||||
// cases this will
|
||||
// result in a value that is not a rounded value itself. We need
|
||||
// to round again
|
||||
// to make sure. This will have no affect in cases where
|
||||
// 'rounded' was already a proper
|
||||
// rounded value
|
||||
rounded = field.roundFloor(rounded);
|
||||
if (timeZone.isFixed() == false) {
|
||||
// special cases for non-fixed time zones with dst transitions
|
||||
if (timeZone.getOffset(utcMillis) != timeZone.getOffset(rounded)) {
|
||||
/*
|
||||
* the offset change indicates a dst transition. In some
|
||||
* edge cases this will result in a value that is not a
|
||||
* rounded value before the transition. We round again to
|
||||
* make sure we really return a rounded value. This will
|
||||
* have no effect in cases where we already had a valid
|
||||
* rounded value
|
||||
*/
|
||||
rounded = field.roundFloor(rounded);
|
||||
} else {
|
||||
/*
|
||||
* check if the current time instant is at a start of a DST
|
||||
* overlap by comparing the offset of the instant and the
|
||||
* previous millisecond. We want to detect negative offset
|
||||
* changes that result in an overlap
|
||||
*/
|
||||
if (timeZone.getOffset(rounded) < timeZone.getOffset(rounded - 1)) {
|
||||
/*
|
||||
* we are rounding a date just after a DST overlap. if
|
||||
* the overlap is smaller than the time unit we are
|
||||
* rounding to, we want to add the overlapping part to
|
||||
* the following rounding interval
|
||||
*/
|
||||
long previousRounded = field.roundFloor(rounded - 1);
|
||||
if (rounded - previousRounded < field.getDurationField().getUnitMillis()) {
|
||||
rounded = previousRounded;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
assert rounded == field.roundFloor(rounded);
|
||||
return rounded;
|
||||
|
@ -140,6 +140,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
|
||||
IndexModule.INDEX_STORE_PRE_LOAD_SETTING,
|
||||
IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING,
|
||||
IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING,
|
||||
IndexModule.INDEX_QUERY_CACHE_TERM_QUERIES_SETTING,
|
||||
PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS_SETTING,
|
||||
FsDirectoryService.INDEX_LOCK_FACTOR_SETTING,
|
||||
EngineConfig.INDEX_CODEC_SETTING,
|
||||
|
@ -102,6 +102,11 @@ public final class IndexModule {
|
||||
public static final Setting<Boolean> INDEX_QUERY_CACHE_EVERYTHING_SETTING =
|
||||
Setting.boolSetting("index.queries.cache.everything", false, Property.IndexScope);
|
||||
|
||||
// This setting is an escape hatch in case not caching term queries would slow some users down
|
||||
// Do not document.
|
||||
public static final Setting<Boolean> INDEX_QUERY_CACHE_TERM_QUERIES_SETTING =
|
||||
Setting.boolSetting("index.queries.cache.term_queries", false, Property.IndexScope);
|
||||
|
||||
private final IndexSettings indexSettings;
|
||||
private final IndexStoreConfig indexStoreConfig;
|
||||
private final AnalysisRegistry analysisRegistry;
|
||||
|
@ -237,17 +237,13 @@ public abstract class Engine implements Closeable {
|
||||
/**
|
||||
* Returns the number of milliseconds this engine was under index throttling.
|
||||
*/
|
||||
public long getIndexThrottleTimeInMillis() {
|
||||
return 0;
|
||||
}
|
||||
public abstract long getIndexThrottleTimeInMillis();
|
||||
|
||||
/**
|
||||
* Returns the <code>true</code> iff this engine is currently under index throttling.
|
||||
* @see #getIndexThrottleTimeInMillis()
|
||||
*/
|
||||
public boolean isThrottled() {
|
||||
return false;
|
||||
}
|
||||
public abstract boolean isThrottled();
|
||||
|
||||
/** A Lock implementation that always allows the lock to be acquired */
|
||||
protected static final class NoOpLock implements Lock {
|
||||
|
@ -1321,6 +1321,12 @@ public class InternalEngine extends Engine {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isThrottled() {
|
||||
return throttle.isThrottled();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getIndexThrottleTimeInMillis() {
|
||||
return throttle.getThrottleTimeInMillis();
|
||||
}
|
||||
|
@ -262,6 +262,16 @@ public class ShadowEngine extends Engine {
|
||||
throw new UnsupportedOperationException("ShadowEngine doesn't track sequence numbers");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isThrottled() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getIndexThrottleTimeInMillis() {
|
||||
return 0L;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine recoverFromTranslog() throws IOException {
|
||||
throw new UnsupportedOperationException("can't recover on a shadow engine");
|
||||
|
@ -152,16 +152,15 @@ public class BooleanFieldMapper extends FieldMapper {
|
||||
} else {
|
||||
sValue = value.toString();
|
||||
}
|
||||
if (sValue.length() == 0) {
|
||||
return Values.FALSE;
|
||||
switch (sValue) {
|
||||
case "true":
|
||||
return Values.TRUE;
|
||||
case "false":
|
||||
return Values.FALSE;
|
||||
default:
|
||||
throw new IllegalArgumentException("Can't parse boolean value [" +
|
||||
sValue + "], expected [true] or [false]");
|
||||
}
|
||||
if (sValue.length() == 1 && sValue.charAt(0) == 'F') {
|
||||
return Values.FALSE;
|
||||
}
|
||||
if (Booleans.parseBoolean(sValue, false)) {
|
||||
return Values.TRUE;
|
||||
}
|
||||
return Values.FALSE;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -321,14 +321,6 @@ public class DateFieldMapper extends FieldMapper {
|
||||
dateParser = this.dateMathParser;
|
||||
}
|
||||
|
||||
if (PointValues.size(reader, name()) == 0) {
|
||||
// no points, so nothing matches
|
||||
return Relation.DISJOINT;
|
||||
}
|
||||
|
||||
long minValue = LongPoint.decodeDimension(PointValues.getMinPackedValue(reader, name()), 0);
|
||||
long maxValue = LongPoint.decodeDimension(PointValues.getMaxPackedValue(reader, name()), 0);
|
||||
|
||||
long fromInclusive = Long.MIN_VALUE;
|
||||
if (from != null) {
|
||||
fromInclusive = parseToMilliseconds(from, !includeLower, timeZone, dateParser, context);
|
||||
@ -351,6 +343,17 @@ public class DateFieldMapper extends FieldMapper {
|
||||
}
|
||||
}
|
||||
|
||||
// This check needs to be done after fromInclusive and toInclusive
|
||||
// are resolved so we can throw an exception if they are invalid
|
||||
// even if there are no points in the shard
|
||||
if (PointValues.size(reader, name()) == 0) {
|
||||
// no points, so nothing matches
|
||||
return Relation.DISJOINT;
|
||||
}
|
||||
|
||||
long minValue = LongPoint.decodeDimension(PointValues.getMinPackedValue(reader, name()), 0);
|
||||
long maxValue = LongPoint.decodeDimension(PointValues.getMaxPackedValue(reader, name()), 0);
|
||||
|
||||
if (minValue >= fromInclusive && maxValue <= toInclusive) {
|
||||
return Relation.WITHIN;
|
||||
} else if (maxValue < fromInclusive || minValue > toInclusive) {
|
||||
|
@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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.index.shard;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.QueryCachingPolicy;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A {@link QueryCachingPolicy} that does not cache {@link TermQuery}s.
|
||||
*/
|
||||
final class ElasticsearchQueryCachingPolicy implements QueryCachingPolicy {
|
||||
|
||||
private final QueryCachingPolicy in;
|
||||
|
||||
ElasticsearchQueryCachingPolicy(QueryCachingPolicy in) {
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUse(Query query) {
|
||||
if (query.getClass() != TermQuery.class) {
|
||||
// Do not waste space in the history for term queries. The assumption
|
||||
// is that these queries are very fast so not worth caching
|
||||
in.onUse(query);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean shouldCache(Query query) throws IOException {
|
||||
if (query.getClass() == TermQuery.class) {
|
||||
return false;
|
||||
}
|
||||
return in.shouldCache(query);
|
||||
}
|
||||
|
||||
}
|
@ -277,7 +277,11 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||
if (IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.get(settings)) {
|
||||
cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE;
|
||||
} else {
|
||||
cachingPolicy = new UsageTrackingQueryCachingPolicy();
|
||||
QueryCachingPolicy cachingPolicy = new UsageTrackingQueryCachingPolicy();
|
||||
if (IndexModule.INDEX_QUERY_CACHE_TERM_QUERIES_SETTING.get(settings) == false) {
|
||||
cachingPolicy = new ElasticsearchQueryCachingPolicy(cachingPolicy);
|
||||
}
|
||||
this.cachingPolicy = cachingPolicy;
|
||||
}
|
||||
indexShardOperationsLock = new IndexShardOperationsLock(shardId, logger, threadPool);
|
||||
searcherWrapper = indexSearcherWrapper;
|
||||
|
@ -116,6 +116,28 @@ public final class IngestDocument {
|
||||
return cast(path, context, clazz);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the value contained in the document for the provided path
|
||||
*
|
||||
* @param path The path within the document in dot-notation
|
||||
* @param clazz The expected class of the field value
|
||||
* @param ignoreMissing The flag to determine whether to throw an exception when `path` is not found in the document.
|
||||
* @return the value for the provided path if existing, null otherwise.
|
||||
* @throws IllegalArgumentException only if ignoreMissing is false and the path is null, empty, invalid, if the field doesn't exist
|
||||
* or if the field that is found at the provided path is not of the expected type.
|
||||
*/
|
||||
public <T> T getFieldValue(String path, Class<T> clazz, boolean ignoreMissing) {
|
||||
try {
|
||||
return getFieldValue(path, clazz);
|
||||
} catch (IllegalArgumentException e) {
|
||||
if (ignoreMissing && hasField(path) != true) {
|
||||
return null;
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the value contained in the document with the provided templated path
|
||||
* @param pathTemplate The path within the document in dot-notation
|
||||
|
@ -71,49 +71,58 @@ public abstract class BaseRestHandler extends AbstractComponent implements RestH
|
||||
request.unconsumedParams().stream().filter(p -> !responseParams().contains(p)).collect(Collectors.toCollection(TreeSet::new));
|
||||
|
||||
// validate the non-response params
|
||||
if (unconsumedParams.isEmpty() == false) {
|
||||
String message = String.format(
|
||||
Locale.ROOT,
|
||||
"request [%s] contains unrecognized parameter%s: ",
|
||||
request.path(),
|
||||
unconsumedParams.size() > 1 ? "s" : "");
|
||||
boolean first = true;
|
||||
for (final String unconsumedParam : unconsumedParams) {
|
||||
final LevensteinDistance ld = new LevensteinDistance();
|
||||
final List<Tuple<Float, String>> scoredParams = new ArrayList<>();
|
||||
final Set<String> candidateParams = new HashSet<>();
|
||||
candidateParams.addAll(request.consumedParams());
|
||||
candidateParams.addAll(responseParams());
|
||||
for (final String candidateParam : candidateParams) {
|
||||
final float distance = ld.getDistance(unconsumedParam, candidateParam);
|
||||
if (distance > 0.5f) {
|
||||
scoredParams.add(new Tuple<>(distance, candidateParam));
|
||||
}
|
||||
}
|
||||
CollectionUtil.timSort(scoredParams, (a, b) -> {
|
||||
// sort by distance in reverse order, then parameter name for equal distances
|
||||
int compare = a.v1().compareTo(b.v1());
|
||||
if (compare != 0) return -compare;
|
||||
else return a.v2().compareTo(b.v2());
|
||||
});
|
||||
if (first == false) {
|
||||
message += ", ";
|
||||
}
|
||||
message += "[" + unconsumedParam + "]";
|
||||
final List<String> keys = scoredParams.stream().map(Tuple::v2).collect(Collectors.toList());
|
||||
if (keys.isEmpty() == false) {
|
||||
message += " -> did you mean " + (keys.size() == 1 ? "[" + keys.get(0) + "]": "any of " + keys.toString()) + "?";
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException(message);
|
||||
if (!unconsumedParams.isEmpty()) {
|
||||
final Set<String> candidateParams = new HashSet<>();
|
||||
candidateParams.addAll(request.consumedParams());
|
||||
candidateParams.addAll(responseParams());
|
||||
throw new IllegalArgumentException(unrecognized(request, unconsumedParams, candidateParams, "parameter"));
|
||||
}
|
||||
|
||||
// execute the action
|
||||
action.accept(channel);
|
||||
}
|
||||
|
||||
protected final String unrecognized(
|
||||
final RestRequest request,
|
||||
final Set<String> invalids,
|
||||
final Set<String> candidates,
|
||||
final String detail) {
|
||||
String message = String.format(
|
||||
Locale.ROOT,
|
||||
"request [%s] contains unrecognized %s%s: ",
|
||||
request.path(),
|
||||
detail,
|
||||
invalids.size() > 1 ? "s" : "");
|
||||
boolean first = true;
|
||||
for (final String invalid : invalids) {
|
||||
final LevensteinDistance ld = new LevensteinDistance();
|
||||
final List<Tuple<Float, String>> scoredParams = new ArrayList<>();
|
||||
for (final String candidate : candidates) {
|
||||
final float distance = ld.getDistance(invalid, candidate);
|
||||
if (distance > 0.5f) {
|
||||
scoredParams.add(new Tuple<>(distance, candidate));
|
||||
}
|
||||
}
|
||||
CollectionUtil.timSort(scoredParams, (a, b) -> {
|
||||
// sort by distance in reverse order, then parameter name for equal distances
|
||||
int compare = a.v1().compareTo(b.v1());
|
||||
if (compare != 0) return -compare;
|
||||
else return a.v2().compareTo(b.v2());
|
||||
});
|
||||
if (first == false) {
|
||||
message += ", ";
|
||||
}
|
||||
message += "[" + invalid + "]";
|
||||
final List<String> keys = scoredParams.stream().map(Tuple::v2).collect(Collectors.toList());
|
||||
if (keys.isEmpty() == false) {
|
||||
message += " -> did you mean " + (keys.size() == 1 ? "[" + keys.get(0) + "]" : "any of " + keys.toString()) + "?";
|
||||
}
|
||||
first = false;
|
||||
}
|
||||
|
||||
return message;
|
||||
}
|
||||
|
||||
/**
|
||||
* REST requests are handled by preparing a channel consumer that represents the execution of
|
||||
* the request against a channel.
|
||||
|
@ -55,7 +55,7 @@ public class RestNodesInfoAction extends BaseRestHandler {
|
||||
public RestNodesInfoAction(Settings settings, RestController controller, SettingsFilter settingsFilter) {
|
||||
super(settings);
|
||||
controller.registerHandler(GET, "/_nodes", this);
|
||||
// this endpoint is used for metrics, not for nodeIds, like /_nodes/fs
|
||||
// this endpoint is used for metrics, not for node IDs, like /_nodes/fs
|
||||
controller.registerHandler(GET, "/_nodes/{nodeId}", this);
|
||||
controller.registerHandler(GET, "/_nodes/{nodeId}/{metrics}", this);
|
||||
// added this endpoint to be aligned with stats
|
||||
|
@ -33,7 +33,13 @@ import org.elasticsearch.rest.action.RestActions.NodesResponseRestListener;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
|
||||
@ -48,9 +54,38 @@ public class RestNodesStatsAction extends BaseRestHandler {
|
||||
controller.registerHandler(GET, "/_nodes/stats/{metric}", this);
|
||||
controller.registerHandler(GET, "/_nodes/{nodeId}/stats/{metric}", this);
|
||||
|
||||
controller.registerHandler(GET, "/_nodes/stats/{metric}/{indexMetric}", this);
|
||||
controller.registerHandler(GET, "/_nodes/stats/{metric}/{index_metric}", this);
|
||||
|
||||
controller.registerHandler(GET, "/_nodes/{nodeId}/stats/{metric}/{indexMetric}", this);
|
||||
controller.registerHandler(GET, "/_nodes/{nodeId}/stats/{metric}/{index_metric}", this);
|
||||
}
|
||||
|
||||
static final Map<String, Consumer<NodesStatsRequest>> METRICS;
|
||||
|
||||
static {
|
||||
final Map<String, Consumer<NodesStatsRequest>> metrics = new HashMap<>();
|
||||
metrics.put("os", r -> r.os(true));
|
||||
metrics.put("jvm", r -> r.jvm(true));
|
||||
metrics.put("thread_pool", r -> r.threadPool(true));
|
||||
metrics.put("fs", r -> r.fs(true));
|
||||
metrics.put("transport", r -> r.transport(true));
|
||||
metrics.put("http", r -> r.http(true));
|
||||
metrics.put("indices", r -> r.indices(true));
|
||||
metrics.put("process", r -> r.process(true));
|
||||
metrics.put("breaker", r -> r.breaker(true));
|
||||
metrics.put("script", r -> r.script(true));
|
||||
metrics.put("discovery", r -> r.discovery(true));
|
||||
metrics.put("ingest", r -> r.ingest(true));
|
||||
METRICS = Collections.unmodifiableMap(metrics);
|
||||
}
|
||||
|
||||
static final Map<String, Consumer<CommonStatsFlags>> FLAGS;
|
||||
|
||||
static {
|
||||
final Map<String, Consumer<CommonStatsFlags>> flags = new HashMap<>();
|
||||
for (final Flag flag : CommonStatsFlags.Flag.values()) {
|
||||
flags.put(flag.getRestName(), f -> f.set(flag, true));
|
||||
}
|
||||
FLAGS = Collections.unmodifiableMap(flags);
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -62,35 +97,72 @@ public class RestNodesStatsAction extends BaseRestHandler {
|
||||
nodesStatsRequest.timeout(request.param("timeout"));
|
||||
|
||||
if (metrics.size() == 1 && metrics.contains("_all")) {
|
||||
if (request.hasParam("index_metric")) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
Locale.ROOT,
|
||||
"request [%s] contains index metrics [%s] but all stats requested",
|
||||
request.path(),
|
||||
request.param("index_metric")));
|
||||
}
|
||||
nodesStatsRequest.all();
|
||||
nodesStatsRequest.indices(CommonStatsFlags.ALL);
|
||||
} else if (metrics.contains("_all")) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(Locale.ROOT,
|
||||
"request [%s] contains _all and individual metrics [%s]",
|
||||
request.path(),
|
||||
request.param("metric")));
|
||||
} else {
|
||||
nodesStatsRequest.clear();
|
||||
nodesStatsRequest.os(metrics.contains("os"));
|
||||
nodesStatsRequest.jvm(metrics.contains("jvm"));
|
||||
nodesStatsRequest.threadPool(metrics.contains("thread_pool"));
|
||||
nodesStatsRequest.fs(metrics.contains("fs"));
|
||||
nodesStatsRequest.transport(metrics.contains("transport"));
|
||||
nodesStatsRequest.http(metrics.contains("http"));
|
||||
nodesStatsRequest.indices(metrics.contains("indices"));
|
||||
nodesStatsRequest.process(metrics.contains("process"));
|
||||
nodesStatsRequest.breaker(metrics.contains("breaker"));
|
||||
nodesStatsRequest.script(metrics.contains("script"));
|
||||
nodesStatsRequest.discovery(metrics.contains("discovery"));
|
||||
nodesStatsRequest.ingest(metrics.contains("ingest"));
|
||||
|
||||
// use a sorted set so the unrecognized parameters appear in a reliable sorted order
|
||||
final Set<String> invalidMetrics = new TreeSet<>();
|
||||
for (final String metric : metrics) {
|
||||
final Consumer<NodesStatsRequest> handler = METRICS.get(metric);
|
||||
if (handler != null) {
|
||||
handler.accept(nodesStatsRequest);
|
||||
} else {
|
||||
invalidMetrics.add(metric);
|
||||
}
|
||||
}
|
||||
|
||||
if (!invalidMetrics.isEmpty()) {
|
||||
throw new IllegalArgumentException(unrecognized(request, invalidMetrics, METRICS.keySet(), "metric"));
|
||||
}
|
||||
|
||||
// check for index specific metrics
|
||||
if (metrics.contains("indices")) {
|
||||
Set<String> indexMetrics = Strings.splitStringByCommaToSet(request.param("indexMetric", "_all"));
|
||||
Set<String> indexMetrics = Strings.splitStringByCommaToSet(request.param("index_metric", "_all"));
|
||||
if (indexMetrics.size() == 1 && indexMetrics.contains("_all")) {
|
||||
nodesStatsRequest.indices(CommonStatsFlags.ALL);
|
||||
} else {
|
||||
CommonStatsFlags flags = new CommonStatsFlags();
|
||||
for (Flag flag : CommonStatsFlags.Flag.values()) {
|
||||
flags.set(flag, indexMetrics.contains(flag.getRestName()));
|
||||
flags.clear();
|
||||
// use a sorted set so the unrecognized parameters appear in a reliable sorted order
|
||||
final Set<String> invalidIndexMetrics = new TreeSet<>();
|
||||
for (final String indexMetric : indexMetrics) {
|
||||
final Consumer<CommonStatsFlags> handler = FLAGS.get(indexMetric);
|
||||
if (handler != null) {
|
||||
handler.accept(flags);
|
||||
} else {
|
||||
invalidIndexMetrics.add(indexMetric);
|
||||
}
|
||||
}
|
||||
|
||||
if (!invalidIndexMetrics.isEmpty()) {
|
||||
throw new IllegalArgumentException(unrecognized(request, invalidIndexMetrics, FLAGS.keySet(), "index metric"));
|
||||
}
|
||||
|
||||
nodesStatsRequest.indices(flags);
|
||||
}
|
||||
} else if (request.hasParam("index_metric")) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(
|
||||
Locale.ROOT,
|
||||
"request [%s] contains index metrics [%s] but indices stats not requested",
|
||||
request.path(),
|
||||
request.param("index_metric")));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -36,7 +36,13 @@ import org.elasticsearch.rest.action.RestBuilderListener;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
import static org.elasticsearch.rest.RestStatus.OK;
|
||||
@ -49,11 +55,34 @@ public class RestIndicesStatsAction extends BaseRestHandler {
|
||||
super(settings);
|
||||
controller.registerHandler(GET, "/_stats", this);
|
||||
controller.registerHandler(GET, "/_stats/{metric}", this);
|
||||
controller.registerHandler(GET, "/_stats/{metric}/{indexMetric}", this);
|
||||
controller.registerHandler(GET, "/{index}/_stats", this);
|
||||
controller.registerHandler(GET, "/{index}/_stats/{metric}", this);
|
||||
}
|
||||
|
||||
static Map<String, Consumer<IndicesStatsRequest>> METRICS;
|
||||
|
||||
static {
|
||||
final Map<String, Consumer<IndicesStatsRequest>> metrics = new HashMap<>();
|
||||
metrics.put("docs", r -> r.docs(true));
|
||||
metrics.put("store", r -> r.store(true));
|
||||
metrics.put("indexing", r -> r.indexing(true));
|
||||
metrics.put("search", r -> r.search(true));
|
||||
metrics.put("suggest", r -> r.search(true));
|
||||
metrics.put("get", r -> r.get(true));
|
||||
metrics.put("merge", r -> r.merge(true));
|
||||
metrics.put("refresh", r -> r.refresh(true));
|
||||
metrics.put("flush", r -> r.flush(true));
|
||||
metrics.put("warmer", r -> r.warmer(true));
|
||||
metrics.put("query_cache", r -> r.queryCache(true));
|
||||
metrics.put("segments", r -> r.segments(true));
|
||||
metrics.put("fielddata", r -> r.fieldData(true));
|
||||
metrics.put("completion", r -> r.completion(true));
|
||||
metrics.put("request_cache", r -> r.requestCache(true));
|
||||
metrics.put("recovery", r -> r.recovery(true));
|
||||
metrics.put("translog", r -> r.translog(true));
|
||||
METRICS = Collections.unmodifiableMap(metrics);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException {
|
||||
IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest();
|
||||
@ -65,24 +94,28 @@ public class RestIndicesStatsAction extends BaseRestHandler {
|
||||
// short cut, if no metrics have been specified in URI
|
||||
if (metrics.size() == 1 && metrics.contains("_all")) {
|
||||
indicesStatsRequest.all();
|
||||
} else if (metrics.contains("_all")) {
|
||||
throw new IllegalArgumentException(
|
||||
String.format(Locale.ROOT,
|
||||
"request [%s] contains _all and individual metrics [%s]",
|
||||
request.path(),
|
||||
request.param("metric")));
|
||||
} else {
|
||||
indicesStatsRequest.clear();
|
||||
indicesStatsRequest.docs(metrics.contains("docs"));
|
||||
indicesStatsRequest.store(metrics.contains("store"));
|
||||
indicesStatsRequest.indexing(metrics.contains("indexing"));
|
||||
indicesStatsRequest.search(metrics.contains("search") || metrics.contains("suggest"));
|
||||
indicesStatsRequest.get(metrics.contains("get"));
|
||||
indicesStatsRequest.merge(metrics.contains("merge"));
|
||||
indicesStatsRequest.refresh(metrics.contains("refresh"));
|
||||
indicesStatsRequest.flush(metrics.contains("flush"));
|
||||
indicesStatsRequest.warmer(metrics.contains("warmer"));
|
||||
indicesStatsRequest.queryCache(metrics.contains("query_cache"));
|
||||
indicesStatsRequest.segments(metrics.contains("segments"));
|
||||
indicesStatsRequest.fieldData(metrics.contains("fielddata"));
|
||||
indicesStatsRequest.completion(metrics.contains("completion"));
|
||||
indicesStatsRequest.requestCache(metrics.contains("request_cache"));
|
||||
indicesStatsRequest.recovery(metrics.contains("recovery"));
|
||||
indicesStatsRequest.translog(metrics.contains("translog"));
|
||||
// use a sorted set so the unrecognized parameters appear in a reliable sorted order
|
||||
final Set<String> invalidMetrics = new TreeSet<>();
|
||||
for (final String metric : metrics) {
|
||||
final Consumer<IndicesStatsRequest> consumer = METRICS.get(metric);
|
||||
if (consumer != null) {
|
||||
consumer.accept(indicesStatsRequest);
|
||||
} else {
|
||||
invalidMetrics.add(metric);
|
||||
}
|
||||
}
|
||||
|
||||
if (!invalidMetrics.isEmpty()) {
|
||||
throw new IllegalArgumentException(unrecognized(request, invalidMetrics, METRICS.keySet(), "metric"));
|
||||
}
|
||||
}
|
||||
|
||||
if (request.hasParam("groups")) {
|
||||
|
@ -19,15 +19,12 @@
|
||||
|
||||
package org.elasticsearch;
|
||||
|
||||
import org.elasticsearch.action.ShardValidateQueryRequestTests;
|
||||
import org.elasticsearch.action.admin.indices.template.put.PutIndexTemplateRequest;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.query.QueryStringQueryBuilder;
|
||||
import org.elasticsearch.monitor.os.OsStats;
|
||||
import org.elasticsearch.index.query.SimpleQueryStringBuilder;
|
||||
import org.elasticsearch.monitor.os.OsStats;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
@ -136,8 +133,11 @@ public class VersionTests extends ESTestCase {
|
||||
assertThat(Version.V_5_0_0_alpha1.minimumCompatibilityVersion(), equalTo(Version.V_5_0_0_alpha1));
|
||||
// from 6.0 on we are supporting the latest minor of the previous major... this might fail once we add a new version ie. 5.x is
|
||||
// released since we need to bump the supported minor in Version#minimumCompatibilityVersion()
|
||||
assertThat("did you miss to bump the minor in Version#minimumCompatibilityVersion()",
|
||||
Version.V_6_0_0_alpha1.minimumCompatibilityVersion(), equalTo(VersionUtils.getPreviousVersion(Version.V_6_0_0_alpha1)));
|
||||
Version lastVersion = VersionUtils.getPreviousVersion(Version.V_6_0_0_alpha1);
|
||||
assertEquals(lastVersion.major, Version.V_6_0_0_alpha1.minimumCompatibilityVersion().major);
|
||||
assertEquals("did you miss to bump the minor in Version#minimumCompatibilityVersion()",
|
||||
lastVersion.minor, Version.V_6_0_0_alpha1.minimumCompatibilityVersion().minor);
|
||||
assertEquals(0, Version.V_6_0_0_alpha1.minimumCompatibilityVersion().revision);
|
||||
}
|
||||
|
||||
public void testToString() {
|
||||
|
@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.action.admin.indices.stats;
|
||||
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.object.HasToString.hasToString;
|
||||
|
||||
|
||||
public class IndicesStatsResponseTests extends ESTestCase {
|
||||
|
||||
public void testInvalidLevel() {
|
||||
final IndicesStatsResponse response = new IndicesStatsResponse();
|
||||
final String level = randomAsciiOfLength(16);
|
||||
final ToXContent.Params params = new ToXContent.MapParams(Collections.singletonMap("level", level));
|
||||
final IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> response.toXContent(null, params));
|
||||
assertThat(
|
||||
e,
|
||||
hasToString(containsString("level parameter must be one of [cluster] or [indices] or [shards] but was [" + level + "]")));
|
||||
}
|
||||
|
||||
}
|
@ -148,6 +148,51 @@ public class ClusterServiceTests extends ESTestCase {
|
||||
return timedClusterService;
|
||||
}
|
||||
|
||||
public void testTimedOutUpdateTaskCleanedUp() throws Exception {
|
||||
final CountDownLatch block = new CountDownLatch(1);
|
||||
clusterService.submitStateUpdateTask("block-task", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
try {
|
||||
block.await();
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return currentState;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
|
||||
final CountDownLatch block2 = new CountDownLatch(1);
|
||||
clusterService.submitStateUpdateTask("test", new ClusterStateUpdateTask() {
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
block2.countDown();
|
||||
return currentState;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TimeValue timeout() {
|
||||
return TimeValue.ZERO;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(String source, Exception e) {
|
||||
block2.countDown();
|
||||
}
|
||||
});
|
||||
block.countDown();
|
||||
block2.await();
|
||||
synchronized (clusterService.updateTasksPerExecutor) {
|
||||
assertTrue("expected empty map but was " + clusterService.updateTasksPerExecutor,
|
||||
clusterService.updateTasksPerExecutor.isEmpty());
|
||||
}
|
||||
}
|
||||
|
||||
public void testTimeoutUpdateTask() throws Exception {
|
||||
final CountDownLatch block = new CountDownLatch(1);
|
||||
clusterService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() {
|
||||
|
@ -514,6 +514,44 @@ public class TimeZoneRoundingTests extends ESTestCase {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* tests for dst transition with overlaps and day roundings.
|
||||
*/
|
||||
public void testDST_END_Edgecases() {
|
||||
// First case, dst happens at 1am local time, switching back one hour.
|
||||
// We want the overlapping hour to count for the next day, making it a 25h interval
|
||||
|
||||
DateTimeUnit timeUnit = DateTimeUnit.DAY_OF_MONTH;
|
||||
DateTimeZone tz = DateTimeZone.forID("Atlantic/Azores");
|
||||
Rounding rounding = new Rounding.TimeUnitRounding(timeUnit, tz);
|
||||
|
||||
// Sunday, 29 October 2000, 01:00:00 clocks were turned backward 1 hour
|
||||
// to Sunday, 29 October 2000, 00:00:00 local standard time instead
|
||||
|
||||
long midnightBeforeTransition = time("2000-10-29T00:00:00", tz);
|
||||
long nextMidnight = time("2000-10-30T00:00:00", tz);
|
||||
|
||||
assertInterval(midnightBeforeTransition, nextMidnight, rounding, 25 * 60, tz);
|
||||
|
||||
// Second case, dst happens at 0am local time, switching back one hour to 23pm local time.
|
||||
// We want the overlapping hour to count for the previous day here
|
||||
|
||||
tz = DateTimeZone.forID("America/Lima");
|
||||
rounding = new Rounding.TimeUnitRounding(timeUnit, tz);
|
||||
|
||||
// Sunday, 1 April 1990, 00:00:00 clocks were turned backward 1 hour to
|
||||
// Saturday, 31 March 1990, 23:00:00 local standard time instead
|
||||
|
||||
midnightBeforeTransition = time("1990-03-31T00:00:00.000-04:00");
|
||||
nextMidnight = time("1990-04-01T00:00:00.000-05:00");
|
||||
assertInterval(midnightBeforeTransition, nextMidnight, rounding, 25 * 60, tz);
|
||||
|
||||
// make sure the next interval is 24h long again
|
||||
long midnightAfterTransition = time("1990-04-01T00:00:00.000-05:00");
|
||||
nextMidnight = time("1990-04-02T00:00:00.000-05:00");
|
||||
assertInterval(midnightAfterTransition, nextMidnight, rounding, 24 * 60, tz);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test that time zones are correctly parsed. There is a bug with
|
||||
* Joda 2.9.4 (see https://github.com/JodaOrg/joda-time/issues/373)
|
||||
|
@ -73,11 +73,12 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
|
||||
}
|
||||
|
||||
public void testIsFieldWithinQueryEmptyReader() throws IOException {
|
||||
QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, null, () -> nowInMillis);
|
||||
IndexReader reader = new MultiReader();
|
||||
DateFieldType ft = new DateFieldType();
|
||||
ft.setName("my_date");
|
||||
assertEquals(Relation.DISJOINT, ft.isFieldWithinQuery(reader, "2015-10-12", "2016-04-03",
|
||||
randomBoolean(), randomBoolean(), null, null, null));
|
||||
randomBoolean(), randomBoolean(), null, null, context));
|
||||
}
|
||||
|
||||
private void doTestIsFieldWithinQuery(DateFieldType ft, DirectoryReader reader,
|
||||
@ -128,7 +129,9 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
|
||||
// Fields with no value indexed.
|
||||
DateFieldType ft2 = new DateFieldType();
|
||||
ft2.setName("my_date2");
|
||||
assertEquals(Relation.DISJOINT, ft2.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", false, false, null, null, null));
|
||||
|
||||
QueryRewriteContext context = new QueryRewriteContext(null, null, null, null, null, null, null, () -> nowInMillis);
|
||||
assertEquals(Relation.DISJOINT, ft2.isFieldWithinQuery(reader, "2015-10-09", "2016-01-02", false, false, null, null, context));
|
||||
IOUtils.close(reader, w, dir);
|
||||
}
|
||||
|
||||
|
@ -106,7 +106,7 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase {
|
||||
SearchResponse response;
|
||||
|
||||
response = client().prepareSearch("test-idx")
|
||||
.setPostFilter(QueryBuilders.termQuery("field.bool", "T"))
|
||||
.setPostFilter(QueryBuilders.termQuery("field.bool", "true"))
|
||||
.execute().actionGet();
|
||||
|
||||
assertThat(response.getHits().totalHits(), equalTo((long) 1));
|
||||
|
@ -0,0 +1,61 @@
|
||||
/*
|
||||
* 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.index.shard;
|
||||
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.PhraseQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.QueryCachingPolicy;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class ElasticsearchQueryCachingPolicyTests extends ESTestCase {
|
||||
|
||||
public void testDoesNotCacheTermQueries() throws IOException {
|
||||
QueryCachingPolicy policy = QueryCachingPolicy.ALWAYS_CACHE;
|
||||
assertTrue(policy.shouldCache(new TermQuery(new Term("foo", "bar"))));
|
||||
assertTrue(policy.shouldCache(new PhraseQuery("foo", "bar", "baz")));
|
||||
policy = new ElasticsearchQueryCachingPolicy(policy);
|
||||
assertFalse(policy.shouldCache(new TermQuery(new Term("foo", "bar"))));
|
||||
assertTrue(policy.shouldCache(new PhraseQuery("foo", "bar", "baz")));
|
||||
}
|
||||
|
||||
public void testDoesNotPutTermQueriesIntoTheHistory() {
|
||||
boolean[] used = new boolean[1];
|
||||
QueryCachingPolicy policy = new QueryCachingPolicy() {
|
||||
@Override
|
||||
public boolean shouldCache(Query query) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
@Override
|
||||
public void onUse(Query query) {
|
||||
used[0] = true;
|
||||
}
|
||||
};
|
||||
policy = new ElasticsearchQueryCachingPolicy(policy);
|
||||
policy.onUse(new TermQuery(new Term("foo", "bar")));
|
||||
assertFalse(used[0]);
|
||||
policy.onUse(new PhraseQuery("foo", "bar", "baz"));
|
||||
assertTrue(used[0]);
|
||||
}
|
||||
|
||||
}
|
@ -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.rest.action.admin.cluster;
|
||||
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.rest.FakeRestRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.object.HasToString.hasToString;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class RestNodesStatsActionTests extends ESTestCase {
|
||||
|
||||
private RestNodesStatsAction action;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
action = new RestNodesStatsAction(Settings.EMPTY, new RestController(Settings.EMPTY, Collections.emptySet()));
|
||||
}
|
||||
|
||||
public void testUnrecognizedMetric() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
final String metric = randomAsciiOfLength(64);
|
||||
params.put("metric", metric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(e, hasToString(containsString("request [/_nodes/stats] contains unrecognized metric: [" + metric + "]")));
|
||||
}
|
||||
|
||||
public void testUnrecognizedMetricDidYouMean() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
params.put("metric", "os,transprot,unrecognized");
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(
|
||||
e,
|
||||
hasToString(
|
||||
containsString(
|
||||
"request [/_nodes/stats] contains unrecognized metrics: [transprot] -> did you mean [transport]?, [unrecognized]")));
|
||||
}
|
||||
|
||||
public void testAllRequestWithOtherMetrics() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
final String metric = randomSubsetOf(1, RestNodesStatsAction.METRICS.keySet()).get(0);
|
||||
params.put("metric", "_all," + metric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(e, hasToString(containsString("request [/_nodes/stats] contains _all and individual metrics [_all," + metric + "]")));
|
||||
}
|
||||
|
||||
public void testUnrecognizedIndexMetric() {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
params.put("metric", "indices");
|
||||
final String indexMetric = randomAsciiOfLength(64);
|
||||
params.put("index_metric", indexMetric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(e, hasToString(containsString("request [/_nodes/stats] contains unrecognized index metric: [" + indexMetric + "]")));
|
||||
}
|
||||
|
||||
public void testUnrecognizedIndexMetricDidYouMean() {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
params.put("metric", "indices");
|
||||
params.put("index_metric", "indexing,stroe,unrecognized");
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(
|
||||
e,
|
||||
hasToString(
|
||||
containsString(
|
||||
"request [/_nodes/stats] contains unrecognized index metrics: [stroe] -> did you mean [store]?, [unrecognized]")));
|
||||
}
|
||||
|
||||
public void testIndexMetricsRequestWithoutIndicesMetric() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
final Set<String> metrics = new HashSet<>(RestNodesStatsAction.METRICS.keySet());
|
||||
metrics.remove("indices");
|
||||
params.put("metric", randomSubsetOf(1, metrics).get(0));
|
||||
final String indexMetric = randomSubsetOf(1, RestNodesStatsAction.FLAGS.keySet()).get(0);
|
||||
params.put("index_metric", indexMetric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(
|
||||
e,
|
||||
hasToString(
|
||||
containsString("request [/_nodes/stats] contains index metrics [" + indexMetric + "] but indices stats not requested")));
|
||||
}
|
||||
|
||||
public void testIndexMetricsRequestOnAllRequest() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
params.put("metric", "_all");
|
||||
final String indexMetric = randomSubsetOf(1, RestNodesStatsAction.FLAGS.keySet()).get(0);
|
||||
params.put("index_metric", indexMetric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_nodes/stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(
|
||||
e,
|
||||
hasToString(
|
||||
containsString("request [/_nodes/stats] contains index metrics [" + indexMetric + "] but all stats requested")));
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,83 @@
|
||||
/*
|
||||
* 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.rest.action.admin.indices;
|
||||
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.rest.FakeRestRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.object.HasToString.hasToString;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class RestIndicesStatsActionTests extends ESTestCase {
|
||||
|
||||
private RestIndicesStatsAction action;
|
||||
|
||||
@Override
|
||||
public void setUp() throws Exception {
|
||||
super.setUp();
|
||||
action = new RestIndicesStatsAction(Settings.EMPTY, new RestController(Settings.EMPTY, Collections.emptySet()));
|
||||
}
|
||||
|
||||
public void testUnrecognizedMetric() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
final String metric = randomAsciiOfLength(64);
|
||||
params.put("metric", metric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(e, hasToString(containsString("request [/_stats] contains unrecognized metric: [" + metric + "]")));
|
||||
}
|
||||
|
||||
public void testUnrecognizedMetricDidYouMean() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
params.put("metric", "request_cache,fieldata,unrecognized");
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(
|
||||
e,
|
||||
hasToString(
|
||||
containsString(
|
||||
"request [/_stats] contains unrecognized metrics: [fieldata] -> did you mean [fielddata]?, [unrecognized]")));
|
||||
}
|
||||
|
||||
public void testAllRequestWithOtherMetrics() throws IOException {
|
||||
final HashMap<String, String> params = new HashMap<>();
|
||||
final String metric = randomSubsetOf(1, RestIndicesStatsAction.METRICS.keySet()).get(0);
|
||||
params.put("metric", "_all," + metric);
|
||||
final RestRequest request = new FakeRestRequest.Builder().withPath("/_stats").withParams(params).build();
|
||||
final IllegalArgumentException e = expectThrows(
|
||||
IllegalArgumentException.class,
|
||||
() -> action.prepareRequest(request, mock(NodeClient.class)));
|
||||
assertThat(e, hasToString(containsString("request [/_stats] contains _all and individual metrics [_all," + metric + "]")));
|
||||
}
|
||||
|
||||
}
|
@ -247,6 +247,14 @@ public class QueryStringIT extends ESIntegTestCase {
|
||||
assertHitCount(resp, 1L);
|
||||
}
|
||||
|
||||
public void testBooleanStrictQuery() throws Exception {
|
||||
Exception e = expectThrows(Exception.class, () ->
|
||||
client().prepareSearch("test").setQuery(
|
||||
queryStringQuery("foo").field("f_bool")).get());
|
||||
assertThat(ExceptionsHelper.detailedMessage(e),
|
||||
containsString("Can't parse boolean value [foo], expected [true] or [false]"));
|
||||
}
|
||||
|
||||
private void assertHits(SearchHits hits, String... ids) {
|
||||
assertThat(hits.totalHits(), equalTo((long) ids.length));
|
||||
Set<String> hitIds = new HashSet<>();
|
||||
|
@ -1904,7 +1904,6 @@ public class SearchQueryIT extends ESIntegTestCase {
|
||||
assertHitCount(client().prepareSearch("test").setSize(0).setQuery(rangeQuery("field").lte(-999999999999L)).get(), 3);
|
||||
}
|
||||
|
||||
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/21501")
|
||||
public void testRangeQueryWithTimeZone() throws Exception {
|
||||
assertAcked(prepareCreate("test")
|
||||
.addMapping("type1", "date", "type=date", "num", "type=integer"));
|
||||
|
BIN
core/src/test/resources/indices/bwc/index-5.0.1.zip
Normal file
BIN
core/src/test/resources/indices/bwc/index-5.0.1.zip
Normal file
Binary file not shown.
BIN
core/src/test/resources/indices/bwc/repo-5.0.1.zip
Normal file
BIN
core/src/test/resources/indices/bwc/repo-5.0.1.zip
Normal file
Binary file not shown.
1
distribution/licenses/joda-time-2.9.5.jar.sha1
Normal file
1
distribution/licenses/joda-time-2.9.5.jar.sha1
Normal file
@ -0,0 +1 @@
|
||||
5f01da7306363fad2028b916f3eab926262de928
|
@ -196,6 +196,17 @@ second document falls into the bucket for 1 October 2015:
|
||||
<1> The `key_as_string` value represents midnight on each day
|
||||
in the specified time zone.
|
||||
|
||||
WARNING: When using time zones that follow DST (daylight savings time) changes,
|
||||
buckets close to the moment when those changes happen can have slightly different
|
||||
sizes than would be expected from the used `interval`.
|
||||
For example, consider a DST start in the `CET` time zone: on 27 March 2016 at 2am,
|
||||
clocks were turned forward 1 hour to 3am local time. When using `day` as `interval`,
|
||||
the bucket covering that day will only hold data for 23 hours instead of the usual
|
||||
24 hours for other buckets. The same is true for shorter intervals like e.g. 12h.
|
||||
Here, we will have only a 11h bucket on the morning of 27 March when the DST shift
|
||||
happens.
|
||||
|
||||
|
||||
==== Offset
|
||||
|
||||
The `offset` parameter is used to change the start value of each bucket by the
|
||||
|
@ -65,12 +65,11 @@ of `indices`, `os`, `process`, `jvm`, `transport`, `http`,
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
# return indices and os
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/os'
|
||||
# return just indices
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/indices'
|
||||
# return just os and process
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/os,process'
|
||||
# specific type endpoint
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/process'
|
||||
# return just process for node with IP address 10.0.0.1
|
||||
curl -XGET 'http://localhost:9200/_nodes/10.0.0.1/stats/process'
|
||||
--------------------------------------------------
|
||||
|
||||
@ -280,27 +279,45 @@ the current running process:
|
||||
`process.mem.total_virtual_in_bytes`::
|
||||
Size in bytes of virtual memory that is guaranteed to be available to the running process
|
||||
|
||||
|
||||
[float]
|
||||
[[field-data]]
|
||||
=== Field data statistics
|
||||
[[node-indices-stats]]
|
||||
=== Indices statistics
|
||||
|
||||
You can get information about field data memory usage on node
|
||||
level or on index level.
|
||||
You can get information about indices stats on node level or on index level.
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
# Node Stats
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/indices/?fields=field1,field2&pretty'
|
||||
# Node level
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/indices/fielddata?fields=field1,field2&pretty'
|
||||
|
||||
# Indices Stat
|
||||
# Index level
|
||||
curl -XGET 'http://localhost:9200/_stats/fielddata/?fields=field1,field2&pretty'
|
||||
|
||||
# You can use wildcards for field names
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/indices/fielddata?fields=field*&pretty'
|
||||
curl -XGET 'http://localhost:9200/_stats/fielddata/?fields=field*&pretty'
|
||||
curl -XGET 'http://localhost:9200/_nodes/stats/indices/?fields=field*&pretty'
|
||||
--------------------------------------------------
|
||||
|
||||
Supported metrics are:
|
||||
|
||||
* `completion`
|
||||
* `docs`
|
||||
* `fielddata`
|
||||
* `flush`
|
||||
* `get`
|
||||
* `indexing`
|
||||
* `merge`
|
||||
* `query_cache`
|
||||
* `recovery`
|
||||
* `refresh`
|
||||
* `request_cache`
|
||||
* `search`
|
||||
* `segments`
|
||||
* `store`
|
||||
* `suggest`
|
||||
* `translog`
|
||||
* `warmer`
|
||||
|
||||
[float]
|
||||
[[search-groups]]
|
||||
=== Search groups
|
||||
|
@ -70,8 +70,7 @@ section for more information on mapping definitions.
|
||||
Automatic index creation can be disabled by setting
|
||||
`action.auto_create_index` to `false` in the config file of all nodes.
|
||||
Automatic mapping creation can be disabled by setting
|
||||
`index.mapper.dynamic` to `false` in the config files of all nodes (or
|
||||
on the specific index settings).
|
||||
`index.mapper.dynamic` to `false` per-index as an index setting.
|
||||
|
||||
Automatic index creation can include a pattern based white/black list,
|
||||
for example, set `action.auto_create_index` to `+aaa*,-bbb*,+ccc*,-*` (+
|
||||
|
@ -523,7 +523,7 @@ instead of deleting all documents with the Delete By Query API.
|
||||
|
||||
=== Batch Processing
|
||||
|
||||
In addition to being able to index, update, and delete individual documents, Elasticsearch also provides the ability to perform any of the above operations in batches using the <<docs-bulk,`_bulk` API>>. This functionality is important in that it provides a very efficient mechanism to do multiple operations as fast as possible with as little network roundtrips as possible.
|
||||
In addition to being able to index, update, and delete individual documents, Elasticsearch also provides the ability to perform any of the above operations in batches using the <<docs-bulk,`_bulk` API>>. This functionality is important in that it provides a very efficient mechanism to do multiple operations as fast as possible with as few network roundtrips as possible.
|
||||
|
||||
As a quick example, the following call indexes two documents (ID 1 - John Doe and ID 2 - Jane Doe) in one bulk operation:
|
||||
|
||||
|
@ -74,7 +74,7 @@ the <<indices-stats,indices stats>> API:
|
||||
|
||||
[source,sh]
|
||||
--------------------------------------------------
|
||||
GET twitter/_stats/commit?level=shards
|
||||
GET twitter/_stats?level=shards
|
||||
--------------------------------------------------
|
||||
// CONSOLE
|
||||
// TEST[s/^/PUT twitter\n/]
|
||||
|
@ -932,14 +932,14 @@ and the result:
|
||||
"date1" : "2016-04-25T12:02:01.789Z"
|
||||
},
|
||||
"_ingest" : {
|
||||
"timestamp" : "2016-08-11T12:00:01.222Z"
|
||||
"timestamp" : "2016-11-08T19:43:03.850+0000"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TESTRESPONSE[s/2016-08-11T12:00:01.222Z/$body.docs.0.doc._ingest.timestamp/]
|
||||
// TESTRESPONSE[s/2016-11-08T19:43:03.850\+0000/$body.docs.0.doc._ingest.timestamp/]
|
||||
|
||||
The above example shows that `_index` was set to `<myindex-{2016-04-25||/M{yyyy-MM-dd|UTC}}>`. Elasticsearch
|
||||
understands this to mean `2016-04-01` as is explained in the <<date-math-index-names, date math index name documentation>>
|
||||
@ -1279,6 +1279,139 @@ Here is an example of a pipeline specifying custom pattern definitions:
|
||||
}
|
||||
--------------------------------------------------
|
||||
|
||||
[[trace-match]]
|
||||
==== Providing Multiple Match Patterns
|
||||
|
||||
Sometimes one pattern is not enough to capture the potential structure of a field. Let's assume we
|
||||
want to match all messages that contain your favorite pet breeds of either cats or dogs. One way to accomplish
|
||||
this is to provide two distinct patterns that can be matched, instead of one really complicated expression capturing
|
||||
the same `or` behavior.
|
||||
|
||||
Here is an example of such a configuration executed against the simulate API:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
POST _ingest/pipeline/_simulate
|
||||
{
|
||||
"pipeline": {
|
||||
"description" : "parse multiple patterns",
|
||||
"processors": [
|
||||
{
|
||||
"grok": {
|
||||
"field": "message",
|
||||
"patterns": ["%{FAVORITE_DOG:pet}", "%{FAVORITE_CAT:pet}"],
|
||||
"pattern_definitions" : {
|
||||
"FAVORITE_DOG" : "beagle",
|
||||
"FAVORITE_CAT" : "burmese"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"docs":[
|
||||
{
|
||||
"_source": {
|
||||
"message": "I love burmese cats!"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// CONSOLE
|
||||
|
||||
response:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"docs": [
|
||||
{
|
||||
"doc": {
|
||||
"_type": "_type",
|
||||
"_index": "_index",
|
||||
"_id": "_id",
|
||||
"_source": {
|
||||
"message": "I love burmese cats!",
|
||||
"pet": "burmese"
|
||||
},
|
||||
"_ingest": {
|
||||
"timestamp": "2016-11-08T19:43:03.850+0000"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TESTRESPONSE[s/2016-11-08T19:43:03.850\+0000/$body.docs.0.doc._ingest.timestamp/]
|
||||
|
||||
Both patterns will set the field `pet` with the appropriate match, but what if we want to trace which of our
|
||||
patterns matched and populated our fields? We can do this with the `trace_match` parameter. Here is the output of
|
||||
that same pipeline, but with `"trace_match": true` configured:
|
||||
|
||||
////
|
||||
Hidden setup for example:
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
POST _ingest/pipeline/_simulate
|
||||
{
|
||||
"pipeline": {
|
||||
"description" : "parse multiple patterns",
|
||||
"processors": [
|
||||
{
|
||||
"grok": {
|
||||
"field": "message",
|
||||
"patterns": ["%{FAVORITE_DOG:pet}", "%{FAVORITE_CAT:pet}"],
|
||||
"trace_match": true,
|
||||
"pattern_definitions" : {
|
||||
"FAVORITE_DOG" : "beagle",
|
||||
"FAVORITE_CAT" : "burmese"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"docs":[
|
||||
{
|
||||
"_source": {
|
||||
"message": "I love burmese cats!"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// CONSOLE
|
||||
////
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"docs": [
|
||||
{
|
||||
"doc": {
|
||||
"_type": "_type",
|
||||
"_index": "_index",
|
||||
"_id": "_id",
|
||||
"_source": {
|
||||
"message": "I love burmese cats!",
|
||||
"pet": "burmese"
|
||||
},
|
||||
"_ingest": {
|
||||
"_grok_match_index": "1",
|
||||
"timestamp": "2016-11-08T19:43:03.850+0000"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TESTRESPONSE[s/2016-11-08T19:43:03.850\+0000/$body.docs.0.doc._ingest.timestamp/]
|
||||
|
||||
In the above response, you can see that the index of the pattern that matched was `"1"`. This is to say that it was the
|
||||
second (index starts at zero) pattern in `patterns` to match.
|
||||
|
||||
This trace metadata enables debugging which of the patterns matched. This information is stored in the ingest
|
||||
metadata and will not be indexed.
|
||||
|
||||
[[gsub-processor]]
|
||||
=== Gsub Processor
|
||||
Converts a string field by applying a regular expression and a replacement.
|
||||
@ -1495,9 +1628,10 @@ Splits a field into an array using a separator character. Only works on string f
|
||||
.Split Options
|
||||
[options="header"]
|
||||
|======
|
||||
| Name | Required | Default | Description
|
||||
| `field` | yes | - | The field to split
|
||||
| `separator` | yes | - | A regex which matches the separator, eg `,` or `\s+`
|
||||
| Name | Required | Default | Description
|
||||
| `field` | yes | - | The field to split
|
||||
| `separator` | yes | - | A regex which matches the separator, eg `,` or `\s+`
|
||||
| `ignore_missing` | no | `false` | If `true` and `field` does not exist, the processor quietly exits without modifying the document
|
||||
|======
|
||||
|
||||
[source,js]
|
||||
|
@ -32,21 +32,21 @@ PUT my_index
|
||||
|
||||
POST my_index/my_type/1
|
||||
{
|
||||
"is_published": true <1>
|
||||
"is_published": 1 <1>
|
||||
}
|
||||
|
||||
GET my_index/_search
|
||||
{
|
||||
"query": {
|
||||
"term": {
|
||||
"is_published": 1 <2>
|
||||
"is_published": true <2>
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// CONSOLE
|
||||
<1> Indexing a document with a JSON `true`.
|
||||
<2> Querying for the document with `1`, which is interpreted as `true`.
|
||||
<1> Indexing a document with `1`, which is interpreted as `true`.
|
||||
<2> Searching for documents with a JSON `true`.
|
||||
|
||||
Aggregations like the <<search-aggregations-bucket-terms-aggregation,`terms`
|
||||
aggregation>> use `1` and `0` for the `key`, and the strings `"true"` and
|
||||
|
@ -5,3 +5,7 @@
|
||||
|
||||
* The `collect_payloads` parameter of the `span_near` query has been removed. Payloads will be
|
||||
loaded when needed.
|
||||
|
||||
* Queries on boolean fields now strictly parse boolean-like values. This means
|
||||
only the strings `"true"` and `"false"` will be parsed into their boolean
|
||||
counterparts. Other strings will cause an error to be thrown.
|
||||
|
@ -109,7 +109,8 @@ which field to search on. It defaults to `_all` field.
|
||||
|
||||
If the `_all` field is disabled, the `query_string` query will automatically
|
||||
attempt to determine the existing fields in the index's mapping that are
|
||||
queryable, and perform the search on those fields.
|
||||
queryable, and perform the search on those fields. Note that this will not
|
||||
include nested documents, use a nested query to search those documents.
|
||||
|
||||
[float]
|
||||
==== Multi Field
|
||||
|
@ -34,7 +34,7 @@ abstract class AbstractStringProcessor extends AbstractProcessor {
|
||||
private final String field;
|
||||
private final boolean ignoreMissing;
|
||||
|
||||
protected AbstractStringProcessor(String tag, String field, boolean ignoreMissing) {
|
||||
AbstractStringProcessor(String tag, String field, boolean ignoreMissing) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.ignoreMissing = ignoreMissing;
|
||||
@ -50,16 +50,8 @@ abstract class AbstractStringProcessor extends AbstractProcessor {
|
||||
|
||||
@Override
|
||||
public final void execute(IngestDocument document) {
|
||||
String val;
|
||||
String val = document.getFieldValue(field, String.class, ignoreMissing);
|
||||
|
||||
try {
|
||||
val = document.getFieldValue(field, String.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
if (ignoreMissing && document.hasField(field) != true) {
|
||||
return;
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
if (val == null && ignoreMissing) {
|
||||
return;
|
||||
} else if (val == null) {
|
||||
@ -72,7 +64,7 @@ abstract class AbstractStringProcessor extends AbstractProcessor {
|
||||
protected abstract String process(String value);
|
||||
|
||||
abstract static class Factory implements Processor.Factory {
|
||||
protected final String processorType;
|
||||
final String processorType;
|
||||
|
||||
protected Factory(String processorType) {
|
||||
this.processorType = processorType;
|
||||
|
@ -142,18 +142,9 @@ public final class ConvertProcessor extends AbstractProcessor {
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
Object oldValue = null;
|
||||
Object oldValue = document.getFieldValue(field, Object.class, ignoreMissing);
|
||||
Object newValue;
|
||||
|
||||
try {
|
||||
oldValue = document.getFieldValue(field, Object.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
if (ignoreMissing) {
|
||||
return;
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
|
||||
if (oldValue == null && ignoreMissing) {
|
||||
return;
|
||||
} else if (oldValue == null) {
|
||||
|
@ -37,6 +37,7 @@ public final class GrokProcessor extends AbstractProcessor {
|
||||
private static final String PATTERN_MATCH_KEY = "_ingest._grok_match_index";
|
||||
|
||||
private final String matchField;
|
||||
private final List<String> matchPatterns;
|
||||
private final Grok grok;
|
||||
private final boolean traceMatch;
|
||||
private final boolean ignoreMissing;
|
||||
@ -45,6 +46,7 @@ public final class GrokProcessor extends AbstractProcessor {
|
||||
boolean traceMatch, boolean ignoreMissing) {
|
||||
super(tag);
|
||||
this.matchField = matchField;
|
||||
this.matchPatterns = matchPatterns;
|
||||
this.grok = new Grok(patternBank, combinePatterns(matchPatterns, traceMatch));
|
||||
this.traceMatch = traceMatch;
|
||||
this.ignoreMissing = ignoreMissing;
|
||||
@ -52,16 +54,7 @@ public final class GrokProcessor extends AbstractProcessor {
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument ingestDocument) throws Exception {
|
||||
String fieldValue;
|
||||
|
||||
try {
|
||||
fieldValue = ingestDocument.getFieldValue(matchField, String.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
if (ignoreMissing && ingestDocument.hasField(matchField) != true) {
|
||||
return;
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
String fieldValue = ingestDocument.getFieldValue(matchField, String.class, ignoreMissing);
|
||||
|
||||
if (fieldValue == null && ignoreMissing) {
|
||||
return;
|
||||
@ -79,11 +72,15 @@ public final class GrokProcessor extends AbstractProcessor {
|
||||
.forEach((e) -> ingestDocument.setFieldValue(e.getKey(), e.getValue()));
|
||||
|
||||
if (traceMatch) {
|
||||
@SuppressWarnings("unchecked")
|
||||
HashMap<String, String> matchMap = (HashMap<String, String>) ingestDocument.getFieldValue(PATTERN_MATCH_KEY, Object.class);
|
||||
matchMap.keySet().stream().findFirst().ifPresent((index) -> {
|
||||
ingestDocument.setFieldValue(PATTERN_MATCH_KEY, index);
|
||||
});
|
||||
if (matchPatterns.size() > 1) {
|
||||
@SuppressWarnings("unchecked")
|
||||
HashMap<String, String> matchMap = (HashMap<String, String>) ingestDocument.getFieldValue(PATTERN_MATCH_KEY, Object.class);
|
||||
matchMap.keySet().stream().findFirst().ifPresent((index) -> {
|
||||
ingestDocument.setFieldValue(PATTERN_MATCH_KEY, index);
|
||||
});
|
||||
} else {
|
||||
ingestDocument.setFieldValue(PATTERN_MATCH_KEY, "0");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -104,6 +101,10 @@ public final class GrokProcessor extends AbstractProcessor {
|
||||
return matchField;
|
||||
}
|
||||
|
||||
List<String> getMatchPatterns() {
|
||||
return matchPatterns;
|
||||
}
|
||||
|
||||
static String combinePatterns(List<String> patterns, boolean traceMatch) {
|
||||
String combinedPattern;
|
||||
if (patterns.size() > 1) {
|
||||
|
@ -40,11 +40,13 @@ public final class SplitProcessor extends AbstractProcessor {
|
||||
|
||||
private final String field;
|
||||
private final String separator;
|
||||
private final boolean ignoreMissing;
|
||||
|
||||
SplitProcessor(String tag, String field, String separator) {
|
||||
SplitProcessor(String tag, String field, String separator, boolean ignoreMissing) {
|
||||
super(tag);
|
||||
this.field = field;
|
||||
this.separator = separator;
|
||||
this.ignoreMissing = ignoreMissing;
|
||||
}
|
||||
|
||||
String getField() {
|
||||
@ -55,12 +57,20 @@ public final class SplitProcessor extends AbstractProcessor {
|
||||
return separator;
|
||||
}
|
||||
|
||||
boolean isIgnoreMissing() {
|
||||
return ignoreMissing;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(IngestDocument document) {
|
||||
String oldVal = document.getFieldValue(field, String.class);
|
||||
if (oldVal == null) {
|
||||
String oldVal = document.getFieldValue(field, String.class, ignoreMissing);
|
||||
|
||||
if (oldVal == null && ignoreMissing) {
|
||||
return;
|
||||
} else if (oldVal == null) {
|
||||
throw new IllegalArgumentException("field [" + field + "] is null, cannot split.");
|
||||
}
|
||||
|
||||
String[] strings = oldVal.split(separator);
|
||||
List<String> splitList = new ArrayList<>(strings.length);
|
||||
Collections.addAll(splitList, strings);
|
||||
@ -77,7 +87,9 @@ public final class SplitProcessor extends AbstractProcessor {
|
||||
public SplitProcessor create(Map<String, Processor.Factory> registry, String processorTag,
|
||||
Map<String, Object> config) throws Exception {
|
||||
String field = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "field");
|
||||
return new SplitProcessor(processorTag, field, ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "separator"));
|
||||
boolean ignoreMissing = ConfigurationUtils.readBooleanProperty(TYPE, processorTag, config, "ignore_missing", false);
|
||||
return new SplitProcessor(processorTag, field,
|
||||
ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "separator"), ignoreMissing);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -158,6 +158,19 @@ public class GrokProcessorTests extends ESTestCase {
|
||||
assertThat(doc.getFieldValue("_ingest._grok_match_index", String.class), equalTo("1"));
|
||||
}
|
||||
|
||||
public void testTraceWithOnePattern() throws Exception {
|
||||
String fieldName = RandomDocumentPicks.randomFieldName(random());
|
||||
IngestDocument doc = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>());
|
||||
doc.setFieldValue(fieldName, "first1");
|
||||
Map<String, String> patternBank = new HashMap<>();
|
||||
patternBank.put("ONE", "1");
|
||||
GrokProcessor processor = new GrokProcessor(randomAsciiOfLength(10), patternBank,
|
||||
Arrays.asList("%{ONE:one}"), fieldName, true, false);
|
||||
processor.execute(doc);
|
||||
assertThat(doc.hasField("one"), equalTo(true));
|
||||
assertThat(doc.getFieldValue("_ingest._grok_match_index", String.class), equalTo("0"));
|
||||
}
|
||||
|
||||
public void testCombinedPatterns() {
|
||||
String combined;
|
||||
combined = GrokProcessor.combinePatterns(Arrays.asList(""), false);
|
||||
|
@ -39,6 +39,7 @@ public class SplitProcessorFactoryTests extends ESTestCase {
|
||||
assertThat(splitProcessor.getTag(), equalTo(processorTag));
|
||||
assertThat(splitProcessor.getField(), equalTo("field1"));
|
||||
assertThat(splitProcessor.getSeparator(), equalTo("\\."));
|
||||
assertFalse(splitProcessor.isIgnoreMissing());
|
||||
}
|
||||
|
||||
public void testCreateNoFieldPresent() throws Exception {
|
||||
|
@ -30,6 +30,7 @@ import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.ingest.IngestDocumentMatcher.assertIngestDocument;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
@ -38,7 +39,7 @@ public class SplitProcessorTests extends ESTestCase {
|
||||
public void testSplit() throws Exception {
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random());
|
||||
String fieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, "127.0.0.1");
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.");
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.", false);
|
||||
processor.execute(ingestDocument);
|
||||
assertThat(ingestDocument.getFieldValue(fieldName, List.class), equalTo(Arrays.asList("127", "0", "0", "1")));
|
||||
}
|
||||
@ -46,7 +47,7 @@ public class SplitProcessorTests extends ESTestCase {
|
||||
public void testSplitFieldNotFound() throws Exception {
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>());
|
||||
String fieldName = RandomDocumentPicks.randomFieldName(random());
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.");
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.", false);
|
||||
try {
|
||||
processor.execute(ingestDocument);
|
||||
fail("split processor should have failed");
|
||||
@ -56,8 +57,9 @@ public class SplitProcessorTests extends ESTestCase {
|
||||
}
|
||||
|
||||
public void testSplitNullValue() throws Exception {
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.singletonMap("field", null));
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), "field", "\\.");
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(),
|
||||
Collections.singletonMap("field", null));
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), "field", "\\.", false);
|
||||
try {
|
||||
processor.execute(ingestDocument);
|
||||
fail("split processor should have failed");
|
||||
@ -66,11 +68,29 @@ public class SplitProcessorTests extends ESTestCase {
|
||||
}
|
||||
}
|
||||
|
||||
public void testSplitNullValueWithIgnoreMissing() throws Exception {
|
||||
String fieldName = RandomDocumentPicks.randomFieldName(random());
|
||||
IngestDocument originalIngestDocument = RandomDocumentPicks.randomIngestDocument(random(),
|
||||
Collections.singletonMap(fieldName, null));
|
||||
IngestDocument ingestDocument = new IngestDocument(originalIngestDocument);
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.", true);
|
||||
processor.execute(ingestDocument);
|
||||
assertIngestDocument(originalIngestDocument, ingestDocument);
|
||||
}
|
||||
|
||||
public void testSplitNonExistentWithIgnoreMissing() throws Exception {
|
||||
IngestDocument originalIngestDocument = RandomDocumentPicks.randomIngestDocument(random(), Collections.emptyMap());
|
||||
IngestDocument ingestDocument = new IngestDocument(originalIngestDocument);
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), "field", "\\.", true);
|
||||
processor.execute(ingestDocument);
|
||||
assertIngestDocument(originalIngestDocument, ingestDocument);
|
||||
}
|
||||
|
||||
public void testSplitNonStringValue() throws Exception {
|
||||
IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), new HashMap<>());
|
||||
String fieldName = RandomDocumentPicks.randomFieldName(random());
|
||||
ingestDocument.setFieldValue(fieldName, randomInt());
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.");
|
||||
Processor processor = new SplitProcessor(randomAsciiOfLength(10), fieldName, "\\.", false);
|
||||
try {
|
||||
processor.execute(ingestDocument);
|
||||
fail("split processor should have failed");
|
||||
|
@ -1 +1,2 @@
|
||||
5.0.0
|
||||
5.0.1
|
||||
|
@ -1,5 +1,8 @@
|
||||
---
|
||||
"Shrink index via API":
|
||||
- skip:
|
||||
features: always
|
||||
reason: Fails consistently for Nik and sometimes for Jenkins. Skip until we can get it passing consistently.
|
||||
# creates an index with one document solely allocated on the master node
|
||||
# and shrinks it into a new index with a single shard
|
||||
# we don't do the relocation to a single node after the index is created
|
||||
|
@ -100,3 +100,16 @@ setup:
|
||||
- is_false: indices.test1
|
||||
- is_true: indices.test2
|
||||
|
||||
---
|
||||
"Indices stats unrecognized parameter":
|
||||
- skip:
|
||||
version: " - 5.0.99"
|
||||
reason: strict stats handling does not exist in 5.0
|
||||
- do:
|
||||
indices.stats:
|
||||
metric: [ fieldata ]
|
||||
ignore: 400
|
||||
|
||||
- match: { status: 400 }
|
||||
- match: { error.type: illegal_argument_exception }
|
||||
- match: { error.reason: "request [/_stats/fieldata] contains unrecognized metric: [fieldata] -> did you mean [fielddata]?" }
|
||||
|
@ -20,3 +20,17 @@
|
||||
level: "indices"
|
||||
|
||||
- is_true: nodes.$master.indices.indices
|
||||
|
||||
---
|
||||
"Nodes stats unrecognized parameter":
|
||||
- skip:
|
||||
version: " - 5.0.99"
|
||||
reason: strict stats handling does not exist in 5.0
|
||||
- do:
|
||||
nodes.stats:
|
||||
metric: [ transprot ]
|
||||
ignore: 400
|
||||
|
||||
- match: { status: 400 }
|
||||
- match: { error.type: illegal_argument_exception }
|
||||
- match: { error.reason: "request [/_nodes/stats/transprot] contains unrecognized metric: [transprot] -> did you mean [transport]?" }
|
||||
|
@ -409,6 +409,9 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
||||
if (randomBoolean()) {
|
||||
randomSettingsBuilder.put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), randomBoolean());
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
randomSettingsBuilder.put(IndexModule.INDEX_QUERY_CACHE_TERM_QUERIES_SETTING.getKey(), randomBoolean());
|
||||
}
|
||||
PutIndexTemplateRequestBuilder putTemplate = client().admin().indices()
|
||||
.preparePutTemplate("random_index_template")
|
||||
.setPatterns(Collections.singletonList("*"))
|
||||
|
@ -1070,10 +1070,15 @@ public final class InternalTestCluster extends TestCluster {
|
||||
|
||||
/** ensure a cluster is form with {@link #nodes}.size() nodes. */
|
||||
private void validateClusterFormed() {
|
||||
final int size = nodes.size();
|
||||
String name = randomFrom(random, getNodeNames());
|
||||
logger.trace("validating cluster formed via [{}], expecting [{}]", name, size);
|
||||
final Client client = client(name);
|
||||
validateClusterFormed(name);
|
||||
}
|
||||
|
||||
/** ensure a cluster is form with {@link #nodes}.size() nodes, but do so by using the client of the specified node */
|
||||
private void validateClusterFormed(String viaNode) {
|
||||
final int size = nodes.size();
|
||||
logger.trace("validating cluster formed via [{}], expecting [{}]", viaNode, size);
|
||||
final Client client = client(viaNode);
|
||||
ClusterHealthResponse response = client.admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(size)).get();
|
||||
if (response.isTimedOut()) {
|
||||
logger.warn("failed to wait for a cluster of size [{}], got [{}]", size, response);
|
||||
@ -1485,7 +1490,9 @@ public final class InternalTestCluster extends TestCluster {
|
||||
if (callback.validateClusterForming() || updateMinMaster) {
|
||||
// we have to validate cluster size if updateMinMaster == true, because we need the
|
||||
// second node to join in order to increment min_master_nodes back to 2.
|
||||
validateClusterFormed();
|
||||
// we also have to do via the node that was just restarted as it may be that the master didn't yet process
|
||||
// the fact it left
|
||||
validateClusterFormed(nodeAndClient.name);
|
||||
}
|
||||
if (updateMinMaster) {
|
||||
updateMinMasterNodes(masterNodesCount);
|
||||
|
Loading…
x
Reference in New Issue
Block a user