Merge branch 'master' into feature/aggs-refactoring
This commit is contained in:
commit
9d7d89d04d
|
@ -45,6 +45,7 @@ org.apache.lucene.search.NumericRangeFilter
|
|||
org.apache.lucene.search.PrefixFilter
|
||||
org.apache.lucene.search.QueryWrapperFilter
|
||||
org.apache.lucene.search.join.BitDocIdSetCachingWrapperFilter
|
||||
org.apache.lucene.index.IndexWriter#isLocked(org.apache.lucene.store.Directory)
|
||||
|
||||
java.nio.file.Paths @ Use org.elasticsearch.common.io.PathUtils.get() instead.
|
||||
java.nio.file.FileSystems#getDefault() @ use org.elasticsearch.common.io.PathUtils.getDefaultFileSystem() instead.
|
||||
|
@ -125,4 +126,6 @@ java.util.Collections#EMPTY_MAP
|
|||
java.util.Collections#EMPTY_SET
|
||||
|
||||
java.util.Collections#shuffle(java.util.List) @ Use java.util.Collections#shuffle(java.util.List, java.util.Random) with a reproducible source of randomness
|
||||
java.util.Random#<init>() @ Use org.elasticsearch.common.random.Randomness#create for reproducible sources of randomness
|
||||
@defaultMessage Use org.elasticsearch.common.Randomness#get for reproducible sources of randomness
|
||||
java.util.Random#<init>()
|
||||
java.util.concurrent.ThreadLocalRandom
|
|
@ -102,8 +102,8 @@ if (isEclipse) {
|
|||
}
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-fallthrough,-overrides,-rawtypes,-serial,-try,-unchecked"
|
||||
compileTestJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-fallthrough,-overrides,-rawtypes,-serial,-try,-unchecked"
|
||||
compileJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-rawtypes,-serial,-try,-unchecked"
|
||||
compileTestJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-rawtypes,-serial,-try,-unchecked"
|
||||
|
||||
forbiddenPatterns {
|
||||
exclude '**/*.json'
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.elasticsearch.index.engine.SegmentsStats;
|
|||
import org.elasticsearch.index.fielddata.FieldDataStats;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.get.GetStats;
|
||||
import org.elasticsearch.index.indexing.IndexingStats;
|
||||
import org.elasticsearch.index.shard.IndexingStats;
|
||||
import org.elasticsearch.index.merge.MergeStats;
|
||||
import org.elasticsearch.index.percolator.PercolateStats;
|
||||
import org.elasticsearch.index.recovery.RecoveryStats;
|
||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
|||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.BigArrays;
|
||||
|
@ -60,7 +61,6 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.atomic.AtomicReferenceArray;
|
||||
|
||||
/**
|
||||
|
@ -112,7 +112,7 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
|
|||
@Override
|
||||
protected GroupShardsIterator shards(ClusterState clusterState, ValidateQueryRequest request, String[] concreteIndices) {
|
||||
// Hard-code routing to limit request to a single shard, but still, randomize it...
|
||||
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, Integer.toString(ThreadLocalRandom.current().nextInt(1000)), request.indices());
|
||||
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, Integer.toString(Randomness.get().nextInt(1000)), request.indices());
|
||||
return clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, "_local");
|
||||
}
|
||||
|
||||
|
|
|
@ -410,7 +410,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
|
|||
}
|
||||
case NONE:
|
||||
UpdateResponse updateResponse = translate.action();
|
||||
indexShard.indexingService().noopUpdate(updateRequest.type());
|
||||
indexShard.noopUpdate(updateRequest.type());
|
||||
return new UpdateResult(translate, updateResponse);
|
||||
default:
|
||||
throw new IllegalStateException("Illegal update operation " + translate.operation());
|
||||
|
|
|
@ -269,7 +269,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
|
|||
if (indexServiceOrNull != null) {
|
||||
IndexShard shard = indexService.getShardOrNull(request.shardId());
|
||||
if (shard != null) {
|
||||
shard.indexingService().noopUpdate(request.type());
|
||||
shard.noopUpdate(request.type());
|
||||
}
|
||||
}
|
||||
listener.onResponse(update);
|
||||
|
|
|
@ -67,7 +67,7 @@ import org.elasticsearch.gateway.GatewayAllocator;
|
|||
import org.elasticsearch.gateway.PrimaryShardAllocator;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.engine.EngineConfig;
|
||||
import org.elasticsearch.index.indexing.IndexingSlowLog;
|
||||
import org.elasticsearch.index.IndexingSlowLog;
|
||||
import org.elasticsearch.index.search.stats.SearchSlowLog;
|
||||
import org.elasticsearch.index.settings.IndexDynamicSettings;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
|
|
|
@ -859,10 +859,16 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
if (parser.currentToken() == XContentParser.Token.START_OBJECT) { // on a start object move to next token
|
||||
parser.nextToken();
|
||||
}
|
||||
if (parser.currentToken() != XContentParser.Token.FIELD_NAME) {
|
||||
throw new IllegalArgumentException("expected field name but got a " + parser.currentToken());
|
||||
}
|
||||
Builder builder = new Builder(parser.currentName());
|
||||
|
||||
String currentFieldName = null;
|
||||
XContentParser.Token token = parser.nextToken();
|
||||
if (token != XContentParser.Token.START_OBJECT) {
|
||||
throw new IllegalArgumentException("expected object but got a " + token);
|
||||
}
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
if (token == XContentParser.Token.FIELD_NAME) {
|
||||
currentFieldName = parser.currentName();
|
||||
|
@ -877,6 +883,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
String mappingType = currentFieldName;
|
||||
Map<String, Object> mappingSource = MapBuilder.<String, Object>newMapBuilder().put(mappingType, parser.mapOrdered()).map();
|
||||
builder.putMapping(new MappingMetaData(mappingType, mappingSource));
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected token: " + token);
|
||||
}
|
||||
}
|
||||
} else if ("aliases".equals(currentFieldName)) {
|
||||
|
@ -896,6 +904,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
}
|
||||
}
|
||||
builder.putActiveAllocationIds(Integer.valueOf(shardId), allocationIds);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected token: " + token);
|
||||
}
|
||||
}
|
||||
} else if ("warmers".equals(currentFieldName)) {
|
||||
|
@ -904,6 +914,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
// ignore: warmers have been removed in 3.0 and are
|
||||
// simply ignored when upgrading from 2.x
|
||||
assert Version.CURRENT.major <= 3;
|
||||
parser.skipChildren();
|
||||
} else {
|
||||
// check if its a custom index metadata
|
||||
Custom proto = lookupPrototype(currentFieldName);
|
||||
|
@ -928,13 +939,19 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected field for an array " + currentFieldName);
|
||||
}
|
||||
} else if (token.isValue()) {
|
||||
if ("state".equals(currentFieldName)) {
|
||||
builder.state(State.fromString(parser.text()));
|
||||
} else if ("version".equals(currentFieldName)) {
|
||||
builder.version(parser.longValue());
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected field [" + currentFieldName + "]");
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected token " + token);
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
|
|
|
@ -237,7 +237,7 @@ public class IndexNameExpressionResolver extends AbstractComponent {
|
|||
public String[] filteringAliases(ClusterState state, String index, String... expressions) {
|
||||
// expand the aliases wildcard
|
||||
List<String> resolvedExpressions = expressions != null ? Arrays.asList(expressions) : Collections.<String>emptyList();
|
||||
Context context = new Context(state, IndicesOptions.lenientExpandOpen());
|
||||
Context context = new Context(state, IndicesOptions.lenientExpandOpen(), true);
|
||||
for (ExpressionResolver expressionResolver : expressionResolvers) {
|
||||
resolvedExpressions = expressionResolver.resolve(context, resolvedExpressions);
|
||||
}
|
||||
|
@ -459,17 +459,25 @@ public class IndexNameExpressionResolver extends AbstractComponent {
|
|||
private final ClusterState state;
|
||||
private final IndicesOptions options;
|
||||
private final long startTime;
|
||||
private final boolean preserveAliases;
|
||||
|
||||
Context(ClusterState state, IndicesOptions options) {
|
||||
this.state = state;
|
||||
this.options = options;
|
||||
startTime = System.currentTimeMillis();
|
||||
this(state, options, System.currentTimeMillis());
|
||||
}
|
||||
|
||||
Context(ClusterState state, IndicesOptions options, boolean preserveAliases) {
|
||||
this(state, options, System.currentTimeMillis(), preserveAliases);
|
||||
}
|
||||
|
||||
public Context(ClusterState state, IndicesOptions options, long startTime) {
|
||||
this(state, options, startTime, false);
|
||||
}
|
||||
|
||||
public Context(ClusterState state, IndicesOptions options, long startTime, boolean preserveAliases) {
|
||||
this.state = state;
|
||||
this.options = options;
|
||||
this.startTime = startTime;
|
||||
this.preserveAliases = preserveAliases;
|
||||
}
|
||||
|
||||
public ClusterState getState() {
|
||||
|
@ -483,6 +491,15 @@ public class IndexNameExpressionResolver extends AbstractComponent {
|
|||
public long getStartTime() {
|
||||
return startTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is used to prevent resolving aliases to concrete indices but this also means
|
||||
* that we might return aliases that point to a closed index. This is currently only used
|
||||
* by {@link #filteringAliases(ClusterState, String, String...)} since it's the only one that needs aliases
|
||||
*/
|
||||
boolean isPreserveAliases() {
|
||||
return preserveAliases;
|
||||
}
|
||||
}
|
||||
|
||||
private interface ExpressionResolver {
|
||||
|
@ -531,6 +548,9 @@ public class IndexNameExpressionResolver extends AbstractComponent {
|
|||
}
|
||||
continue;
|
||||
}
|
||||
if (Strings.isEmpty(expression)) {
|
||||
throw new IndexNotFoundException(expression);
|
||||
}
|
||||
boolean add = true;
|
||||
if (expression.charAt(0) == '+') {
|
||||
// if its the first, add empty result set
|
||||
|
@ -612,20 +632,23 @@ public class IndexNameExpressionResolver extends AbstractComponent {
|
|||
.filter(e -> Regex.simpleMatch(pattern, e.getKey()))
|
||||
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
}
|
||||
Set<String> expand = new HashSet<>();
|
||||
for (Map.Entry<String, AliasOrIndex> entry : matches.entrySet()) {
|
||||
AliasOrIndex aliasOrIndex = entry.getValue();
|
||||
if (aliasOrIndex.isAlias() == false) {
|
||||
AliasOrIndex.Index index = (AliasOrIndex.Index) aliasOrIndex;
|
||||
if (excludeState != null && index.getIndex().getState() == excludeState) {
|
||||
continue;
|
||||
if (context.isPreserveAliases() && aliasOrIndex.isAlias()) {
|
||||
expand.add(entry.getKey());
|
||||
} else {
|
||||
for (IndexMetaData meta : aliasOrIndex.getIndices()) {
|
||||
if (excludeState == null || meta.getState() != excludeState) {
|
||||
expand.add(meta.getIndex());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (add) {
|
||||
result.add(entry.getKey());
|
||||
} else {
|
||||
result.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
if (add) {
|
||||
result.addAll(expand);
|
||||
} else {
|
||||
result.removeAll(expand);
|
||||
}
|
||||
|
||||
if (matches.isEmpty() && options.allowNoIndices() == false) {
|
||||
|
|
|
@ -1074,14 +1074,20 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
|
|||
if (token == XContentParser.Token.START_OBJECT) {
|
||||
// move to the field name (meta-data)
|
||||
token = parser.nextToken();
|
||||
if (token != XContentParser.Token.FIELD_NAME) {
|
||||
throw new IllegalArgumentException("Expected a field name but got " + token);
|
||||
}
|
||||
// move to the next object
|
||||
token = parser.nextToken();
|
||||
}
|
||||
currentFieldName = parser.currentName();
|
||||
if (token == null) {
|
||||
// no data...
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
|
||||
if (!"meta-data".equals(parser.currentName())) {
|
||||
throw new IllegalArgumentException("Expected [meta-data] as a field name but got " + currentFieldName);
|
||||
}
|
||||
if (token != XContentParser.Token.START_OBJECT) {
|
||||
throw new IllegalArgumentException("Expected a START_OBJECT but got " + token);
|
||||
}
|
||||
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
|
||||
|
@ -1114,7 +1120,11 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
|
|||
builder.version = parser.longValue();
|
||||
} else if ("cluster_uuid".equals(currentFieldName) || "uuid".equals(currentFieldName)) {
|
||||
builder.clusterUUID = parser.text();
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected field [" + currentFieldName + "]");
|
||||
}
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected token " + token);
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.util.CollectionUtil;
|
|||
import org.elasticsearch.cluster.AbstractDiffable;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -39,7 +40,6 @@ import java.util.HashSet;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
/**
|
||||
* The {@link IndexRoutingTable} represents routing information for a single
|
||||
|
@ -71,7 +71,7 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
|
|||
|
||||
IndexRoutingTable(String index, ImmutableOpenIntMap<IndexShardRoutingTable> shards) {
|
||||
this.index = index;
|
||||
this.shuffler = new RotationShardShuffler(ThreadLocalRandom.current().nextInt());
|
||||
this.shuffler = new RotationShardShuffler(Randomness.get().nextInt());
|
||||
this.shards = shards;
|
||||
List<ShardRouting> allActiveShards = new ArrayList<>();
|
||||
for (IntObjectCursor<IndexShardRoutingTable> cursor : shards) {
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.cluster.routing;
|
|||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
@ -36,7 +37,6 @@ import java.util.LinkedList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
|
||||
|
@ -73,7 +73,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
|
||||
IndexShardRoutingTable(ShardId shardId, List<ShardRouting> shards) {
|
||||
this.shardId = shardId;
|
||||
this.shuffler = new RotationShardShuffler(ThreadLocalRandom.current().nextInt());
|
||||
this.shuffler = new RotationShardShuffler(Randomness.get().nextInt());
|
||||
this.shards = Collections.unmodifiableList(shards);
|
||||
|
||||
ShardRouting primary = null;
|
||||
|
|
|
@ -109,6 +109,7 @@ public final class Randomness {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "ThreadLocalRandom is okay when not running tests")
|
||||
private static Random getWithoutSeed() {
|
||||
assert currentMethod == null && getRandomMethod == null : "running under tests but tried to create non-reproducible random";
|
||||
return ThreadLocalRandom.current();
|
||||
|
|
|
@ -59,6 +59,7 @@ public enum MurmurHash3 {
|
|||
* Note, this hashing function might be used to persist hashes, so if the way hashes are computed
|
||||
* changes for some reason, it needs to be addressed (like in BloomFilter and MurmurHashField).
|
||||
*/
|
||||
@SuppressWarnings("fallthrough") // Intentionally uses fallthrough to implement a well known hashing algorithm
|
||||
public static Hash128 hash128(byte[] key, int offset, int length, long seed, Hash128 hash) {
|
||||
long h1 = seed;
|
||||
long h2 = seed;
|
||||
|
|
|
@ -258,6 +258,12 @@ public final class FactoryProvider2<F> implements InvocationHandler, Provider<F>
|
|||
return o == this || o == factory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
// This way both this and its factory hash to the same spot, making hashCode consistent.
|
||||
return factory.hashCode();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if {@code thrown} can be thrown by {@code invoked} without wrapping.
|
||||
*/
|
||||
|
|
|
@ -39,10 +39,12 @@ public class BytesStreamOutput extends StreamOutput implements BytesStream {
|
|||
protected int count;
|
||||
|
||||
/**
|
||||
* Create a non recycling {@link BytesStreamOutput} with 1 initial page acquired.
|
||||
* Create a non recycling {@link BytesStreamOutput} with an initial capacity of 0.
|
||||
*/
|
||||
public BytesStreamOutput() {
|
||||
this(BigArrays.PAGE_SIZE_IN_BYTES);
|
||||
// since this impl is not recycling anyway, don't bother aligning to
|
||||
// the page size, this will even save memory
|
||||
this(0);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -1,593 +0,0 @@
|
|||
/*
|
||||
* 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.common.math;
|
||||
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class UnboxedMathUtils {
|
||||
|
||||
public static double sin(Short a) {
|
||||
return Math.sin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sin(Integer a) {
|
||||
return Math.sin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sin(Float a) {
|
||||
return Math.sin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sin(Long a) {
|
||||
return Math.sin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sin(Double a) {
|
||||
return Math.sin(a);
|
||||
}
|
||||
|
||||
public static double cos(Short a) {
|
||||
return Math.cos(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cos(Integer a) {
|
||||
return Math.cos(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cos(Float a) {
|
||||
return Math.cos(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cos(Long a) {
|
||||
return Math.cos(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cos(Double a) {
|
||||
return Math.cos(a);
|
||||
}
|
||||
|
||||
public static double tan(Short a) {
|
||||
return Math.tan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double tan(Integer a) {
|
||||
return Math.tan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double tan(Float a) {
|
||||
return Math.tan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double tan(Long a) {
|
||||
return Math.tan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double tan(Double a) {
|
||||
return Math.tan(a);
|
||||
}
|
||||
|
||||
public static double asin(Short a) {
|
||||
return Math.asin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double asin(Integer a) {
|
||||
return Math.asin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double asin(Float a) {
|
||||
return Math.asin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double asin(Long a) {
|
||||
return Math.asin(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double asin(Double a) {
|
||||
return Math.asin(a);
|
||||
}
|
||||
|
||||
public static double acos(Short a) {
|
||||
return Math.acos(a.doubleValue());
|
||||
}
|
||||
|
||||
|
||||
public static double acos(Integer a) {
|
||||
return Math.acos(a.doubleValue());
|
||||
}
|
||||
|
||||
|
||||
public static double acos(Float a) {
|
||||
return Math.acos(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double acos(Long a) {
|
||||
return Math.acos(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double acos(Double a) {
|
||||
return Math.acos(a);
|
||||
}
|
||||
|
||||
public static double atan(Short a) {
|
||||
return Math.atan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan(Integer a) {
|
||||
return Math.atan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan(Float a) {
|
||||
return Math.atan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan(Long a) {
|
||||
return Math.atan(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan(Double a) {
|
||||
return Math.atan(a);
|
||||
}
|
||||
|
||||
public static double toRadians(Short angdeg) {
|
||||
return Math.toRadians(angdeg.doubleValue());
|
||||
}
|
||||
|
||||
public static double toRadians(Integer angdeg) {
|
||||
return Math.toRadians(angdeg.doubleValue());
|
||||
}
|
||||
|
||||
public static double toRadians(Float angdeg) {
|
||||
return Math.toRadians(angdeg.doubleValue());
|
||||
}
|
||||
|
||||
public static double toRadians(Long angdeg) {
|
||||
return Math.toRadians(angdeg.doubleValue());
|
||||
}
|
||||
|
||||
public static double toRadians(Double angdeg) {
|
||||
return Math.toRadians(angdeg);
|
||||
}
|
||||
|
||||
public static double toDegrees(Short angrad) {
|
||||
return Math.toDegrees(angrad.doubleValue());
|
||||
}
|
||||
|
||||
public static double toDegrees(Integer angrad) {
|
||||
return Math.toDegrees(angrad.doubleValue());
|
||||
}
|
||||
|
||||
public static double toDegrees(Float angrad) {
|
||||
return Math.toDegrees(angrad.doubleValue());
|
||||
}
|
||||
|
||||
public static double toDegrees(Long angrad) {
|
||||
return Math.toDegrees(angrad.doubleValue());
|
||||
}
|
||||
|
||||
public static double toDegrees(Double angrad) {
|
||||
return Math.toDegrees(angrad);
|
||||
}
|
||||
|
||||
public static double exp(Short a) {
|
||||
return Math.exp(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double exp(Integer a) {
|
||||
return Math.exp(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double exp(Float a) {
|
||||
return Math.exp(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double exp(Long a) {
|
||||
return Math.exp(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double exp(Double a) {
|
||||
return Math.exp(a);
|
||||
}
|
||||
|
||||
public static double log(Short a) {
|
||||
return Math.log(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log(Integer a) {
|
||||
return Math.log(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log(Float a) {
|
||||
return Math.log(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log(Long a) {
|
||||
return Math.log(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log(Double a) {
|
||||
return Math.log(a);
|
||||
}
|
||||
|
||||
public static double log10(Short a) {
|
||||
return Math.log10(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log10(Integer a) {
|
||||
return Math.log10(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log10(Float a) {
|
||||
return Math.log10(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log10(Long a) {
|
||||
return Math.log10(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double log10(Double a) {
|
||||
return Math.log10(a);
|
||||
}
|
||||
|
||||
public static double sqrt(Short a) {
|
||||
return Math.sqrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sqrt(Integer a) {
|
||||
return Math.sqrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sqrt(Float a) {
|
||||
return Math.sqrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sqrt(Long a) {
|
||||
return Math.sqrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double sqrt(Double a) {
|
||||
return Math.sqrt(a);
|
||||
}
|
||||
|
||||
public static double cbrt(Short a) {
|
||||
return Math.cbrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cbrt(Integer a) {
|
||||
return Math.cbrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cbrt(Float a) {
|
||||
return Math.cbrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cbrt(Long a) {
|
||||
return Math.cbrt(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double cbrt(Double a) {
|
||||
return Math.cbrt(a);
|
||||
}
|
||||
|
||||
public static double IEEEremainder(Short f1, Short f2) {
|
||||
return Math.IEEEremainder(f1.doubleValue(), f2.doubleValue());
|
||||
}
|
||||
|
||||
public static double IEEEremainder(Integer f1, Integer f2) {
|
||||
return Math.IEEEremainder(f1.doubleValue(), f2.doubleValue());
|
||||
}
|
||||
|
||||
public static double IEEEremainder(Float f1, Float f2) {
|
||||
return Math.IEEEremainder(f1.doubleValue(), f2.doubleValue());
|
||||
}
|
||||
|
||||
public static double IEEEremainder(Long f1, Long f2) {
|
||||
return Math.IEEEremainder(f1.doubleValue(), f2.doubleValue());
|
||||
}
|
||||
|
||||
public static double IEEEremainder(Double f1, Double f2) {
|
||||
return Math.IEEEremainder(f1, f2);
|
||||
}
|
||||
|
||||
public static double ceil(Short a) {
|
||||
return Math.ceil(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double ceil(Integer a) {
|
||||
return Math.ceil(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double ceil(Float a) {
|
||||
return Math.ceil(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double ceil(Long a) {
|
||||
return Math.ceil(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double ceil(Double a) {
|
||||
return Math.ceil(a);
|
||||
}
|
||||
|
||||
public static double floor(Short a) {
|
||||
return Math.floor(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double floor(Integer a) {
|
||||
return Math.floor(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double floor(Float a) {
|
||||
return Math.floor(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double floor(Long a) {
|
||||
return Math.floor(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double floor(Double a) {
|
||||
return Math.floor(a);
|
||||
}
|
||||
|
||||
public static double rint(Short a) {
|
||||
return Math.rint(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double rint(Integer a) {
|
||||
return Math.rint(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double rint(Float a) {
|
||||
return Math.rint(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double rint(Long a) {
|
||||
return Math.rint(a.doubleValue());
|
||||
}
|
||||
|
||||
public static double rint(Double a) {
|
||||
return Math.rint(a);
|
||||
}
|
||||
|
||||
public static double atan2(Short y, Short x) {
|
||||
return Math.atan2(y.doubleValue(), x.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan2(Integer y, Integer x) {
|
||||
return Math.atan2(y.doubleValue(), x.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan2(Float y, Float x) {
|
||||
return Math.atan2(y.doubleValue(), x.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan2(Long y, Long x) {
|
||||
return Math.atan2(y.doubleValue(), x.doubleValue());
|
||||
}
|
||||
|
||||
public static double atan2(Double y, Double x) {
|
||||
return Math.atan2(y, x);
|
||||
}
|
||||
|
||||
public static double pow(Short a, Short b) {
|
||||
return Math.pow(a.doubleValue(), b.doubleValue());
|
||||
}
|
||||
|
||||
public static double pow(Integer a, Integer b) {
|
||||
return Math.pow(a.doubleValue(), b.doubleValue());
|
||||
}
|
||||
|
||||
public static double pow(Float a, Float b) {
|
||||
return Math.pow(a.doubleValue(), b.doubleValue());
|
||||
}
|
||||
|
||||
public static double pow(Long a, Long b) {
|
||||
return Math.pow(a.doubleValue(), b.doubleValue());
|
||||
}
|
||||
|
||||
public static double pow(Double a, Double b) {
|
||||
return Math.pow(a, b);
|
||||
}
|
||||
|
||||
public static int round(Short a) {
|
||||
return Math.round(a.floatValue());
|
||||
}
|
||||
|
||||
public static int round(Integer a) {
|
||||
return Math.round(a.floatValue());
|
||||
}
|
||||
|
||||
public static int round(Float a) {
|
||||
return Math.round(a);
|
||||
}
|
||||
|
||||
public static long round(Long a) {
|
||||
return Math.round(a.doubleValue());
|
||||
}
|
||||
|
||||
public static long round(Double a) {
|
||||
return Math.round(a);
|
||||
}
|
||||
|
||||
public static double random() {
|
||||
return ThreadLocalRandom.current().nextDouble();
|
||||
}
|
||||
|
||||
public static double randomDouble() {
|
||||
return ThreadLocalRandom.current().nextDouble();
|
||||
}
|
||||
|
||||
public static double randomFloat() {
|
||||
return ThreadLocalRandom.current().nextFloat();
|
||||
}
|
||||
|
||||
public static double randomInt() {
|
||||
return ThreadLocalRandom.current().nextInt();
|
||||
}
|
||||
|
||||
public static double randomInt(Integer i) {
|
||||
return ThreadLocalRandom.current().nextInt(i);
|
||||
}
|
||||
|
||||
public static double randomLong() {
|
||||
return ThreadLocalRandom.current().nextLong();
|
||||
}
|
||||
|
||||
public static double randomLong(Long l) {
|
||||
return ThreadLocalRandom.current().nextLong(l);
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Math#abs is trappy")
|
||||
public static int abs(Integer a) {
|
||||
return Math.abs(a);
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Math#abs is trappy")
|
||||
public static long abs(Long a) {
|
||||
return Math.abs(a);
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Math#abs is trappy")
|
||||
public static float abs(Float a) {
|
||||
return Math.abs(a);
|
||||
}
|
||||
|
||||
@SuppressForbidden(reason = "Math#abs is trappy")
|
||||
public static double abs(Double a) {
|
||||
return Math.abs(a);
|
||||
}
|
||||
|
||||
public static int max(Integer a, Integer b) {
|
||||
return Math.max(a, b);
|
||||
}
|
||||
|
||||
public static long max(Long a, Long b) {
|
||||
return Math.max(a, b);
|
||||
}
|
||||
|
||||
public static float max(Float a, Float b) {
|
||||
return Math.max(a, b);
|
||||
}
|
||||
|
||||
public static double max(Double a, Double b) {
|
||||
return Math.max(a, b);
|
||||
}
|
||||
|
||||
public static int min(Integer a, Integer b) {
|
||||
return Math.min(a, b);
|
||||
}
|
||||
|
||||
public static long min(Long a, Long b) {
|
||||
return Math.min(a, b);
|
||||
}
|
||||
|
||||
public static float min(Float a, Float b) {
|
||||
return Math.min(a, b);
|
||||
}
|
||||
|
||||
public static double min(Double a, Double b) {
|
||||
return Math.min(a, b);
|
||||
}
|
||||
|
||||
public static double ulp(Double d) {
|
||||
return Math.ulp(d);
|
||||
}
|
||||
|
||||
public static float ulp(Float f) {
|
||||
return Math.ulp(f);
|
||||
}
|
||||
|
||||
public static double signum(Double d) {
|
||||
return Math.signum(d);
|
||||
}
|
||||
|
||||
public static float signum(Float f) {
|
||||
return Math.signum(f);
|
||||
}
|
||||
|
||||
public static double sinh(Double x) {
|
||||
return Math.sinh(x);
|
||||
}
|
||||
|
||||
public static double cosh(Double x) {
|
||||
return Math.cosh(x);
|
||||
}
|
||||
|
||||
public static double tanh(Double x) {
|
||||
return Math.tanh(x);
|
||||
}
|
||||
|
||||
public static double hypot(Double x, Double y) {
|
||||
return Math.hypot(x, y);
|
||||
}
|
||||
|
||||
public static double expm1(Double x) {
|
||||
return Math.expm1(x);
|
||||
}
|
||||
|
||||
public static double log1p(Double x) {
|
||||
return Math.log1p(x);
|
||||
}
|
||||
|
||||
public static double copySign(Double magnitude, Double sign) {
|
||||
return Math.copySign(magnitude, sign);
|
||||
}
|
||||
|
||||
public static float copySign(Float magnitude, Float sign) {
|
||||
return Math.copySign(magnitude, sign);
|
||||
}
|
||||
|
||||
public static int getExponent(Float f) {
|
||||
return Math.getExponent(f);
|
||||
}
|
||||
|
||||
public static int getExponent(Double d) {
|
||||
return Math.getExponent(d);
|
||||
}
|
||||
|
||||
public static double nextAfter(Double start, Double direction) {
|
||||
return Math.nextAfter(start, direction);
|
||||
}
|
||||
|
||||
public static float nextAfter(Float start, Double direction) {
|
||||
return Math.nextAfter(start, direction);
|
||||
}
|
||||
|
||||
public static double nextUp(Double d) {
|
||||
return Math.nextUp(d);
|
||||
}
|
||||
|
||||
public static float nextUp(Float f) {
|
||||
return Math.nextUp(f);
|
||||
}
|
||||
|
||||
|
||||
public static double scalb(Double d, Integer scaleFactor) {
|
||||
return Math.scalb(d, scaleFactor);
|
||||
}
|
||||
|
||||
public static float scalb(Float f, Integer scaleFactor) {
|
||||
return Math.scalb(f, scaleFactor);
|
||||
}
|
||||
}
|
|
@ -519,6 +519,7 @@ public class BloomFilter {
|
|||
return k;
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough") // Uses fallthrough to implement a well know hashing algorithm
|
||||
public static long hash3_x64_128(byte[] key, int offset, int length, long seed) {
|
||||
final int nblocks = length >> 4; // Process as 128-bit blocks.
|
||||
|
||||
|
@ -598,7 +599,7 @@ public class BloomFilter {
|
|||
case 2:
|
||||
k1 ^= ((long) key[offset + 1]) << 8;
|
||||
case 1:
|
||||
k1 ^= ((long) key[offset]);
|
||||
k1 ^= (key[offset]);
|
||||
k1 *= c1;
|
||||
k1 = rotl64(k1, 31);
|
||||
k1 *= c2;
|
||||
|
|
|
@ -45,7 +45,6 @@ import org.elasticsearch.index.engine.EngineFactory;
|
|||
import org.elasticsearch.index.fielddata.FieldDataType;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.ParsedQuery;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
|
@ -68,6 +67,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Path;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
@ -101,6 +101,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
|
|||
private final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
private final AtomicBoolean deleted = new AtomicBoolean(false);
|
||||
private final IndexSettings indexSettings;
|
||||
private final IndexingSlowLog slowLog;
|
||||
|
||||
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
|
||||
SimilarityService similarityService,
|
||||
|
@ -130,6 +131,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
|
|||
this.engineFactory = engineFactory;
|
||||
// initialize this last -- otherwise if the wrapper requires any other member to be non-null we fail with an NPE
|
||||
this.searcherWrapper = wrapperFactory.newWrapper(this);
|
||||
this.slowLog = new IndexingSlowLog(indexSettings.getSettings());
|
||||
}
|
||||
|
||||
public int numberOfShards() {
|
||||
|
@ -292,9 +294,9 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
|
|||
(primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
|
||||
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> nodeServicesProvider.getIndicesQueryCache().onClose(shardId)));
|
||||
if (useShadowEngine(primary, indexSettings)) {
|
||||
indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider);
|
||||
indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider); // no indexing listeners - shadow engines don't index
|
||||
} else {
|
||||
indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider);
|
||||
indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider, slowLog);
|
||||
}
|
||||
|
||||
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
||||
|
@ -552,6 +554,11 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
|
|||
} catch (Exception e) {
|
||||
logger.warn("failed to refresh index store settings", e);
|
||||
}
|
||||
try {
|
||||
slowLog.onRefreshSettings(settings); // this will be refactored soon anyway so duplication is ok here
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to refresh slowlog settings", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.indexing;
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -28,6 +28,7 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.index.shard.IndexingOperationListener;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
|
@ -35,7 +36,7 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
/**
|
||||
*/
|
||||
public final class IndexingSlowLog {
|
||||
public final class IndexingSlowLog implements IndexingOperationListener {
|
||||
|
||||
private boolean reformat;
|
||||
|
||||
|
@ -124,8 +125,9 @@ public final class IndexingSlowLog {
|
|||
}
|
||||
}
|
||||
|
||||
void postIndex(Engine.Index index, long tookInNanos) {
|
||||
postIndexing(index.parsedDoc(), tookInNanos);
|
||||
public void postIndex(Engine.Index index) {
|
||||
final long took = index.endTime() - index.startTime();
|
||||
postIndexing(index.parsedDoc(), took);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -192,4 +194,4 @@ public final class IndexingSlowLog {
|
|||
return sb.toString();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -118,6 +118,12 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L
|
|||
private BitSet getAndLoadIfNotPresent(final Query query, final LeafReaderContext context) throws IOException, ExecutionException {
|
||||
final Object coreCacheReader = context.reader().getCoreCacheKey();
|
||||
final ShardId shardId = ShardUtils.extractShardId(context.reader());
|
||||
if (shardId != null // can't require it because of the percolator
|
||||
&& indexSettings.getIndex().getName().equals(shardId.getIndex()) == false) {
|
||||
// insanity
|
||||
throw new IllegalStateException("Trying to load bit set for index [" + shardId.getIndex()
|
||||
+ "] with cache of index [" + indexSettings.getIndex().getName() + "]");
|
||||
}
|
||||
Cache<Query, Value> filterToFbs = loadedFilters.computeIfAbsent(coreCacheReader, key -> {
|
||||
context.reader().addCoreClosedListener(BitsetFilterCache.this);
|
||||
return CacheBuilder.<Query, Value>builder().build();
|
||||
|
@ -208,6 +214,11 @@ public final class BitsetFilterCache extends AbstractIndexComponent implements L
|
|||
|
||||
@Override
|
||||
public IndicesWarmer.TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
|
||||
if (indexSettings.getIndex().equals(indexShard.getIndexSettings().getIndex()) == false) {
|
||||
// this is from a different index
|
||||
return TerminationHandle.NO_WAIT;
|
||||
}
|
||||
|
||||
if (!loadRandomAccessFiltersEagerly) {
|
||||
return TerminationHandle.NO_WAIT;
|
||||
}
|
||||
|
|
|
@ -30,13 +30,12 @@ import org.elasticsearch.common.unit.ByteSizeValue;
|
|||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.codec.CodecService;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.shard.MergeSchedulerConfig;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
|
||||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.index.translog.TranslogConfig;
|
||||
import org.elasticsearch.indices.memory.IndexingMemoryController;
|
||||
import org.elasticsearch.indices.IndexingMemoryController;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -193,7 +192,7 @@ public final class EngineConfig {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the initial index buffer size. This setting is only read on startup and otherwise controlled by {@link org.elasticsearch.indices.memory.IndexingMemoryController}
|
||||
* Returns the initial index buffer size. This setting is only read on startup and otherwise controlled by {@link IndexingMemoryController}
|
||||
*/
|
||||
public ByteSizeValue getIndexingBufferSize() {
|
||||
return indexingBufferSize;
|
||||
|
|
|
@ -55,12 +55,10 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
|||
import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader;
|
||||
import org.elasticsearch.common.lucene.uid.Versions;
|
||||
import org.elasticsearch.common.math.MathUtils;
|
||||
import org.elasticsearch.common.metrics.CounterMetric;
|
||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.common.util.concurrent.ReleasableLock;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
import org.elasticsearch.index.merge.MergeStats;
|
||||
import org.elasticsearch.index.merge.OnGoingMerge;
|
||||
|
@ -961,8 +959,7 @@ public class InternalEngine extends Engine {
|
|||
});
|
||||
return new IndexWriter(store.directory(), iwc);
|
||||
} catch (LockObtainFailedException ex) {
|
||||
boolean isLocked = IndexWriter.isLocked(store.directory());
|
||||
logger.warn("Could not lock IndexWriter isLocked [{}]", ex, isLocked);
|
||||
logger.warn("could not lock IndexWriter", ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,11 +58,16 @@ public class SingleFieldsVisitor extends FieldsVisitor {
|
|||
|
||||
public void postProcess(MappedFieldType fieldType) {
|
||||
if (uid != null) {
|
||||
// TODO: this switch seems very wrong...either each case should be breaking, or this should not be a switch
|
||||
switch (field) {
|
||||
case UidFieldMapper.NAME: addValue(field, uid.toString());
|
||||
case IdFieldMapper.NAME: addValue(field, uid.id());
|
||||
case TypeFieldMapper.NAME: addValue(field, uid.type());
|
||||
case UidFieldMapper.NAME:
|
||||
addValue(field, uid.toString());
|
||||
break;
|
||||
case IdFieldMapper.NAME:
|
||||
addValue(field, uid.id());
|
||||
break;
|
||||
case TypeFieldMapper.NAME:
|
||||
addValue(field, uid.type());
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -1,70 +0,0 @@
|
|||
/*
|
||||
* 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.indexing;
|
||||
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
|
||||
/**
|
||||
* An indexing listener for indexing, delete, events.
|
||||
*/
|
||||
public abstract class IndexingOperationListener {
|
||||
|
||||
/**
|
||||
* Called before the indexing occurs.
|
||||
*/
|
||||
public Engine.Index preIndex(Engine.Index operation) {
|
||||
return operation;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called after the indexing operation occurred.
|
||||
*/
|
||||
public void postIndex(Engine.Index index) {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Called after the indexing operation occurred with exception.
|
||||
*/
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Called before the delete occurs.
|
||||
*/
|
||||
public Engine.Delete preDelete(Engine.Delete delete) {
|
||||
return delete;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Called after the delete operation occurred.
|
||||
*/
|
||||
public void postDelete(Engine.Delete delete) {
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Called after the delete operation occurred with exception.
|
||||
*/
|
||||
public void postDelete(Engine.Delete delete, Throwable ex) {
|
||||
|
||||
}
|
||||
}
|
|
@ -81,8 +81,6 @@ import org.elasticsearch.index.fielddata.ShardFieldData;
|
|||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.get.GetStats;
|
||||
import org.elasticsearch.index.get.ShardGetService;
|
||||
import org.elasticsearch.index.indexing.IndexingStats;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.DocumentMapperForType;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
@ -113,7 +111,7 @@ import org.elasticsearch.index.warmer.ShardIndexWarmerService;
|
|||
import org.elasticsearch.index.warmer.WarmerStats;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
|
||||
import org.elasticsearch.indices.memory.IndexingMemoryController;
|
||||
import org.elasticsearch.indices.IndexingMemoryController;
|
||||
import org.elasticsearch.indices.recovery.RecoveryFailedException;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.percolator.PercolatorService;
|
||||
|
@ -125,6 +123,8 @@ import java.io.IOException;
|
|||
import java.io.PrintStream;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -143,7 +143,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
private final IndexCache indexCache;
|
||||
private final Store store;
|
||||
private final MergeSchedulerConfig mergeSchedulerConfig;
|
||||
private final ShardIndexingService indexingService;
|
||||
private final InternalIndexingStats internalIndexingStats;
|
||||
private final ShardSearchStats searchService;
|
||||
private final ShardGetService getService;
|
||||
private final ShardIndexWarmerService shardWarmerService;
|
||||
|
@ -167,7 +167,6 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
private final IndexEventListener indexEventListener;
|
||||
private final IndexSettings idxSettings;
|
||||
private final NodeServicesProvider provider;
|
||||
|
||||
private TimeValue refreshInterval;
|
||||
|
||||
private volatile ScheduledFuture<?> refreshScheduledFuture;
|
||||
|
@ -176,6 +175,8 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
protected final AtomicReference<Engine> currentEngineReference = new AtomicReference<>();
|
||||
protected final EngineFactory engineFactory;
|
||||
|
||||
private final IndexingOperationListener indexingOperationListeners;
|
||||
|
||||
@Nullable
|
||||
private RecoveryState recoveryState;
|
||||
|
||||
|
@ -215,7 +216,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache,
|
||||
MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService,
|
||||
@Nullable EngineFactory engineFactory,
|
||||
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider) {
|
||||
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider, IndexingOperationListener... listeners) {
|
||||
super(shardId, indexSettings);
|
||||
final Settings settings = indexSettings.getSettings();
|
||||
this.inactiveTime = settings.getAsTime(INDEX_SHARD_INACTIVE_TIME_SETTING, settings.getAsTime(INDICES_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5)));
|
||||
|
@ -232,7 +233,10 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
this.threadPool = provider.getThreadPool();
|
||||
this.mapperService = mapperService;
|
||||
this.indexCache = indexCache;
|
||||
this.indexingService = new ShardIndexingService(shardId, indexSettings);
|
||||
this.internalIndexingStats = new InternalIndexingStats();
|
||||
final List<IndexingOperationListener> listenersList = new ArrayList<>(Arrays.asList(listeners));
|
||||
listenersList.add(internalIndexingStats);
|
||||
this.indexingOperationListeners = new IndexingOperationListener.CompositeListener(listenersList, logger);
|
||||
this.getService = new ShardGetService(indexSettings, this, mapperService);
|
||||
this.termVectorsService = provider.getTermVectorsService();
|
||||
this.searchService = new ShardSearchStats(settings);
|
||||
|
@ -285,10 +289,6 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
return true;
|
||||
}
|
||||
|
||||
public ShardIndexingService indexingService() {
|
||||
return this.indexingService;
|
||||
}
|
||||
|
||||
public ShardGetService getService() {
|
||||
return this.getService;
|
||||
}
|
||||
|
@ -489,7 +489,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
public boolean index(Engine.Index index) {
|
||||
ensureWriteAllowed(index);
|
||||
markLastWrite();
|
||||
index = indexingService.preIndex(index);
|
||||
index = indexingOperationListeners.preIndex(index);
|
||||
final boolean created;
|
||||
try {
|
||||
if (logger.isTraceEnabled()) {
|
||||
|
@ -503,10 +503,10 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
}
|
||||
index.endTime(System.nanoTime());
|
||||
} catch (Throwable ex) {
|
||||
indexingService.postIndex(index, ex);
|
||||
indexingOperationListeners.postIndex(index, ex);
|
||||
throw ex;
|
||||
}
|
||||
indexingService.postIndex(index);
|
||||
indexingOperationListeners.postIndex(index);
|
||||
return created;
|
||||
}
|
||||
|
||||
|
@ -532,7 +532,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
public void delete(Engine.Delete delete) {
|
||||
ensureWriteAllowed(delete);
|
||||
markLastWrite();
|
||||
delete = indexingService.preDelete(delete);
|
||||
delete = indexingOperationListeners.preDelete(delete);
|
||||
try {
|
||||
if (logger.isTraceEnabled()) {
|
||||
logger.trace("delete [{}]", delete.uid().text());
|
||||
|
@ -545,10 +545,10 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
}
|
||||
delete.endTime(System.nanoTime());
|
||||
} catch (Throwable ex) {
|
||||
indexingService.postDelete(delete, ex);
|
||||
indexingOperationListeners.postDelete(delete, ex);
|
||||
throw ex;
|
||||
}
|
||||
indexingService.postDelete(delete);
|
||||
indexingOperationListeners.postDelete(delete);
|
||||
}
|
||||
|
||||
public Engine.GetResult get(Engine.Get get) {
|
||||
|
@ -600,7 +600,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
throttled = engine.isThrottled();
|
||||
throttleTimeInMillis = engine.getIndexThrottleTimeInMillis();
|
||||
}
|
||||
return indexingService.stats(throttled, throttleTimeInMillis, types);
|
||||
return internalIndexingStats.stats(throttled, throttleTimeInMillis, types);
|
||||
}
|
||||
|
||||
public SearchStats searchStats(String... groups) {
|
||||
|
@ -1222,7 +1222,6 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
}
|
||||
mergePolicyConfig.onRefreshSettings(settings);
|
||||
searchService.onRefreshSettings(settings);
|
||||
indexingService.onRefreshSettings(settings);
|
||||
if (change) {
|
||||
getEngine().onSettingsChanged();
|
||||
}
|
||||
|
@ -1258,6 +1257,14 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
return inactiveTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should be called for each no-op update operation to increment relevant statistics.
|
||||
* @param type the doc type of the update
|
||||
*/
|
||||
public void noopUpdate(String type) {
|
||||
internalIndexingStats.noopUpdate(type);
|
||||
}
|
||||
|
||||
class EngineRefresher implements Runnable {
|
||||
@Override
|
||||
public void run() {
|
||||
|
|
|
@ -0,0 +1,152 @@
|
|||
/*
|
||||
* 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.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* An indexing listener for indexing, delete, events.
|
||||
*/
|
||||
public interface IndexingOperationListener {
|
||||
|
||||
/**
|
||||
* Called before the indexing occurs.
|
||||
*/
|
||||
default Engine.Index preIndex(Engine.Index operation) {
|
||||
return operation;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called after the indexing operation occurred.
|
||||
*/
|
||||
default void postIndex(Engine.Index index) {}
|
||||
|
||||
/**
|
||||
* Called after the indexing operation occurred with exception.
|
||||
*/
|
||||
default void postIndex(Engine.Index index, Throwable ex) {}
|
||||
|
||||
/**
|
||||
* Called before the delete occurs.
|
||||
*/
|
||||
default Engine.Delete preDelete(Engine.Delete delete) {
|
||||
return delete;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Called after the delete operation occurred.
|
||||
*/
|
||||
default void postDelete(Engine.Delete delete) {}
|
||||
|
||||
/**
|
||||
* Called after the delete operation occurred with exception.
|
||||
*/
|
||||
default void postDelete(Engine.Delete delete, Throwable ex) {}
|
||||
|
||||
/**
|
||||
* A Composite listener that multiplexes calls to each of the listeners methods.
|
||||
*/
|
||||
final class CompositeListener implements IndexingOperationListener{
|
||||
private final List<IndexingOperationListener> listeners;
|
||||
private final ESLogger logger;
|
||||
|
||||
public CompositeListener(List<IndexingOperationListener> listeners, ESLogger logger) {
|
||||
this.listeners = listeners;
|
||||
this.logger = logger;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Index preIndex(Engine.Index operation) {
|
||||
assert operation != null;
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.preIndex(operation);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("preIndex listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
return operation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index) {
|
||||
assert index != null;
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postIndex(index);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("postIndex listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
assert index != null && ex != null;
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postIndex(index, ex);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("postIndex listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Delete preDelete(Engine.Delete delete) {
|
||||
assert delete != null;
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.preDelete(delete);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("preDelete listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
return delete;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete) {
|
||||
assert delete != null;
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postDelete(delete);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("postDelete listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete, Throwable ex) {
|
||||
assert delete != null && ex != null;
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postDelete(delete, ex);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("postDelete listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.indexing;
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Nullable;
|
|
@ -17,49 +17,34 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.indexing;
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.metrics.CounterMetric;
|
||||
import org.elasticsearch.common.metrics.MeanMetric;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
|
||||
/**
|
||||
* Internal class that maintains relevant indexing statistics / metrics.
|
||||
* @see IndexShard
|
||||
*/
|
||||
public class ShardIndexingService extends AbstractIndexShardComponent {
|
||||
|
||||
private final IndexingSlowLog slowLog;
|
||||
|
||||
final class InternalIndexingStats implements IndexingOperationListener {
|
||||
private final StatsHolder totalStats = new StatsHolder();
|
||||
|
||||
private final CopyOnWriteArrayList<IndexingOperationListener> listeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
private volatile Map<String, StatsHolder> typesStats = emptyMap();
|
||||
|
||||
public ShardIndexingService(ShardId shardId, IndexSettings indexSettings) {
|
||||
super(shardId, indexSettings);
|
||||
this.slowLog = new IndexingSlowLog(this.indexSettings.getSettings());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the stats, including type specific stats. If the types are null/0 length, then nothing
|
||||
* is returned for them. If they are set, then only types provided will be returned, or
|
||||
* <tt>_all</tt> for all types.
|
||||
*/
|
||||
public IndexingStats stats(boolean isThrottled, long currentThrottleInMillis, String... types) {
|
||||
IndexingStats stats(boolean isThrottled, long currentThrottleInMillis, String... types) {
|
||||
IndexingStats.Stats total = totalStats.stats(isThrottled, currentThrottleInMillis);
|
||||
Map<String, IndexingStats.Stats> typesSt = null;
|
||||
if (types != null && types.length > 0) {
|
||||
|
@ -79,23 +64,14 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
|
|||
return new IndexingStats(total, typesSt);
|
||||
}
|
||||
|
||||
public void addListener(IndexingOperationListener listener) {
|
||||
listeners.add(listener);
|
||||
}
|
||||
|
||||
public void removeListener(IndexingOperationListener listener) {
|
||||
listeners.remove(listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Index preIndex(Engine.Index operation) {
|
||||
totalStats.indexCurrent.inc();
|
||||
typeStats(operation.type()).indexCurrent.inc();
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
operation = listener.preIndex(operation);
|
||||
}
|
||||
return operation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index) {
|
||||
long took = index.endTime() - index.startTime();
|
||||
totalStats.indexMetric.inc(took);
|
||||
|
@ -103,40 +79,24 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
|
|||
StatsHolder typeStats = typeStats(index.type());
|
||||
typeStats.indexMetric.inc(took);
|
||||
typeStats.indexCurrent.dec();
|
||||
slowLog.postIndex(index, took);
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postIndex(index);
|
||||
} catch (Exception e) {
|
||||
logger.warn("postIndex listener [{}] failed", e, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
totalStats.indexCurrent.dec();
|
||||
typeStats(index.type()).indexCurrent.dec();
|
||||
totalStats.indexFailed.inc();
|
||||
typeStats(index.type()).indexFailed.inc();
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postIndex(index, ex);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("postIndex listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Delete preDelete(Engine.Delete delete) {
|
||||
totalStats.deleteCurrent.inc();
|
||||
typeStats(delete.type()).deleteCurrent.inc();
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
delete = listener.preDelete(delete);
|
||||
}
|
||||
return delete;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete) {
|
||||
long took = delete.endTime() - delete.startTime();
|
||||
totalStats.deleteMetric.inc(took);
|
||||
|
@ -144,25 +104,12 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
|
|||
StatsHolder typeStats = typeStats(delete.type());
|
||||
typeStats.deleteMetric.inc(took);
|
||||
typeStats.deleteCurrent.dec();
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener.postDelete(delete);
|
||||
} catch (Exception e) {
|
||||
logger.warn("postDelete listener [{}] failed", e, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete, Throwable ex) {
|
||||
totalStats.deleteCurrent.dec();
|
||||
typeStats(delete.type()).deleteCurrent.dec();
|
||||
for (IndexingOperationListener listener : listeners) {
|
||||
try {
|
||||
listener. postDelete(delete, ex);
|
||||
} catch (Throwable t) {
|
||||
logger.warn("postDelete listener [{}] failed", t, listener);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void noopUpdate(String type) {
|
||||
|
@ -170,22 +117,6 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
|
|||
typeStats(type).noopUpdates.inc();
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
totalStats.clear();
|
||||
synchronized (this) {
|
||||
if (!typesStats.isEmpty()) {
|
||||
MapBuilder<String, StatsHolder> typesStatsBuilder = MapBuilder.newMapBuilder();
|
||||
for (Map.Entry<String, StatsHolder> typeStats : typesStats.entrySet()) {
|
||||
if (typeStats.getValue().totalCurrent() > 0) {
|
||||
typeStats.getValue().clear();
|
||||
typesStatsBuilder.put(typeStats.getKey(), typeStats.getValue());
|
||||
}
|
||||
}
|
||||
typesStats = typesStatsBuilder.immutableMap();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private StatsHolder typeStats(String type) {
|
||||
StatsHolder stats = typesStats.get(type);
|
||||
if (stats == null) {
|
||||
|
@ -200,34 +131,24 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
|
|||
return stats;
|
||||
}
|
||||
|
||||
public void onRefreshSettings(Settings settings) {
|
||||
slowLog.onRefreshSettings(settings);
|
||||
}
|
||||
|
||||
static class StatsHolder {
|
||||
public final MeanMetric indexMetric = new MeanMetric();
|
||||
public final MeanMetric deleteMetric = new MeanMetric();
|
||||
public final CounterMetric indexCurrent = new CounterMetric();
|
||||
public final CounterMetric indexFailed = new CounterMetric();
|
||||
public final CounterMetric deleteCurrent = new CounterMetric();
|
||||
public final CounterMetric noopUpdates = new CounterMetric();
|
||||
private final MeanMetric indexMetric = new MeanMetric();
|
||||
private final MeanMetric deleteMetric = new MeanMetric();
|
||||
private final CounterMetric indexCurrent = new CounterMetric();
|
||||
private final CounterMetric indexFailed = new CounterMetric();
|
||||
private final CounterMetric deleteCurrent = new CounterMetric();
|
||||
private final CounterMetric noopUpdates = new CounterMetric();
|
||||
|
||||
public IndexingStats.Stats stats(boolean isThrottled, long currentThrottleMillis) {
|
||||
IndexingStats.Stats stats(boolean isThrottled, long currentThrottleMillis) {
|
||||
return new IndexingStats.Stats(
|
||||
indexMetric.count(), TimeUnit.NANOSECONDS.toMillis(indexMetric.sum()), indexCurrent.count(), indexFailed.count(),
|
||||
deleteMetric.count(), TimeUnit.NANOSECONDS.toMillis(deleteMetric.sum()), deleteCurrent.count(),
|
||||
noopUpdates.count(), isThrottled, TimeUnit.MILLISECONDS.toMillis(currentThrottleMillis));
|
||||
indexMetric.count(), TimeUnit.NANOSECONDS.toMillis(indexMetric.sum()), indexCurrent.count(), indexFailed.count(),
|
||||
deleteMetric.count(), TimeUnit.NANOSECONDS.toMillis(deleteMetric.sum()), deleteCurrent.count(),
|
||||
noopUpdates.count(), isThrottled, TimeUnit.MILLISECONDS.toMillis(currentThrottleMillis));
|
||||
}
|
||||
|
||||
public long totalCurrent() {
|
||||
return indexCurrent.count() + deleteMetric.count();
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
void clear() {
|
||||
indexMetric.clear();
|
||||
deleteMetric.clear();
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
}
|
|
@ -70,6 +70,7 @@ public class TranslogRecoveryPerformer {
|
|||
performRecoveryOperation(engine, operation, false);
|
||||
numOps++;
|
||||
}
|
||||
engine.getTranslog().sync();
|
||||
} catch (Throwable t) {
|
||||
throw new BatchOperationException(shardId, "failed to apply batch translog operation", numOps, t);
|
||||
}
|
||||
|
|
|
@ -17,10 +17,9 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.indices.memory;
|
||||
package org.elasticsearch.indices;
|
||||
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
|
@ -32,16 +31,16 @@ import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
|
|||
import org.elasticsearch.index.shard.IndexEventListener;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.monitor.jvm.JvmInfo;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
|
||||
public class IndexingMemoryController extends AbstractLifecycleComponent<IndexingMemoryController> implements IndexEventListener {
|
||||
public class IndexingMemoryController extends AbstractComponent implements IndexEventListener, Closeable {
|
||||
|
||||
/** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */
|
||||
public static final String INDEX_BUFFER_SIZE_SETTING = "indices.memory.index_buffer_size";
|
||||
|
@ -70,10 +69,6 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
|
|||
/** Once a shard becomes inactive, we reduce the {@code IndexWriter} buffer to this value (500 KB) to let active shards use the heap instead. */
|
||||
public static final ByteSizeValue INACTIVE_SHARD_INDEXING_BUFFER = ByteSizeValue.parseBytesSizeValue("500kb", "INACTIVE_SHARD_INDEXING_BUFFER");
|
||||
|
||||
/** Once a shard becomes inactive, we reduce the {@code Translog} buffer to this value (1 KB) to let active shards use the heap instead. */
|
||||
public static final ByteSizeValue INACTIVE_SHARD_TRANSLOG_BUFFER = ByteSizeValue.parseBytesSizeValue("1kb", "INACTIVE_SHARD_TRANSLOG_BUFFER");
|
||||
|
||||
private final ThreadPool threadPool;
|
||||
private final IndicesService indicesService;
|
||||
|
||||
private final ByteSizeValue indexingBuffer;
|
||||
|
@ -81,22 +76,20 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
|
|||
private final ByteSizeValue maxShardIndexBufferSize;
|
||||
private final TimeValue interval;
|
||||
|
||||
private volatile ScheduledFuture scheduler;
|
||||
private final ScheduledFuture scheduler;
|
||||
|
||||
private static final EnumSet<IndexShardState> CAN_UPDATE_INDEX_BUFFER_STATES = EnumSet.of(
|
||||
IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED, IndexShardState.RELOCATED);
|
||||
|
||||
private final ShardsIndicesStatusChecker statusChecker;
|
||||
|
||||
@Inject
|
||||
public IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) {
|
||||
IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService) {
|
||||
this(settings, threadPool, indicesService, JvmInfo.jvmInfo().getMem().getHeapMax().bytes());
|
||||
}
|
||||
|
||||
// for testing
|
||||
protected IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService, long jvmMemoryInBytes) {
|
||||
IndexingMemoryController(Settings settings, ThreadPool threadPool, IndicesService indicesService, long jvmMemoryInBytes) {
|
||||
super(settings);
|
||||
this.threadPool = threadPool;
|
||||
this.indicesService = indicesService;
|
||||
|
||||
ByteSizeValue indexingBuffer;
|
||||
|
@ -131,29 +124,24 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
|
|||
MIN_SHARD_INDEX_BUFFER_SIZE_SETTING, this.minShardIndexBufferSize,
|
||||
MAX_SHARD_INDEX_BUFFER_SIZE_SETTING, this.maxShardIndexBufferSize,
|
||||
SHARD_INACTIVE_INTERVAL_TIME_SETTING, this.interval);
|
||||
this.scheduler = scheduleTask(threadPool);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doStart() {
|
||||
protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
|
||||
// it's fine to run it on the scheduler thread, no busy work
|
||||
this.scheduler = threadPool.scheduleWithFixedDelay(statusChecker, interval);
|
||||
return threadPool.scheduleWithFixedDelay(statusChecker, interval);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doStop() {
|
||||
public void close() {
|
||||
FutureUtils.cancel(scheduler);
|
||||
scheduler = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doClose() {
|
||||
}
|
||||
|
||||
/**
|
||||
* returns the current budget for the total amount of indexing buffers of
|
||||
* active shards on this node
|
||||
*/
|
||||
public ByteSizeValue indexingBufferSize() {
|
||||
ByteSizeValue indexingBufferSize() {
|
||||
return indexingBuffer;
|
||||
}
|
||||
|
||||
|
@ -188,7 +176,7 @@ public class IndexingMemoryController extends AbstractLifecycleComponent<Indexin
|
|||
}
|
||||
|
||||
/** check if any shards active status changed, now. */
|
||||
public void forceCheck() {
|
||||
void forceCheck() {
|
||||
statusChecker.run();
|
||||
}
|
||||
|
|
@ -111,7 +111,6 @@ import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
|
|||
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener;
|
||||
import org.elasticsearch.indices.flush.SyncedFlushService;
|
||||
import org.elasticsearch.indices.mapper.MapperRegistry;
|
||||
import org.elasticsearch.indices.memory.IndexingMemoryController;
|
||||
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.indices.recovery.RecoverySource;
|
||||
|
@ -273,7 +272,6 @@ public class IndicesModule extends AbstractModule {
|
|||
bind(RecoverySource.class).asEagerSingleton();
|
||||
bind(IndicesStore.class).asEagerSingleton();
|
||||
bind(IndicesClusterStateService.class).asEagerSingleton();
|
||||
bind(IndexingMemoryController.class).asEagerSingleton();
|
||||
bind(SyncedFlushService.class).asEagerSingleton();
|
||||
bind(IndicesQueryCache.class).asEagerSingleton();
|
||||
bind(IndicesRequestCache.class).asEagerSingleton();
|
||||
|
|
|
@ -52,7 +52,7 @@ import org.elasticsearch.index.NodeServicesProvider;
|
|||
import org.elasticsearch.index.analysis.AnalysisRegistry;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.get.GetStats;
|
||||
import org.elasticsearch.index.indexing.IndexingStats;
|
||||
import org.elasticsearch.index.shard.IndexingStats;
|
||||
import org.elasticsearch.index.merge.MergeStats;
|
||||
import org.elasticsearch.index.recovery.RecoveryStats;
|
||||
import org.elasticsearch.index.refresh.RefreshStats;
|
||||
|
@ -65,6 +65,7 @@ import org.elasticsearch.index.store.IndexStoreConfig;
|
|||
import org.elasticsearch.indices.mapper.MapperRegistry;
|
||||
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
|
||||
import org.elasticsearch.plugins.PluginsService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
|
@ -105,6 +106,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
private final OldShardsStats oldShardsStats = new OldShardsStats();
|
||||
private final IndexStoreConfig indexStoreConfig;
|
||||
private final MapperRegistry mapperRegistry;
|
||||
private final IndexingMemoryController indexingMemoryController;
|
||||
|
||||
@Override
|
||||
protected void doStart() {
|
||||
|
@ -114,7 +116,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv,
|
||||
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
|
||||
IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
ClusterService clusterService, MapperRegistry mapperRegistry) {
|
||||
ClusterService clusterService, MapperRegistry mapperRegistry, ThreadPool threadPool) {
|
||||
super(settings);
|
||||
this.pluginsService = pluginsService;
|
||||
this.nodeEnv = nodeEnv;
|
||||
|
@ -127,7 +129,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
this.mapperRegistry = mapperRegistry;
|
||||
clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE_SETTING, indexStoreConfig::setRateLimitingType);
|
||||
clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, indexStoreConfig::setRateLimitingThrottle);
|
||||
|
||||
indexingMemoryController = new IndexingMemoryController(settings, threadPool, this);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -161,7 +163,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
|
||||
@Override
|
||||
protected void doClose() {
|
||||
IOUtils.closeWhileHandlingException(analysisRegistry);
|
||||
IOUtils.closeWhileHandlingException(analysisRegistry, indexingMemoryController);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -291,6 +293,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
|
||||
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, analysisRegistry);
|
||||
pluginsService.onIndexModule(indexModule);
|
||||
indexModule.addIndexEventListener(indexingMemoryController);
|
||||
for (IndexEventListener listener : builtInListeners) {
|
||||
indexModule.addIndexEventListener(listener);
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.elasticsearch.index.engine.SegmentsStats;
|
|||
import org.elasticsearch.index.fielddata.FieldDataStats;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.get.GetStats;
|
||||
import org.elasticsearch.index.indexing.IndexingStats;
|
||||
import org.elasticsearch.index.shard.IndexingStats;
|
||||
import org.elasticsearch.index.merge.MergeStats;
|
||||
import org.elasticsearch.index.percolator.PercolateStats;
|
||||
import org.elasticsearch.index.recovery.RecoveryStats;
|
||||
|
|
|
@ -63,7 +63,6 @@ import org.elasticsearch.index.shard.ShardNotFoundException;
|
|||
import org.elasticsearch.index.snapshots.IndexShardRepository;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.flush.SyncedFlushService;
|
||||
import org.elasticsearch.indices.memory.IndexingMemoryController;
|
||||
import org.elasticsearch.indices.recovery.RecoveryFailedException;
|
||||
import org.elasticsearch.indices.recovery.RecoverySource;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
|
@ -130,9 +129,9 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
|
|||
NodeMappingRefreshAction nodeMappingRefreshAction,
|
||||
RepositoriesService repositoriesService, RestoreService restoreService,
|
||||
SearchService searchService, SyncedFlushService syncedFlushService,
|
||||
RecoverySource recoverySource, NodeServicesProvider nodeServicesProvider, IndexingMemoryController indexingMemoryController) {
|
||||
RecoverySource recoverySource, NodeServicesProvider nodeServicesProvider) {
|
||||
super(settings);
|
||||
this.buildInIndexListener = Arrays.asList(recoverySource, recoveryTarget, searchService, syncedFlushService, indexingMemoryController);
|
||||
this.buildInIndexListener = Arrays.asList(recoverySource, recoveryTarget, searchService, syncedFlushService);
|
||||
this.indicesService = indicesService;
|
||||
this.clusterService = clusterService;
|
||||
this.threadPool = threadPool;
|
||||
|
|
|
@ -69,8 +69,6 @@ import org.elasticsearch.indices.breaker.CircuitBreakerModule;
|
|||
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
|
||||
import org.elasticsearch.indices.cluster.IndicesClusterStateService;
|
||||
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
|
||||
import org.elasticsearch.indices.memory.IndexingMemoryController;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.indices.store.IndicesStore;
|
||||
import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
import org.elasticsearch.monitor.MonitorService;
|
||||
|
@ -249,7 +247,6 @@ public class Node implements Releasable {
|
|||
|
||||
injector.getInstance(MappingUpdatedAction.class).setClient(client);
|
||||
injector.getInstance(IndicesService.class).start();
|
||||
injector.getInstance(IndexingMemoryController.class).start();
|
||||
injector.getInstance(IndicesClusterStateService.class).start();
|
||||
injector.getInstance(IndicesTTLService.class).start();
|
||||
injector.getInstance(SnapshotsService.class).start();
|
||||
|
@ -308,7 +305,6 @@ public class Node implements Releasable {
|
|||
// stop any changes happening as a result of cluster state changes
|
||||
injector.getInstance(IndicesClusterStateService.class).stop();
|
||||
// we close indices first, so operations won't be allowed on it
|
||||
injector.getInstance(IndexingMemoryController.class).stop();
|
||||
injector.getInstance(IndicesTTLService.class).stop();
|
||||
injector.getInstance(RoutingService.class).stop();
|
||||
injector.getInstance(ClusterService.class).stop();
|
||||
|
@ -360,7 +356,6 @@ public class Node implements Releasable {
|
|||
stopWatch.stop().start("indices_cluster");
|
||||
injector.getInstance(IndicesClusterStateService.class).close();
|
||||
stopWatch.stop().start("indices");
|
||||
injector.getInstance(IndexingMemoryController.class).close();
|
||||
injector.getInstance(IndicesTTLService.class).close();
|
||||
injector.getInstance(IndicesService.class).close();
|
||||
// close filter/fielddata caches after indices
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.elasticsearch.node.internal;
|
|||
import org.elasticsearch.bootstrap.BootstrapInfo;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.cli.Terminal;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
|
@ -41,7 +42,6 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
import static org.elasticsearch.common.Strings.cleanPath;
|
||||
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
||||
|
@ -207,7 +207,7 @@ public class InternalSettingsPreparer {
|
|||
name = reader.readLine();
|
||||
}
|
||||
}
|
||||
int index = ThreadLocalRandom.current().nextInt(names.size());
|
||||
int index = Randomness.get().nextInt(names.size());
|
||||
return names.get(index);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Could not read node names list", e);
|
||||
|
|
|
@ -41,7 +41,7 @@ import org.elasticsearch.index.engine.SegmentsStats;
|
|||
import org.elasticsearch.index.fielddata.FieldDataStats;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.get.GetStats;
|
||||
import org.elasticsearch.index.indexing.IndexingStats;
|
||||
import org.elasticsearch.index.shard.IndexingStats;
|
||||
import org.elasticsearch.index.merge.MergeStats;
|
||||
import org.elasticsearch.index.percolator.PercolateStats;
|
||||
import org.elasticsearch.index.refresh.RefreshStats;
|
||||
|
|
|
@ -357,7 +357,7 @@ public class ThreadPool extends AbstractComponent {
|
|||
if (!Names.SAME.equals(name)) {
|
||||
command = new ThreadedRunnable(command, executor(name));
|
||||
}
|
||||
return scheduler.schedule(command, delay.millis(), TimeUnit.MILLISECONDS);
|
||||
return scheduler.schedule(new LoggingRunnable(command), delay.millis(), TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
public void shutdown() {
|
||||
|
@ -458,7 +458,7 @@ public class ThreadPool extends AbstractComponent {
|
|||
if (ThreadPoolType.FIXED == previousInfo.getThreadPoolType()) {
|
||||
SizeValue updatedQueueSize = getAsSizeOrUnbounded(settings, "capacity", getAsSizeOrUnbounded(settings, "queue", getAsSizeOrUnbounded(settings, "queue_size", previousInfo.getQueueSize())));
|
||||
if (Objects.equals(previousInfo.getQueueSize(), updatedQueueSize)) {
|
||||
int updatedSize = settings.getAsInt("size", previousInfo.getMax());
|
||||
int updatedSize = applyHardSizeLimit(name, settings.getAsInt("size", previousInfo.getMax()));
|
||||
if (previousInfo.getMax() != updatedSize) {
|
||||
logger.debug("updating thread_pool [{}], type [{}], size [{}], queue_size [{}]", name, type, updatedSize, updatedQueueSize);
|
||||
// if you think this code is crazy: that's because it is!
|
||||
|
@ -480,7 +480,7 @@ public class ThreadPool extends AbstractComponent {
|
|||
defaultQueueSize = previousInfo.getQueueSize();
|
||||
}
|
||||
|
||||
int size = settings.getAsInt("size", defaultSize);
|
||||
int size = applyHardSizeLimit(name, settings.getAsInt("size", defaultSize));
|
||||
SizeValue queueSize = getAsSizeOrUnbounded(settings, "capacity", getAsSizeOrUnbounded(settings, "queue", getAsSizeOrUnbounded(settings, "queue_size", defaultQueueSize)));
|
||||
logger.debug("creating thread_pool [{}], type [{}], size [{}], queue_size [{}]", name, type, size, queueSize);
|
||||
Executor executor = EsExecutors.newFixed(name, size, queueSize == null ? -1 : (int) queueSize.singles(), threadFactory);
|
||||
|
@ -533,6 +533,21 @@ public class ThreadPool extends AbstractComponent {
|
|||
throw new IllegalArgumentException("No type found [" + type + "], for [" + name + "]");
|
||||
}
|
||||
|
||||
private int applyHardSizeLimit(String name, int size) {
|
||||
int availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
|
||||
if ((name.equals(Names.BULK) || name.equals(Names.INDEX)) && size > availableProcessors) {
|
||||
// We use a hard max size for the indexing pools, because if too many threads enter Lucene's IndexWriter, it means
|
||||
// too many segments written, too frequently, too much merging, etc:
|
||||
// TODO: I would love to be loud here (throw an exception if you ask for a too-big size), but I think this is dangerous
|
||||
// because on upgrade this setting could be in cluster state and hard for the user to correct?
|
||||
logger.warn("requested thread pool size [{}] for [{}] is too large; setting to maximum [{}] instead",
|
||||
size, name, availableProcessors);
|
||||
size = availableProcessors;
|
||||
}
|
||||
|
||||
return size;
|
||||
}
|
||||
|
||||
private void updateSettings(Settings settings) {
|
||||
Map<String, Settings> groupSettings = settings.getAsGroups();
|
||||
if (groupSettings.isEmpty()) {
|
||||
|
@ -633,6 +648,7 @@ public class ThreadPool extends AbstractComponent {
|
|||
runnable.run();
|
||||
} catch (Throwable t) {
|
||||
logger.warn("failed to run {}", t, runnable.toString());
|
||||
throw t;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -192,7 +192,7 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
|
|||
|
||||
context = new IndexNameExpressionResolver.Context(state, lenientExpand);
|
||||
results = indexNameExpressionResolver.concreteIndices(context, Strings.EMPTY_ARRAY);
|
||||
assertEquals(4, results.length);
|
||||
assertEquals(Arrays.toString(results), 4, results.length);
|
||||
|
||||
context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen());
|
||||
results = indexNameExpressionResolver.concreteIndices(context, "foofoo*");
|
||||
|
@ -867,4 +867,37 @@ public class IndexNameExpressionResolverTests extends ESTestCase {
|
|||
}
|
||||
return mdBuilder.build();
|
||||
}
|
||||
|
||||
public void testFilterClosedIndicesOnAliases() {
|
||||
MetaData.Builder mdBuilder = MetaData.builder()
|
||||
.put(indexBuilder("test-0").state(State.OPEN).putAlias(AliasMetaData.builder("alias-0")))
|
||||
.put(indexBuilder("test-1").state(IndexMetaData.State.CLOSE).putAlias(AliasMetaData.builder("alias-1")));
|
||||
ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build();
|
||||
|
||||
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen());
|
||||
String[] strings = indexNameExpressionResolver.concreteIndices(context, "alias-*");
|
||||
assertArrayEquals(new String[] {"test-0"}, strings);
|
||||
|
||||
context = new IndexNameExpressionResolver.Context(state, IndicesOptions.strictExpandOpen());
|
||||
strings = indexNameExpressionResolver.concreteIndices(context, "alias-*");
|
||||
|
||||
assertArrayEquals(new String[] {"test-0"}, strings);
|
||||
}
|
||||
|
||||
public void testFilteringAliases() {
|
||||
MetaData.Builder mdBuilder = MetaData.builder()
|
||||
.put(indexBuilder("test-0").state(State.OPEN).putAlias(AliasMetaData.builder("alias-0").filter("{ \"term\": \"foo\"}")))
|
||||
.put(indexBuilder("test-1").state(State.OPEN).putAlias(AliasMetaData.builder("alias-1")));
|
||||
ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build();
|
||||
|
||||
String[] strings = indexNameExpressionResolver.filteringAliases(state, "test-0", "alias-*");
|
||||
assertArrayEquals(new String[] {"alias-0"}, strings);
|
||||
|
||||
// concrete index supersedes filtering alias
|
||||
strings = indexNameExpressionResolver.filteringAliases(state, "test-0", "test-0,alias-*");
|
||||
assertNull(strings);
|
||||
|
||||
strings = indexNameExpressionResolver.filteringAliases(state, "test-0", "test-*,alias-*");
|
||||
assertNull(strings);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,9 +20,15 @@
|
|||
package org.elasticsearch.cluster.metadata;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
|
@ -110,4 +116,36 @@ public class MetaDataTests extends ESTestCase {
|
|||
assertThat(ex.getMessage(), is("index/alias [alias2] provided with routing value [1,2] that resolved to several routing values, rejecting operation"));
|
||||
}
|
||||
}
|
||||
|
||||
public void testUnknownFieldClusterMetaData() throws IOException {
|
||||
BytesReference metadata = JsonXContent.contentBuilder()
|
||||
.startObject()
|
||||
.startObject("meta-data")
|
||||
.field("random", "value")
|
||||
.endObject()
|
||||
.endObject().bytes();
|
||||
XContentParser parser = JsonXContent.jsonXContent.createParser(metadata);
|
||||
try {
|
||||
MetaData.Builder.fromXContent(parser);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals("Unexpected field [random]", e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
public void testUnknownFieldIndexMetaData() throws IOException {
|
||||
BytesReference metadata = JsonXContent.contentBuilder()
|
||||
.startObject()
|
||||
.startObject("index_name")
|
||||
.field("random", "value")
|
||||
.endObject()
|
||||
.endObject().bytes();
|
||||
XContentParser parser = JsonXContent.jsonXContent.createParser(metadata);
|
||||
try {
|
||||
IndexMetaData.Builder.fromXContent(parser);
|
||||
fail();
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals("Unexpected field [random]", e.getMessage());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -59,7 +59,7 @@ public class WildcardExpressionResolverTests extends ESTestCase {
|
|||
IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver();
|
||||
|
||||
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen());
|
||||
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testYY*", "alias*"))), equalTo(newHashSet("alias1", "alias2", "alias3", "testYYY")));
|
||||
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testYY*", "alias*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
|
||||
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("-kuku"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
|
||||
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+test*", "-testYYY"))), equalTo(newHashSet("testXXX", "testXYY")));
|
||||
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("+testX*", "+testYYY"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
|
||||
|
|
|
@ -56,7 +56,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
|
@ -292,7 +291,6 @@ public class VersionsTests extends ESTestCase {
|
|||
}
|
||||
|
||||
iw.close();
|
||||
assertThat(IndexWriter.isLocked(iw.getDirectory()), is(false));
|
||||
ir.close();
|
||||
dir.close();
|
||||
}
|
||||
|
|
|
@ -17,14 +17,14 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.indexing;
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import org.apache.lucene.document.Field.Store;
|
||||
import org.apache.lucene.document.IntField;
|
||||
import org.apache.lucene.document.StringField;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.elasticsearch.index.indexing.IndexingSlowLog.SlowLogParsedDocumentPrinter;
|
||||
import org.elasticsearch.index.IndexingSlowLog.SlowLogParsedDocumentPrinter;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
|
@ -31,8 +31,11 @@ import org.apache.lucene.index.LeafReaderContext;
|
|||
import org.apache.lucene.index.LogByteSizeMergePolicy;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.join.BitSetProducer;
|
||||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.RAMDirectory;
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
|
@ -89,7 +92,8 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
writer.addDocument(document);
|
||||
writer.commit();
|
||||
|
||||
IndexReader reader = DirectoryReader.open(writer, false);
|
||||
DirectoryReader reader = DirectoryReader.open(writer, false);
|
||||
reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId(new Index("test"), 0));
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
|
||||
BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() {
|
||||
|
@ -114,6 +118,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
writer.forceMerge(1);
|
||||
reader.close();
|
||||
reader = DirectoryReader.open(writer, false);
|
||||
reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId(new Index("test"), 0));
|
||||
searcher = new IndexSearcher(reader);
|
||||
|
||||
assertThat(matchCount(filter, reader), equalTo(3));
|
||||
|
@ -139,7 +144,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
writer.addDocument(document);
|
||||
writer.commit();
|
||||
final DirectoryReader writerReader = DirectoryReader.open(writer, false);
|
||||
final IndexReader reader = randomBoolean() ? writerReader : ElasticsearchDirectoryReader.wrap(writerReader, new ShardId("test", 0));
|
||||
final IndexReader reader = ElasticsearchDirectoryReader.wrap(writerReader, new ShardId("test", 0));
|
||||
|
||||
final AtomicLong stats = new AtomicLong();
|
||||
final AtomicInteger onCacheCalls = new AtomicInteger();
|
||||
|
@ -192,4 +197,39 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testRejectOtherIndex() throws IOException {
|
||||
BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() {
|
||||
@Override
|
||||
public void onCache(ShardId shardId, Accountable accountable) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRemoval(ShardId shardId, Accountable accountable) {
|
||||
|
||||
}
|
||||
});
|
||||
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(
|
||||
dir,
|
||||
newIndexWriterConfig()
|
||||
);
|
||||
writer.addDocument(new Document());
|
||||
DirectoryReader reader = DirectoryReader.open(writer, true);
|
||||
writer.close();
|
||||
reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId(new Index("test2"), 0));
|
||||
|
||||
BitSetProducer producer = cache.getBitSetProducer(new MatchAllDocsQuery());
|
||||
|
||||
try {
|
||||
producer.getBitSet(reader.leaves().get(0));
|
||||
fail();
|
||||
} catch (IllegalStateException expected) {
|
||||
assertEquals("Trying to load bit set for index [test2] with cache of index [test]", expected.getMessage());
|
||||
} finally {
|
||||
IOUtils.close(reader, dir);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -46,12 +46,15 @@ import org.apache.lucene.util.BytesRef;
|
|||
import org.apache.lucene.util.FixedBitSet;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
|
||||
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
|
||||
import org.elasticsearch.index.fielddata.ordinals.GlobalOrdinalsIndexFieldData;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -385,7 +388,9 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
|
|||
writer.commit();
|
||||
}
|
||||
}
|
||||
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer, true));
|
||||
DirectoryReader directoryReader = DirectoryReader.open(writer, true);
|
||||
directoryReader = ElasticsearchDirectoryReader.wrap(directoryReader, new ShardId(new Index("test"), 0));
|
||||
IndexSearcher searcher = new IndexSearcher(directoryReader);
|
||||
IndexFieldData<?> fieldData = getForField("text");
|
||||
final Object missingValue;
|
||||
switch (randomInt(4)) {
|
||||
|
|
|
@ -36,11 +36,14 @@ import org.apache.lucene.search.TopFieldDocs;
|
|||
import org.apache.lucene.search.join.QueryBitSetProducer;
|
||||
import org.apache.lucene.search.join.ScoreMode;
|
||||
import org.apache.lucene.search.join.ToParentBlockJoinQuery;
|
||||
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.fielddata.AbstractFieldDataTestCase;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -216,7 +219,9 @@ public abstract class AbstractNumberNestedSortingTestCase extends AbstractFieldD
|
|||
writer.addDocument(document);
|
||||
|
||||
MultiValueMode sortMode = MultiValueMode.SUM;
|
||||
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer, false));
|
||||
DirectoryReader directoryReader = DirectoryReader.open(writer, false);
|
||||
directoryReader = ElasticsearchDirectoryReader.wrap(directoryReader, new ShardId(new Index("test"), 0));
|
||||
IndexSearcher searcher = new IndexSearcher(directoryReader);
|
||||
Query parentFilter = new TermQuery(new Term("__type", "parent"));
|
||||
Query childFilter = Queries.not(parentFilter);
|
||||
XFieldComparatorSource nestedComparatorSource = createFieldComparator("field2", sortMode, null, createNested(searcher, parentFilter, childFilter));
|
||||
|
|
|
@ -40,8 +40,10 @@ import org.apache.lucene.search.join.ScoreMode;
|
|||
import org.apache.lucene.search.join.ToParentBlockJoinQuery;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.TestUtil;
|
||||
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.fielddata.AbstractFieldDataTestCase;
|
||||
import org.elasticsearch.index.fielddata.FieldDataType;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
|
@ -49,6 +51,7 @@ import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource;
|
|||
import org.elasticsearch.index.fielddata.NoOrdinalsStringFieldDataTests;
|
||||
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
|
||||
import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -91,7 +94,9 @@ public class NestedSortingTests extends AbstractFieldDataTestCase {
|
|||
writer.commit();
|
||||
|
||||
MultiValueMode sortMode = randomFrom(Arrays.asList(MultiValueMode.MIN, MultiValueMode.MAX));
|
||||
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer, false));
|
||||
DirectoryReader reader = DirectoryReader.open(writer, false);
|
||||
reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId(new Index("test"), 0));
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
PagedBytesIndexFieldData indexFieldData1 = getForField("f");
|
||||
IndexFieldData<?> indexFieldData2 = NoOrdinalsStringFieldDataTests.hideOrdinals(indexFieldData1);
|
||||
final String missingValue = randomBoolean() ? null : TestUtil.randomSimpleString(getRandom(), 2);
|
||||
|
@ -274,7 +279,9 @@ public class NestedSortingTests extends AbstractFieldDataTestCase {
|
|||
writer.addDocument(document);
|
||||
|
||||
MultiValueMode sortMode = MultiValueMode.MIN;
|
||||
IndexSearcher searcher = new IndexSearcher(DirectoryReader.open(writer, false));
|
||||
DirectoryReader reader = DirectoryReader.open(writer, false);
|
||||
reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId(new Index("test"), 0));
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
PagedBytesIndexFieldData indexFieldData = getForField("field2");
|
||||
Query parentFilter = new TermQuery(new Term("__type", "parent"));
|
||||
Query childFilter = Queries.not(parentFilter);
|
||||
|
|
|
@ -76,8 +76,6 @@ import org.elasticsearch.index.engine.EngineException;
|
|||
import org.elasticsearch.index.fielddata.FieldDataStats;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.indexing.IndexingOperationListener;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.Mapping;
|
||||
import org.elasticsearch.index.mapper.ParseContext;
|
||||
|
@ -100,19 +98,23 @@ import java.io.IOException;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.StandardCopyOption;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.BrokenBarrierException;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
|
||||
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
|
||||
import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS;
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
|
@ -609,77 +611,77 @@ public class IndexShardTests extends ESSingleNodeTestCase {
|
|||
return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate);
|
||||
}
|
||||
|
||||
public void testPreIndex() throws IOException {
|
||||
createIndex("testpreindex");
|
||||
public void testIndexingOperationsListeners() throws IOException {
|
||||
createIndex("test_iol");
|
||||
ensureGreen();
|
||||
client().prepareIndex("test_iol", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get();
|
||||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
IndexService test = indicesService.indexService("testpreindex");
|
||||
IndexService test = indicesService.indexService("test_iol");
|
||||
IndexShard shard = test.getShardOrNull(0);
|
||||
ShardIndexingService shardIndexingService = shard.indexingService();
|
||||
final AtomicBoolean preIndexCalled = new AtomicBoolean(false);
|
||||
|
||||
shardIndexingService.addListener(new IndexingOperationListener() {
|
||||
AtomicInteger preIndex = new AtomicInteger();
|
||||
AtomicInteger postIndex = new AtomicInteger();
|
||||
AtomicInteger postIndexException = new AtomicInteger();
|
||||
AtomicInteger preDelete = new AtomicInteger();
|
||||
AtomicInteger postDelete = new AtomicInteger();
|
||||
AtomicInteger postDeleteException = new AtomicInteger();
|
||||
shard = reinitWithWrapper(test, shard, null, new IndexingOperationListener() {
|
||||
@Override
|
||||
public Engine.Index preIndex(Engine.Index operation) {
|
||||
preIndexCalled.set(true);
|
||||
return super.preIndex(operation);
|
||||
preIndex.incrementAndGet();
|
||||
return operation;
|
||||
}
|
||||
});
|
||||
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null);
|
||||
Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc);
|
||||
shard.index(index);
|
||||
assertTrue(preIndexCalled.get());
|
||||
}
|
||||
|
||||
public void testPostIndex() throws IOException {
|
||||
createIndex("testpostindex");
|
||||
ensureGreen();
|
||||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
IndexService test = indicesService.indexService("testpostindex");
|
||||
IndexShard shard = test.getShardOrNull(0);
|
||||
ShardIndexingService shardIndexingService = shard.indexingService();
|
||||
final AtomicBoolean postIndexCalled = new AtomicBoolean(false);
|
||||
|
||||
shardIndexingService.addListener(new IndexingOperationListener() {
|
||||
@Override
|
||||
public void postIndex(Engine.Index index) {
|
||||
postIndexCalled.set(true);
|
||||
super.postIndex(index);
|
||||
postIndex.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
postIndexException.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Delete preDelete(Engine.Delete delete) {
|
||||
preDelete.incrementAndGet();
|
||||
return delete;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete) {
|
||||
postDelete.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete, Throwable ex) {
|
||||
postDeleteException.incrementAndGet();
|
||||
|
||||
}
|
||||
});
|
||||
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null);
|
||||
Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc);
|
||||
shard.index(index);
|
||||
assertTrue(postIndexCalled.get());
|
||||
}
|
||||
assertEquals(1, preIndex.get());
|
||||
assertEquals(1, postIndex.get());
|
||||
assertEquals(0, postIndexException.get());
|
||||
assertEquals(0, preDelete.get());
|
||||
assertEquals(0, postDelete.get());
|
||||
assertEquals(0, postDeleteException.get());
|
||||
|
||||
public void testPostIndexWithException() throws IOException {
|
||||
createIndex("testpostindexwithexception");
|
||||
ensureGreen();
|
||||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
IndexService test = indicesService.indexService("testpostindexwithexception");
|
||||
IndexShard shard = test.getShardOrNull(0);
|
||||
ShardIndexingService shardIndexingService = shard.indexingService();
|
||||
Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1"));
|
||||
shard.delete(delete);
|
||||
|
||||
assertEquals(1, preIndex.get());
|
||||
assertEquals(1, postIndex.get());
|
||||
assertEquals(0, postIndexException.get());
|
||||
assertEquals(1, preDelete.get());
|
||||
assertEquals(1, postDelete.get());
|
||||
assertEquals(0, postDeleteException.get());
|
||||
|
||||
shard.close("Unexpected close", true);
|
||||
shard.state = IndexShardState.STARTED; // It will generate exception
|
||||
|
||||
final AtomicBoolean postIndexWithExceptionCalled = new AtomicBoolean(false);
|
||||
|
||||
shardIndexingService.addListener(new IndexingOperationListener() {
|
||||
@Override
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
assertNotNull(ex);
|
||||
postIndexWithExceptionCalled.set(true);
|
||||
super.postIndex(index, ex);
|
||||
}
|
||||
});
|
||||
|
||||
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null);
|
||||
Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc);
|
||||
|
||||
try {
|
||||
shard.index(index);
|
||||
fail();
|
||||
|
@ -687,7 +689,26 @@ public class IndexShardTests extends ESSingleNodeTestCase {
|
|||
|
||||
}
|
||||
|
||||
assertTrue(postIndexWithExceptionCalled.get());
|
||||
assertEquals(2, preIndex.get());
|
||||
assertEquals(1, postIndex.get());
|
||||
assertEquals(1, postIndexException.get());
|
||||
assertEquals(1, preDelete.get());
|
||||
assertEquals(1, postDelete.get());
|
||||
assertEquals(0, postDeleteException.get());
|
||||
try {
|
||||
shard.delete(delete);
|
||||
fail();
|
||||
}catch (IllegalIndexShardStateException e){
|
||||
|
||||
}
|
||||
|
||||
assertEquals(2, preIndex.get());
|
||||
assertEquals(1, postIndex.get());
|
||||
assertEquals(1, postIndexException.get());
|
||||
assertEquals(2, preDelete.get());
|
||||
assertEquals(1, postDelete.get());
|
||||
assertEquals(1, postDeleteException.get());
|
||||
|
||||
}
|
||||
|
||||
public void testMaybeFlush() throws Exception {
|
||||
|
@ -1038,11 +1059,11 @@ public class IndexShardTests extends ESSingleNodeTestCase {
|
|||
// test will fail due to unclosed searchers if the searcher is not released
|
||||
}
|
||||
|
||||
private final IndexShard reinitWithWrapper(IndexService indexService, IndexShard shard, IndexSearcherWrapper wrapper) throws IOException {
|
||||
private final IndexShard reinitWithWrapper(IndexService indexService, IndexShard shard, IndexSearcherWrapper wrapper, IndexingOperationListener... listeners) throws IOException {
|
||||
ShardRouting routing = new ShardRouting(shard.routingEntry());
|
||||
shard.close("simon says", true);
|
||||
NodeServicesProvider indexServices = indexService.getIndexServices();
|
||||
IndexShard newShard = new IndexShard(shard.shardId(), indexService.getIndexSettings(), shard.shardPath(), shard.store(), indexService.cache(), indexService.mapperService(), indexService.similarityService(), indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, indexServices);
|
||||
IndexShard newShard = new IndexShard(shard.shardId(), indexService.getIndexSettings(), shard.shardPath(), shard.store(), indexService.cache(), indexService.mapperService(), indexService.similarityService(), indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, indexServices, listeners);
|
||||
ShardRoutingHelper.reinit(routing);
|
||||
newShard.updateRoutingEntry(routing, false);
|
||||
DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT);
|
||||
|
@ -1054,4 +1075,29 @@ public class IndexShardTests extends ESSingleNodeTestCase {
|
|||
return newShard;
|
||||
}
|
||||
|
||||
public void testTranslogRecoverySyncsTranslog() throws IOException {
|
||||
createIndex("testindexfortranslogsync");
|
||||
client().admin().indices().preparePutMapping("testindexfortranslogsync").setType("testtype").setSource(jsonBuilder().startObject()
|
||||
.startObject("testtype")
|
||||
.startObject("properties")
|
||||
.startObject("foo")
|
||||
.field("type", "string")
|
||||
.endObject()
|
||||
.endObject().endObject().endObject()).get();
|
||||
ensureGreen();
|
||||
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
IndexService test = indicesService.indexService("testindexfortranslogsync");
|
||||
IndexShard shard = test.getShardOrNull(0);
|
||||
ShardRouting routing = new ShardRouting(shard.routingEntry());
|
||||
test.removeShard(0, "b/c britta says so");
|
||||
IndexShard newShard = test.createShard(routing);
|
||||
DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT);
|
||||
newShard.markAsRecovering("for testing", new RecoveryState(newShard.shardId(), routing.primary(), RecoveryState.Type.REPLICA, localNode, localNode));
|
||||
List<Translog.Operation> operations = new ArrayList<>();
|
||||
operations.add(new Translog.Index("testtype", "1", jsonBuilder().startObject().field("foo", "bar").endObject().bytes().toBytes()));
|
||||
newShard.prepareForIndexRecovery();
|
||||
newShard.performTranslogRecovery(true);
|
||||
newShard.performBatchRecovery(operations);
|
||||
assertFalse(newShard.getTranslog().syncNeeded());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,162 @@
|
|||
/*
|
||||
* 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.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class IndexingOperationListenerTests extends ESTestCase{
|
||||
|
||||
// this test also tests if calls are correct if one or more listeners throw exceptions
|
||||
public void testListenersAreExecuted() {
|
||||
AtomicInteger preIndex = new AtomicInteger();
|
||||
AtomicInteger postIndex = new AtomicInteger();
|
||||
AtomicInteger postIndexException = new AtomicInteger();
|
||||
AtomicInteger preDelete = new AtomicInteger();
|
||||
AtomicInteger postDelete = new AtomicInteger();
|
||||
AtomicInteger postDeleteException = new AtomicInteger();
|
||||
IndexingOperationListener listener = new IndexingOperationListener() {
|
||||
@Override
|
||||
public Engine.Index preIndex(Engine.Index operation) {
|
||||
preIndex.incrementAndGet();
|
||||
return operation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index) {
|
||||
postIndex.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
postIndexException.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Engine.Delete preDelete(Engine.Delete delete) {
|
||||
preDelete.incrementAndGet();
|
||||
return delete;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete) {
|
||||
postDelete.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete, Throwable ex) {
|
||||
postDeleteException.incrementAndGet();
|
||||
}
|
||||
};
|
||||
|
||||
IndexingOperationListener throwingListener = new IndexingOperationListener() {
|
||||
@Override
|
||||
public Engine.Index preIndex(Engine.Index operation) {
|
||||
throw new RuntimeException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index) {
|
||||
throw new RuntimeException(); }
|
||||
|
||||
@Override
|
||||
public void postIndex(Engine.Index index, Throwable ex) {
|
||||
throw new RuntimeException(); }
|
||||
|
||||
@Override
|
||||
public Engine.Delete preDelete(Engine.Delete delete) {
|
||||
throw new RuntimeException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete) {
|
||||
throw new RuntimeException(); }
|
||||
|
||||
@Override
|
||||
public void postDelete(Engine.Delete delete, Throwable ex) {
|
||||
throw new RuntimeException();
|
||||
}
|
||||
};
|
||||
final List<IndexingOperationListener> indexingOperationListeners = new ArrayList<>(Arrays.asList(listener, listener));
|
||||
if (randomBoolean()) {
|
||||
indexingOperationListeners.add(throwingListener);
|
||||
if (randomBoolean()) {
|
||||
indexingOperationListeners.add(throwingListener);
|
||||
}
|
||||
}
|
||||
Collections.shuffle(indexingOperationListeners, random());
|
||||
IndexingOperationListener.CompositeListener compositeListener = new IndexingOperationListener.CompositeListener(indexingOperationListeners, logger);
|
||||
Engine.Delete delete = new Engine.Delete("test", "1", new Term("_uid", "1"));
|
||||
Engine.Index index = new Engine.Index(new Term("_uid", "1"), null);
|
||||
compositeListener.postDelete(delete);
|
||||
assertEquals(0, preIndex.get());
|
||||
assertEquals(0, postIndex.get());
|
||||
assertEquals(0, postIndexException.get());
|
||||
assertEquals(0, preDelete.get());
|
||||
assertEquals(2, postDelete.get());
|
||||
assertEquals(0, postDeleteException.get());
|
||||
|
||||
compositeListener.postDelete(delete, new RuntimeException());
|
||||
assertEquals(0, preIndex.get());
|
||||
assertEquals(0, postIndex.get());
|
||||
assertEquals(0, postIndexException.get());
|
||||
assertEquals(0, preDelete.get());
|
||||
assertEquals(2, postDelete.get());
|
||||
assertEquals(2, postDeleteException.get());
|
||||
|
||||
compositeListener.preDelete(delete);
|
||||
assertEquals(0, preIndex.get());
|
||||
assertEquals(0, postIndex.get());
|
||||
assertEquals(0, postIndexException.get());
|
||||
assertEquals(2, preDelete.get());
|
||||
assertEquals(2, postDelete.get());
|
||||
assertEquals(2, postDeleteException.get());
|
||||
|
||||
compositeListener.postIndex(index);
|
||||
assertEquals(0, preIndex.get());
|
||||
assertEquals(2, postIndex.get());
|
||||
assertEquals(0, postIndexException.get());
|
||||
assertEquals(2, preDelete.get());
|
||||
assertEquals(2, postDelete.get());
|
||||
assertEquals(2, postDeleteException.get());
|
||||
|
||||
compositeListener.postIndex(index, new RuntimeException());
|
||||
assertEquals(0, preIndex.get());
|
||||
assertEquals(2, postIndex.get());
|
||||
assertEquals(2, postIndexException.get());
|
||||
assertEquals(2, preDelete.get());
|
||||
assertEquals(2, postDelete.get());
|
||||
assertEquals(2, postDeleteException.get());
|
||||
|
||||
compositeListener.preIndex(index);
|
||||
assertEquals(2, preIndex.get());
|
||||
assertEquals(2, postIndex.get());
|
||||
assertEquals(2, postIndexException.get());
|
||||
assertEquals(2, preDelete.get());
|
||||
assertEquals(2, postDelete.get());
|
||||
assertEquals(2, postDeleteException.get());
|
||||
}
|
||||
}
|
|
@ -1836,6 +1836,12 @@ public class TranslogTests extends ESTestCase {
|
|||
syncedDocs.clear();
|
||||
}
|
||||
}
|
||||
// we survived all the randomness!!!
|
||||
// lets close the translog and if it succeeds we are all synced again. If we don't do this we will close
|
||||
// it in the finally block but miss to copy over unsynced docs to syncedDocs and fail the assertion down the road...
|
||||
failableTLog.close();
|
||||
syncedDocs.addAll(unsynced);
|
||||
unsynced.clear();
|
||||
} catch (TranslogException | MockDirectoryWrapper.FakeIOException ex) {
|
||||
// fair enough
|
||||
} catch (IOException ex) {
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.indices.memory;
|
||||
package org.elasticsearch.indices;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
|
@ -16,7 +16,7 @@
|
|||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.elasticsearch.indices.memory;
|
||||
package org.elasticsearch.indices;
|
||||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
|
@ -24,8 +24,8 @@ import org.elasticsearch.common.unit.ByteSizeValue;
|
|||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
|
@ -33,6 +33,7 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
|
||||
|
@ -120,6 +121,11 @@ public class IndexingMemoryControllerTests extends ESSingleNodeTestCase {
|
|||
activeShards.add(shard);
|
||||
forceCheck();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
public void testShardAdditionAndRemoval() {
|
|
@ -151,18 +151,10 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
|
|||
|
||||
for (int i = 0; i < numSearches; i++) {
|
||||
SearchRequestBuilder searchRequestBuilder = client().prepareSearch().setQuery(QueryBuilders.matchAllQuery());
|
||||
switch (randomIntBetween(0, 5)) {
|
||||
case 5:
|
||||
case 4:
|
||||
case 3:
|
||||
searchRequestBuilder.addSort("test-str", SortOrder.ASC);
|
||||
// fall through - sometimes get both fields
|
||||
case 2:
|
||||
case 1:
|
||||
default:
|
||||
searchRequestBuilder.addSort("test-num", SortOrder.ASC);
|
||||
|
||||
if (random().nextBoolean()) {
|
||||
searchRequestBuilder.addSort("test-str", SortOrder.ASC);
|
||||
}
|
||||
searchRequestBuilder.addSort("test-num", SortOrder.ASC);
|
||||
boolean success = false;
|
||||
try {
|
||||
// Sort by the string and numeric fields, to load them into field data
|
||||
|
@ -249,6 +241,7 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
|
|||
if (random.nextDouble() < topLevelRatio) {
|
||||
throw new IOException("Forced top level Exception on [" + flag.name() + "]");
|
||||
}
|
||||
break;
|
||||
case Intersect:
|
||||
break;
|
||||
case Norms:
|
||||
|
|
|
@ -219,18 +219,9 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
assertThat(response.getMatches()[0].getId().string(), equalTo("100"));
|
||||
}
|
||||
|
||||
public void testSinglePercolatorRecovery() throws Exception {
|
||||
percolatorRecovery(false);
|
||||
}
|
||||
|
||||
@AwaitsFix(bugUrl = "sometimes reprodes with: gradle :core:integTest -Dtests.seed=21DDCAA92013B00C -Dtests.class=org.elasticsearch.percolator.RecoveryPercolatorIT -Dtests.method=\"testMultiPercolatorRecovery\"")
|
||||
public void testMultiPercolatorRecovery() throws Exception {
|
||||
percolatorRecovery(true);
|
||||
}
|
||||
|
||||
// 3 nodes, 2 primary + 2 replicas per primary, so each node should have a copy of the data.
|
||||
// We only start and stop nodes 2 and 3, so all requests should succeed and never be partial.
|
||||
private void percolatorRecovery(final boolean multiPercolate) throws Exception {
|
||||
public void testPercolatorRecovery() throws Exception {
|
||||
// 3 nodes, 2 primary + 2 replicas per primary, so each node should have a copy of the data.
|
||||
// We only start and stop nodes 2 and 3, so all requests should succeed and never be partial.
|
||||
internalCluster().startNode(settingsBuilder().put("node.stay", true));
|
||||
internalCluster().startNode(settingsBuilder().put("node.stay", false));
|
||||
internalCluster().startNode(settingsBuilder().put("node.stay", false));
|
||||
|
@ -260,56 +251,26 @@ public class RecoveryPercolatorIT extends ESIntegTestCase {
|
|||
|
||||
final AtomicBoolean run = new AtomicBoolean(true);
|
||||
final AtomicReference<Throwable> error = new AtomicReference<>();
|
||||
Runnable r = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
while (run.get()) {
|
||||
if (multiPercolate) {
|
||||
MultiPercolateRequestBuilder builder = client
|
||||
.prepareMultiPercolate();
|
||||
int numPercolateRequest = randomIntBetween(50, 100);
|
||||
|
||||
for (int i = 0; i < numPercolateRequest; i++) {
|
||||
PercolateRequestBuilder percolateBuilder = client.preparePercolate()
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries);
|
||||
if (randomBoolean()) {
|
||||
percolateBuilder.setGetRequest(Requests.getRequest("test").type("type").id("1"));
|
||||
} else {
|
||||
percolateBuilder.setPercolateDoc(docBuilder().setDoc(document));
|
||||
}
|
||||
builder.add(percolateBuilder);
|
||||
}
|
||||
|
||||
MultiPercolateResponse response = builder.get();
|
||||
assertThat(response.items().length, equalTo(numPercolateRequest));
|
||||
for (MultiPercolateResponse.Item item : response) {
|
||||
assertThat(item.isFailure(), equalTo(false));
|
||||
assertNoFailures(item.getResponse());
|
||||
assertThat(item.getResponse().getSuccessfulShards(), equalTo(item.getResponse().getTotalShards()));
|
||||
assertThat(item.getResponse().getCount(), equalTo((long) numQueries));
|
||||
assertThat(item.getResponse().getMatches().length, equalTo(numQueries));
|
||||
}
|
||||
} else {
|
||||
PercolateRequestBuilder percolateBuilder = client.preparePercolate()
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries);
|
||||
if (randomBoolean()) {
|
||||
percolateBuilder.setPercolateDoc(docBuilder().setDoc(document));
|
||||
} else {
|
||||
percolateBuilder.setGetRequest(Requests.getRequest("test").type("type").id("1"));
|
||||
}
|
||||
PercolateResponse response = percolateBuilder.get();
|
||||
assertNoFailures(response);
|
||||
assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards()));
|
||||
assertThat(response.getCount(), equalTo((long) numQueries));
|
||||
assertThat(response.getMatches().length, equalTo(numQueries));
|
||||
}
|
||||
Runnable r = () -> {
|
||||
try {
|
||||
while (run.get()) {
|
||||
PercolateRequestBuilder percolateBuilder = client.preparePercolate()
|
||||
.setIndices("test").setDocumentType("type").setSize(numQueries);
|
||||
if (randomBoolean()) {
|
||||
percolateBuilder.setPercolateDoc(docBuilder().setDoc(document));
|
||||
} else {
|
||||
percolateBuilder.setGetRequest(Requests.getRequest("test").type("type").id("1"));
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
logger.info("Error in percolate thread...", t);
|
||||
run.set(false);
|
||||
error.set(t);
|
||||
PercolateResponse response = percolateBuilder.get();
|
||||
assertNoFailures(response);
|
||||
assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards()));
|
||||
assertThat(response.getCount(), equalTo((long) numQueries));
|
||||
assertThat(response.getMatches().length, equalTo(numQueries));
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
logger.info("Error in percolate thread...", t);
|
||||
run.set(false);
|
||||
error.set(t);
|
||||
}
|
||||
};
|
||||
Thread t = new Thread(r);
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.routing;
|
||||
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.common.Priority;
|
||||
|
@ -27,9 +28,12 @@ import org.elasticsearch.test.ESIntegTestCase;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import static org.elasticsearch.cluster.metadata.AliasAction.newAddAliasAction;
|
||||
import static org.elasticsearch.common.util.set.Sets.newHashSet;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
|
@ -37,6 +41,22 @@ import static org.hamcrest.Matchers.nullValue;
|
|||
*
|
||||
*/
|
||||
public class AliasResolveRoutingIT extends ESIntegTestCase {
|
||||
|
||||
|
||||
// see https://github.com/elastic/elasticsearch/issues/13278
|
||||
public void testSearchClosedWildcardIndex() throws ExecutionException, InterruptedException {
|
||||
createIndex("test-0");
|
||||
createIndex("test-1");
|
||||
ensureGreen();
|
||||
client().admin().indices().prepareAliases().addAlias("test-0", "alias-0").addAlias("test-1", "alias-1").get();
|
||||
client().admin().indices().prepareClose("test-1").get();
|
||||
indexRandom(true, client().prepareIndex("test-0", "type1", "1").setSource("field1", "the quick brown fox jumps"),
|
||||
client().prepareIndex("test-0", "type1", "2").setSource("field1", "quick brown"),
|
||||
client().prepareIndex("test-0", "type1", "3").setSource("field1", "quick"));
|
||||
refresh("test-*");
|
||||
assertHitCount(client().prepareSearch().setIndices("alias-*").setIndicesOptions(IndicesOptions.lenientExpandOpen()).setQuery(matchQuery("_all", "quick")).get(), 3l);
|
||||
}
|
||||
|
||||
public void testResolveIndexRouting() throws Exception {
|
||||
createIndex("test1");
|
||||
createIndex("test2");
|
||||
|
|
|
@ -34,10 +34,13 @@ import org.apache.lucene.search.TermQuery;
|
|||
import org.apache.lucene.store.Directory;
|
||||
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
|
||||
import org.elasticsearch.common.compress.CompressedXContent;
|
||||
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.aggregations.Aggregator;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.BucketCollector;
|
||||
|
@ -110,7 +113,8 @@ public class NestedAggregatorTests extends ESSingleNodeTestCase {
|
|||
indexWriter.commit();
|
||||
indexWriter.close();
|
||||
|
||||
DirectoryReader directoryReader = DirectoryReader.open(directory);
|
||||
DirectoryReader directoryReader = DirectoryReader.open(directory);
|
||||
directoryReader = ElasticsearchDirectoryReader.wrap(directoryReader, new ShardId(new Index("test"), 0));
|
||||
IndexSearcher searcher = new IndexSearcher(directoryReader);
|
||||
|
||||
IndexService indexService = createIndex("test");
|
||||
|
|
|
@ -176,7 +176,10 @@ public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase {
|
|||
private long stopWaitingAt = -1;
|
||||
|
||||
public BlockingClusterStateListener(ClusterService clusterService, String blockOn, String countOn, Priority passThroughPriority) {
|
||||
this(clusterService, blockOn, countOn, passThroughPriority, TimeValue.timeValueMinutes(1));
|
||||
// Waiting for the 70 seconds here to make sure that the last check at 65 sec mark in assertBusyPendingTasks has a chance
|
||||
// to finish before we timeout on the cluster state block. Otherwise the last check in assertBusyPendingTasks kicks in
|
||||
// after the cluster state block clean up takes place and it's assert doesn't reflect the actual failure
|
||||
this(clusterService, blockOn, countOn, passThroughPriority, TimeValue.timeValueSeconds(70));
|
||||
}
|
||||
|
||||
public BlockingClusterStateListener(ClusterService clusterService, final String blockOn, final String countOn, Priority passThroughPriority, TimeValue timeout) {
|
||||
|
|
|
@ -1943,7 +1943,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
|||
.put("compress", randomBoolean())
|
||||
.put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
||||
|
||||
assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 20))
|
||||
assertAcked(prepareCreate("test-idx", 0, settingsBuilder().put("number_of_shards", between(1, 10))
|
||||
.put("number_of_replicas", 0)));
|
||||
ensureGreen();
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.threadpool;
|
|||
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.ThreadPool.Names;
|
||||
|
@ -89,6 +90,51 @@ public class UpdateThreadPoolSettingsTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testIndexingThreadPoolsMaxSize() throws InterruptedException {
|
||||
String threadPoolName = randomThreadPoolName();
|
||||
for (String name : new String[] {ThreadPool.Names.BULK, ThreadPool.Names.INDEX}) {
|
||||
ThreadPool threadPool = null;
|
||||
try {
|
||||
|
||||
int maxSize = EsExecutors.boundedNumberOfProcessors(Settings.EMPTY);
|
||||
|
||||
// try to create a too-big (maxSize+1) thread pool
|
||||
threadPool = new ThreadPool(settingsBuilder()
|
||||
.put("name", "testIndexingThreadPoolsMaxSize")
|
||||
.put("threadpool." + name + ".size", maxSize+1)
|
||||
.build());
|
||||
|
||||
// confirm it clipped us at the maxSize:
|
||||
assertEquals(maxSize, ((ThreadPoolExecutor) threadPool.executor(name)).getMaximumPoolSize());
|
||||
|
||||
ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
|
||||
threadPool.setClusterSettings(clusterSettings);
|
||||
|
||||
// update it to a tiny size:
|
||||
clusterSettings.applySettings(
|
||||
settingsBuilder()
|
||||
.put("threadpool." + name + ".size", 1)
|
||||
.build()
|
||||
);
|
||||
|
||||
// confirm it worked:
|
||||
assertEquals(1, ((ThreadPoolExecutor) threadPool.executor(name)).getMaximumPoolSize());
|
||||
|
||||
// try to update to too-big size:
|
||||
clusterSettings.applySettings(
|
||||
settingsBuilder()
|
||||
.put("threadpool." + name + ".size", maxSize+1)
|
||||
.build()
|
||||
);
|
||||
|
||||
// confirm it clipped us at the maxSize:
|
||||
assertEquals(maxSize, ((ThreadPoolExecutor) threadPool.executor(name)).getMaximumPoolSize());
|
||||
} finally {
|
||||
terminateThreadPoolIfNeeded(threadPool);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testUpdateSettingsCanNotChangeThreadPoolType() throws InterruptedException {
|
||||
String threadPoolName = randomThreadPoolName();
|
||||
ThreadPool.ThreadPoolType invalidThreadPoolType = randomIncorrectThreadPoolType(threadPoolName);
|
||||
|
@ -165,6 +211,14 @@ public class UpdateThreadPoolSettingsTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private static int getExpectedThreadPoolSize(Settings settings, String name, int size) {
|
||||
if (name.equals(ThreadPool.Names.BULK) || name.equals(ThreadPool.Names.INDEX)) {
|
||||
return Math.min(size, EsExecutors.boundedNumberOfProcessors(settings));
|
||||
} else {
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
public void testFixedExecutorType() throws InterruptedException {
|
||||
String threadPoolName = randomThreadPool(ThreadPool.ThreadPoolType.FIXED);
|
||||
ThreadPool threadPool = null;
|
||||
|
@ -179,12 +233,14 @@ public class UpdateThreadPoolSettingsTests extends ESTestCase {
|
|||
Settings settings = clusterSettings.applySettings(settingsBuilder()
|
||||
.put("threadpool." + threadPoolName + ".size", "15")
|
||||
.build());
|
||||
|
||||
int expectedSize = getExpectedThreadPoolSize(nodeSettings, threadPoolName, 15);
|
||||
assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED);
|
||||
assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(15));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(15));
|
||||
assertThat(info(threadPool, threadPoolName).getMin(), equalTo(15));
|
||||
assertThat(info(threadPool, threadPoolName).getMax(), equalTo(15));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(expectedSize));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(expectedSize));
|
||||
assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedSize));
|
||||
assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize));
|
||||
// keep alive does not apply to fixed thread pools
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getKeepAliveTime(TimeUnit.MINUTES), equalTo(0L));
|
||||
|
||||
|
@ -194,20 +250,23 @@ public class UpdateThreadPoolSettingsTests extends ESTestCase {
|
|||
// Make sure keep alive value is not used
|
||||
assertThat(info(threadPool, threadPoolName).getKeepAlive(), nullValue());
|
||||
// Make sure keep pool size value were reused
|
||||
assertThat(info(threadPool, threadPoolName).getMin(), equalTo(15));
|
||||
assertThat(info(threadPool, threadPoolName).getMax(), equalTo(15));
|
||||
assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedSize));
|
||||
assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize));
|
||||
assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(15));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(15));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(expectedSize));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(expectedSize));
|
||||
|
||||
// Change size
|
||||
Executor oldExecutor = threadPool.executor(threadPoolName);
|
||||
settings = clusterSettings.applySettings(settingsBuilder().put(settings).put("threadpool." + threadPoolName + ".size", "10").build());
|
||||
|
||||
expectedSize = getExpectedThreadPoolSize(nodeSettings, threadPoolName, 10);
|
||||
|
||||
// Make sure size values changed
|
||||
assertThat(info(threadPool, threadPoolName).getMax(), equalTo(10));
|
||||
assertThat(info(threadPool, threadPoolName).getMin(), equalTo(10));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(10));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(10));
|
||||
assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize));
|
||||
assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedSize));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(expectedSize));
|
||||
assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(expectedSize));
|
||||
// Make sure executor didn't change
|
||||
assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED);
|
||||
assertThat(threadPool.executor(threadPoolName), sameInstance(oldExecutor));
|
||||
|
|
|
@ -340,7 +340,7 @@ Possible options:
|
|||
multiple alias names separated by a comma.
|
||||
|
||||
`ignore_unavailable`::
|
||||
What to do is an specified index name doesn't
|
||||
What to do if an specified index name doesn't
|
||||
exist. If set to `true` then those indices are ignored.
|
||||
|
||||
The rest endpoint is: `/{index}/_alias/{alias}`.
|
||||
|
|
|
@ -11,9 +11,8 @@ $ curl -XDELETE 'http://localhost:9200/twitter/'
|
|||
The above example deletes an index called `twitter`. Specifying an index,
|
||||
alias or wildcard expression is required.
|
||||
|
||||
The delete index API can also be applied to more than one index, or on
|
||||
all indices (be careful!) by using `_all` or `*` as index.
|
||||
The delete index API can also be applied to more than one index, by either using a comma separated list, or on all indices (be careful!) by using `_all` or `*` as index.
|
||||
|
||||
In order to disable allowing to delete indices via wildcards or `_all`,
|
||||
set `action.destructive_requires_name` setting in the config to `true`.
|
||||
This setting can also be changed via the cluster update settings api.
|
||||
This setting can also be changed via the cluster update settings api.
|
||||
|
|
|
@ -55,7 +55,7 @@ GET my_index/_search
|
|||
"bool": {
|
||||
"must": [
|
||||
{ "match": { "user.first": "Alice" }},
|
||||
{ "match": { "user.last": "White" }}
|
||||
{ "match": { "user.last": "Smith" }}
|
||||
]
|
||||
}
|
||||
}
|
||||
|
|
|
@ -160,7 +160,7 @@ shared file system repository.
|
|||
Other repository backends are available in these official plugins:
|
||||
|
||||
* {plugins}/repository-s3.html[repository-s3] for S3 repository support
|
||||
* https://github.com/elasticsearch/elasticsearch-hadoop/tree/master/repository-hdfs[HDFS Plugin] for Hadoop environments
|
||||
* {plugins}/repository-hdfs.html[repository-hdfs] for HDFS repository support in Hadoop environments
|
||||
* {plugins}/repository-azure.html[repository-azure] for Azure storage repositories
|
||||
|
||||
[float]
|
||||
|
|
|
@ -73,6 +73,19 @@ the request with two different groups:
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
[float]
|
||||
[[global-search-timeout]]
|
||||
== Global Search Timeout
|
||||
|
||||
Individual searches can have a timeout as part of the
|
||||
<<search-request-body>>. Since search requests can originate from many
|
||||
sources, Elasticsearch has a dynamic cluster-level setting for a global
|
||||
search timeout that applies to all search requests that do not set a
|
||||
timeout in the <<search-request-body>>. The default value is no global
|
||||
timeout. The setting key is `search.default_search_timeout` and can be
|
||||
set using the <<cluster-update-settings>> endpoints. Setting this value
|
||||
to `-1` resets the global search timeout to no timeout.
|
||||
|
||||
--
|
||||
|
||||
include::search/search.asciidoc[]
|
||||
|
|
|
@ -21,6 +21,5 @@ defaults to `10`.
|
|||
--------------------------------------------------
|
||||
|
||||
Note that `from` + `size` can not be more than the `index.max_result_window`
|
||||
index setting which defaults to 10,000. See the
|
||||
{ref}/search-request-scroll.html[Scroll] api for more efficient ways to do deep
|
||||
scrolling.
|
||||
index setting which defaults to 10,000. See the <<search-request-scroll,Scroll>>
|
||||
API for more efficient ways to do deep scrolling.
|
||||
|
|
|
@ -11,7 +11,7 @@ it's very useful to know which inner nested objects (in the case of nested) or c
|
|||
of parent/child) caused certain information to be returned. The inner hits feature can be used for this. This feature
|
||||
returns per search hit in the search response additional nested hits that caused a search hit to match in a different scope.
|
||||
|
||||
Inner hits can be used by defining a `inner_hits` definition on a `nested`, `has_child` or `has_parent` query and filter.
|
||||
Inner hits can be used by defining an `inner_hits` definition on a `nested`, `has_child` or `has_parent` query and filter.
|
||||
The structure looks like this:
|
||||
|
||||
[source,js]
|
||||
|
@ -23,7 +23,7 @@ The structure looks like this:
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
If `_inner_hits` is defined on a query that supports it then each search hit will contain a `inner_hits` json object with the following structure:
|
||||
If `_inner_hits` is defined on a query that supports it then each search hit will contain an `inner_hits` json object with the following structure:
|
||||
|
||||
[source,js]
|
||||
--------------------------------------------------
|
||||
|
@ -234,7 +234,7 @@ An example of a response snippet that could be generated from the above search r
|
|||
Besides defining inner hits on query and filters, inner hits can also be defined as a top level construct alongside the
|
||||
`query` and `aggregations` definition. The main reason for using the top level inner hits definition is to let the
|
||||
inner hits return documents that don't match with the main query. Also inner hits definitions can be nested via the
|
||||
top level notation. Other then that the inner hit definition inside the query should be used, because that is the most
|
||||
top level notation. Other than that, the inner hit definition inside the query should be used because that is the most
|
||||
compact way for defining inner hits.
|
||||
|
||||
The following snippet explains the basic structure of inner hits defined at the top level of the search request body:
|
||||
|
@ -254,7 +254,7 @@ The following snippet explains the basic structure of inner hits defined at the
|
|||
}
|
||||
--------------------------------------------------
|
||||
|
||||
Inside the `inner_hits` definition, first the name if the inner hit is defined then whether the inner_hit
|
||||
Inside the `inner_hits` definition, first the name of the inner hit is defined then whether the inner_hit
|
||||
is a nested by defining `path` or a parent/child based definition by defining `type`. The next object layer contains
|
||||
the name of the nested object field if the inner_hits is nested or the parent or child type if the inner_hit definition
|
||||
is parent/child based.
|
||||
|
|
|
@ -35,6 +35,3 @@ dependencyLicenses {
|
|||
mapping from: /asm-.*/, to: 'asm'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << '-Xlint:-rawtypes'
|
||||
compileTestJava.options.compilerArgs << '-Xlint:-rawtypes'
|
||||
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
|
||||
package org.elasticsearch.script.expression;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
|
@ -26,10 +30,6 @@ import org.elasticsearch.index.fielddata.AtomicFieldData;
|
|||
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A ValueSource to create FunctionValues to get the count of the number of values in a field for a document.
|
||||
*/
|
||||
|
@ -43,6 +43,7 @@ public class CountMethodValueSource extends ValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes") // ValueSource uses a rawtype
|
||||
public FunctionValues getValues(Map context, LeafReaderContext leaf) throws IOException {
|
||||
AtomicFieldData leafData = fieldData.load(leaf);
|
||||
assert(leafData instanceof AtomicNumericFieldData);
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
|
||||
package org.elasticsearch.script.expression;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.elasticsearch.index.fielddata.AtomicFieldData;
|
||||
|
@ -26,10 +30,6 @@ import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
|
|||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
class DateMethodValueSource extends FieldDataValueSource {
|
||||
|
||||
protected final String methodName;
|
||||
|
@ -45,6 +45,7 @@ class DateMethodValueSource extends FieldDataValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes") // ValueSource uses a rawtype
|
||||
public FunctionValues getValues(Map context, LeafReaderContext leaf) throws IOException {
|
||||
AtomicFieldData leafData = fieldData.load(leaf);
|
||||
assert(leafData instanceof AtomicNumericFieldData);
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
|
||||
package org.elasticsearch.script.expression;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
|
@ -27,10 +31,6 @@ import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
|
|||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.search.MultiValueMode;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A {@link ValueSource} wrapper for field data.
|
||||
*/
|
||||
|
@ -67,6 +67,7 @@ class FieldDataValueSource extends ValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes") // ValueSource uses a rawtype
|
||||
public FunctionValues getValues(Map context, LeafReaderContext leaf) throws IOException {
|
||||
AtomicFieldData leafData = fieldData.load(leaf);
|
||||
assert(leafData instanceof AtomicNumericFieldData);
|
||||
|
|
|
@ -19,13 +19,13 @@
|
|||
|
||||
package org.elasticsearch.script.expression;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.queries.function.FunctionValues;
|
||||
import org.apache.lucene.queries.function.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A {@link ValueSource} which has a stub {@link FunctionValues} that holds a dynamically replaceable constant double.
|
||||
*/
|
||||
|
@ -37,6 +37,7 @@ class ReplaceableConstValueSource extends ValueSource {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes") // ValueSource uses a rawtype
|
||||
public FunctionValues getValues(Map map, LeafReaderContext atomicReaderContext) throws IOException {
|
||||
return fv;
|
||||
}
|
||||
|
|
|
@ -19,6 +19,12 @@
|
|||
|
||||
package org.elasticsearch.script.expression;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.expressions.Expression;
|
||||
import org.apache.lucene.expressions.js.JavascriptCompiler;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
|
@ -47,12 +53,6 @@ import org.elasticsearch.search.sort.SortOrder;
|
|||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.hamcrest.ElasticsearchAssertions;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
|
||||
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
|
||||
import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.bucketScript;
|
||||
|
@ -121,7 +121,7 @@ public class MoreExpressionTests extends ESIntegTestCase {
|
|||
client().prepareIndex("test", "doc", "1").setSource("text", "hello goodbye"),
|
||||
client().prepareIndex("test", "doc", "2").setSource("text", "hello hello hello goodbye"),
|
||||
client().prepareIndex("test", "doc", "3").setSource("text", "hello hello goodebye"));
|
||||
ScoreFunctionBuilder score = ScoreFunctionBuilders.scriptFunction(new Script("1 / _score", ScriptType.INLINE, "expression", null));
|
||||
ScoreFunctionBuilder<?> score = ScoreFunctionBuilders.scriptFunction(new Script("1 / _score", ScriptType.INLINE, "expression", null));
|
||||
SearchRequestBuilder req = client().prepareSearch().setIndices("test");
|
||||
req.setQuery(QueryBuilders.functionScoreQuery(QueryBuilders.termQuery("text", "hello"), score).boostMode(CombineFunction.REPLACE));
|
||||
req.setSearchType(SearchType.DFS_QUERY_THEN_FETCH); // make sure DF is consistent
|
||||
|
|
|
@ -30,6 +30,3 @@ dependencies {
|
|||
dependencyLicenses {
|
||||
mapping from: /lucene-.*/, to: 'lucene'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked"
|
||||
|
||||
|
|
|
@ -19,6 +19,9 @@
|
|||
|
||||
package org.elasticsearch.index.analysis;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
|
||||
import org.apache.commons.codec.Encoder;
|
||||
import org.apache.commons.codec.language.Caverphone1;
|
||||
import org.apache.commons.codec.language.Caverphone2;
|
||||
|
@ -43,9 +46,6 @@ import org.elasticsearch.index.analysis.phonetic.HaasePhonetik;
|
|||
import org.elasticsearch.index.analysis.phonetic.KoelnerPhonetik;
|
||||
import org.elasticsearch.index.analysis.phonetic.Nysiis;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -122,7 +122,7 @@ public class PhoneticTokenFilterFactory extends AbstractTokenFilterFactory {
|
|||
if (encoder == null) {
|
||||
if (ruletype != null && nametype != null) {
|
||||
if (languageset != null) {
|
||||
final LanguageSet languages = LanguageSet.from(new HashSet(Arrays.asList(languageset)));
|
||||
final LanguageSet languages = LanguageSet.from(new HashSet<>(Arrays.asList(languageset)));
|
||||
return new BeiderMorseFilter(tokenStream, new PhoneticEngine(nametype, ruletype, true), languages);
|
||||
}
|
||||
return new BeiderMorseFilter(tokenStream, new PhoneticEngine(nametype, ruletype, true));
|
||||
|
|
|
@ -19,9 +19,6 @@
|
|||
|
||||
package org.elasticsearch.index.analysis.phonetic;
|
||||
|
||||
import org.apache.commons.codec.EncoderException;
|
||||
import org.apache.commons.codec.StringEncoder;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
|
@ -31,6 +28,9 @@ import java.util.Set;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.codec.EncoderException;
|
||||
import org.apache.commons.codec.StringEncoder;
|
||||
|
||||
/**
|
||||
* Kölner Phonetik
|
||||
*
|
||||
|
@ -49,13 +49,13 @@ public class KoelnerPhonetik implements StringEncoder {
|
|||
private static final String[] POSTEL_VARIATIONS_REPLACEMENTS = {"OWN", "AUN", "RW", "RB", "RSK", "WSK"};
|
||||
private Pattern[] variationsPatterns;
|
||||
private boolean primary = false;
|
||||
private final Set<Character> csz = new HashSet(Arrays.asList(
|
||||
private final Set<Character> csz = new HashSet<>(Arrays.asList(
|
||||
'C', 'S', 'Z'));
|
||||
private final Set<Character> ckq = new HashSet(Arrays.asList(
|
||||
private final Set<Character> ckq = new HashSet<>(Arrays.asList(
|
||||
'C', 'K', 'Q'));
|
||||
private final Set<Character> aouhkxq = new HashSet(Arrays.asList(
|
||||
private final Set<Character> aouhkxq = new HashSet<>(Arrays.asList(
|
||||
'A', 'O', 'U', 'H', 'K', 'X', 'Q'));
|
||||
private final Set<Character> ahkloqrux = new HashSet(Arrays.asList(
|
||||
private final Set<Character> ahkloqrux = new HashSet<>(Arrays.asList(
|
||||
'A', 'H', 'K', 'L', 'O', 'Q', 'R', 'U', 'X'));
|
||||
|
||||
/**
|
||||
|
@ -139,10 +139,10 @@ public class KoelnerPhonetik implements StringEncoder {
|
|||
|
||||
private List<String> partition(String str) {
|
||||
String primaryForm = str;
|
||||
List<String> parts = new ArrayList();
|
||||
List<String> parts = new ArrayList<>();
|
||||
parts.add(primaryForm.replaceAll("[^\\p{L}\\p{N}]", ""));
|
||||
if (!primary) {
|
||||
List<String> tmpParts = new ArrayList();
|
||||
List<String> tmpParts = new ArrayList<>();
|
||||
tmpParts.addAll((Arrays.asList(str.split("[\\p{Z}\\p{C}\\p{P}]"))));
|
||||
int numberOfParts = tmpParts.size();
|
||||
while (tmpParts.size() > 0) {
|
||||
|
@ -156,9 +156,9 @@ public class KoelnerPhonetik implements StringEncoder {
|
|||
tmpParts.remove(0);
|
||||
}
|
||||
}
|
||||
List<String> variations = new ArrayList();
|
||||
List<String> variations = new ArrayList<>();
|
||||
for (int i = 0; i < parts.size(); i++) {
|
||||
List variation = getVariations(parts.get(i));
|
||||
List<String> variation = getVariations(parts.get(i));
|
||||
if (variation != null) {
|
||||
variations.addAll(variation);
|
||||
}
|
||||
|
@ -166,9 +166,9 @@ public class KoelnerPhonetik implements StringEncoder {
|
|||
return variations;
|
||||
}
|
||||
|
||||
private List getVariations(String str) {
|
||||
private List<String> getVariations(String str) {
|
||||
int position = 0;
|
||||
List<String> variations = new ArrayList();
|
||||
List<String> variations = new ArrayList<>();
|
||||
variations.add("");
|
||||
while (position < str.length()) {
|
||||
int i = 0;
|
||||
|
@ -182,7 +182,7 @@ public class KoelnerPhonetik implements StringEncoder {
|
|||
}
|
||||
if (substPos >= position) {
|
||||
i--;
|
||||
List<String> varNew = new ArrayList();
|
||||
List<String> varNew = new ArrayList<>();
|
||||
String prevPart = str.substring(position, substPos);
|
||||
for (int ii = 0; ii < variations.size(); ii++) {
|
||||
String tmp = variations.get(ii);
|
||||
|
|
|
@ -56,11 +56,9 @@ dependencyLicenses {
|
|||
mapping from: /jaxb-.*/, to: 'jaxb'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << '-Xlint:-path,-serial,-static,-unchecked'
|
||||
compileJava.options.compilerArgs << '-Xlint:-path,-serial,-unchecked'
|
||||
// TODO: why is deprecation needed here but not in maven....?
|
||||
compileJava.options.compilerArgs << '-Xlint:-deprecation'
|
||||
// TODO: and why does this static not show up in maven...
|
||||
compileTestJava.options.compilerArgs << '-Xlint:-static'
|
||||
|
||||
thirdPartyAudit.excludes = [
|
||||
// classes are missing
|
||||
|
|
|
@ -47,11 +47,7 @@ public class AzureDiscoveryModule extends AbstractModule {
|
|||
private Settings settings;
|
||||
|
||||
// pkg private so it is settable by tests
|
||||
static Class<? extends AzureComputeService> computeServiceImpl = AzureComputeServiceImpl.class;
|
||||
|
||||
public static Class<? extends AzureComputeService> getComputeServiceImpl() {
|
||||
return computeServiceImpl;
|
||||
}
|
||||
Class<? extends AzureComputeService> computeServiceImpl = AzureComputeServiceImpl.class;
|
||||
|
||||
@Inject
|
||||
public AzureDiscoveryModule(Settings settings) {
|
||||
|
|
|
@ -25,8 +25,6 @@ esplugin {
|
|||
// no unit tests
|
||||
test.enabled = false
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-rawtypes"
|
||||
|
||||
configurations {
|
||||
exampleFixture
|
||||
}
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
|
||||
package org.elasticsearch.plugin.example;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.elasticsearch.common.component.LifecycleComponent;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.Module;
|
||||
|
@ -28,10 +32,6 @@ import org.elasticsearch.plugins.Plugin;
|
|||
import org.elasticsearch.repositories.RepositoriesModule;
|
||||
import org.elasticsearch.rest.action.cat.AbstractCatAction;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Example of a plugin.
|
||||
*/
|
||||
|
@ -59,6 +59,7 @@ public class JvmExamplePlugin extends Plugin {
|
|||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("rawtypes") // Plugin use a rawtype
|
||||
public Collection<Class<? extends LifecycleComponent>> nodeServices() {
|
||||
Collection<Class<? extends LifecycleComponent>> services = new ArrayList<>();
|
||||
return services;
|
||||
|
|
|
@ -26,13 +26,9 @@ dependencies {
|
|||
compile 'org.mozilla:rhino:1.7.7'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked"
|
||||
compileTestJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked"
|
||||
|
||||
integTest {
|
||||
cluster {
|
||||
systemProperty 'es.script.inline', 'on'
|
||||
systemProperty 'es.script.indexed', 'on'
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,18 @@
|
|||
|
||||
package org.elasticsearch.script.javascript;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.security.AccessControlContext;
|
||||
import java.security.AccessController;
|
||||
import java.security.CodeSource;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.security.cert.Certificate;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.elasticsearch.SpecialPermission;
|
||||
|
@ -49,18 +61,6 @@ import org.mozilla.javascript.ScriptableObject;
|
|||
import org.mozilla.javascript.SecurityController;
|
||||
import org.mozilla.javascript.WrapFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.MalformedURLException;
|
||||
import java.net.URL;
|
||||
import java.security.AccessControlContext;
|
||||
import java.security.AccessController;
|
||||
import java.security.CodeSource;
|
||||
import java.security.PrivilegedAction;
|
||||
import java.security.cert.Certificate;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -348,12 +348,14 @@ public class JavaScriptScriptEngineService extends AbstractComponent implements
|
|||
setJavaPrimitiveWrap(false); // RingoJS does that..., claims its annoying...
|
||||
}
|
||||
|
||||
public Scriptable wrapAsJavaObject(Context cx, Scriptable scope, Object javaObject, Class staticType) {
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Scriptable wrapAsJavaObject(Context cx, Scriptable scope, Object javaObject, Class<?> staticType) {
|
||||
if (javaObject instanceof Map) {
|
||||
return NativeMap.wrap(scope, (Map) javaObject);
|
||||
return NativeMap.wrap(scope, (Map<Object, Object>) javaObject);
|
||||
}
|
||||
if (javaObject instanceof List) {
|
||||
return NativeList.wrap(scope, (List) javaObject, staticType);
|
||||
return NativeList.wrap(scope, (List<Object>) javaObject, staticType);
|
||||
}
|
||||
return super.wrapAsJavaObject(cx, scope, javaObject, staticType);
|
||||
}
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package org.elasticsearch.script.javascript.support;
|
||||
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
import org.mozilla.javascript.Wrapper;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
import org.mozilla.javascript.Wrapper;
|
||||
|
||||
/**
|
||||
* Wrapper for exposing maps in Rhino scripts.
|
||||
*
|
||||
|
@ -55,26 +55,17 @@ public class NativeMap implements Scriptable, Wrapper {
|
|||
this.map = map;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Wrapper#unwrap()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object unwrap() {
|
||||
return map;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getClassName()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public String getClassName() {
|
||||
return "NativeMap";
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#get(java.lang.String, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object get(String name, Scriptable start) {
|
||||
// get the property from the underlying QName map
|
||||
if ("length".equals(name)) {
|
||||
|
@ -84,69 +75,47 @@ public class NativeMap implements Scriptable, Wrapper {
|
|||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#get(int, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object get(int index, Scriptable start) {
|
||||
Object value = null;
|
||||
int i = 0;
|
||||
Iterator itrValues = map.values().iterator();
|
||||
Iterator<Object> itrValues = map.values().iterator();
|
||||
while (i++ <= index && itrValues.hasNext()) {
|
||||
value = itrValues.next();
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#has(java.lang.String, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean has(String name, Scriptable start) {
|
||||
// locate the property in the underlying map
|
||||
return map.containsKey(name);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#has(int, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean has(int index, Scriptable start) {
|
||||
return (index >= 0 && map.values().size() > index);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#put(java.lang.String, org.mozilla.javascript.Scriptable, java.lang.Object)
|
||||
*/
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void put(String name, Scriptable start, Object value) {
|
||||
map.put(name, value);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#put(int, org.mozilla.javascript.Scriptable, java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void put(int index, Scriptable start, Object value) {
|
||||
// TODO: implement?
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#delete(java.lang.String)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void delete(String name) {
|
||||
map.remove(name);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#delete(int)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void delete(int index) {
|
||||
int i = 0;
|
||||
Iterator itrKeys = map.keySet().iterator();
|
||||
Iterator<Object> itrKeys = map.keySet().iterator();
|
||||
while (i <= index && itrKeys.hasNext()) {
|
||||
Object key = itrKeys.next();
|
||||
if (i == index) {
|
||||
|
@ -156,58 +125,37 @@ public class NativeMap implements Scriptable, Wrapper {
|
|||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getPrototype()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Scriptable getPrototype() {
|
||||
return this.prototype;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#setPrototype(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void setPrototype(Scriptable prototype) {
|
||||
this.prototype = prototype;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getParentScope()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Scriptable getParentScope() {
|
||||
return this.parentScope;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#setParentScope(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void setParentScope(Scriptable parent) {
|
||||
this.parentScope = parent;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getIds()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object[] getIds() {
|
||||
return map.keySet().toArray();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getDefaultValue(java.lang.Class)
|
||||
*/
|
||||
|
||||
public Object getDefaultValue(Class hint) {
|
||||
@Override
|
||||
public Object getDefaultValue(Class<?> hint) {
|
||||
return null;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#hasInstance(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean hasInstance(Scriptable value) {
|
||||
if (!(value instanceof Wrapper))
|
||||
return false;
|
||||
|
|
|
@ -19,13 +19,6 @@
|
|||
|
||||
package org.elasticsearch.script.javascript.support;
|
||||
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.IdScriptableObject;
|
||||
import org.mozilla.javascript.NativeArray;
|
||||
import org.mozilla.javascript.ScriptRuntime;
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
import org.mozilla.javascript.Wrapper;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
|
@ -33,6 +26,13 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.mozilla.javascript.Context;
|
||||
import org.mozilla.javascript.IdScriptableObject;
|
||||
import org.mozilla.javascript.NativeArray;
|
||||
import org.mozilla.javascript.ScriptRuntime;
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
import org.mozilla.javascript.Wrapper;
|
||||
|
||||
/**
|
||||
* Value Converter to marshal objects between Java and Javascript.
|
||||
*
|
||||
|
@ -126,6 +126,7 @@ public final class ScriptValueConverter {
|
|||
value = list;
|
||||
} else if (value instanceof Map) {
|
||||
// ensure each value in the Map is unwrapped (which may have been an unwrapped NativeMap!)
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<Object, Object> map = (Map<Object, Object>) value;
|
||||
Map<Object, Object> copyMap = new HashMap<Object, Object>(map.size());
|
||||
for (Object key : map.keySet()) {
|
||||
|
@ -157,6 +158,7 @@ public final class ScriptValueConverter {
|
|||
Context.getCurrentContext(), scope, TYPE_DATE, new Object[]{date.getTime()});
|
||||
} else if (value instanceof Collection) {
|
||||
// recursively convert each value in the collection
|
||||
@SuppressWarnings("unchecked")
|
||||
Collection<Object> collection = (Collection<Object>) value;
|
||||
Object[] array = new Object[collection.size()];
|
||||
int index = 0;
|
||||
|
@ -166,7 +168,9 @@ public final class ScriptValueConverter {
|
|||
// convert array to a native JavaScript Array
|
||||
value = Context.getCurrentContext().newArray(scope, array);
|
||||
} else if (value instanceof Map) {
|
||||
value = NativeMap.wrap(scope, (Map) value);
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<Object, Object> map = (Map<Object, Object>) value;
|
||||
value = NativeMap.wrap(scope, map);
|
||||
}
|
||||
|
||||
// simple numbers, strings and booleans are wrapped automatically by Rhino
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package org.elasticsearch.script.javascript.support;
|
||||
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
|
||||
/**
|
||||
* Implementation of a Scriptable Map. This is the best choice for maps that want to represent
|
||||
* JavaScript associative arrays - allowing access via key and integer index. It maintains and
|
||||
|
@ -53,6 +53,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#getClassName()
|
||||
*/
|
||||
@Override
|
||||
public String getClassName() {
|
||||
return "ScriptableMap";
|
||||
}
|
||||
|
@ -60,6 +61,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#get(java.lang.String, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public Object get(String name, Scriptable start) {
|
||||
// get the property from the underlying QName map
|
||||
if ("length".equals(name)) {
|
||||
|
@ -72,10 +74,11 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#get(int, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public Object get(int index, Scriptable start) {
|
||||
Object value = null;
|
||||
int i = 0;
|
||||
Iterator itrValues = this.values().iterator();
|
||||
Iterator<V> itrValues = this.values().iterator();
|
||||
while (i++ <= index && itrValues.hasNext()) {
|
||||
value = itrValues.next();
|
||||
}
|
||||
|
@ -85,6 +88,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#has(java.lang.String, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public boolean has(String name, Scriptable start) {
|
||||
// locate the property in the underlying map
|
||||
return containsKey(name);
|
||||
|
@ -93,6 +97,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#has(int, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public boolean has(int index, Scriptable start) {
|
||||
return (index >= 0 && this.values().size() > index);
|
||||
}
|
||||
|
@ -100,6 +105,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#put(java.lang.String, org.mozilla.javascript.Scriptable, java.lang.Object)
|
||||
*/
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public void put(String name, Scriptable start, Object value) {
|
||||
// add the property to the underlying QName map
|
||||
|
@ -109,6 +115,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#put(int, org.mozilla.javascript.Scriptable, java.lang.Object)
|
||||
*/
|
||||
@Override
|
||||
public void put(int index, Scriptable start, Object value) {
|
||||
// TODO: implement?
|
||||
}
|
||||
|
@ -116,6 +123,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#delete(java.lang.String)
|
||||
*/
|
||||
@Override
|
||||
public void delete(String name) {
|
||||
// remove the property from the underlying QName map
|
||||
remove(name);
|
||||
|
@ -124,9 +132,10 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#delete(int)
|
||||
*/
|
||||
@Override
|
||||
public void delete(int index) {
|
||||
int i = 0;
|
||||
Iterator itrKeys = this.keySet().iterator();
|
||||
Iterator<K> itrKeys = this.keySet().iterator();
|
||||
while (i <= index && itrKeys.hasNext()) {
|
||||
Object key = itrKeys.next();
|
||||
if (i == index) {
|
||||
|
@ -139,6 +148,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#getPrototype()
|
||||
*/
|
||||
@Override
|
||||
public Scriptable getPrototype() {
|
||||
return this.prototype;
|
||||
}
|
||||
|
@ -146,6 +156,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#setPrototype(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public void setPrototype(Scriptable prototype) {
|
||||
this.prototype = prototype;
|
||||
}
|
||||
|
@ -153,6 +164,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#getParentScope()
|
||||
*/
|
||||
@Override
|
||||
public Scriptable getParentScope() {
|
||||
return this.parentScope;
|
||||
}
|
||||
|
@ -160,6 +172,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#setParentScope(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public void setParentScope(Scriptable parent) {
|
||||
this.parentScope = parent;
|
||||
}
|
||||
|
@ -167,6 +180,7 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#getIds()
|
||||
*/
|
||||
@Override
|
||||
public Object[] getIds() {
|
||||
return keySet().toArray();
|
||||
}
|
||||
|
@ -174,13 +188,15 @@ public class ScriptableLinkedHashMap<K, V> extends LinkedHashMap<K, V> implement
|
|||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#getDefaultValue(java.lang.Class)
|
||||
*/
|
||||
public Object getDefaultValue(Class hint) {
|
||||
@Override
|
||||
public Object getDefaultValue(Class<?> hint) {
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* @see org.mozilla.javascript.Scriptable#hasInstance(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
@Override
|
||||
public boolean hasInstance(Scriptable instance) {
|
||||
return instance instanceof ScriptableLinkedHashMap;
|
||||
}
|
||||
|
|
|
@ -19,14 +19,14 @@
|
|||
|
||||
package org.elasticsearch.script.javascript.support;
|
||||
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
import org.mozilla.javascript.Wrapper;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.mozilla.javascript.Scriptable;
|
||||
import org.mozilla.javascript.Wrapper;
|
||||
|
||||
/**
|
||||
* Implementation of a Scriptable Map. This is the best choice where you want values to be
|
||||
* persisted directly to an underlying map supplied on construction. The class automatically
|
||||
|
@ -37,8 +37,8 @@ import java.util.Set;
|
|||
*
|
||||
*
|
||||
*/
|
||||
public class ScriptableWrappedMap implements ScriptableMap, Wrapper {
|
||||
private Map map;
|
||||
public class ScriptableWrappedMap implements ScriptableMap<Object, Object>, Wrapper {
|
||||
private Map<Object, Object> map;
|
||||
private Scriptable parentScope;
|
||||
private Scriptable prototype;
|
||||
|
||||
|
@ -54,38 +54,29 @@ public class ScriptableWrappedMap implements ScriptableMap, Wrapper {
|
|||
/**
|
||||
* Construct
|
||||
*/
|
||||
public ScriptableWrappedMap(Map map) {
|
||||
public ScriptableWrappedMap(Map<Object, Object> map) {
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct
|
||||
*/
|
||||
public ScriptableWrappedMap(Scriptable scope, Map map) {
|
||||
public ScriptableWrappedMap(Scriptable scope, Map<Object, Object> map) {
|
||||
this.parentScope = scope;
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Wrapper#unwrap()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object unwrap() {
|
||||
return map;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getClassName()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public String getClassName() {
|
||||
return "ScriptableWrappedMap";
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#get(java.lang.String, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object get(String name, Scriptable start) {
|
||||
// get the property from the underlying QName map
|
||||
if ("length".equals(name)) {
|
||||
|
@ -95,69 +86,47 @@ public class ScriptableWrappedMap implements ScriptableMap, Wrapper {
|
|||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#get(int, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object get(int index, Scriptable start) {
|
||||
Object value = null;
|
||||
int i = 0;
|
||||
Iterator itrValues = map.values().iterator();
|
||||
Iterator<Object> itrValues = map.values().iterator();
|
||||
while (i++ <= index && itrValues.hasNext()) {
|
||||
value = itrValues.next();
|
||||
}
|
||||
return ScriptValueConverter.wrapValue(this.parentScope != null ? this.parentScope : start, value);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#has(java.lang.String, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean has(String name, Scriptable start) {
|
||||
// locate the property in the underlying map
|
||||
return map.containsKey(name);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#has(int, org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean has(int index, Scriptable start) {
|
||||
return (index >= 0 && map.values().size() > index);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#put(java.lang.String, org.mozilla.javascript.Scriptable, java.lang.Object)
|
||||
*/
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public void put(String name, Scriptable start, Object value) {
|
||||
map.put(name, ScriptValueConverter.unwrapValue(value));
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#put(int, org.mozilla.javascript.Scriptable, java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void put(int index, Scriptable start, Object value) {
|
||||
// TODO: implement?
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#delete(java.lang.String)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void delete(String name) {
|
||||
map.remove(name);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#delete(int)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void delete(int index) {
|
||||
int i = 0;
|
||||
Iterator itrKeys = map.keySet().iterator();
|
||||
Iterator<Object> itrKeys = map.keySet().iterator();
|
||||
while (i <= index && itrKeys.hasNext()) {
|
||||
Object key = itrKeys.next();
|
||||
if (i == index) {
|
||||
|
@ -167,58 +136,37 @@ public class ScriptableWrappedMap implements ScriptableMap, Wrapper {
|
|||
}
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getPrototype()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Scriptable getPrototype() {
|
||||
return this.prototype;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#setPrototype(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void setPrototype(Scriptable prototype) {
|
||||
this.prototype = prototype;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getParentScope()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Scriptable getParentScope() {
|
||||
return this.parentScope;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#setParentScope(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void setParentScope(Scriptable parent) {
|
||||
this.parentScope = parent;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getIds()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object[] getIds() {
|
||||
return map.keySet().toArray();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#getDefaultValue(java.lang.Class)
|
||||
*/
|
||||
|
||||
public Object getDefaultValue(Class hint) {
|
||||
@Override
|
||||
public Object getDefaultValue(Class<?> hint) {
|
||||
return null;
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see org.mozilla.javascript.Scriptable#hasInstance(org.mozilla.javascript.Scriptable)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean hasInstance(Scriptable value) {
|
||||
if (!(value instanceof Wrapper))
|
||||
return false;
|
||||
|
@ -226,106 +174,66 @@ public class ScriptableWrappedMap implements ScriptableMap, Wrapper {
|
|||
return Map.class.isInstance(instance);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#clear()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
this.map.clear();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#containsKey(java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
return this.map.containsKey(key);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#containsValue(java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean containsValue(Object value) {
|
||||
return this.map.containsValue(value);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#entrySet()
|
||||
*/
|
||||
|
||||
public Set entrySet() {
|
||||
@Override
|
||||
public Set<Map.Entry<Object, Object>> entrySet() {
|
||||
return this.map.entrySet();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#get(java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object get(Object key) {
|
||||
return this.map.get(key);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#isEmpty()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return (this.map.size() == 0);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#keySet()
|
||||
*/
|
||||
|
||||
public Set keySet() {
|
||||
@Override
|
||||
public Set<Object> keySet() {
|
||||
return this.map.keySet();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#put(java.lang.Object, java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object put(Object key, Object value) {
|
||||
return this.map.put(key, value);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#putAll(java.util.Map)
|
||||
*/
|
||||
|
||||
public void putAll(Map t) {
|
||||
@Override
|
||||
public void putAll(Map<? extends Object, ? extends Object> t) {
|
||||
this.map.putAll(t);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#remove(java.lang.Object)
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Object remove(Object key) {
|
||||
return this.map.remove(key);
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#size()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return this.map.size();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.util.Map#values()
|
||||
*/
|
||||
|
||||
public Collection values() {
|
||||
@Override
|
||||
public Collection<Object> values() {
|
||||
return this.map.values();
|
||||
}
|
||||
|
||||
/* (non-Javadoc)
|
||||
* @see java.lang.Object#toString()
|
||||
*/
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return (this.map != null ? this.map.toString() : super.toString());
|
||||
|
|
|
@ -19,6 +19,12 @@
|
|||
|
||||
package org.elasticsearch.script.javascript;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.CompiledScript;
|
||||
|
@ -28,12 +34,6 @@ import org.elasticsearch.test.ESTestCase;
|
|||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
|
||||
|
@ -59,6 +59,7 @@ public class JavaScriptScriptEngineTests extends ESTestCase {
|
|||
assertThat(((Number) o).intValue(), equalTo(3));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testMapAccess() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
|
||||
|
@ -75,15 +76,17 @@ public class JavaScriptScriptEngineTests extends ESTestCase {
|
|||
assertThat(((String) o), equalTo("2"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testJavaScriptObjectToMap() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
Object o = se.executable(new CompiledScript(ScriptService.ScriptType.INLINE, "testJavaScriptObjectToMap", "js",
|
||||
se.compile("var obj1 = {}; obj1.prop1 = 'value1'; obj1.obj2 = {}; obj1.obj2.prop2 = 'value2'; obj1", Collections.emptyMap())), vars).run();
|
||||
Map obj1 = (Map) o;
|
||||
Map<String, Object> obj1 = (Map<String, Object>) o;
|
||||
assertThat((String) obj1.get("prop1"), equalTo("value1"));
|
||||
assertThat((String) ((Map<String, Object>) obj1.get("obj2")).get("prop2"), equalTo("value2"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testJavaScriptObjectMapInter() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
Map<String, Object> ctx = new HashMap<String, Object>();
|
||||
|
@ -102,6 +105,7 @@ public class JavaScriptScriptEngineTests extends ESTestCase {
|
|||
assertThat((String) ((Map<String, Object>) ctx.get("obj2")).get("prop2"), equalTo("value2"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testJavaScriptInnerArrayCreation() {
|
||||
Map<String, Object> ctx = new HashMap<String, Object>();
|
||||
Map<String, Object> doc = new HashMap<String, Object>();
|
||||
|
@ -115,9 +119,10 @@ public class JavaScriptScriptEngineTests extends ESTestCase {
|
|||
|
||||
Map<String, Object> unwrap = (Map<String, Object>) script.unwrap(ctx);
|
||||
|
||||
assertThat(((Map) unwrap.get("doc")).get("field1"), instanceOf(List.class));
|
||||
assertThat(((Map<String, Object>) unwrap.get("doc")).get("field1"), instanceOf(List.class));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testAccessListInScript() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
Map<String, Object> obj2 = MapBuilder.<String, Object>newMapBuilder().put("prop2", "value2").map();
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.script.javascript;
|
||||
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.CompiledScript;
|
||||
import org.elasticsearch.script.ExecutableScript;
|
||||
|
@ -30,7 +31,6 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -53,8 +53,8 @@ public class JavaScriptScriptMultiThreadedTests extends ESTestCase {
|
|||
public void run() {
|
||||
try {
|
||||
barrier.await();
|
||||
long x = ThreadLocalRandom.current().nextInt();
|
||||
long y = ThreadLocalRandom.current().nextInt();
|
||||
long x = Randomness.get().nextInt();
|
||||
long y = Randomness.get().nextInt();
|
||||
long addition = x + y;
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
vars.put("x", x);
|
||||
|
@ -95,12 +95,12 @@ public class JavaScriptScriptMultiThreadedTests extends ESTestCase {
|
|||
public void run() {
|
||||
try {
|
||||
barrier.await();
|
||||
long x = ThreadLocalRandom.current().nextInt();
|
||||
long x = Randomness.get().nextInt();
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
vars.put("x", x);
|
||||
ExecutableScript script = se.executable(new CompiledScript(ScriptService.ScriptType.INLINE, "testExecutableNoRuntimeParams", "js", compiled), vars);
|
||||
for (int i = 0; i < 100000; i++) {
|
||||
long y = ThreadLocalRandom.current().nextInt();
|
||||
long y = Randomness.get().nextInt();
|
||||
long addition = x + y;
|
||||
script.setNextVar("y", y);
|
||||
long result = ((Number) script.run()).longValue();
|
||||
|
@ -139,8 +139,8 @@ public class JavaScriptScriptMultiThreadedTests extends ESTestCase {
|
|||
barrier.await();
|
||||
Map<String, Object> runtimeVars = new HashMap<String, Object>();
|
||||
for (int i = 0; i < 100000; i++) {
|
||||
long x = ThreadLocalRandom.current().nextInt();
|
||||
long y = ThreadLocalRandom.current().nextInt();
|
||||
long x = Randomness.get().nextInt();
|
||||
long y = Randomness.get().nextInt();
|
||||
long addition = x + y;
|
||||
runtimeVars.put("x", x);
|
||||
runtimeVars.put("y", y);
|
||||
|
|
|
@ -35,7 +35,7 @@ dependencyLicenses {
|
|||
mapping from: /asm-.*/, to: 'asm'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << '-Xlint:-cast,-fallthrough,-rawtypes'
|
||||
compileJava.options.compilerArgs << '-Xlint:-cast,-rawtypes'
|
||||
compileTestJava.options.compilerArgs << '-Xlint:-unchecked'
|
||||
|
||||
// regeneration logic, comes in via ant right now
|
||||
|
|
|
@ -2223,12 +2223,15 @@ class Analyzer extends PlanAParserBaseVisitor<Void> {
|
|||
case LONG:
|
||||
incremd.preConst = positive ? 1L : -1L;
|
||||
incremd.from = definition.longType;
|
||||
break;
|
||||
case FLOAT:
|
||||
incremd.preConst = positive ? 1.0F : -1.0F;
|
||||
incremd.from = definition.floatType;
|
||||
break;
|
||||
case DOUBLE:
|
||||
incremd.preConst = positive ? 1.0 : -1.0;
|
||||
incremd.from = definition.doubleType;
|
||||
break;
|
||||
default:
|
||||
incremd.preConst = positive ? 1 : -1;
|
||||
incremd.from = definition.intType;
|
||||
|
|
|
@ -19,18 +19,18 @@
|
|||
|
||||
package org.elasticsearch.plan.a;
|
||||
|
||||
import org.elasticsearch.plan.a.Definition.Cast;
|
||||
import org.elasticsearch.plan.a.Definition.Field;
|
||||
import org.elasticsearch.plan.a.Definition.Method;
|
||||
import org.elasticsearch.plan.a.Definition.Struct;
|
||||
import org.elasticsearch.plan.a.Definition.Transform;
|
||||
import org.elasticsearch.plan.a.Definition.Type;
|
||||
|
||||
import java.lang.invoke.MethodHandle;
|
||||
import java.lang.reflect.Array;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.plan.a.Definition.Cast;
|
||||
import static org.elasticsearch.plan.a.Definition.Field;
|
||||
import static org.elasticsearch.plan.a.Definition.Method;
|
||||
import static org.elasticsearch.plan.a.Definition.Struct;
|
||||
import static org.elasticsearch.plan.a.Definition.Transform;
|
||||
import static org.elasticsearch.plan.a.Definition.Type;
|
||||
|
||||
public class Def {
|
||||
public static Object methodCall(final Object owner, final String name, final Definition definition,
|
||||
final Object[] arguments, final boolean[] typesafe) {
|
||||
|
@ -70,7 +70,7 @@ public class Def {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({ "unchecked", "rawtypes" })
|
||||
public static void fieldStore(final Object owner, Object value, final String name,
|
||||
final Definition definition, final boolean typesafe) {
|
||||
final Field field = getField(owner, name, definition);
|
||||
|
@ -117,7 +117,7 @@ public class Def {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings("rawtypes")
|
||||
public static Object fieldLoad(final Object owner, final String name, final Definition definition) {
|
||||
if (owner.getClass().isArray() && "length".equals(name)) {
|
||||
return Array.getLength(owner);
|
||||
|
@ -163,7 +163,7 @@ public class Def {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings({ "unchecked", "rawtypes" })
|
||||
public static void arrayStore(final Object array, Object index, Object value, final Definition definition,
|
||||
final boolean indexsafe, final boolean valuesafe) {
|
||||
if (array instanceof Map) {
|
||||
|
@ -206,7 +206,7 @@ public class Def {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@SuppressWarnings("rawtypes")
|
||||
public static Object arrayLoad(final Object array, Object index,
|
||||
final Definition definition, final boolean indexsafe) {
|
||||
if (array instanceof Map) {
|
||||
|
@ -257,7 +257,7 @@ public class Def {
|
|||
}
|
||||
}
|
||||
|
||||
for (final Class iface : clazz.getInterfaces()) {
|
||||
for (final Class<?> iface : clazz.getInterfaces()) {
|
||||
struct = definition.classes.get(iface);
|
||||
|
||||
if (struct != null) {
|
||||
|
@ -303,7 +303,7 @@ public class Def {
|
|||
}
|
||||
}
|
||||
|
||||
for (final Class iface : clazz.getInterfaces()) {
|
||||
for (final Class<?> iface : clazz.getInterfaces()) {
|
||||
struct = definition.classes.get(iface);
|
||||
|
||||
if (struct != null) {
|
||||
|
@ -348,7 +348,7 @@ public class Def {
|
|||
break;
|
||||
}
|
||||
|
||||
for (final Class iface : fromClass.getInterfaces()) {
|
||||
for (final Class<?> iface : fromClass.getInterfaces()) {
|
||||
fromStruct = definition.classes.get(iface);
|
||||
|
||||
if (fromStruct != null) {
|
||||
|
@ -371,7 +371,7 @@ public class Def {
|
|||
break;
|
||||
}
|
||||
|
||||
for (final Class iface : toClass.getInterfaces()) {
|
||||
for (final Class<?> iface : toClass.getInterfaces()) {
|
||||
toStruct = definition.classes.get(iface);
|
||||
|
||||
if (toStruct != null) {
|
||||
|
@ -442,28 +442,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() * (double)(char)right;
|
||||
return ((Number)left).doubleValue() * (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() * (float)(char)right;
|
||||
return ((Number)left).floatValue() * (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() * (long)(char)right;
|
||||
return ((Number)left).longValue() * (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() * (int)(char)right;
|
||||
return ((Number)left).intValue() * (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left * ((Number)right).doubleValue();
|
||||
return (char)left * ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left * ((Number)right).floatValue();
|
||||
return (char)left * ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left * ((Number)right).longValue();
|
||||
return (char)left * ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left * ((Number)right).intValue();
|
||||
return (char)left * ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left * (int)(char)right;
|
||||
return (char)left * (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -485,28 +485,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() / (double)(char)right;
|
||||
return ((Number)left).doubleValue() / (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() / (float)(char)right;
|
||||
return ((Number)left).floatValue() / (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() / (long)(char)right;
|
||||
return ((Number)left).longValue() / (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() / (int)(char)right;
|
||||
return ((Number)left).intValue() / (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left / ((Number)right).doubleValue();
|
||||
return (char)left / ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left / ((Number)right).floatValue();
|
||||
return (char)left / ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left / ((Number)right).longValue();
|
||||
return (char)left / ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left / ((Number)right).intValue();
|
||||
return (char)left / ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left / (int)(char)right;
|
||||
return (char)left / (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -528,28 +528,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() % (double)(char)right;
|
||||
return ((Number)left).doubleValue() % (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() % (float)(char)right;
|
||||
return ((Number)left).floatValue() % (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() % (long)(char)right;
|
||||
return ((Number)left).longValue() % (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() % (int)(char)right;
|
||||
return ((Number)left).intValue() % (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left % ((Number)right).doubleValue();
|
||||
return (char)left % ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left % ((Number)right).floatValue();
|
||||
return (char)left % ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left % ((Number)right).longValue();
|
||||
return (char)left % ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left % ((Number)right).intValue();
|
||||
return (char)left % ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left % (int)(char)right;
|
||||
return (char)left % (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -573,28 +573,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() + (double)(char)right;
|
||||
return ((Number)left).doubleValue() + (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() + (float)(char)right;
|
||||
return ((Number)left).floatValue() + (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() + (long)(char)right;
|
||||
return ((Number)left).longValue() + (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() + (int)(char)right;
|
||||
return ((Number)left).intValue() + (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left + ((Number)right).doubleValue();
|
||||
return (char)left + ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left + ((Number)right).floatValue();
|
||||
return (char)left + ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left + ((Number)right).longValue();
|
||||
return (char)left + ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left + ((Number)right).intValue();
|
||||
return (char)left + ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left + (int)(char)right;
|
||||
return (char)left + (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -616,28 +616,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() - (double)(char)right;
|
||||
return ((Number)left).doubleValue() - (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() - (float)(char)right;
|
||||
return ((Number)left).floatValue() - (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() - (long)(char)right;
|
||||
return ((Number)left).longValue() - (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() - (int)(char)right;
|
||||
return ((Number)left).intValue() - (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left - ((Number)right).doubleValue();
|
||||
return (char)left - ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left - ((Number)right).floatValue();
|
||||
return (char)left - ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left - ((Number)right).longValue();
|
||||
return (char)left - ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left - ((Number)right).intValue();
|
||||
return (char)left - ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left - (int)(char)right;
|
||||
return (char)left - (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -657,9 +657,9 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double || left instanceof Float || left instanceof Long) {
|
||||
return ((Number)left).longValue() << (long)(char)right;
|
||||
return ((Number)left).longValue() << (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() << (int)(char)right;
|
||||
return ((Number)left).intValue() << (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
|
@ -667,10 +667,10 @@ public class Def {
|
|||
if (right instanceof Double || right instanceof Float || right instanceof Long) {
|
||||
return (long)(char)left << ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left << ((Number)right).intValue();
|
||||
return (char)left << ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left << (int)(char)right;
|
||||
return (char)left << (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -690,9 +690,9 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double || left instanceof Float || left instanceof Long) {
|
||||
return ((Number)left).longValue() >> (long)(char)right;
|
||||
return ((Number)left).longValue() >> (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() >> (int)(char)right;
|
||||
return ((Number)left).intValue() >> (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
|
@ -700,10 +700,10 @@ public class Def {
|
|||
if (right instanceof Double || right instanceof Float || right instanceof Long) {
|
||||
return (long)(char)left >> ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left >> ((Number)right).intValue();
|
||||
return (char)left >> ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left >> (int)(char)right;
|
||||
return (char)left >> (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -723,9 +723,9 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double || left instanceof Float || left instanceof Long) {
|
||||
return ((Number)left).longValue() >>> (long)(char)right;
|
||||
return ((Number)left).longValue() >>> (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() >>> (int)(char)right;
|
||||
return ((Number)left).intValue() >>> (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
|
@ -733,10 +733,10 @@ public class Def {
|
|||
if (right instanceof Double || right instanceof Float || right instanceof Long) {
|
||||
return (long)(char)left >>> ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left >>> ((Number)right).intValue();
|
||||
return (char)left >>> ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left >>> (int)(char)right;
|
||||
return (char)left >>> (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -758,20 +758,20 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double || left instanceof Float || left instanceof Long) {
|
||||
return ((Number)left).longValue() & (long)(char)right;
|
||||
return ((Number)left).longValue() & (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() & (int)(char)right;
|
||||
return ((Number)left).intValue() & (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double || right instanceof Float || right instanceof Long) {
|
||||
return (long)(char)left & ((Number)right).longValue();
|
||||
return (char)left & ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left & ((Number)right).intValue();
|
||||
return (char)left & ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left & (int)(char)right;
|
||||
return (char)left & (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -793,20 +793,20 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double || left instanceof Float || left instanceof Long) {
|
||||
return ((Number)left).longValue() ^ (long)(char)right;
|
||||
return ((Number)left).longValue() ^ (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() ^ (int)(char)right;
|
||||
return ((Number)left).intValue() ^ (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double || right instanceof Float || right instanceof Long) {
|
||||
return (long)(char)left ^ ((Number)right).longValue();
|
||||
return (char)left ^ ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left ^ ((Number)right).intValue();
|
||||
return (char)left ^ ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left ^ (int)(char)right;
|
||||
return (char)left ^ (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -828,20 +828,20 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double || left instanceof Float || left instanceof Long) {
|
||||
return ((Number)left).longValue() | (long)(char)right;
|
||||
return ((Number)left).longValue() | (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() | (int)(char)right;
|
||||
return ((Number)left).intValue() | (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double || right instanceof Float || right instanceof Long) {
|
||||
return (long)(char)left | ((Number)right).longValue();
|
||||
return (char)left | ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left | ((Number)right).intValue();
|
||||
return (char)left | ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left | (int)(char)right;
|
||||
return (char)left | (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -855,48 +855,48 @@ public class Def {
|
|||
if (right instanceof Number) {
|
||||
return (double)left == ((Number)right).doubleValue();
|
||||
} else if (right instanceof Character) {
|
||||
return (double)left == (double)(char)right;
|
||||
return (double)left == (char)right;
|
||||
}
|
||||
} else if (right instanceof Double) {
|
||||
if (left instanceof Number) {
|
||||
return ((Number)left).doubleValue() == (double)right;
|
||||
} else if (left instanceof Character) {
|
||||
return (double)(char)left == ((Number)right).doubleValue();
|
||||
return (char)left == ((Number)right).doubleValue();
|
||||
}
|
||||
} else if (left instanceof Float) {
|
||||
if (right instanceof Number) {
|
||||
return (float)left == ((Number)right).floatValue();
|
||||
} else if (right instanceof Character) {
|
||||
return (float)left == (float)(char)right;
|
||||
return (float)left == (char)right;
|
||||
}
|
||||
} else if (right instanceof Float) {
|
||||
if (left instanceof Number) {
|
||||
return ((Number)left).floatValue() == (float)right;
|
||||
} else if (left instanceof Character) {
|
||||
return (float)(char)left == ((Number)right).floatValue();
|
||||
return (char)left == ((Number)right).floatValue();
|
||||
}
|
||||
} else if (left instanceof Long) {
|
||||
if (right instanceof Number) {
|
||||
return (long)left == ((Number)right).longValue();
|
||||
} else if (right instanceof Character) {
|
||||
return (long)left == (long)(char)right;
|
||||
return (long)left == (char)right;
|
||||
}
|
||||
} else if (right instanceof Long) {
|
||||
if (left instanceof Number) {
|
||||
return ((Number)left).longValue() == (long)right;
|
||||
} else if (left instanceof Character) {
|
||||
return (long)(char)left == ((Number)right).longValue();
|
||||
return (char)left == ((Number)right).longValue();
|
||||
}
|
||||
} else if (left instanceof Number) {
|
||||
if (right instanceof Number) {
|
||||
return ((Number)left).intValue() == ((Number)right).intValue();
|
||||
} else if (right instanceof Character) {
|
||||
return ((Number)left).intValue() == (int)(char)right;
|
||||
return ((Number)left).intValue() == (char)right;
|
||||
}
|
||||
} else if (right instanceof Number && left instanceof Character) {
|
||||
return (int)(char)left == ((Number)right).intValue();
|
||||
return (char)left == ((Number)right).intValue();
|
||||
} else if (left instanceof Character && right instanceof Character) {
|
||||
return (int)(char)left == (int)(char)right;
|
||||
return (char)left == (char)right;
|
||||
}
|
||||
|
||||
return left.equals(right);
|
||||
|
@ -919,28 +919,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() < (double)(char)right;
|
||||
return ((Number)left).doubleValue() < (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() < (float)(char)right;
|
||||
return ((Number)left).floatValue() < (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() < (long)(char)right;
|
||||
return ((Number)left).longValue() < (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() < (int)(char)right;
|
||||
return ((Number)left).intValue() < (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left < ((Number)right).doubleValue();
|
||||
return (char)left < ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left < ((Number)right).floatValue();
|
||||
return (char)left < ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left < ((Number)right).longValue();
|
||||
return (char)left < ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left < ((Number)right).intValue();
|
||||
return (char)left < ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left < (int)(char)right;
|
||||
return (char)left < (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -962,28 +962,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() <= (double)(char)right;
|
||||
return ((Number)left).doubleValue() <= (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() <= (float)(char)right;
|
||||
return ((Number)left).floatValue() <= (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() <= (long)(char)right;
|
||||
return ((Number)left).longValue() <= (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() <= (int)(char)right;
|
||||
return ((Number)left).intValue() <= (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left <= ((Number)right).doubleValue();
|
||||
return (char)left <= ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left <= ((Number)right).floatValue();
|
||||
return (char)left <= ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left <= ((Number)right).longValue();
|
||||
return (char)left <= ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left <= ((Number)right).intValue();
|
||||
return (char)left <= ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left <= (int)(char)right;
|
||||
return (char)left <= (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1005,28 +1005,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() > (double)(char)right;
|
||||
return ((Number)left).doubleValue() > (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() > (float)(char)right;
|
||||
return ((Number)left).floatValue() > (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() > (long)(char)right;
|
||||
return ((Number)left).longValue() > (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() > (int)(char)right;
|
||||
return ((Number)left).intValue() > (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left > ((Number)right).doubleValue();
|
||||
return (char)left > ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left > ((Number)right).floatValue();
|
||||
return (char)left > ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left > ((Number)right).longValue();
|
||||
return (char)left > ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left > ((Number)right).intValue();
|
||||
return (char)left > ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left > (int)(char)right;
|
||||
return (char)left > (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1048,28 +1048,28 @@ public class Def {
|
|||
}
|
||||
} else if (right instanceof Character) {
|
||||
if (left instanceof Double) {
|
||||
return ((Number)left).doubleValue() >= (double)(char)right;
|
||||
return ((Number)left).doubleValue() >= (char)right;
|
||||
} else if (left instanceof Float) {
|
||||
return ((Number)left).floatValue() >= (float)(char)right;
|
||||
return ((Number)left).floatValue() >= (char)right;
|
||||
} else if (left instanceof Long) {
|
||||
return ((Number)left).longValue() >= (long)(char)right;
|
||||
return ((Number)left).longValue() >= (char)right;
|
||||
} else {
|
||||
return ((Number)left).intValue() >= (int)(char)right;
|
||||
return ((Number)left).intValue() >= (char)right;
|
||||
}
|
||||
}
|
||||
} else if (left instanceof Character) {
|
||||
if (right instanceof Number) {
|
||||
if (right instanceof Double) {
|
||||
return (double)(char)left >= ((Number)right).doubleValue();
|
||||
return (char)left >= ((Number)right).doubleValue();
|
||||
} else if (right instanceof Float) {
|
||||
return (float)(char)left >= ((Number)right).floatValue();
|
||||
return (char)left >= ((Number)right).floatValue();
|
||||
} else if (right instanceof Long) {
|
||||
return (long)(char)left >= ((Number)right).longValue();
|
||||
return (char)left >= ((Number)right).longValue();
|
||||
} else {
|
||||
return (int)(char)left >= ((Number)right).intValue();
|
||||
return (char)left >= ((Number)right).intValue();
|
||||
}
|
||||
} else if (right instanceof Character) {
|
||||
return (int)(char)left >= (int)(char)right;
|
||||
return (char)left >= (char)right;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1121,7 +1121,7 @@ public class Def {
|
|||
if (value instanceof Boolean) {
|
||||
return ((Boolean)value) ? 1 : 0;
|
||||
} else if (value instanceof Character) {
|
||||
return (int)(char)value;
|
||||
return (char)value;
|
||||
} else {
|
||||
return ((Number)value).intValue();
|
||||
}
|
||||
|
@ -1131,7 +1131,7 @@ public class Def {
|
|||
if (value instanceof Boolean) {
|
||||
return ((Boolean)value) ? 1L : 0;
|
||||
} else if (value instanceof Character) {
|
||||
return (long)(char)value;
|
||||
return (char)value;
|
||||
} else {
|
||||
return ((Number)value).longValue();
|
||||
}
|
||||
|
@ -1141,7 +1141,7 @@ public class Def {
|
|||
if (value instanceof Boolean) {
|
||||
return ((Boolean)value) ? (float)1 : 0;
|
||||
} else if (value instanceof Character) {
|
||||
return (float)(char)value;
|
||||
return (char)value;
|
||||
} else {
|
||||
return ((Number)value).floatValue();
|
||||
}
|
||||
|
@ -1151,7 +1151,7 @@ public class Def {
|
|||
if (value instanceof Boolean) {
|
||||
return ((Boolean)value) ? (double)1 : 0;
|
||||
} else if (value instanceof Character) {
|
||||
return (double)(char)value;
|
||||
return (char)value;
|
||||
} else {
|
||||
return ((Number)value).doubleValue();
|
||||
}
|
||||
|
|
|
@ -26,9 +26,6 @@ dependencies {
|
|||
compile 'org.python:jython-standalone:2.7.0'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-unchecked"
|
||||
compileTestJava.options.compilerArgs << "-Xlint:-unchecked"
|
||||
|
||||
integTest {
|
||||
cluster {
|
||||
systemProperty 'es.script.inline', 'on'
|
||||
|
|
|
@ -19,6 +19,11 @@
|
|||
|
||||
package org.elasticsearch.script.python;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.CompiledScript;
|
||||
|
@ -28,11 +33,6 @@ import org.elasticsearch.test.ESTestCase;
|
|||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
|
||||
|
@ -58,6 +58,7 @@ public class PythonScriptEngineTests extends ESTestCase {
|
|||
assertThat(((Number) o).intValue(), equalTo(3));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testMapAccess() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
|
||||
|
@ -74,6 +75,7 @@ public class PythonScriptEngineTests extends ESTestCase {
|
|||
assertThat(((String) o), equalTo("2"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testObjectMapInter() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
Map<String, Object> ctx = new HashMap<String, Object>();
|
||||
|
@ -92,6 +94,7 @@ public class PythonScriptEngineTests extends ESTestCase {
|
|||
assertThat((String) ((Map<String, Object>) ctx.get("obj2")).get("prop2"), equalTo("value2"));
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public void testAccessListInScript() {
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
Map<String, Object> obj2 = MapBuilder.<String, Object>newMapBuilder().put("prop2", "value2").map();
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package org.elasticsearch.script.python;
|
||||
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.script.CompiledScript;
|
||||
import org.elasticsearch.script.ExecutableScript;
|
||||
|
@ -30,7 +31,6 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -55,8 +55,8 @@ public class PythonScriptMultiThreadedTests extends ESTestCase {
|
|||
public void run() {
|
||||
try {
|
||||
barrier.await();
|
||||
long x = ThreadLocalRandom.current().nextInt();
|
||||
long y = ThreadLocalRandom.current().nextInt();
|
||||
long x = Randomness.get().nextInt();
|
||||
long y = Randomness.get().nextInt();
|
||||
long addition = x + y;
|
||||
Map<String, Object> vars = new HashMap<String, Object>();
|
||||
vars.put("x", x);
|
||||
|
@ -97,13 +97,13 @@ public class PythonScriptMultiThreadedTests extends ESTestCase {
|
|||
// @Override public void run() {
|
||||
// try {
|
||||
// barrier.await();
|
||||
// long x = ThreadLocalRandom.current().nextInt();
|
||||
// long x = Randomness.get().nextInt();
|
||||
// Map<String, Object> vars = new HashMap<String, Object>();
|
||||
// vars.put("x", x);
|
||||
// ExecutableScript script = se.executable(compiled, vars);
|
||||
// Map<String, Object> runtimeVars = new HashMap<String, Object>();
|
||||
// for (int i = 0; i < 100000; i++) {
|
||||
// long y = ThreadLocalRandom.current().nextInt();
|
||||
// long y = Randomness.get().nextInt();
|
||||
// long addition = x + y;
|
||||
// runtimeVars.put("y", y);
|
||||
// long result = ((Number) script.run(runtimeVars)).longValue();
|
||||
|
@ -143,8 +143,8 @@ public class PythonScriptMultiThreadedTests extends ESTestCase {
|
|||
barrier.await();
|
||||
Map<String, Object> runtimeVars = new HashMap<String, Object>();
|
||||
for (int i = 0; i < 10000; i++) {
|
||||
long x = ThreadLocalRandom.current().nextInt();
|
||||
long y = ThreadLocalRandom.current().nextInt();
|
||||
long x = Randomness.get().nextInt();
|
||||
long y = Randomness.get().nextInt();
|
||||
long addition = x + y;
|
||||
runtimeVars.put("x", x);
|
||||
runtimeVars.put("y", y);
|
||||
|
|
|
@ -21,5 +21,3 @@ esplugin {
|
|||
description 'The Mapper Murmur3 plugin allows to compute hashes of a field\'s values at index-time and to store them in the index.'
|
||||
classname 'org.elasticsearch.plugin.mapper.MapperMurmur3Plugin'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-rawtypes"
|
||||
|
|
|
@ -19,6 +19,10 @@
|
|||
|
||||
package org.elasticsearch.index.mapper.murmur3;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexOptions;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -35,10 +39,6 @@ import org.elasticsearch.index.mapper.ParseContext;
|
|||
import org.elasticsearch.index.mapper.core.LongFieldMapper;
|
||||
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.index.mapper.core.TypeParsers.parseNumberField;
|
||||
|
||||
public class Murmur3FieldMapper extends LongFieldMapper {
|
||||
|
@ -93,8 +93,7 @@ public class Murmur3FieldMapper extends LongFieldMapper {
|
|||
|
||||
public static class TypeParser implements Mapper.TypeParser {
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
|
||||
public Mapper.Builder<?, ?> parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
|
||||
Builder builder = new Builder(name);
|
||||
|
||||
// tweaking these settings is no longer allowed, the entire purpose of murmur3 fields is to store a hash
|
||||
|
|
|
@ -33,7 +33,7 @@ dependencies {
|
|||
compile 'org.elasticsearch:securemock:1.2'
|
||||
}
|
||||
|
||||
compileJava.options.compilerArgs << '-Xlint:-cast,-deprecation,-fallthrough,-overrides,-rawtypes,-serial,-try,-unchecked'
|
||||
compileJava.options.compilerArgs << '-Xlint:-cast,-deprecation,-rawtypes,-serial,-try,-unchecked'
|
||||
compileTestJava.options.compilerArgs << '-Xlint:-rawtypes'
|
||||
|
||||
// the main files are actually test files, so use the appopriate forbidden api sigs
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.lucene.index.IndexWriter;
|
|||
import org.apache.lucene.store.BaseDirectoryWrapper;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.LockFactory;
|
||||
import org.apache.lucene.store.LockObtainFailedException;
|
||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||
import org.apache.lucene.store.StoreRateLimiting;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -113,10 +114,6 @@ public class MockFSDirectoryService extends FsDirectoryService {
|
|||
if (!Lucene.indexExists(dir)) {
|
||||
return;
|
||||
}
|
||||
if (IndexWriter.isLocked(dir)) {
|
||||
ESTestCase.checkIndexFailed = true;
|
||||
throw new IllegalStateException("IndexWriter is still open on shard " + shardId);
|
||||
}
|
||||
try (CheckIndex checkIndex = new CheckIndex(dir)) {
|
||||
BytesStreamOutput os = new BytesStreamOutput();
|
||||
PrintStream out = new PrintStream(os, false, StandardCharsets.UTF_8.name());
|
||||
|
@ -134,6 +131,9 @@ public class MockFSDirectoryService extends FsDirectoryService {
|
|||
logger.debug("check index [success]\n{}", new String(os.bytes().toBytes(), StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
||||
} catch (LockObtainFailedException e) {
|
||||
ESTestCase.checkIndexFailed = true;
|
||||
throw new IllegalStateException("IndexWriter is still open on shard " + shardId, e);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
logger.warn("failed to check index", e);
|
||||
|
|
Loading…
Reference in New Issue