Merge branch 'master' into feature/rank-eval

This commit is contained in:
Christoph Büscher 2016-09-07 09:57:43 +02:00
commit dab771a0b4
138 changed files with 1563 additions and 572 deletions

2
.gitignore vendored
View File

@ -21,7 +21,7 @@ nbactions.xml
build/
# gradle wrapper
gradle/
/gradle/
gradlew
gradlew.bat

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.gradle.doc
import org.elasticsearch.gradle.VersionProperties
import org.elasticsearch.gradle.test.RestTestPlugin
import org.gradle.api.Project
import org.gradle.api.Task
@ -30,9 +31,19 @@ public class DocsTestPlugin extends RestTestPlugin {
@Override
public void apply(Project project) {
super.apply(project)
Map<String, String> defaultSubstitutions = [
/* These match up with the asciidoc syntax for substitutions but
* the values may differ. In particular {version} needs to resolve
* to the version being built for testing but needs to resolve to
* the last released version for docs. */
'\\{version\\}':
VersionProperties.elasticsearch.replace('-SNAPSHOT', ''),
'\\{lucene_version\\}' : VersionProperties.lucene,
]
Task listSnippets = project.tasks.create('listSnippets', SnippetsTask)
listSnippets.group 'Docs'
listSnippets.description 'List each snippet'
listSnippets.defaultSubstitutions = defaultSubstitutions
listSnippets.perSnippet { println(it.toString()) }
Task listConsoleCandidates = project.tasks.create(
@ -40,6 +51,7 @@ public class DocsTestPlugin extends RestTestPlugin {
listConsoleCandidates.group 'Docs'
listConsoleCandidates.description
'List snippets that probably should be marked // CONSOLE'
listConsoleCandidates.defaultSubstitutions = defaultSubstitutions
listConsoleCandidates.perSnippet {
if (
it.console != null // Already marked, nothing to do
@ -47,19 +59,17 @@ public class DocsTestPlugin extends RestTestPlugin {
) {
return
}
List<String> languages = [
// This language should almost always be marked console
'js',
// These are often curl commands that should be converted but
// are probably false positives
'sh', 'shell',
]
if (false == languages.contains(it.language)) {
return
if ( // js almost always should be `// CONSOLE`
it.language == 'js' ||
// snippets containing `curl` *probably* should
// be `// CONSOLE`
it.curl) {
println(it.toString())
}
println(it.toString())
}
project.tasks.create('buildRestTests', RestTestsFromSnippetsTask)
Task buildRestTests = project.tasks.create(
'buildRestTests', RestTestsFromSnippetsTask)
buildRestTests.defaultSubstitutions = defaultSubstitutions
}
}

View File

@ -146,6 +146,9 @@ public class RestTestsFromSnippetsTask extends SnippetsTask {
void emitDo(String method, String pathAndQuery, String body,
String catchPart, List warnings, boolean inSetup) {
def (String path, String query) = pathAndQuery.tokenize('?')
if (path == null) {
path = '' // Catch requests to the root...
}
current.println(" - do:")
if (catchPart != null) {
current.println(" catch: $catchPart")

View File

@ -22,6 +22,7 @@ package org.elasticsearch.gradle.doc
import org.gradle.api.DefaultTask
import org.gradle.api.InvalidUserDataException
import org.gradle.api.file.ConfigurableFileTree
import org.gradle.api.tasks.Input
import org.gradle.api.tasks.InputFiles
import org.gradle.api.tasks.TaskAction
@ -60,6 +61,12 @@ public class SnippetsTask extends DefaultTask {
exclude 'build'
}
/**
* Substitutions done on every snippet's contents.
*/
@Input
Map<String, String> defaultSubstitutions = [:]
@TaskAction
public void executeTask() {
/*
@ -75,21 +82,39 @@ public class SnippetsTask extends DefaultTask {
Closure emit = {
snippet.contents = contents.toString()
contents = null
Closure doSubstitution = { String pattern, String subst ->
/*
* $body is really common but it looks like a
* backreference so we just escape it here to make the
* tests cleaner.
*/
subst = subst.replace('$body', '\\$body')
// \n is a new line....
subst = subst.replace('\\n', '\n')
snippet.contents = snippet.contents.replaceAll(
pattern, subst)
}
defaultSubstitutions.each doSubstitution
if (substitutions != null) {
substitutions.each { String pattern, String subst ->
/*
* $body is really common but it looks like a
* backreference so we just escape it here to make the
* tests cleaner.
*/
subst = subst.replace('$body', '\\$body')
// \n is a new line....
subst = subst.replace('\\n', '\n')
snippet.contents = snippet.contents.replaceAll(
pattern, subst)
}
substitutions.each doSubstitution
substitutions = null
}
if (snippet.language == null) {
throw new InvalidUserDataException("$snippet: "
+ "Snippet missing a language. This is required by "
+ "Elasticsearch's doc testing infrastructure so we "
+ "be sure we don't accidentally forget to test a "
+ "snippet.")
}
// Try to detect snippets that contain `curl`
if (snippet.language == 'sh' || snippet.language == 'shell') {
snippet.curl = snippet.contents.contains('curl')
if (snippet.console == false && snippet.curl == false) {
throw new InvalidUserDataException("$snippet: "
+ "No need for NOTCONSOLE if snippet doesn't "
+ "contain `curl`.")
}
}
perSnippet(snippet)
snippet = null
}
@ -107,7 +132,7 @@ public class SnippetsTask extends DefaultTask {
}
return
}
matcher = line =~ /\[source,(\w+)]\s*/
matcher = line =~ /\["?source"?,\s*"?(\w+)"?(,.*)?].*/
if (matcher.matches()) {
lastLanguage = matcher.group(1)
lastLanguageLine = lineNumber
@ -250,6 +275,7 @@ public class SnippetsTask extends DefaultTask {
String language = null
String catchPart = null
String setup = null
boolean curl
List warnings = new ArrayList()
@Override
@ -285,6 +311,9 @@ public class SnippetsTask extends DefaultTask {
if (testSetup) {
result += '// TESTSETUP'
}
if (curl) {
result += '(curl)'
}
return result
}
}

View File

@ -470,7 +470,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]node[/\\]Node.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]node[/\\]internal[/\\]InternalSettingsPreparer.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]plugins[/\\]PluginsService.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]plugins[/\\]RemovePluginCommand.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]RepositoriesService.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]Repository.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]RepositoryModule.java" checks="LineLength" />

View File

@ -38,10 +38,12 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.AllocationId;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.settings.Settings;
@ -53,14 +55,17 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportChannelResponseHandler;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponse;
@ -69,6 +74,7 @@ import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.Objects;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
import java.util.function.Supplier;
@ -115,9 +121,12 @@ public abstract class TransportReplicationAction<
this.transportPrimaryAction = actionName + "[p]";
this.transportReplicaAction = actionName + "[r]";
transportService.registerRequestHandler(actionName, request, ThreadPool.Names.SAME, new OperationTransportHandler());
transportService.registerRequestHandler(transportPrimaryAction, request, executor, new PrimaryOperationTransportHandler());
transportService.registerRequestHandler(transportPrimaryAction, () -> new ConcreteShardRequest<>(request), executor,
new PrimaryOperationTransportHandler());
// we must never reject on because of thread pool capacity on replicas
transportService.registerRequestHandler(transportReplicaAction, replicaRequest, executor, true, true,
transportService.registerRequestHandler(transportReplicaAction,
() -> new ConcreteShardRequest<>(replicaRequest),
executor, true, true,
new ReplicaOperationTransportHandler());
this.transportOptions = transportOptions();
@ -163,7 +172,7 @@ public abstract class TransportReplicationAction<
/**
* Synchronous replica operation on nodes with replica copies. This is done under the lock form
* {@link #acquireReplicaOperationLock(ShardId, long, ActionListener)}.
* {@link #acquireReplicaOperationLock(ShardId, long, String, ActionListener)}.
*/
protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest);
@ -230,33 +239,36 @@ public abstract class TransportReplicationAction<
}
}
class PrimaryOperationTransportHandler implements TransportRequestHandler<Request> {
class PrimaryOperationTransportHandler implements TransportRequestHandler<ConcreteShardRequest<Request>> {
@Override
public void messageReceived(final Request request, final TransportChannel channel) throws Exception {
public void messageReceived(final ConcreteShardRequest<Request> request, final TransportChannel channel) throws Exception {
throw new UnsupportedOperationException("the task parameter is required for this operation");
}
@Override
public void messageReceived(Request request, TransportChannel channel, Task task) {
new AsyncPrimaryAction(request, channel, (ReplicationTask) task).run();
public void messageReceived(ConcreteShardRequest<Request> request, TransportChannel channel, Task task) {
new AsyncPrimaryAction(request.request, request.targetAllocationID, channel, (ReplicationTask) task).run();
}
}
class AsyncPrimaryAction extends AbstractRunnable implements ActionListener<PrimaryShardReference> {
private final Request request;
/** targetAllocationID of the shard this request is meant for */
private final String targetAllocationID;
private final TransportChannel channel;
private final ReplicationTask replicationTask;
AsyncPrimaryAction(Request request, TransportChannel channel, ReplicationTask replicationTask) {
AsyncPrimaryAction(Request request, String targetAllocationID, TransportChannel channel, ReplicationTask replicationTask) {
this.request = request;
this.targetAllocationID = targetAllocationID;
this.channel = channel;
this.replicationTask = replicationTask;
}
@Override
protected void doRun() throws Exception {
acquirePrimaryShardReference(request.shardId(), this);
acquirePrimaryShardReference(request.shardId(), targetAllocationID, this);
}
@Override
@ -271,7 +283,9 @@ public abstract class TransportReplicationAction<
final ShardRouting primary = primaryShardReference.routingEntry();
assert primary.relocating() : "indexShard is marked as relocated but routing isn't" + primary;
DiscoveryNode relocatingNode = clusterService.state().nodes().get(primary.relocatingNodeId());
transportService.sendRequest(relocatingNode, transportPrimaryAction, request, transportOptions,
transportService.sendRequest(relocatingNode, transportPrimaryAction,
new ConcreteShardRequest<>(request, primary.allocationId().getRelocationId()),
transportOptions,
new TransportChannelResponseHandler<Response>(logger, channel, "rerouting indexing to target primary " + primary,
TransportReplicationAction.this::newResponseInstance) {
@ -391,15 +405,17 @@ public abstract class TransportReplicationAction<
}
}
class ReplicaOperationTransportHandler implements TransportRequestHandler<ReplicaRequest> {
class ReplicaOperationTransportHandler implements TransportRequestHandler<ConcreteShardRequest<ReplicaRequest>> {
@Override
public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception {
public void messageReceived(final ConcreteShardRequest<ReplicaRequest> request, final TransportChannel channel)
throws Exception {
throw new UnsupportedOperationException("the task parameter is required for this operation");
}
@Override
public void messageReceived(ReplicaRequest request, TransportChannel channel, Task task) throws Exception {
new AsyncReplicaAction(request, channel, (ReplicationTask) task).run();
public void messageReceived(ConcreteShardRequest<ReplicaRequest> requestWithAID, TransportChannel channel, Task task)
throws Exception {
new AsyncReplicaAction(requestWithAID.request, requestWithAID.targetAllocationID, channel, (ReplicationTask) task).run();
}
}
@ -417,6 +433,8 @@ public abstract class TransportReplicationAction<
private final class AsyncReplicaAction extends AbstractRunnable implements ActionListener<Releasable> {
private final ReplicaRequest request;
// allocation id of the replica this request is meant for
private final String targetAllocationID;
private final TransportChannel channel;
/**
* The task on the node with the replica shard.
@ -426,10 +444,11 @@ public abstract class TransportReplicationAction<
// something we want to avoid at all costs
private final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext());
AsyncReplicaAction(ReplicaRequest request, TransportChannel channel, ReplicationTask task) {
AsyncReplicaAction(ReplicaRequest request, String targetAllocationID, TransportChannel channel, ReplicationTask task) {
this.request = request;
this.channel = channel;
this.task = task;
this.targetAllocationID = targetAllocationID;
}
@Override
@ -464,7 +483,9 @@ public abstract class TransportReplicationAction<
String extraMessage = "action [" + transportReplicaAction + "], request[" + request + "]";
TransportChannelResponseHandler<TransportResponse.Empty> handler =
new TransportChannelResponseHandler<>(logger, channel, extraMessage, () -> TransportResponse.Empty.INSTANCE);
transportService.sendRequest(clusterService.localNode(), transportReplicaAction, request, handler);
transportService.sendRequest(clusterService.localNode(), transportReplicaAction,
new ConcreteShardRequest<>(request, targetAllocationID),
handler);
}
@Override
@ -501,7 +522,7 @@ public abstract class TransportReplicationAction<
protected void doRun() throws Exception {
setPhase(task, "replica");
assert request.shardId() != null : "request shardId must be set";
acquireReplicaOperationLock(request.shardId(), request.primaryTerm(), this);
acquireReplicaOperationLock(request.shardId(), request.primaryTerm(), targetAllocationID, this);
}
/**
@ -598,7 +619,7 @@ public abstract class TransportReplicationAction<
logger.trace("send action [{}] on primary [{}] for request [{}] with cluster state version [{}] to [{}] ",
transportPrimaryAction, request.shardId(), request, state.version(), primary.currentNodeId());
}
performAction(node, transportPrimaryAction, true);
performAction(node, transportPrimaryAction, true, new ConcreteShardRequest<>(request, primary.allocationId().getId()));
}
private void performRemoteAction(ClusterState state, ShardRouting primary, DiscoveryNode node) {
@ -620,7 +641,7 @@ public abstract class TransportReplicationAction<
request.shardId(), request, state.version(), primary.currentNodeId());
}
setPhase(task, "rerouted");
performAction(node, actionName, false);
performAction(node, actionName, false, request);
}
private boolean retryIfUnavailable(ClusterState state, ShardRouting primary) {
@ -671,8 +692,9 @@ public abstract class TransportReplicationAction<
}
}
private void performAction(final DiscoveryNode node, final String action, final boolean isPrimaryAction) {
transportService.sendRequest(node, action, request, transportOptions, new TransportResponseHandler<Response>() {
private void performAction(final DiscoveryNode node, final String action, final boolean isPrimaryAction,
final TransportRequest requestToPerform) {
transportService.sendRequest(node, action, requestToPerform, transportOptions, new TransportResponseHandler<Response>() {
@Override
public Response newInstance() {
@ -700,7 +722,7 @@ public abstract class TransportReplicationAction<
(org.apache.logging.log4j.util.Supplier<?>) () -> new ParameterizedMessage(
"received an error from node [{}] for request [{}], scheduling a retry",
node.getId(),
request),
requestToPerform),
exp);
retry(exp);
} else {
@ -794,7 +816,8 @@ public abstract class TransportReplicationAction<
* tries to acquire reference to {@link IndexShard} to perform a primary operation. Released after performing primary operation locally
* and replication of the operation to all replica shards is completed / failed (see {@link ReplicationOperation}).
*/
protected void acquirePrimaryShardReference(ShardId shardId, ActionListener<PrimaryShardReference> onReferenceAcquired) {
protected void acquirePrimaryShardReference(ShardId shardId, String allocationId,
ActionListener<PrimaryShardReference> onReferenceAcquired) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
// we may end up here if the cluster state used to route the primary is so stale that the underlying
@ -804,6 +827,10 @@ public abstract class TransportReplicationAction<
throw new ReplicationOperation.RetryOnPrimaryException(indexShard.shardId(),
"actual shard is not a primary " + indexShard.routingEntry());
}
final String actualAllocationId = indexShard.routingEntry().allocationId().getId();
if (actualAllocationId.equals(allocationId) == false) {
throw new ShardNotFoundException(shardId, "expected aID [{}] but found [{}]", allocationId, actualAllocationId);
}
ActionListener<Releasable> onAcquired = new ActionListener<Releasable>() {
@Override
@ -823,9 +850,14 @@ public abstract class TransportReplicationAction<
/**
* tries to acquire an operation on replicas. The lock is closed as soon as replication is completed on the node.
*/
protected void acquireReplicaOperationLock(ShardId shardId, long primaryTerm, ActionListener<Releasable> onLockAcquired) {
protected void acquireReplicaOperationLock(ShardId shardId, long primaryTerm, final String allocationId,
ActionListener<Releasable> onLockAcquired) {
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
final String actualAllocationId = indexShard.routingEntry().allocationId().getId();
if (actualAllocationId.equals(allocationId) == false) {
throw new ShardNotFoundException(shardId, "expected aID [{}] but found [{}]", allocationId, actualAllocationId);
}
indexShard.acquireReplicaOperationLock(primaryTerm, onLockAcquired, executor);
}
@ -888,7 +920,8 @@ public abstract class TransportReplicationAction<
listener.onFailure(new NoNodeAvailableException("unknown node [" + nodeId + "]"));
return;
}
transportService.sendRequest(node, transportReplicaAction, request, transportOptions,
transportService.sendRequest(node, transportReplicaAction,
new ConcreteShardRequest<>(request, replica.allocationId().getId()), transportOptions,
new ActionListenerResponseHandler<>(listener, () -> TransportResponse.Empty.INSTANCE));
}
@ -930,6 +963,72 @@ public abstract class TransportReplicationAction<
}
}
/** a wrapper class to encapsulate a request when being sent to a specific allocation id **/
public static final class ConcreteShardRequest<R extends TransportRequest> extends TransportRequest {
/** {@link AllocationId#getId()} of the shard this request is sent to **/
private String targetAllocationID;
private R request;
ConcreteShardRequest(Supplier<R> requestSupplier) {
request = requestSupplier.get();
// null now, but will be populated by reading from the streams
targetAllocationID = null;
}
ConcreteShardRequest(R request, String targetAllocationID) {
Objects.requireNonNull(request);
Objects.requireNonNull(targetAllocationID);
this.request = request;
this.targetAllocationID = targetAllocationID;
}
@Override
public void setParentTask(String parentTaskNode, long parentTaskId) {
request.setParentTask(parentTaskNode, parentTaskId);
}
@Override
public void setParentTask(TaskId taskId) {
request.setParentTask(taskId);
}
@Override
public TaskId getParentTask() {
return request.getParentTask();
}
@Override
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return request.createTask(id, type, action, parentTaskId);
}
@Override
public String getDescription() {
return "[" + request.getDescription() + "] for aID [" + targetAllocationID + "]";
}
@Override
public void readFrom(StreamInput in) throws IOException {
targetAllocationID = in.readString();
request.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeString(targetAllocationID);
request.writeTo(out);
}
public R getRequest() {
return request;
}
public String getTargetAllocationID() {
return targetAllocationID;
}
}
/**
* Sets the current phase on the task if it isn't null. Pulled into its own
* method because its more convenient that way.

View File

@ -271,9 +271,6 @@ public class JarHell {
"class: " + clazz + System.lineSeparator() +
"exists multiple times in jar: " + jarpath + " !!!!!!!!!");
} else {
if (clazz.startsWith("org.apache.log4j")) {
return; // go figure, jar hell for what should be System.out.println...
}
if (clazz.startsWith("org.apache.logging.log4j.core.impl.ThrowableProxy")) {
/*
* deliberate to hack around a bug in Log4j

View File

@ -27,8 +27,8 @@ import org.elasticsearch.cluster.Diffable;
import org.elasticsearch.cluster.DiffableUtils;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -88,6 +88,11 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
return indicesRouting.containsKey(index);
}
public boolean hasIndex(Index index) {
IndexRoutingTable indexRouting = index(index.getName());
return indexRouting != null && indexRouting.getIndex().equals(index);
}
public IndexRoutingTable index(String index) {
return indicesRouting.get(index);
}

View File

@ -78,11 +78,10 @@ public class DiskThresholdDecider extends AllocationDecider {
* Returns the size of all shards that are currently being relocated to
* the node, but may not be finished transferring yet.
*
* If subtractShardsMovingAway is set then the size of shards moving away is subtracted from the total size
* of all shards
* If subtractShardsMovingAway is true then the size of shards moving away is subtracted from the total size of all shards
*/
static long sizeOfRelocatingShards(RoutingNode node, RoutingAllocation allocation,
boolean subtractShardsMovingAway, String dataPath) {
boolean subtractShardsMovingAway, String dataPath) {
ClusterInfo clusterInfo = allocation.clusterInfo();
long totalSize = 0;
for (ShardRouting routing : node.shardsWithState(ShardRoutingState.RELOCATING, ShardRoutingState.INITIALIZING)) {
@ -111,7 +110,9 @@ public class DiskThresholdDecider extends AllocationDecider {
final double usedDiskThresholdLow = 100.0 - diskThresholdSettings.getFreeDiskThresholdLow();
final double usedDiskThresholdHigh = 100.0 - diskThresholdSettings.getFreeDiskThresholdHigh();
DiskUsage usage = getDiskUsage(node, allocation, usages);
// subtractLeavingShards is passed as false here, because they still use disk space, and therefore should we should be extra careful
// and take the size into account
DiskUsage usage = getDiskUsage(node, allocation, usages, false);
// First, check that the node currently over the low watermark
double freeDiskPercentage = usage.getFreeDiskAsPercentage();
// Cache the used disk percentage for displaying disk percentages consistent with documentation
@ -243,7 +244,9 @@ public class DiskThresholdDecider extends AllocationDecider {
return decision;
}
final DiskUsage usage = getDiskUsage(node, allocation, usages);
// subtractLeavingShards is passed as true here, since this is only for shards remaining, we will *eventually* have enough disk
// since shards are moving away. No new shards will be incoming since in canAllocate we pass false for this check.
final DiskUsage usage = getDiskUsage(node, allocation, usages, true);
final String dataPath = clusterInfo.getDataPath(shardRouting);
// If this node is already above the high threshold, the shard cannot remain (get it off!)
final double freeDiskPercentage = usage.getFreeDiskAsPercentage();
@ -280,7 +283,8 @@ public class DiskThresholdDecider extends AllocationDecider {
"there is enough disk on this node for the shard to remain, free: [%s]", new ByteSizeValue(freeBytes));
}
private DiskUsage getDiskUsage(RoutingNode node, RoutingAllocation allocation, ImmutableOpenMap<String, DiskUsage> usages) {
private DiskUsage getDiskUsage(RoutingNode node, RoutingAllocation allocation,
ImmutableOpenMap<String, DiskUsage> usages, boolean subtractLeavingShards) {
DiskUsage usage = usages.get(node.nodeId());
if (usage == null) {
// If there is no usage, and we have other nodes in the cluster,
@ -293,7 +297,7 @@ public class DiskThresholdDecider extends AllocationDecider {
}
if (diskThresholdSettings.includeRelocations()) {
long relocatingShardsSize = sizeOfRelocatingShards(node, allocation, true, usage.getPath());
long relocatingShardsSize = sizeOfRelocatingShards(node, allocation, subtractLeavingShards, usage.getPath());
DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().getName(), usage.getPath(),
usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize);
if (logger.isTraceEnabled()) {

View File

@ -551,10 +551,6 @@ public class Setting<T> extends ToXContentToBytes {
return new Setting<>(key, defaultValueFn, Booleans::parseBooleanExact, properties);
}
public static Setting<ByteSizeValue> byteSizeSetting(String key, String percentage, Property... properties) {
return new Setting<>(key, (s) -> percentage, (s) -> MemorySizeValue.parseBytesSizeValueOrHeapRatio(s, key), properties);
}
public static Setting<ByteSizeValue> byteSizeSetting(String key, ByteSizeValue value, Property... properties) {
return byteSizeSetting(key, (s) -> value.toString(), properties);
}
@ -591,6 +587,49 @@ public class Setting<T> extends ToXContentToBytes {
return value;
}
/**
* Creates a setting which specifies a memory size. This can either be
* specified as an absolute bytes value or as a percentage of the heap
* memory.
*
* @param key the key for the setting
* @param defaultValue the default value for this setting
* @param properties properties properties for this setting like scope, filtering...
* @return the setting object
*/
public static Setting<ByteSizeValue> memorySizeSetting(String key, ByteSizeValue defaultValue, Property... properties) {
return memorySizeSetting(key, (s) -> defaultValue.toString(), properties);
}
/**
* Creates a setting which specifies a memory size. This can either be
* specified as an absolute bytes value or as a percentage of the heap
* memory.
*
* @param key the key for the setting
* @param defaultValue a function that supplies the default value for this setting
* @param properties properties properties for this setting like scope, filtering...
* @return the setting object
*/
public static Setting<ByteSizeValue> memorySizeSetting(String key, Function<Settings, String> defaultValue, Property... properties) {
return new Setting<>(key, defaultValue, (s) -> MemorySizeValue.parseBytesSizeValueOrHeapRatio(s, key), properties);
}
/**
* Creates a setting which specifies a memory size. This can either be
* specified as an absolute bytes value or as a percentage of the heap
* memory.
*
* @param key the key for the setting
* @param defaultPercentage the default value of this setting as a percentage of the heap memory
* @param properties properties properties for this setting like scope, filtering...
* @return the setting object
*/
public static Setting<ByteSizeValue> memorySizeSetting(String key, String defaultPercentage, Property... properties) {
return new Setting<>(key, (s) -> defaultPercentage, (s) -> MemorySizeValue.parseBytesSizeValueOrHeapRatio(s, key), properties);
}
public static Setting<TimeValue> positiveTimeSetting(String key, TimeValue defaultValue, Property... properties) {
return timeSetting(key, defaultValue, TimeValue.timeValueMillis(0), properties);
}

View File

@ -46,7 +46,7 @@ public class PageCacheRecycler extends AbstractComponent implements Releasable {
public static final Setting<Type> TYPE_SETTING =
new Setting<>("cache.recycler.page.type", Type.CONCURRENT.name(), Type::parse, Property.NodeScope);
public static final Setting<ByteSizeValue> LIMIT_HEAP_SETTING =
Setting.byteSizeSetting("cache.recycler.page.limit.heap", "10%", Property.NodeScope);
Setting.memorySizeSetting("cache.recycler.page.limit.heap", "10%", Property.NodeScope);
public static final Setting<Double> WEIGHT_BYTES_SETTING =
Setting.doubleSetting("cache.recycler.page.weight.bytes", 1d, 0d, Property.NodeScope);
public static final Setting<Double> WEIGHT_LONG_SETTING =

View File

@ -0,0 +1,57 @@
/*
* 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.analysis;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.minhash.MinHashFilterFactory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexSettings;
import java.util.HashMap;
import java.util.Map;
/**
* TokenFilterFactoryAdapter for {@link MinHashFilterFactory}
*
*/
public class MinHashTokenFilterFactory extends AbstractTokenFilterFactory {
private final MinHashFilterFactory minHashFilterFactory;
public MinHashTokenFilterFactory(IndexSettings indexSettings, Environment environment, String name, Settings settings) {
super(indexSettings, name, settings);
minHashFilterFactory = new MinHashFilterFactory(convertSettings(settings));
}
@Override
public TokenStream create(TokenStream tokenStream) {
return minHashFilterFactory.create(tokenStream);
}
private Map<String, String> convertSettings(Settings settings) {
Map<String, String> settingMap = new HashMap<>();
settingMap.put("hashCount", settings.get("hash_count"));
settingMap.put("bucketCount", settings.get("bucket_count"));
settingMap.put("hashSetSize", settings.get("hash_set_size"));
settingMap.put("withRotation", settings.get("with_rotation"));
return settingMap;
}
}

View File

@ -25,8 +25,11 @@ import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptSettings;
import java.io.IOException;
import java.util.Objects;
@ -42,11 +45,18 @@ public class QueryParseContext implements ParseFieldMatcherSupplier {
private final XContentParser parser;
private final IndicesQueriesRegistry indicesQueriesRegistry;
private final ParseFieldMatcher parseFieldMatcher;
private final String defaultScriptLanguage;
public QueryParseContext(IndicesQueriesRegistry registry, XContentParser parser, ParseFieldMatcher parseFieldMatcher) {
this(Script.DEFAULT_SCRIPT_LANG, registry, parser, parseFieldMatcher);
}
public QueryParseContext(String defaultScriptLanguage, IndicesQueriesRegistry registry, XContentParser parser,
ParseFieldMatcher parseFieldMatcher) {
this.indicesQueriesRegistry = Objects.requireNonNull(registry, "indices queries registry cannot be null");
this.parser = Objects.requireNonNull(parser, "parser cannot be null");
this.parseFieldMatcher = Objects.requireNonNull(parseFieldMatcher, "parse field matcher cannot be null");
this.defaultScriptLanguage = defaultScriptLanguage;
}
public XContentParser parser() {
@ -127,4 +137,12 @@ public class QueryParseContext implements ParseFieldMatcherSupplier {
public ParseFieldMatcher getParseFieldMatcher() {
return parseFieldMatcher;
}
/**
* Returns the default scripting language, that should be used if scripts don't specify the script language
* explicitly.
*/
public String getDefaultScriptLanguage() {
return defaultScriptLanguage;
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.ScriptSettings;
/**
* Context object used to rewrite {@link QueryBuilder} instances into simplified version.
@ -101,9 +102,18 @@ public class QueryRewriteContext implements ParseFieldMatcherSupplier {
/**
* Returns a new {@link QueryParseContext} that wraps the provided parser, using the ParseFieldMatcher settings that
* are configured in the index settings
* are configured in the index settings. The default script language will always default to Painless.
*/
public QueryParseContext newParseContext(XContentParser parser) {
return new QueryParseContext(indicesQueriesRegistry, parser, indexSettings.getParseFieldMatcher());
}
/**
* Returns a new {@link QueryParseContext} like {@link #newParseContext(XContentParser)} with the only diffence, that
* the default script language will default to what has been set in the 'script.legacy.default_lang' setting.
*/
public QueryParseContext newParseContextWithLegacyScriptLanguage(XContentParser parser) {
String defaultScriptLanguage = ScriptSettings.getLegacyDefaultLang(indexSettings.getNodeSettings());
return new QueryParseContext(defaultScriptLanguage, indicesQueriesRegistry, parser, indexSettings.getParseFieldMatcher());
}
}

View File

@ -106,7 +106,7 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
// skip
} else if (token == XContentParser.Token.START_OBJECT) {
if (parseContext.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, parseContext.getParseFieldMatcher());
script = Script.parse(parser, parseContext.getParseFieldMatcher(), parseContext.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(), "[script] query does not support [" + currentFieldName + "]");
}
@ -116,7 +116,7 @@ public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder>
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.BOOST_FIELD)) {
boost = parser.floatValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, parseContext.getParseFieldMatcher());
script = Script.parse(parser, parseContext.getParseFieldMatcher(), parseContext.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(), "[script] query does not support [" + currentFieldName + "]");
}

View File

@ -115,7 +115,7 @@ public class ScriptScoreFunctionBuilder extends ScoreFunctionBuilder<ScriptScore
currentFieldName = parser.currentName();
} else {
if (parseContext.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, parseContext.getParseFieldMatcher());
script = Script.parse(parser, parseContext.getParseFieldMatcher(), parseContext.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(), NAME + " query does not support [" + currentFieldName + "]");
}

View File

@ -33,10 +33,18 @@ public class ShardNotFoundException extends ResourceNotFoundException {
}
public ShardNotFoundException(ShardId shardId, Throwable ex) {
super("no such shard", ex);
setShard(shardId);
this(shardId, "no such shard", ex);
}
public ShardNotFoundException(ShardId shardId, String msg, Object... args) {
this(shardId, msg, null, args);
}
public ShardNotFoundException(ShardId shardId, String msg, Throwable ex, Object... args) {
super(msg, ex, args);
setShard(shardId);
}
public ShardNotFoundException(StreamInput in) throws IOException{
super(in);
}

View File

@ -52,7 +52,8 @@ import java.util.concurrent.locks.ReentrantLock;
public class IndexingMemoryController extends AbstractComponent implements IndexingOperationListener, Closeable {
/** How much heap (% or bytes) we will share across all actively indexing shards on this node (default: 10%). */
public static final Setting<ByteSizeValue> INDEX_BUFFER_SIZE_SETTING = Setting.byteSizeSetting("indices.memory.index_buffer_size", "10%", Property.NodeScope);
public static final Setting<ByteSizeValue> INDEX_BUFFER_SIZE_SETTING =
Setting.memorySizeSetting("indices.memory.index_buffer_size", "10%", Property.NodeScope);
/** Only applies when <code>indices.memory.index_buffer_size</code> is a %, to set a floor on the actual size in bytes (default: 48 MB). */
public static final Setting<ByteSizeValue> MIN_INDEX_BUFFER_SIZE_SETTING = Setting.byteSizeSetting("indices.memory.min_index_buffer_size",

View File

@ -49,13 +49,13 @@ import java.util.function.Predicate;
public class IndicesQueryCache extends AbstractComponent implements QueryCache, Closeable {
public static final Setting<ByteSizeValue> INDICES_CACHE_QUERY_SIZE_SETTING = Setting.byteSizeSetting(
"indices.queries.cache.size", "10%", Property.NodeScope);
public static final Setting<Integer> INDICES_CACHE_QUERY_COUNT_SETTING = Setting.intSetting(
"indices.queries.cache.count", 10000, 1, Property.NodeScope);
public static final Setting<ByteSizeValue> INDICES_CACHE_QUERY_SIZE_SETTING =
Setting.memorySizeSetting("indices.queries.cache.size", "10%", Property.NodeScope);
public static final Setting<Integer> INDICES_CACHE_QUERY_COUNT_SETTING =
Setting.intSetting("indices.queries.cache.count", 10000, 1, Property.NodeScope);
// enables caching on all segments instead of only the larger ones, for testing only
public static final Setting<Boolean> INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING = Setting.boolSetting(
"indices.queries.cache.all_segments", false, Property.NodeScope);
public static final Setting<Boolean> INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING =
Setting.boolSetting("indices.queries.cache.all_segments", false, Property.NodeScope);
private final LRUQueryCache cache;
private final ShardCoreKeyMap shardKeyMap = new ShardCoreKeyMap();

View File

@ -72,7 +72,7 @@ public final class IndicesRequestCache extends AbstractComponent implements Remo
public static final Setting<Boolean> INDEX_CACHE_REQUEST_ENABLED_SETTING =
Setting.boolSetting("index.requests.cache.enable", true, Property.Dynamic, Property.IndexScope);
public static final Setting<ByteSizeValue> INDICES_CACHE_QUERY_SIZE =
Setting.byteSizeSetting("indices.requests.cache.size", "1%", Property.NodeScope);
Setting.memorySizeSetting("indices.requests.cache.size", "1%", Property.NodeScope);
public static final Setting<TimeValue> INDICES_CACHE_QUERY_EXPIRE =
Setting.positiveTimeSetting("indices.requests.cache.expire", new TimeValue(0), Property.NodeScope);

View File

@ -90,6 +90,7 @@ import org.elasticsearch.index.analysis.LithuanianAnalyzerProvider;
import org.elasticsearch.index.analysis.LowerCaseTokenFilterFactory;
import org.elasticsearch.index.analysis.LowerCaseTokenizerFactory;
import org.elasticsearch.index.analysis.MappingCharFilterFactory;
import org.elasticsearch.index.analysis.MinHashTokenFilterFactory;
import org.elasticsearch.index.analysis.NGramTokenFilterFactory;
import org.elasticsearch.index.analysis.NGramTokenizerFactory;
import org.elasticsearch.index.analysis.NorwegianAnalyzerProvider;
@ -214,6 +215,7 @@ public final class AnalysisModule {
tokenFilters.register("edgeNGram", EdgeNGramTokenFilterFactory::new);
tokenFilters.register("edge_ngram", EdgeNGramTokenFilterFactory::new);
tokenFilters.register("shingle", ShingleTokenFilterFactory::new);
tokenFilters.register("min_hash", MinHashTokenFilterFactory::new);
tokenFilters.register("unique", UniqueTokenFilterFactory::new);
tokenFilters.register("truncate", requriesAnalysisSettings(TruncateTokenFilterFactory::new));
tokenFilters.register("trim", TrimTokenFilterFactory::new);

View File

@ -47,24 +47,24 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
private final ConcurrentMap<String, CircuitBreaker> breakers = new ConcurrentHashMap<>();
public static final Setting<ByteSizeValue> TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING =
Setting.byteSizeSetting("indices.breaker.total.limit", "70%", Property.Dynamic, Property.NodeScope);
Setting.memorySizeSetting("indices.breaker.total.limit", "70%", Property.Dynamic, Property.NodeScope);
public static final Setting<ByteSizeValue> FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING =
Setting.byteSizeSetting("indices.breaker.fielddata.limit", "60%", Property.Dynamic, Property.NodeScope);
Setting.memorySizeSetting("indices.breaker.fielddata.limit", "60%", Property.Dynamic, Property.NodeScope);
public static final Setting<Double> FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING =
Setting.doubleSetting("indices.breaker.fielddata.overhead", 1.03d, 0.0d, Property.Dynamic, Property.NodeScope);
public static final Setting<CircuitBreaker.Type> FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING =
new Setting<>("indices.breaker.fielddata.type", "memory", CircuitBreaker.Type::parseValue, Property.NodeScope);
public static final Setting<ByteSizeValue> REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING =
Setting.byteSizeSetting("indices.breaker.request.limit", "60%", Property.Dynamic, Property.NodeScope);
Setting.memorySizeSetting("indices.breaker.request.limit", "60%", Property.Dynamic, Property.NodeScope);
public static final Setting<Double> REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING =
Setting.doubleSetting("indices.breaker.request.overhead", 1.0d, 0.0d, Property.Dynamic, Property.NodeScope);
public static final Setting<CircuitBreaker.Type> REQUEST_CIRCUIT_BREAKER_TYPE_SETTING =
new Setting<>("indices.breaker.request.type", "memory", CircuitBreaker.Type::parseValue, Property.NodeScope);
public static final Setting<ByteSizeValue> IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING =
Setting.byteSizeSetting("network.breaker.inflight_requests.limit", "100%", Property.Dynamic, Property.NodeScope);
Setting.memorySizeSetting("network.breaker.inflight_requests.limit", "100%", Property.Dynamic, Property.NodeScope);
public static final Setting<Double> IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_OVERHEAD_SETTING =
Setting.doubleSetting("network.breaker.inflight_requests.overhead", 1.0d, 0.0d, Property.Dynamic, Property.NodeScope);
public static final Setting<CircuitBreaker.Type> IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_TYPE_SETTING =

View File

@ -51,7 +51,7 @@ import java.util.function.ToLongBiFunction;
public class IndicesFieldDataCache extends AbstractComponent implements RemovalListener<IndicesFieldDataCache.Key, Accountable>, Releasable{
public static final Setting<ByteSizeValue> INDICES_FIELDDATA_CACHE_SIZE_KEY =
Setting.byteSizeSetting("indices.fielddata.cache.size", new ByteSizeValue(-1), Property.NodeScope);
Setting.memorySizeSetting("indices.fielddata.cache.size", new ByteSizeValue(-1), Property.NodeScope);
private final IndexFieldDataCache.Listener indicesFieldDataCacheListener;
private final Cache<Key, Accountable> cache;

View File

@ -43,7 +43,7 @@ import static org.elasticsearch.cli.Terminal.Verbosity.VERBOSE;
/**
* A command for the plugin cli to remove a plugin from elasticsearch.
*/
class RemovePluginCommand extends SettingCommand {
final class RemovePluginCommand extends SettingCommand {
private final OptionSpec<String> arguments;
@ -64,14 +64,16 @@ class RemovePluginCommand extends SettingCommand {
terminal.println("-> Removing " + Strings.coalesceToEmpty(pluginName) + "...");
Path pluginDir = env.pluginsFile().resolve(pluginName);
final Path pluginDir = env.pluginsFile().resolve(pluginName);
if (Files.exists(pluginDir) == false) {
throw new UserException(ExitCodes.USAGE, "plugin " + pluginName + " not found; run 'elasticsearch-plugin list' to get list of installed plugins");
throw new UserException(
ExitCodes.USAGE,
"plugin " + pluginName + " not found; run 'elasticsearch-plugin list' to get list of installed plugins");
}
List<Path> pluginPaths = new ArrayList<>();
final List<Path> pluginPaths = new ArrayList<>();
Path pluginBinDir = env.binFile().resolve(pluginName);
final Path pluginBinDir = env.binFile().resolve(pluginName);
if (Files.exists(pluginBinDir)) {
if (Files.isDirectory(pluginBinDir) == false) {
throw new UserException(ExitCodes.IO_ERROR, "Bin dir for " + pluginName + " is not a directory");
@ -81,10 +83,19 @@ class RemovePluginCommand extends SettingCommand {
}
terminal.println(VERBOSE, "Removing: " + pluginDir);
Path tmpPluginDir = env.pluginsFile().resolve(".removing-" + pluginName);
final Path tmpPluginDir = env.pluginsFile().resolve(".removing-" + pluginName);
Files.move(pluginDir, tmpPluginDir, StandardCopyOption.ATOMIC_MOVE);
pluginPaths.add(tmpPluginDir);
IOUtils.rm(pluginPaths.toArray(new Path[pluginPaths.size()]));
// we preserve the config files in case the user is upgrading the plugin, but we print
// a message so the user knows in case they want to remove manually
final Path pluginConfigDir = env.configFile().resolve(pluginName);
if (Files.exists(pluginConfigDir)) {
terminal.println(
"-> Preserving plugin config files [" + pluginConfigDir + "] in case of upgrade, delete manually if not needed");
}
}
}

View File

@ -54,9 +54,9 @@ public class FsRepository extends BlobStoreRepository {
public static final Setting<String> REPOSITORIES_LOCATION_SETTING =
new Setting<>("repositories.fs.location", LOCATION_SETTING, Function.identity(), Property.NodeScope);
public static final Setting<ByteSizeValue> CHUNK_SIZE_SETTING =
Setting.byteSizeSetting("chunk_size", "-1", Property.NodeScope);
Setting.byteSizeSetting("chunk_size", new ByteSizeValue(-1), Property.NodeScope);
public static final Setting<ByteSizeValue> REPOSITORIES_CHUNK_SIZE_SETTING =
Setting.byteSizeSetting("repositories.fs.chunk_size", "-1", Property.NodeScope);
Setting.byteSizeSetting("repositories.fs.chunk_size", new ByteSizeValue(-1), Property.NodeScope);
public static final Setting<Boolean> COMPRESS_SETTING = Setting.boolSetting("compress", false, Property.NodeScope);
public static final Setting<Boolean> REPOSITORIES_COMPRESS_SETTING =
Setting.boolSetting("repositories.fs.compress", false, Property.NodeScope);

View File

@ -44,6 +44,7 @@ import java.util.Objects;
public final class Script implements ToXContent, Writeable {
public static final ScriptType DEFAULT_TYPE = ScriptType.INLINE;
public static final String DEFAULT_SCRIPT_LANG = "painless";
private String script;
private ScriptType type;
@ -60,7 +61,7 @@ public final class Script implements ToXContent, Writeable {
this(script, ScriptType.INLINE, null, null);
}
public Script(String script, ScriptType type, @Nullable String lang, @Nullable Map<String, ?> params) {
public Script(String script, ScriptType type, String lang, @Nullable Map<String, ?> params) {
this(script, type, lang, params, null);
}
@ -78,14 +79,14 @@ public final class Script implements ToXContent, Writeable {
* when serializing the script back to xcontent.
*/
@SuppressWarnings("unchecked")
public Script(String script, ScriptType type, @Nullable String lang, @Nullable Map<String, ?> params,
public Script(String script, ScriptType type, String lang, @Nullable Map<String, ?> params,
@Nullable XContentType contentType) {
if (contentType != null && type != ScriptType.INLINE) {
throw new IllegalArgumentException("The parameter contentType only makes sense for inline scripts");
}
this.script = Objects.requireNonNull(script);
this.type = Objects.requireNonNull(type);
this.lang = lang;
this.lang = lang == null ? DEFAULT_SCRIPT_LANG : lang;
this.params = (Map<String, Object>) params;
this.contentType = contentType;
}
@ -135,7 +136,7 @@ public final class Script implements ToXContent, Writeable {
* @return The type of script -- inline, stored, or file.
*/
public ScriptType getType() {
return type == null ? DEFAULT_TYPE : type;
return type;
}
/**
@ -196,7 +197,7 @@ public final class Script implements ToXContent, Writeable {
token = parser.nextToken();
}
if (token == XContentParser.Token.VALUE_STRING) {
return new Script(parser.text());
return new Script(parser.text(), ScriptType.INLINE, lang, null);
}
if (token != XContentParser.Token.START_OBJECT) {
throw new ElasticsearchParseException("expected a string value or an object, but found [{}] instead", token);

View File

@ -92,8 +92,6 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
public static final Setting<Integer> SCRIPT_MAX_COMPILATIONS_PER_MINUTE =
Setting.intSetting("script.max_compilations_per_minute", 15, 0, Property.Dynamic, Property.NodeScope);
private final String defaultLang;
private final Collection<ScriptEngineService> scriptEngines;
private final Map<String, ScriptEngineService> scriptEnginesByLang;
private final Map<String, ScriptEngineService> scriptEnginesByExt;
@ -131,8 +129,6 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
this.scriptContextRegistry = scriptContextRegistry;
int cacheMaxSize = SCRIPT_CACHE_SIZE_SETTING.get(settings);
this.defaultLang = scriptSettings.getDefaultScriptLanguageSetting().get(settings);
CacheBuilder<CacheKey, CompiledScript> cacheBuilder = CacheBuilder.builder();
if (cacheMaxSize >= 0) {
cacheBuilder.setMaximumWeight(cacheMaxSize);
@ -222,11 +218,6 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
}
String lang = script.getLang();
if (lang == null) {
lang = defaultLang;
}
ScriptEngineService scriptEngineService = getScriptEngineServiceForLang(lang);
if (canExecuteScript(lang, script.getType(), scriptContext) == false) {
throw new IllegalStateException("scripts of type [" + script.getType() + "], operation [" + scriptContext.getKey() + "] and lang [" + lang + "] are disabled");
@ -285,7 +276,7 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
throw new IllegalArgumentException("The parameter script (Script) must not be null.");
}
String lang = script.getLang() == null ? defaultLang : script.getLang();
String lang = script.getLang();
ScriptType type = script.getType();
//script.getScript() could return either a name or code for a script,
//but we check for a file script name first and an indexed script name second
@ -364,9 +355,8 @@ public class ScriptService extends AbstractComponent implements Closeable, Clust
}
private String validateScriptLanguage(String scriptLang) {
if (scriptLang == null) {
scriptLang = defaultLang;
} else if (scriptEnginesByLang.containsKey(scriptLang) == false) {
Objects.requireNonNull(scriptLang);
if (scriptEnginesByLang.containsKey(scriptLang) == false) {
throw new IllegalArgumentException("script_lang not supported [" + scriptLang + "]");
}
return scriptLang;

View File

@ -32,7 +32,16 @@ import java.util.function.Function;
public class ScriptSettings {
public static final String DEFAULT_LANG = "painless";
static final String LEGACY_DEFAULT_LANG = "groovy";
/**
* The default script language to use for scripts that are stored in documents that have no script lang set explicitly.
* This setting is legacy setting and only applies for indices created on ES versions prior to version 5.0
*
* This constant will be removed in the next major release.
*/
@Deprecated
public static final String LEGACY_SCRIPT_SETTING = "script.legacy.default_lang";
private static final Map<ScriptService.ScriptType, Setting<Boolean>> SCRIPT_TYPE_SETTING_MAP;
@ -49,7 +58,7 @@ public class ScriptSettings {
private final Map<ScriptContext, Setting<Boolean>> scriptContextSettingMap;
private final List<Setting<Boolean>> scriptLanguageSettings;
private final Setting<String> defaultScriptLanguageSetting;
private final Setting<String> defaultLegacyScriptLanguageSetting;
public ScriptSettings(ScriptEngineRegistry scriptEngineRegistry, ScriptContextRegistry scriptContextRegistry) {
Map<ScriptContext, Setting<Boolean>> scriptContextSettingMap = contextSettings(scriptContextRegistry);
@ -58,8 +67,8 @@ public class ScriptSettings {
List<Setting<Boolean>> scriptLanguageSettings = languageSettings(SCRIPT_TYPE_SETTING_MAP, scriptContextSettingMap, scriptEngineRegistry, scriptContextRegistry);
this.scriptLanguageSettings = Collections.unmodifiableList(scriptLanguageSettings);
this.defaultScriptLanguageSetting = new Setting<>("script.default_lang", DEFAULT_LANG, setting -> {
if (!DEFAULT_LANG.equals(setting) && !scriptEngineRegistry.getRegisteredLanguages().containsKey(setting)) {
this.defaultLegacyScriptLanguageSetting = new Setting<>(LEGACY_SCRIPT_SETTING, LEGACY_DEFAULT_LANG, setting -> {
if (!LEGACY_DEFAULT_LANG.equals(setting) && !scriptEngineRegistry.getRegisteredLanguages().containsKey(setting)) {
throw new IllegalArgumentException("unregistered default language [" + setting + "]");
}
return setting;
@ -160,7 +169,7 @@ public class ScriptSettings {
settings.addAll(SCRIPT_TYPE_SETTING_MAP.values());
settings.addAll(scriptContextSettingMap.values());
settings.addAll(scriptLanguageSettings);
settings.add(defaultScriptLanguageSetting);
settings.add(defaultLegacyScriptLanguageSetting);
return settings;
}
@ -168,7 +177,11 @@ public class ScriptSettings {
return scriptLanguageSettings;
}
public Setting<String> getDefaultScriptLanguageSetting() {
return defaultScriptLanguageSetting;
public Setting<String> getDefaultLegacyScriptLanguageSetting() {
return defaultLegacyScriptLanguageSetting;
}
public static String getLegacyDefaultLang(Settings settings) {
return settings.get(LEGACY_SCRIPT_SETTING, ScriptSettings.LEGACY_DEFAULT_LANG);
}
}

View File

@ -19,11 +19,11 @@
package org.elasticsearch.search.aggregations.bucket.geogrid;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -65,16 +65,17 @@ public class GeoHashGridParser extends GeoPointValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token == XContentParser.Token.VALUE_NUMBER || token == XContentParser.Token.VALUE_STRING) {
if (parseFieldMatcher.match(currentFieldName, GeoHashGridParams.FIELD_PRECISION)) {
if (context.matchField(currentFieldName, GeoHashGridParams.FIELD_PRECISION)) {
otherOptions.put(GeoHashGridParams.FIELD_PRECISION, parser.intValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, GeoHashGridParams.FIELD_SIZE)) {
} else if (context.matchField(currentFieldName, GeoHashGridParams.FIELD_SIZE)) {
otherOptions.put(GeoHashGridParams.FIELD_SIZE, parser.intValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, GeoHashGridParams.FIELD_SHARD_SIZE)) {
} else if (context.matchField(currentFieldName, GeoHashGridParams.FIELD_SHARD_SIZE)) {
otherOptions.put(GeoHashGridParams.FIELD_SHARD_SIZE, parser.intValue());
return true;
}

View File

@ -19,11 +19,11 @@
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -79,10 +79,11 @@ public class DateHistogramParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token.isValue()) {
if (parseFieldMatcher.match(currentFieldName, Histogram.INTERVAL_FIELD)) {
if (context.matchField(currentFieldName, Histogram.INTERVAL_FIELD)) {
if (token == XContentParser.Token.VALUE_STRING) {
otherOptions.put(Histogram.INTERVAL_FIELD, new DateHistogramInterval(parser.text()));
return true;
@ -90,13 +91,13 @@ public class DateHistogramParser extends NumericValuesSourceParser {
otherOptions.put(Histogram.INTERVAL_FIELD, parser.longValue());
return true;
}
} else if (parseFieldMatcher.match(currentFieldName, Histogram.MIN_DOC_COUNT_FIELD)) {
} else if (context.matchField(currentFieldName, Histogram.MIN_DOC_COUNT_FIELD)) {
otherOptions.put(Histogram.MIN_DOC_COUNT_FIELD, parser.longValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, Histogram.KEYED_FIELD)) {
} else if (context.matchField(currentFieldName, Histogram.KEYED_FIELD)) {
otherOptions.put(Histogram.KEYED_FIELD, parser.booleanValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, Histogram.OFFSET_FIELD)) {
} else if (context.matchField(currentFieldName, Histogram.OFFSET_FIELD)) {
if (token == XContentParser.Token.VALUE_STRING) {
otherOptions.put(Histogram.OFFSET_FIELD,
DateHistogramAggregationBuilder.parseStringOffset(parser.text()));
@ -109,7 +110,7 @@ public class DateHistogramParser extends NumericValuesSourceParser {
return false;
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (parseFieldMatcher.match(currentFieldName, Histogram.ORDER_FIELD)) {
if (context.matchField(currentFieldName, Histogram.ORDER_FIELD)) {
InternalOrder order = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
@ -127,9 +128,10 @@ public class DateHistogramParser extends NumericValuesSourceParser {
}
otherOptions.put(Histogram.ORDER_FIELD, order);
return true;
} else if (parseFieldMatcher.match(currentFieldName, ExtendedBounds.EXTENDED_BOUNDS_FIELD)) {
} else if (context.matchField(currentFieldName, ExtendedBounds.EXTENDED_BOUNDS_FIELD)) {
try {
otherOptions.put(ExtendedBounds.EXTENDED_BOUNDS_FIELD, ExtendedBounds.PARSER.apply(parser, () -> parseFieldMatcher));
otherOptions.put(ExtendedBounds.EXTENDED_BOUNDS_FIELD,
ExtendedBounds.PARSER.apply(parser, context::getParseFieldMatcher));
} catch (Exception e) {
throw new ParsingException(parser.getTokenLocation(), "Error parsing [{}]", e, aggregationName);
}

View File

@ -19,13 +19,13 @@
package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -85,26 +85,27 @@ public class HistogramParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token.isValue()) {
if (parseFieldMatcher.match(currentFieldName, Histogram.INTERVAL_FIELD)) {
if (context.matchField(currentFieldName, Histogram.INTERVAL_FIELD)) {
otherOptions.put(Histogram.INTERVAL_FIELD, parser.doubleValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, Histogram.MIN_DOC_COUNT_FIELD)) {
} else if (context.matchField(currentFieldName, Histogram.MIN_DOC_COUNT_FIELD)) {
otherOptions.put(Histogram.MIN_DOC_COUNT_FIELD, parser.longValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, Histogram.KEYED_FIELD)) {
} else if (context.matchField(currentFieldName, Histogram.KEYED_FIELD)) {
otherOptions.put(Histogram.KEYED_FIELD, parser.booleanValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, Histogram.OFFSET_FIELD)) {
} else if (context.matchField(currentFieldName, Histogram.OFFSET_FIELD)) {
otherOptions.put(Histogram.OFFSET_FIELD, parser.doubleValue());
return true;
} else {
return false;
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (parseFieldMatcher.match(currentFieldName, Histogram.ORDER_FIELD)) {
if (context.matchField(currentFieldName, Histogram.ORDER_FIELD)) {
InternalOrder order = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
@ -122,8 +123,8 @@ public class HistogramParser extends NumericValuesSourceParser {
}
otherOptions.put(Histogram.ORDER_FIELD, order);
return true;
} else if (parseFieldMatcher.match(currentFieldName, Histogram.EXTENDED_BOUNDS_FIELD)) {
double[] bounds = EXTENDED_BOUNDS_PARSER.apply(parser, () -> parseFieldMatcher);
} else if (context.matchField(currentFieldName, Histogram.EXTENDED_BOUNDS_FIELD)) {
double[] bounds = EXTENDED_BOUNDS_PARSER.apply(parser, context::getParseFieldMatcher);
otherOptions.put(Histogram.EXTENDED_BOUNDS_FIELD, bounds);
return true;
} else {

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -35,8 +35,8 @@ public class MissingParser extends AnyValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -65,20 +66,21 @@ public class RangeParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token == XContentParser.Token.START_ARRAY) {
if (parseFieldMatcher.match(currentFieldName, RangeAggregator.RANGES_FIELD)) {
if (context.matchField(currentFieldName, RangeAggregator.RANGES_FIELD)) {
List<Range> ranges = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
Range range = parseRange(parser, parseFieldMatcher);
Range range = parseRange(parser, context.getParseFieldMatcher());
ranges.add(range);
}
otherOptions.put(RangeAggregator.RANGES_FIELD, ranges);
return true;
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, RangeAggregator.KEYED_FIELD)) {
if (context.matchField(currentFieldName, RangeAggregator.KEYED_FIELD)) {
boolean keyed = parser.booleanValue();
otherOptions.put(RangeAggregator.KEYED_FIELD, keyed);
return true;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.search.aggregations.bucket.range.geodistance;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.io.stream.StreamInput;
@ -30,6 +29,7 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.GeoPointParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -110,28 +110,29 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
if (geoPointParser.token(aggregationName, currentFieldName, token, parser, parseFieldMatcher, otherOptions)) {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (geoPointParser.token(aggregationName, currentFieldName, token, parser, context.getParseFieldMatcher(), otherOptions)) {
return true;
} else if (token == XContentParser.Token.VALUE_STRING) {
if (parseFieldMatcher.match(currentFieldName, UNIT_FIELD)) {
if (context.matchField(currentFieldName, UNIT_FIELD)) {
DistanceUnit unit = DistanceUnit.fromString(parser.text());
otherOptions.put(UNIT_FIELD, unit);
return true;
} else if (parseFieldMatcher.match(currentFieldName, DISTANCE_TYPE_FIELD)) {
} else if (context.matchField(currentFieldName, DISTANCE_TYPE_FIELD)) {
GeoDistance distanceType = GeoDistance.fromString(parser.text());
otherOptions.put(DISTANCE_TYPE_FIELD, distanceType);
return true;
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, RangeAggregator.KEYED_FIELD)) {
if (context.matchField(currentFieldName, RangeAggregator.KEYED_FIELD)) {
boolean keyed = parser.booleanValue();
otherOptions.put(RangeAggregator.KEYED_FIELD, keyed);
return true;
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (parseFieldMatcher.match(currentFieldName, RangeAggregator.RANGES_FIELD)) {
if (context.matchField(currentFieldName, RangeAggregator.RANGES_FIELD)) {
List<Range> ranges = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
String fromAsStr = null;
@ -144,17 +145,17 @@ public class GeoDistanceParser extends GeoPointValuesSourceParser {
if (token == XContentParser.Token.FIELD_NAME) {
toOrFromOrKey = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(toOrFromOrKey, Range.FROM_FIELD)) {
if (context.matchField(toOrFromOrKey, Range.FROM_FIELD)) {
from = parser.doubleValue();
} else if (parseFieldMatcher.match(toOrFromOrKey, Range.TO_FIELD)) {
} else if (context.matchField(toOrFromOrKey, Range.TO_FIELD)) {
to = parser.doubleValue();
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if (parseFieldMatcher.match(toOrFromOrKey, Range.KEY_FIELD)) {
if (context.matchField(toOrFromOrKey, Range.KEY_FIELD)) {
key = parser.text();
} else if (parseFieldMatcher.match(toOrFromOrKey, Range.FROM_FIELD)) {
} else if (context.matchField(toOrFromOrKey, Range.FROM_FIELD)) {
fromAsStr = parser.text();
} else if (parseFieldMatcher.match(toOrFromOrKey, Range.TO_FIELD)) {
} else if (context.matchField(toOrFromOrKey, Range.TO_FIELD)) {
toAsStr = parser.text();
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.BytesValuesSourceParser;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.ip.IpRangeAggregationBuilder.Range;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -102,21 +103,22 @@ public class IpRangeParser extends BytesValuesSourceParser {
@Override
protected boolean token(String aggregationName, String currentFieldName,
Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher,
Map<ParseField, Object> otherOptions) throws IOException {
if (parseFieldMatcher.match(currentFieldName, RangeAggregator.RANGES_FIELD)) {
Token token,
XContentParseContext context,
Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (context.matchField(currentFieldName, RangeAggregator.RANGES_FIELD)) {
if (parser.currentToken() != Token.START_ARRAY) {
throw new ParsingException(parser.getTokenLocation(), "[ranges] must be passed as an array, but got a " + token);
}
List<Range> ranges = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
Range range = parseRange(parser, parseFieldMatcher);
Range range = parseRange(parser, context.getParseFieldMatcher());
ranges.add(range);
}
otherOptions.put(RangeAggregator.RANGES_FIELD, ranges);
return true;
} else if (parseFieldMatcher.match(parser.currentName(), RangeAggregator.KEYED_FIELD)) {
} else if (context.matchField(parser.currentName(), RangeAggregator.KEYED_FIELD)) {
otherOptions.put(RangeAggregator.KEYED_FIELD, parser.booleanValue());
return true;
}

View File

@ -20,9 +20,9 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -57,20 +57,21 @@ public class DiversifiedSamplerParser extends AnyValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(currentFieldName, SamplerAggregator.SHARD_SIZE_FIELD)) {
if (context.matchField(currentFieldName, SamplerAggregator.SHARD_SIZE_FIELD)) {
int shardSize = parser.intValue();
otherOptions.put(SamplerAggregator.SHARD_SIZE_FIELD, shardSize);
return true;
} else if (parseFieldMatcher.match(currentFieldName, SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD)) {
} else if (context.matchField(currentFieldName, SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD)) {
int maxDocsPerValue = parser.intValue();
otherOptions.put(SamplerAggregator.MAX_DOCS_PER_VALUE_FIELD, maxDocsPerValue);
return true;
}
} else if (token == XContentParser.Token.VALUE_STRING) {
if (parseFieldMatcher.match(currentFieldName, SamplerAggregator.EXECUTION_HINT_FIELD)) {
if (context.matchField(currentFieldName, SamplerAggregator.EXECUTION_HINT_FIELD)) {
String executionHint = parser.text();
otherOptions.put(SamplerAggregator.EXECUTION_HINT_FIELD, executionHint);
return true;

View File

@ -19,7 +19,6 @@
package org.elasticsearch.search.aggregations.bucket.significant;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
@ -33,6 +32,7 @@ import org.elasticsearch.search.aggregations.bucket.terms.AbstractTermsParser;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -81,17 +81,18 @@ public class SignificantTermsParser extends AbstractTermsParser {
}
@Override
public boolean parseSpecial(String aggregationName, XContentParser parser, ParseFieldMatcher parseFieldMatcher, Token token,
String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException {
public boolean parseSpecial(String aggregationName, XContentParseContext context, Token token,
String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException {
if (token == XContentParser.Token.START_OBJECT) {
SignificanceHeuristicParser significanceHeuristicParser = significanceHeuristicParserRegistry
.lookupReturningNullIfNotFound(currentFieldName, parseFieldMatcher);
.lookupReturningNullIfNotFound(currentFieldName, context.getParseFieldMatcher());
if (significanceHeuristicParser != null) {
SignificanceHeuristic significanceHeuristic = significanceHeuristicParser.parse(parser, parseFieldMatcher);
SignificanceHeuristic significanceHeuristic = significanceHeuristicParser.parse(context);
otherOptions.put(SignificantTermsAggregationBuilder.HEURISTIC, significanceHeuristic);
return true;
} else if (parseFieldMatcher.match(currentFieldName, SignificantTermsAggregationBuilder.BACKGROUND_FILTER)) {
QueryParseContext queryParseContext = new QueryParseContext(queriesRegistry, parser, parseFieldMatcher);
} else if (context.matchField(currentFieldName, SignificantTermsAggregationBuilder.BACKGROUND_FILTER)) {
QueryParseContext queryParseContext = new QueryParseContext(context.getDefaultScriptLanguage(), queriesRegistry,
context.getParser(), context.getParseFieldMatcher());
Optional<QueryBuilder> filter = queryParseContext.parseInnerQueryBuilder();
if (filter.isPresent()) {
otherOptions.put(SignificantTermsAggregationBuilder.BACKGROUND_FILTER, filter.get());

View File

@ -22,12 +22,12 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import java.io.IOException;
@ -113,13 +113,13 @@ public class GND extends NXYSignificanceHeuristic {
}
@Override
public SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher)
throws IOException, QueryShardException {
public SignificanceHeuristic parse(XContentParseContext context) throws IOException, QueryShardException {
XContentParser parser = context.getParser();
String givenName = parser.currentName();
boolean backgroundIsSuperset = true;
XContentParser.Token token = parser.nextToken();
while (!token.equals(XContentParser.Token.END_OBJECT)) {
if (parseFieldMatcher.match(parser.currentName(), BACKGROUND_IS_SUPERSET)) {
if (context.matchField(parser.currentName(), BACKGROUND_IS_SUPERSET)) {
parser.nextToken();
backgroundIsSuperset = parser.booleanValue();
} else {

View File

@ -22,12 +22,12 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import java.io.IOException;
@ -104,8 +104,9 @@ public class JLHScore extends SignificanceHeuristic {
return builder;
}
public static SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher)
public static SignificanceHeuristic parse(XContentParseContext context)
throws IOException, QueryShardException {
XContentParser parser = context.getParser();
// move to the closing bracket
if (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) {
throw new ElasticsearchParseException(

View File

@ -23,12 +23,12 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import java.io.IOException;
@ -152,17 +152,18 @@ public abstract class NXYSignificanceHeuristic extends SignificanceHeuristic {
public abstract static class NXYParser implements SignificanceHeuristicParser {
@Override
public SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher)
public SignificanceHeuristic parse(XContentParseContext context)
throws IOException, QueryShardException {
XContentParser parser = context.getParser();
String givenName = parser.currentName();
boolean includeNegatives = false;
boolean backgroundIsSuperset = true;
XContentParser.Token token = parser.nextToken();
while (!token.equals(XContentParser.Token.END_OBJECT)) {
if (parseFieldMatcher.match(parser.currentName(), INCLUDE_NEGATIVES_FIELD)) {
if (context.matchField(parser.currentName(), INCLUDE_NEGATIVES_FIELD)) {
parser.nextToken();
includeNegatives = parser.booleanValue();
} else if (parseFieldMatcher.match(parser.currentName(), BACKGROUND_IS_SUPERSET)) {
} else if (context.matchField(parser.currentName(), BACKGROUND_IS_SUPERSET)) {
parser.nextToken();
backgroundIsSuperset = parser.booleanValue();
} else {

View File

@ -22,12 +22,12 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import java.io.IOException;
@ -56,8 +56,9 @@ public class PercentageScore extends SignificanceHeuristic {
return builder;
}
public static SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher)
public static SignificanceHeuristic parse(XContentParseContext context)
throws IOException, QueryShardException {
XContentParser parser = context.getParser();
// move to the closing bracket
if (!parser.nextToken().equals(XContentParser.Token.END_OBJECT)) {
throw new ElasticsearchParseException("failed to parse [percentage] significance heuristic. expected an empty object, but got [{}] instead", parser.currentToken());

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLoggerFactory;
@ -35,6 +34,7 @@ import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -146,8 +146,9 @@ public class ScriptHeuristic extends SignificanceHeuristic {
return Objects.equals(script, other.script);
}
public static SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher)
public static SignificanceHeuristic parse(XContentParseContext context)
throws IOException, QueryShardException {
XContentParser parser = context.getParser();
String heuristicName = parser.currentName();
Script script = null;
XContentParser.Token token;
@ -156,8 +157,8 @@ public class ScriptHeuristic extends SignificanceHeuristic {
if (token.equals(XContentParser.Token.FIELD_NAME)) {
currentFieldName = parser.currentName();
} else {
if (parseFieldMatcher.match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, parseFieldMatcher);
if (context.matchField(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else {
throw new ElasticsearchParseException("failed to parse [{}] significance heuristic. unknown object [{}]", heuristicName, currentFieldName);
}

View File

@ -20,9 +20,9 @@
package org.elasticsearch.search.aggregations.bucket.significant.heuristics;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import java.io.IOException;
@ -31,6 +31,5 @@ import java.io.IOException;
*/
@FunctionalInterface
public interface SignificanceHeuristicParser {
SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher) throws IOException,
ParsingException;
SignificanceHeuristic parse(XContentParseContext context) throws IOException, ParsingException;
}

View File

@ -20,13 +20,13 @@
package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -89,47 +89,48 @@ public abstract class AbstractTermsParser extends AnyValuesSourceParser {
Map<ParseField, Object> otherOptions);
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
if (incExcParser.token(currentFieldName, token, parser, parseFieldMatcher, otherOptions)) {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (incExcParser.token(currentFieldName, token, parser, context.getParseFieldMatcher(), otherOptions)) {
return true;
} else if (token == XContentParser.Token.VALUE_STRING) {
if (parseFieldMatcher.match(currentFieldName, EXECUTION_HINT_FIELD_NAME)) {
if (context.matchField(currentFieldName, EXECUTION_HINT_FIELD_NAME)) {
otherOptions.put(EXECUTION_HINT_FIELD_NAME, parser.text());
return true;
} else if (parseFieldMatcher.match(currentFieldName, SubAggCollectionMode.KEY)) {
otherOptions.put(SubAggCollectionMode.KEY, SubAggCollectionMode.parse(parser.text(), parseFieldMatcher));
} else if (context.matchField(currentFieldName, SubAggCollectionMode.KEY)) {
otherOptions.put(SubAggCollectionMode.KEY, SubAggCollectionMode.parse(parser.text(), context.getParseFieldMatcher()));
return true;
} else if (parseFieldMatcher.match(currentFieldName, REQUIRED_SIZE_FIELD_NAME)) {
} else if (context.matchField(currentFieldName, REQUIRED_SIZE_FIELD_NAME)) {
otherOptions.put(REQUIRED_SIZE_FIELD_NAME, parser.intValue());
return true;
} else if (parseSpecial(aggregationName, parser, parseFieldMatcher, token, currentFieldName, otherOptions)) {
} else if (parseSpecial(aggregationName, context, token, currentFieldName, otherOptions)) {
return true;
}
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(currentFieldName, REQUIRED_SIZE_FIELD_NAME)) {
if (context.matchField(currentFieldName, REQUIRED_SIZE_FIELD_NAME)) {
otherOptions.put(REQUIRED_SIZE_FIELD_NAME, parser.intValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, SHARD_SIZE_FIELD_NAME)) {
} else if (context.matchField(currentFieldName, SHARD_SIZE_FIELD_NAME)) {
otherOptions.put(SHARD_SIZE_FIELD_NAME, parser.intValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, MIN_DOC_COUNT_FIELD_NAME)) {
} else if (context.matchField(currentFieldName, MIN_DOC_COUNT_FIELD_NAME)) {
otherOptions.put(MIN_DOC_COUNT_FIELD_NAME, parser.longValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, SHARD_MIN_DOC_COUNT_FIELD_NAME)) {
} else if (context.matchField(currentFieldName, SHARD_MIN_DOC_COUNT_FIELD_NAME)) {
otherOptions.put(SHARD_MIN_DOC_COUNT_FIELD_NAME, parser.longValue());
return true;
} else if (parseSpecial(aggregationName, parser, parseFieldMatcher, token, currentFieldName, otherOptions)) {
} else if (parseSpecial(aggregationName, context, token, currentFieldName, otherOptions)) {
return true;
}
} else if (parseSpecial(aggregationName, parser, parseFieldMatcher, token, currentFieldName, otherOptions)) {
} else if (parseSpecial(aggregationName, context, token, currentFieldName, otherOptions)) {
return true;
}
return false;
}
public abstract boolean parseSpecial(String aggregationName, XContentParser parser, ParseFieldMatcher parseFieldMatcher,
XContentParser.Token token, String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException;
public abstract boolean parseSpecial(String aggregationName, XContentParseContext context,
Token token, String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException;
protected abstract TermsAggregator.BucketCountThresholds getDefaultBucketCountThresholds();

View File

@ -19,7 +19,6 @@
package org.elasticsearch.search.aggregations.bucket.terms;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
@ -27,6 +26,7 @@ import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -75,15 +75,16 @@ public class TermsParser extends AbstractTermsParser {
}
@Override
public boolean parseSpecial(String aggregationName, XContentParser parser, ParseFieldMatcher parseFieldMatcher, Token token,
String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException {
public boolean parseSpecial(String aggregationName, XContentParseContext context, Token token,
String currentFieldName, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token == XContentParser.Token.START_OBJECT) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregationBuilder.ORDER_FIELD)) {
if (context.matchField(currentFieldName, TermsAggregationBuilder.ORDER_FIELD)) {
otherOptions.put(TermsAggregationBuilder.ORDER_FIELD, Collections.singletonList(parseOrderParam(aggregationName, parser)));
return true;
}
} else if (token == XContentParser.Token.START_ARRAY) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregationBuilder.ORDER_FIELD)) {
if (context.matchField(currentFieldName, TermsAggregationBuilder.ORDER_FIELD)) {
List<OrderElement> orderElements = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token == XContentParser.Token.START_OBJECT) {
@ -98,7 +99,7 @@ public class TermsParser extends AbstractTermsParser {
return true;
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, TermsAggregationBuilder.SHOW_TERM_DOC_COUNT_ERROR)) {
if (context.matchField(currentFieldName, TermsAggregationBuilder.SHOW_TERM_DOC_COUNT_ERROR)) {
otherOptions.put(TermsAggregationBuilder.SHOW_TERM_DOC_COUNT_ERROR, parser.booleanValue());
return true;
}

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.avg;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -38,8 +38,8 @@ public class AvgParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -20,10 +20,9 @@
package org.elasticsearch.search.aggregations.metrics.cardinality;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -51,13 +50,13 @@ public class CardinalityParser extends AnyValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
if (token.isValue()) {
if (parseFieldMatcher.match(currentFieldName, CardinalityAggregationBuilder.PRECISION_THRESHOLD_FIELD)) {
otherOptions.put(CardinalityAggregationBuilder.PRECISION_THRESHOLD_FIELD, parser.longValue());
if (context.matchField(currentFieldName, CardinalityAggregationBuilder.PRECISION_THRESHOLD_FIELD)) {
otherOptions.put(CardinalityAggregationBuilder.PRECISION_THRESHOLD_FIELD, context.getParser().longValue());
return true;
} else if (parseFieldMatcher.match(currentFieldName, REHASH)) {
} else if (context.matchField(currentFieldName, REHASH)) {
// ignore
return true;
}

View File

@ -20,10 +20,10 @@
package org.elasticsearch.search.aggregations.metrics.geobounds;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -48,11 +48,11 @@ public class GeoBoundsParser extends GeoPointValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, GeoBoundsAggregator.WRAP_LONGITUDE_FIELD)) {
otherOptions.put(GeoBoundsAggregator.WRAP_LONGITUDE_FIELD, parser.booleanValue());
if (context.matchField(currentFieldName, GeoBoundsAggregator.WRAP_LONGITUDE_FIELD)) {
otherOptions.put(GeoBoundsAggregator.WRAP_LONGITUDE_FIELD, context.getParser().booleanValue());
return true;
}
}

View File

@ -20,10 +20,9 @@
package org.elasticsearch.search.aggregations.metrics.geocentroid;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.GeoPointValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -40,8 +39,8 @@ public class GeoCentroidParser extends GeoPointValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.max;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -38,8 +38,8 @@ public class MaxParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -19,10 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.min;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -39,8 +38,8 @@ public class MinParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -21,10 +21,10 @@ package org.elasticsearch.search.aggregations.metrics.percentiles;
import com.carrotsearch.hppc.DoubleArrayList;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -45,10 +45,11 @@ public abstract class AbstractPercentilesParser extends NumericValuesSourceParse
}
@Override
protected boolean token(String aggregationName, String currentFieldName, Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
XContentParser parser = context.getParser();
if (token == XContentParser.Token.START_ARRAY) {
if (parseFieldMatcher.match(currentFieldName, keysField())) {
if (context.matchField(currentFieldName, keysField())) {
DoubleArrayList values = new DoubleArrayList(10);
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
double value = parser.doubleValue();
@ -61,7 +62,7 @@ public abstract class AbstractPercentilesParser extends NumericValuesSourceParse
return false;
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
if (parseFieldMatcher.match(currentFieldName, KEYED_FIELD)) {
if (context.matchField(currentFieldName, KEYED_FIELD)) {
boolean keyed = parser.booleanValue();
otherOptions.put(KEYED_FIELD, keyed);
return true;
@ -80,7 +81,7 @@ public abstract class AbstractPercentilesParser extends NumericValuesSourceParse
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(currentFieldName, COMPRESSION_FIELD)) {
if (context.matchField(currentFieldName, COMPRESSION_FIELD)) {
double compression = parser.doubleValue();
otherOptions.put(COMPRESSION_FIELD, compression);
} else {
@ -96,7 +97,7 @@ public abstract class AbstractPercentilesParser extends NumericValuesSourceParse
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
if (parseFieldMatcher.match(currentFieldName, NUMBER_SIGNIFICANT_DIGITS_FIELD)) {
if (context.matchField(currentFieldName, NUMBER_SIGNIFICANT_DIGITS_FIELD)) {
int numberOfSignificantValueDigits = parser.intValue();
otherOptions.put(NUMBER_SIGNIFICANT_DIGITS_FIELD, numberOfSignificantValueDigits);
} else {

View File

@ -232,13 +232,13 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.START_OBJECT || token == XContentParser.Token.VALUE_STRING) {
if (context.getParseFieldMatcher().match(currentFieldName, INIT_SCRIPT_FIELD)) {
initScript = Script.parse(parser, context.getParseFieldMatcher());
initScript = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName, MAP_SCRIPT_FIELD)) {
mapScript = Script.parse(parser, context.getParseFieldMatcher());
mapScript = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName, COMBINE_SCRIPT_FIELD)) {
combineScript = Script.parse(parser, context.getParseFieldMatcher());
combineScript = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName, REDUCE_SCRIPT_FIELD)) {
reduceScript = Script.parse(parser, context.getParseFieldMatcher());
reduceScript = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (token == XContentParser.Token.START_OBJECT &&
context.getParseFieldMatcher().match(currentFieldName, PARAMS_FIELD)) {
params = parser.map();

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.stats;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -38,8 +38,8 @@ public class StatsParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.stats.extended;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -38,11 +38,11 @@ public class ExtendedStatsParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
if (parseFieldMatcher.match(currentFieldName, ExtendedStatsAggregator.SIGMA_FIELD)) {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
if (context.matchField(currentFieldName, ExtendedStatsAggregator.SIGMA_FIELD)) {
if (token.isValue()) {
otherOptions.put(ExtendedStatsAggregator.SIGMA_FIELD, parser.doubleValue());
otherOptions.put(ExtendedStatsAggregator.SIGMA_FIELD, context.getParser().doubleValue());
return true;
}
}

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.sum;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.NumericValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSourceType;
@ -38,8 +38,8 @@ public class SumParser extends NumericValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -622,7 +622,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (context.getParseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SCRIPT_FIELD)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName,
SearchSourceBuilder.IGNORE_FAILURE_FIELD)) {
ignoreFailure = parser.booleanValue();
@ -633,7 +633,7 @@ public class TopHitsAggregationBuilder extends AbstractAggregationBuilder<TopHit
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.getParseFieldMatcher().match(currentFieldName, SearchSourceBuilder.SCRIPT_FIELD)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + currentFieldName + "].",

View File

@ -19,9 +19,9 @@
package org.elasticsearch.search.aggregations.metrics.valuecount;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.support.AbstractValuesSourceParser.AnyValuesSourceParser;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -40,8 +40,8 @@ public class ValueCountParser extends AnyValuesSourceParser {
}
@Override
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException {
protected boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException {
return false;
}

View File

@ -179,7 +179,7 @@ public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggr
} else if (context.getParseFieldMatcher().match(currentFieldName, GAP_POLICY)) {
gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
} else if (context.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
@ -201,7 +201,7 @@ public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggr
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
Map<String, Object> map = parser.map();
bucketsPathsMap = new HashMap<>();
@ -260,4 +260,4 @@ public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggr
public String getWriteableName() {
return NAME;
}
}
}

View File

@ -142,7 +142,7 @@ public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAg
} else if (context.getParseFieldMatcher().match(currentFieldName, GAP_POLICY)) {
gapPolicy = GapPolicy.parse(context, parser.text(), parser.getTokenLocation());
} else if (context.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(),
"Unknown key for a " + token + " in [" + reducerName + "]: [" + currentFieldName + "].");
@ -164,7 +164,7 @@ public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAg
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName, BUCKETS_PATH)) {
Map<String, Object> map = parser.map();
bucketsPathsMap = new HashMap<>();
@ -219,4 +219,4 @@ public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAg
public String getWriteableName() {
return NAME;
}
}
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.search.aggregations.support;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
@ -95,6 +94,8 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource>
Object missing = null;
DateTimeZone timezone = null;
Map<ParseField, Object> otherOptions = new HashMap<>();
XContentParseContext parserContext =
new XContentParseContext(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
XContentParser.Token token;
String currentFieldName = null;
@ -126,22 +127,22 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource>
+ valueType + "]. It can only work on value of type ["
+ targetValueType + "]");
}
} else if (!token(aggregationName, currentFieldName, token, parser, context.getParseFieldMatcher(), otherOptions)) {
} else if (!token(aggregationName, currentFieldName, token, parserContext, otherOptions)) {
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (!token(aggregationName, currentFieldName, token, parser, context.getParseFieldMatcher(), otherOptions)) {
} else if (!token(aggregationName, currentFieldName, token, parserContext, otherOptions)) {
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (scriptable && token == XContentParser.Token.START_OBJECT) {
if (context.getParseFieldMatcher().match(currentFieldName, ScriptField.SCRIPT)) {
script = Script.parse(parser, context.getParseFieldMatcher());
} else if (!token(aggregationName, currentFieldName, token, parser, context.getParseFieldMatcher(), otherOptions)) {
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (!token(aggregationName, currentFieldName, token, parserContext, otherOptions)) {
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
} else if (!token(aggregationName, currentFieldName, token, parser, context.getParseFieldMatcher(), otherOptions)) {
} else if (!token(aggregationName, currentFieldName, token, parserContext, otherOptions)) {
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " [" + currentFieldName + "] in [" + aggregationName + "].");
}
@ -184,8 +185,7 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource>
* the target type of the final value output by the aggregation
* @param otherOptions
* a {@link Map} containing the extra options parsed by the
* {@link #token(String, String, org.elasticsearch.common.xcontent.XContentParser.Token,
* XContentParser, ParseFieldMatcher, Map)}
* {@link #token(String, String, XContentParser.Token, XContentParseContext, Map)}
* method
* @return the created factory
*/
@ -203,10 +203,8 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource>
* the name of the current field being parsed
* @param token
* the current token for the parser
* @param parser
* the parser
* @param parseFieldMatcher
* the {@link ParseFieldMatcher} to use to match field names
* @param context
* the query context
* @param otherOptions
* a {@link Map} of options to be populated by successive calls
* to this method which will then be passed to the
@ -217,6 +215,6 @@ public abstract class AbstractValuesSourceParser<VS extends ValuesSource>
* @throws IOException
* if an error occurs whilst parsing
*/
protected abstract boolean token(String aggregationName, String currentFieldName, XContentParser.Token token, XContentParser parser,
ParseFieldMatcher parseFieldMatcher, Map<ParseField, Object> otherOptions) throws IOException;
protected abstract boolean token(String aggregationName, String currentFieldName, XContentParser.Token token,
XContentParseContext context, Map<ParseField, Object> otherOptions) throws IOException;
}

View File

@ -0,0 +1,65 @@
/*
* 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.search.aggregations.support;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.xcontent.XContentParser;
/**
* A minimal context for parsing xcontent into aggregation builders.
* Only a minimal set of dependencies and settings are available.
*/
public final class XContentParseContext {
private final XContentParser parser;
private final ParseFieldMatcher parseFieldMatcher;
private final String defaultScriptLanguage;
public XContentParseContext(XContentParser parser, ParseFieldMatcher parseFieldMatcher, String defaultScriptLanguage) {
this.parser = parser;
this.parseFieldMatcher = parseFieldMatcher;
this.defaultScriptLanguage = defaultScriptLanguage;
}
public XContentParser getParser() {
return parser;
}
public ParseFieldMatcher getParseFieldMatcher() {
return parseFieldMatcher;
}
public String getDefaultScriptLanguage() {
return defaultScriptLanguage;
}
/**
* Returns whether the parse field we're looking for matches with the found field name.
*
* Helper that delegates to {@link ParseFieldMatcher#match(String, ParseField)}.
*/
public boolean matchField(String fieldName, ParseField parseField) {
return parseFieldMatcher.match(fieldName, parseField);
}
}

View File

@ -1273,7 +1273,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
currentFieldName = parser.currentName();
} else if (token.isValue()) {
if (context.getParseFieldMatcher().match(currentFieldName, SCRIPT_FIELD)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else if (context.getParseFieldMatcher().match(currentFieldName, IGNORE_FAILURE_FIELD)) {
ignoreFailure = parser.booleanValue();
} else {
@ -1282,7 +1282,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
}
} else if (token == XContentParser.Token.START_OBJECT) {
if (context.getParseFieldMatcher().match(currentFieldName, SCRIPT_FIELD)) {
script = Script.parse(parser, context.getParseFieldMatcher());
script = Script.parse(parser, context.getParseFieldMatcher(), context.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName
+ "].", parser.getTokenLocation());

View File

@ -244,7 +244,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
currentName = parser.currentName();
} else if (token == XContentParser.Token.START_OBJECT) {
if (parseField.match(currentName, ScriptField.SCRIPT)) {
script = Script.parse(parser, parseField);
script = Script.parse(parser, parseField, context.getDefaultScriptLanguage());
} else if (parseField.match(currentName, NESTED_FILTER_FIELD)) {
nestedFilter = context.parseInnerQueryBuilder();
} else {
@ -260,7 +260,7 @@ public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
} else if (parseField.match(currentName, NESTED_PATH_FIELD)) {
nestedPath = parser.text();
} else if (parseField.match(currentName, ScriptField.SCRIPT)) {
script = Script.parse(parser, parseField);
script = Script.parse(parser, parseField, context.getDefaultScriptLanguage());
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + NAME + "] failed to parse field [" + currentName + "]");
}

View File

@ -69,6 +69,7 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.replication.TransportReplicationActionTests;
import org.elasticsearch.action.termvectors.MultiTermVectorsAction;
import org.elasticsearch.action.termvectors.MultiTermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsAction;
@ -117,7 +118,6 @@ import static org.hamcrest.Matchers.emptyIterable;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.instanceOf;
@ClusterScope(scope = Scope.SUITE, numClientNodes = 1, minNumDataNodes = 2)
public class IndicesRequestIT extends ESIntegTestCase {
@ -638,8 +638,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
assertThat("no internal requests intercepted for action [" + action + "]", requests.size(), greaterThan(0));
}
for (TransportRequest internalRequest : requests) {
assertThat(internalRequest, instanceOf(IndicesRequest.class));
IndicesRequest indicesRequest = (IndicesRequest) internalRequest;
IndicesRequest indicesRequest = convertRequest(internalRequest);
assertThat(internalRequest.getClass().getName(), indicesRequest.indices(), equalTo(originalRequest.indices()));
assertThat(indicesRequest.indicesOptions(), equalTo(originalRequest.indicesOptions()));
}
@ -651,14 +650,24 @@ public class IndicesRequestIT extends ESIntegTestCase {
List<TransportRequest> requests = consumeTransportRequests(action);
assertThat("no internal requests intercepted for action [" + action + "]", requests.size(), greaterThan(0));
for (TransportRequest internalRequest : requests) {
assertThat(internalRequest, instanceOf(IndicesRequest.class));
for (String index : ((IndicesRequest) internalRequest).indices()) {
IndicesRequest indicesRequest = convertRequest(internalRequest);
for (String index : indicesRequest.indices()) {
assertThat(indices, hasItem(index));
}
}
}
}
static IndicesRequest convertRequest(TransportRequest request) {
final IndicesRequest indicesRequest;
if (request instanceof IndicesRequest) {
indicesRequest = (IndicesRequest) request;
} else {
indicesRequest = TransportReplicationActionTests.resolveRequest(request);
}
return indicesRequest;
}
private String randomIndexOrAlias() {
String index = randomFrom(indices);
if (randomBoolean()) {

View File

@ -60,14 +60,6 @@ import static org.hamcrest.Matchers.nullValue;
public class BulkWithUpdatesIT extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put("script.default_lang", CustomScriptPlugin.NAME)
.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Collections.singleton(CustomScriptPlugin.class);
@ -557,6 +549,7 @@ public class BulkWithUpdatesIT extends ESIntegTestCase {
" \"script\" : {" +
" \"inline\" : \"ctx._source.field2 = 'value2'\"" +
" }," +
" \"lang\" : \"" + CustomScriptPlugin.NAME + "\"," +
" \"upsert\" : {" +
" \"field1\" : \"value1'\"" +
" }" +
@ -589,7 +582,9 @@ public class BulkWithUpdatesIT extends ESIntegTestCase {
assertThat(bulkResponse.getItems().length, equalTo(3));
assertThat(bulkResponse.getItems()[0].isFailed(), equalTo(false));
assertThat(bulkResponse.getItems()[1].isFailed(), equalTo(false));
assertThat(bulkResponse.getItems()[2].isFailed(), equalTo(false));
assertThat(bulkResponse.getItems()[2].isFailed(), equalTo(true));
assertThat(bulkResponse.getItems()[2].getFailure().getCause().getCause().getMessage(),
equalTo("script_lang not supported [painless]"));
client().admin().indices().prepareRefresh("test").get();

View File

@ -26,6 +26,7 @@ import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.client.transport.NoNodeAvailableException;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ESAllocationTestCase;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockException;
@ -36,6 +37,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
@ -47,21 +49,25 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.EngineClosedException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardClosedException;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.cluster.ESAllocationTestCase;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponseOptions;
import org.elasticsearch.transport.TransportService;
@ -75,12 +81,12 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state;
@ -93,12 +99,32 @@ import static org.hamcrest.Matchers.arrayWithSize;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyInt;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class TransportReplicationActionTests extends ESTestCase {
/**
* takes a request that was sent by a {@link TransportReplicationAction} and captured
* and returns the underlying request if it's wrapped or the original (cast to the expected type).
*
* This will throw a {@link ClassCastException} if the request is of the wrong type.
*/
public static <R extends ReplicationRequest> R resolveRequest(TransportRequest requestOrWrappedRequest) {
if (requestOrWrappedRequest instanceof TransportReplicationAction.ConcreteShardRequest) {
requestOrWrappedRequest = ((TransportReplicationAction.ConcreteShardRequest<?>)requestOrWrappedRequest).getRequest();
}
return (R) requestOrWrappedRequest;
}
private static ThreadPool threadPool;
private ClusterService clusterService;
@ -411,7 +437,7 @@ public class TransportReplicationActionTests extends ESTestCase {
isRelocated.set(true);
executeOnPrimary = false;
}
action.new AsyncPrimaryAction(request, createTransportChannel(listener), task) {
action.new AsyncPrimaryAction(request, primaryShard.allocationId().getId(), createTransportChannel(listener), task) {
@Override
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
@ -452,7 +478,8 @@ public class TransportReplicationActionTests extends ESTestCase {
final String index = "test";
final ShardId shardId = new ShardId(index, "_na_", 0);
ClusterState state = state(index, true, ShardRoutingState.RELOCATING);
String primaryTargetNodeId = state.getRoutingTable().shardRoutingTable(shardId).primaryShard().relocatingNodeId();
final ShardRouting primaryShard = state.getRoutingTable().shardRoutingTable(shardId).primaryShard();
String primaryTargetNodeId = primaryShard.relocatingNodeId();
// simulate execution of the primary phase on the relocation target node
state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(primaryTargetNodeId)).build();
setState(clusterService, state);
@ -460,7 +487,7 @@ public class TransportReplicationActionTests extends ESTestCase {
PlainActionFuture<Response> listener = new PlainActionFuture<>();
ReplicationTask task = maybeTask();
AtomicBoolean executed = new AtomicBoolean();
action.new AsyncPrimaryAction(request, createTransportChannel(listener), task) {
action.new AsyncPrimaryAction(request, primaryShard.allocationId().getRelocationId(), createTransportChannel(listener), task) {
@Override
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
@ -473,6 +500,11 @@ public class TransportReplicationActionTests extends ESTestCase {
}
};
}
@Override
public void onFailure(Exception e) {
throw new RuntimeException(e);
}
}.run();
assertThat(executed.get(), equalTo(true));
assertPhase(task, "finished");
@ -596,7 +628,9 @@ public class TransportReplicationActionTests extends ESTestCase {
state = ClusterState.builder(state).metaData(metaData).build();
setState(clusterService, state);
AtomicBoolean executed = new AtomicBoolean();
action.new AsyncPrimaryAction(new Request(shardId), createTransportChannel(new PlainActionFuture<>()), null) {
ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard();
action.new AsyncPrimaryAction(new Request(shardId), primaryShard.allocationId().getId(),
createTransportChannel(new PlainActionFuture<>()), null) {
@Override
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
@ -613,8 +647,10 @@ public class TransportReplicationActionTests extends ESTestCase {
final String index = "test";
final ShardId shardId = new ShardId(index, "_na_", 0);
// no replica, we only want to test on primary
setState(clusterService, state(index, true, ShardRoutingState.STARTED));
final ClusterState state = state(index, true, ShardRoutingState.STARTED);
setState(clusterService, state);
logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
final ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard();
Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
ReplicationTask task = maybeTask();
@ -622,7 +658,7 @@ public class TransportReplicationActionTests extends ESTestCase {
final boolean throwExceptionOnCreation = i == 1;
final boolean throwExceptionOnRun = i == 2;
final boolean respondWithError = i == 3;
action.new AsyncPrimaryAction(request, createTransportChannel(listener), task) {
action.new AsyncPrimaryAction(request, primaryShard.allocationId().getId(), createTransportChannel(listener), task) {
@Override
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
@ -666,8 +702,9 @@ public class TransportReplicationActionTests extends ESTestCase {
public void testReplicasCounter() throws Exception {
final ShardId shardId = new ShardId("test", "_na_", 0);
setState(clusterService, state(shardId.getIndexName(), true,
ShardRoutingState.STARTED, ShardRoutingState.STARTED));
final ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
setState(clusterService, state);
final ShardRouting replicaRouting = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0);
boolean throwException = randomBoolean();
final ReplicationTask task = maybeTask();
Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) {
@ -683,7 +720,9 @@ public class TransportReplicationActionTests extends ESTestCase {
};
final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
try {
replicaOperationTransportHandler.messageReceived(new Request().setShardId(shardId),
replicaOperationTransportHandler.messageReceived(
new TransportReplicationAction.ConcreteShardRequest<>(
new Request().setShardId(shardId), replicaRouting.allocationId().getId()),
createTransportChannel(new PlainActionFuture<>()), task);
} catch (ElasticsearchException e) {
assertThat(e.getMessage(), containsString("simulated"));
@ -725,6 +764,111 @@ public class TransportReplicationActionTests extends ESTestCase {
assertEquals(ActiveShardCount.from(requestWaitForActiveShards), request.waitForActiveShards());
}
/** test that a primary request is rejected if it arrives at a shard with a wrong allocation id */
public void testPrimaryActionRejectsWrongAid() throws Exception {
final String index = "test";
final ShardId shardId = new ShardId(index, "_na_", 0);
setState(clusterService, state(index, true, ShardRoutingState.STARTED));
PlainActionFuture<Response> listener = new PlainActionFuture<>();
Request request = new Request(shardId).timeout("1ms");
action.new PrimaryOperationTransportHandler().messageReceived(
new TransportReplicationAction.ConcreteShardRequest<>(request, "_not_a_valid_aid_"),
createTransportChannel(listener), maybeTask()
);
try {
listener.get();
fail("using a wrong aid didn't fail the operation");
} catch (ExecutionException execException) {
Throwable throwable = execException.getCause();
logger.debug("got exception:" , throwable);
assertTrue(throwable.getClass() + " is not a retry exception", action.retryPrimaryException(throwable));
}
}
/** test that a replica request is rejected if it arrives at a shard with a wrong allocation id */
public void testReplicaActionRejectsWrongAid() throws Exception {
final String index = "test";
final ShardId shardId = new ShardId(index, "_na_", 0);
ClusterState state = state(index, false, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
final ShardRouting replica = state.routingTable().shardRoutingTable(shardId).replicaShards().get(0);
// simulate execution of the node holding the replica
state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(replica.currentNodeId())).build();
setState(clusterService, state);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
Request request = new Request(shardId).timeout("1ms");
action.new ReplicaOperationTransportHandler().messageReceived(
new TransportReplicationAction.ConcreteShardRequest<>(request, "_not_a_valid_aid_"),
createTransportChannel(listener), maybeTask()
);
try {
listener.get();
fail("using a wrong aid didn't fail the operation");
} catch (ExecutionException execException) {
Throwable throwable = execException.getCause();
if (action.retryPrimaryException(throwable) == false) {
throw new AssertionError("thrown exception is not retriable", throwable);
}
assertThat(throwable.getMessage(), containsString("_not_a_valid_aid_"));
}
}
/**
* test throwing a {@link org.elasticsearch.action.support.replication.TransportReplicationAction.RetryOnReplicaException}
* causes a retry
*/
public void testRetryOnReplica() throws Exception {
final ShardId shardId = new ShardId("test", "_na_", 0);
ClusterState state = state(shardId.getIndexName(), true, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
final ShardRouting replica = state.getRoutingTable().shardRoutingTable(shardId).replicaShards().get(0);
// simulate execution of the node holding the replica
state = ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(replica.currentNodeId())).build();
setState(clusterService, state);
AtomicBoolean throwException = new AtomicBoolean(true);
final ReplicationTask task = maybeTask();
Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) {
@Override
protected ReplicaResult shardOperationOnReplica(Request request) {
assertPhase(task, "replica");
if (throwException.get()) {
throw new RetryOnReplicaException(shardId, "simulation");
}
return new ReplicaResult();
}
};
final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
final PlainActionFuture<Response> listener = new PlainActionFuture<>();
final Request request = new Request().setShardId(shardId);
request.primaryTerm(state.metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
replicaOperationTransportHandler.messageReceived(
new TransportReplicationAction.ConcreteShardRequest<>(request, replica.allocationId().getId()),
createTransportChannel(listener), task);
if (listener.isDone()) {
listener.get(); // fail with the exception if there
fail("listener shouldn't be done");
}
// no retry yet
List<CapturingTransport.CapturedRequest> capturedRequests =
transport.getCapturedRequestsByTargetNodeAndClear().get(replica.currentNodeId());
assertThat(capturedRequests, nullValue());
// release the waiting
throwException.set(false);
setState(clusterService, state);
capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear().get(replica.currentNodeId());
assertThat(capturedRequests, notNullValue());
assertThat(capturedRequests.size(), equalTo(1));
final CapturingTransport.CapturedRequest capturedRequest = capturedRequests.get(0);
assertThat(capturedRequest.action, equalTo("testActionWithExceptions[r]"));
assertThat(capturedRequest.request, instanceOf(TransportReplicationAction.ConcreteShardRequest.class));
assertThat(((TransportReplicationAction.ConcreteShardRequest<?>) capturedRequest.request).getRequest(), equalTo(request));
assertThat(((TransportReplicationAction.ConcreteShardRequest<?>) capturedRequest.request).getTargetAllocationID(),
equalTo(replica.allocationId().getId()));
}
private void assertIndexShardCounter(int expected) {
assertThat(count.get(), equalTo(expected));
}
@ -797,7 +941,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Action(Settings settings, String actionName, TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool) {
super(settings, actionName, transportService, clusterService, null, threadPool,
super(settings, actionName, transportService, clusterService, mockIndicesService(clusterService), threadPool,
new ShardStateAction(settings, clusterService, transportService, null, null, threadPool),
new ActionFilters(new HashSet<>()), new IndexNameExpressionResolver(Settings.EMPTY),
Request::new, Request::new, ThreadPool.Names.SAME);
@ -825,43 +969,76 @@ public class TransportReplicationActionTests extends ESTestCase {
protected boolean resolveIndex() {
return false;
}
}
@Override
protected void acquirePrimaryShardReference(ShardId shardId, ActionListener<PrimaryShardReference> onReferenceAcquired) {
final IndicesService mockIndicesService(ClusterService clusterService) {
final IndicesService indicesService = mock(IndicesService.class);
when(indicesService.indexServiceSafe(any(Index.class))).then(invocation -> {
Index index = (Index)invocation.getArguments()[0];
final ClusterState state = clusterService.state();
final IndexMetaData indexSafe = state.metaData().getIndexSafe(index);
return mockIndexService(indexSafe, clusterService);
});
when(indicesService.indexService(any(Index.class))).then(invocation -> {
Index index = (Index) invocation.getArguments()[0];
final ClusterState state = clusterService.state();
if (state.metaData().hasIndex(index.getName())) {
final IndexMetaData indexSafe = state.metaData().getIndexSafe(index);
return mockIndexService(clusterService.state().metaData().getIndexSafe(index), clusterService);
} else {
return null;
}
});
return indicesService;
}
final IndexService mockIndexService(final IndexMetaData indexMetaData, ClusterService clusterService) {
final IndexService indexService = mock(IndexService.class);
when(indexService.getShard(anyInt())).then(invocation -> {
int shard = (Integer) invocation.getArguments()[0];
final ShardId shardId = new ShardId(indexMetaData.getIndex(), shard);
if (shard > indexMetaData.getNumberOfShards()) {
throw new ShardNotFoundException(shardId);
}
return mockIndexShard(shardId, clusterService);
});
return indexService;
}
private IndexShard mockIndexShard(ShardId shardId, ClusterService clusterService) {
final IndexShard indexShard = mock(IndexShard.class);
doAnswer(invocation -> {
ActionListener<Releasable> callback = (ActionListener<Releasable>) invocation.getArguments()[0];
count.incrementAndGet();
PrimaryShardReference primaryShardReference = new PrimaryShardReference(null, null) {
@Override
public boolean isRelocated() {
return isRelocated.get();
}
@Override
public void failShard(String reason, @Nullable Exception e) {
throw new UnsupportedOperationException();
}
@Override
public ShardRouting routingEntry() {
ShardRouting shardRouting = clusterService.state().getRoutingTable()
.shardRoutingTable(shardId).primaryShard();
assert shardRouting != null;
return shardRouting;
}
@Override
public void close() {
count.decrementAndGet();
}
};
onReferenceAcquired.onResponse(primaryShardReference);
}
@Override
protected void acquireReplicaOperationLock(ShardId shardId, long primaryTerm, ActionListener<Releasable> onLockAcquired) {
callback.onResponse(count::decrementAndGet);
return null;
}).when(indexShard).acquirePrimaryOperationLock(any(ActionListener.class), anyString());
doAnswer(invocation -> {
long term = (Long)invocation.getArguments()[0];
ActionListener<Releasable> callback = (ActionListener<Releasable>) invocation.getArguments()[1];
final long primaryTerm = indexShard.getPrimaryTerm();
if (term < primaryTerm) {
throw new IllegalArgumentException(String.format(Locale.ROOT, "%s operation term [%d] is too old (current [%d])",
shardId, term, primaryTerm));
}
count.incrementAndGet();
onLockAcquired.onResponse(count::decrementAndGet);
}
callback.onResponse(count::decrementAndGet);
return null;
}).when(indexShard).acquireReplicaOperationLock(anyLong(), any(ActionListener.class), anyString());
when(indexShard.routingEntry()).thenAnswer(invocationOnMock -> {
final ClusterState state = clusterService.state();
final RoutingNode node = state.getRoutingNodes().node(state.nodes().getLocalNodeId());
final ShardRouting routing = node.getByShardId(shardId);
if (routing == null) {
throw new ShardNotFoundException(shardId, "shard is no longer assigned to current node");
}
return routing;
});
when(indexShard.state()).thenAnswer(invocationOnMock -> isRelocated.get() ? IndexShardState.RELOCATED : IndexShardState.STARTED);
doThrow(new AssertionError("failed shard is not supported")).when(indexShard).failShard(anyString(), any(Exception.class));
when(indexShard.getPrimaryTerm()).thenAnswer(i ->
clusterService.state().metaData().getIndexSafe(shardId.getIndex()).primaryTerm(shardId.id()));
return indexShard;
}
class NoopReplicationOperation extends ReplicationOperation<Request, Request, Action.PrimaryResult> {
@ -879,11 +1056,6 @@ public class TransportReplicationActionTests extends ESTestCase {
* Transport channel that is needed for replica operation testing.
*/
public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener) {
return createTransportChannel(listener, error -> {
});
}
public TransportChannel createTransportChannel(final PlainActionFuture<Response> listener, Consumer<Throwable> consumer) {
return new TransportChannel() {
@Override
@ -908,7 +1080,6 @@ public class TransportReplicationActionTests extends ESTestCase {
@Override
public void sendResponse(Exception exception) throws IOException {
consumer.accept(exception);
listener.onFailure(exception);
}

View File

@ -55,7 +55,7 @@ public class UpdateRequestTests extends ESTestCase {
assertThat(script, notNullValue());
assertThat(script.getScript(), equalTo("script1"));
assertThat(script.getType(), equalTo(ScriptType.INLINE));
assertThat(script.getLang(), nullValue());
assertThat(script.getLang(), equalTo(Script.DEFAULT_SCRIPT_LANG));
Map<String, Object> params = script.getParams();
assertThat(params, nullValue());
@ -67,7 +67,7 @@ public class UpdateRequestTests extends ESTestCase {
assertThat(script, notNullValue());
assertThat(script.getScript(), equalTo("script1"));
assertThat(script.getType(), equalTo(ScriptType.INLINE));
assertThat(script.getLang(), nullValue());
assertThat(script.getLang(), equalTo(Script.DEFAULT_SCRIPT_LANG));
params = script.getParams();
assertThat(params, nullValue());
@ -79,7 +79,7 @@ public class UpdateRequestTests extends ESTestCase {
assertThat(script, notNullValue());
assertThat(script.getScript(), equalTo("script1"));
assertThat(script.getType(), equalTo(ScriptType.INLINE));
assertThat(script.getLang(), nullValue());
assertThat(script.getLang(), equalTo(Script.DEFAULT_SCRIPT_LANG));
params = script.getParams();
assertThat(params, notNullValue());
assertThat(params.size(), equalTo(1));
@ -92,7 +92,7 @@ public class UpdateRequestTests extends ESTestCase {
assertThat(script, notNullValue());
assertThat(script.getScript(), equalTo("script1"));
assertThat(script.getType(), equalTo(ScriptType.INLINE));
assertThat(script.getLang(), nullValue());
assertThat(script.getLang(), equalTo(Script.DEFAULT_SCRIPT_LANG));
params = script.getParams();
assertThat(params, notNullValue());
assertThat(params.size(), equalTo(1));
@ -107,7 +107,7 @@ public class UpdateRequestTests extends ESTestCase {
assertThat(script, notNullValue());
assertThat(script.getScript(), equalTo("script1"));
assertThat(script.getType(), equalTo(ScriptType.INLINE));
assertThat(script.getLang(), nullValue());
assertThat(script.getLang(), equalTo(Script.DEFAULT_SCRIPT_LANG));
params = script.getParams();
assertThat(params, notNullValue());
assertThat(params.size(), equalTo(1));
@ -124,7 +124,7 @@ public class UpdateRequestTests extends ESTestCase {
assertThat(script, notNullValue());
assertThat(script.getScript(), equalTo("script1"));
assertThat(script.getType(), equalTo(ScriptType.INLINE));
assertThat(script.getLang(), nullValue());
assertThat(script.getLang(), equalTo(Script.DEFAULT_SCRIPT_LANG));
params = script.getParams();
assertThat(params, notNullValue());
assertThat(params.size(), equalTo(1));

View File

@ -111,12 +111,6 @@ public class JarHellTests extends ESTestCase {
}
}
public void testLog4jLeniency() throws Exception {
Path dir = createTempDir();
URL[] jars = {makeJar(dir, "foo.jar", null, "org/apache/log4j/DuplicateClass.class"), makeJar(dir, "bar.jar", null, "org/apache/log4j/DuplicateClass.class")};
JarHell.checkJarHell(jars);
}
public void testLog4jThrowableProxyLeniency() throws Exception {
Path dir = createTempDir();
URL[] jars = {makeJar(dir, "foo.jar", null, "org.apache.logging.log4j.core.impl.ThrowableProxy.class"), makeJar(dir, "bar.jar", null, "org.apache.logging.log4j.core.impl.ThrowableProxy.class")};

View File

@ -56,6 +56,7 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singleton;
@ -729,10 +730,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
ImmutableOpenMap<String, Long> shardSizes = shardSizesBuilder.build();
final ClusterInfo clusterInfo = new DevNullClusterInfo(usages, usages, shardSizes);
DiskThresholdDecider decider = makeDecider(diskSettings);
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList(
new SameShardAllocationDecider(Settings.EMPTY),
makeDecider(diskSettings))));
new HashSet<>(Arrays.asList(new SameShardAllocationDecider(Settings.EMPTY), decider)));
ClusterInfoService cis = new ClusterInfoService() {
@Override
@ -832,6 +832,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
ImmutableOpenMap.Builder<String, Long> shardSizesBuilder = ImmutableOpenMap.builder();
shardSizesBuilder.put("[test][0][p]", 40L);
shardSizesBuilder.put("[test][1][p]", 40L);
shardSizesBuilder.put("[foo][0][p]", 10L);
ImmutableOpenMap<String, Long> shardSizes = shardSizesBuilder.build();
final ClusterInfo clusterInfo = new DevNullClusterInfo(usages, usages, shardSizes);
@ -839,10 +840,12 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
DiskThresholdDecider diskThresholdDecider = makeDecider(diskSettings);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0))
.put(IndexMetaData.builder("foo").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
.build();
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.addAsNew(metaData.index("foo"))
.build();
DiscoveryNode discoveryNode1 = new DiscoveryNode("node1", new LocalTransportAddress("1"), emptyMap(),
@ -881,6 +884,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// Two shards consuming each 80% of disk space while 70% is allowed, but one is relocating, so shard 0 can stay
firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, true, ShardRoutingState.STARTED);
secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", "node2", true, ShardRoutingState.RELOCATING);
ShardRouting fooRouting = TestShardRouting.newShardRouting("foo", 0, "node1", null, true, ShardRoutingState.UNASSIGNED);
firstRoutingNode = new RoutingNode("node1", discoveryNode1, firstRouting, secondRouting);
builder = RoutingTable.builder().add(
IndexRoutingTable.builder(firstRouting.index())
@ -898,6 +902,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
false);
decision = diskThresholdDecider.canRemain(firstRouting, firstRoutingNode, routingAllocation);
assertThat(decision.type(), equalTo(Decision.Type.YES));
decision = diskThresholdDecider.canAllocate(fooRouting, firstRoutingNode, routingAllocation);
assertThat(decision.type(), equalTo(Decision.Type.NO));
// Creating AllocationService instance and the services it depends on...
ClusterInfoService cis = new ClusterInfoService() {

View File

@ -0,0 +1,88 @@
/*
* 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.settings;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.util.PageCacheRecycler;
import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.indices.IndicesQueryCache;
import org.elasticsearch.indices.IndicesRequestCache;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.test.ESTestCase;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasItem;
public class MemorySizeSettingsTests extends ESTestCase {
public void testPageCacheLimitHeapSetting() {
assertMemorySizeSetting(PageCacheRecycler.LIMIT_HEAP_SETTING, "cache.recycler.page.limit.heap",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.1)));
}
public void testIndexBufferSizeSetting() {
assertMemorySizeSetting(IndexingMemoryController.INDEX_BUFFER_SIZE_SETTING, "indices.memory.index_buffer_size",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.1)));
}
public void testQueryCacheSizeSetting() {
assertMemorySizeSetting(IndicesQueryCache.INDICES_CACHE_QUERY_SIZE_SETTING, "indices.queries.cache.size",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.1)));
}
public void testIndicesRequestCacheSetting() {
assertMemorySizeSetting(IndicesRequestCache.INDICES_CACHE_QUERY_SIZE, "indices.requests.cache.size",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.01)));
}
public void testCircuitBreakerSettings() {
assertMemorySizeSetting(HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, "indices.breaker.total.limit",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.7)));
assertMemorySizeSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, "indices.breaker.fielddata.limit",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.6)));
assertMemorySizeSetting(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, "indices.breaker.request.limit",
new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.6)));
assertMemorySizeSetting(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING,
"network.breaker.inflight_requests.limit", new ByteSizeValue((JvmInfo.jvmInfo().getMem().getHeapMax().bytes())));
}
public void testIndicesFieldDataCacheSetting() {
assertMemorySizeSetting(IndicesFieldDataCache.INDICES_FIELDDATA_CACHE_SIZE_KEY, "indices.fielddata.cache.size",
new ByteSizeValue(-1));
}
private void assertMemorySizeSetting(Setting<ByteSizeValue> setting, String settingKey, ByteSizeValue defaultValue) {
assertThat(setting, notNullValue());
assertThat(setting.getKey(), equalTo(settingKey));
assertThat(setting.getProperties(), hasItem(Property.NodeScope));
assertThat(setting.getDefault(Settings.EMPTY),
equalTo(defaultValue));
Settings settingWithPercentage = Settings.builder().put(settingKey, "25%").build();
assertThat(setting.get(settingWithPercentage),
equalTo(new ByteSizeValue((long) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.25))));
Settings settingWithBytesValue = Settings.builder().put(settingKey, "1024b").build();
assertThat(setting.get(settingWithBytesValue), equalTo(new ByteSizeValue(1024)));
}
}

View File

@ -22,6 +22,7 @@ import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.test.ESTestCase;
import java.util.Arrays;
@ -68,6 +69,44 @@ public class SettingTests extends ESTestCase {
assertEquals(new ByteSizeValue(12), value.get());
}
public void testMemorySize() {
Setting<ByteSizeValue> memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", new ByteSizeValue(1024), Property.Dynamic,
Property.NodeScope);
assertFalse(memorySizeValueSetting.isGroupSetting());
ByteSizeValue memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), 1024);
memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", s -> "2048b", Property.Dynamic, Property.NodeScope);
memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), 2048);
memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", "50%", Property.Dynamic, Property.NodeScope);
assertFalse(memorySizeValueSetting.isGroupSetting());
memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.5, 1.0);
memorySizeValueSetting = Setting.memorySizeSetting("a.byte.size", s -> "25%", Property.Dynamic, Property.NodeScope);
memorySizeValue = memorySizeValueSetting.get(Settings.EMPTY);
assertEquals(memorySizeValue.bytes(), JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.25, 1.0);
AtomicReference<ByteSizeValue> value = new AtomicReference<>(null);
ClusterSettings.SettingUpdater<ByteSizeValue> settingUpdater = memorySizeValueSetting.newUpdater(value::set, logger);
try {
settingUpdater.apply(Settings.builder().put("a.byte.size", 12).build(), Settings.EMPTY);
fail("no unit");
} catch (IllegalArgumentException ex) {
assertEquals("failed to parse setting [a.byte.size] with value [12] as a size in bytes: unit is missing or unrecognized",
ex.getMessage());
}
assertTrue(settingUpdater.apply(Settings.builder().put("a.byte.size", "12b").build(), Settings.EMPTY));
assertEquals(new ByteSizeValue(12), value.get());
assertTrue(settingUpdater.apply(Settings.builder().put("a.byte.size", "20%").build(), Settings.EMPTY));
assertEquals(new ByteSizeValue((int) (JvmInfo.jvmInfo().getMem().getHeapMax().bytes() * 0.2)), value.get());
}
public void testSimpleUpdate() {
Setting<Boolean> booleanSetting = Setting.boolSetting("foo.bar", false, Property.Dynamic, Property.NodeScope);
AtomicReference<Boolean> atomicBoolean = new AtomicReference<>(null);

View File

@ -0,0 +1,70 @@
/*
* 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.analysis;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.core.WhitespaceTokenizer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.test.ESTokenStreamTestCase;
import java.io.IOException;
import java.io.StringReader;
public class MinHashFilterFactoryTests extends ESTokenStreamTestCase {
public void testDefault() throws IOException {
int default_hash_count = 1;
int default_bucket_size = 512;
int default_hash_set_size = 1;
Settings settings = Settings.builder()
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
.build();
AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings);
TokenFilterFactory tokenFilter = analysisService.tokenFilter("min_hash");
String source = "the quick brown fox";
Tokenizer tokenizer = new WhitespaceTokenizer();
tokenizer.setReader(new StringReader(source));
// with_rotation is true by default, and hash_set_size is 1, so even though the source doesn't
// have enough tokens to fill all the buckets, we still expect 512 tokens.
assertStreamHasNumberOfTokens(tokenFilter.create(tokenizer),
default_hash_count * default_bucket_size * default_hash_set_size);
}
public void testSettings() throws IOException {
Settings settings = Settings.builder()
.put("index.analysis.filter.test_min_hash.type", "min_hash")
.put("index.analysis.filter.test_min_hash.hash_count", "1")
.put("index.analysis.filter.test_min_hash.bucket_count", "2")
.put("index.analysis.filter.test_min_hash.hash_set_size", "1")
.put("index.analysis.filter.test_min_hash.with_rotation", false)
.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString())
.build();
AnalysisService analysisService = AnalysisTestsHelper.createAnalysisServiceFromSettings(settings);
TokenFilterFactory tokenFilter = analysisService.tokenFilter("test_min_hash");
String source = "sushi";
Tokenizer tokenizer = new WhitespaceTokenizer();
tokenizer.setReader(new StringReader(source));
// despite the fact that bucket_count is 2 and hash_set_size is 1,
// because with_rotation is false, we only expect 1 token here.
assertStreamHasNumberOfTokens(tokenFilter.create(tokenizer), 1);
}
}

View File

@ -0,0 +1,63 @@
/*
* 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.query;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptSettings;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.test.ESTestCase;
import static java.util.Collections.emptyList;
public class QueryRewriteContextTests extends ESTestCase {
public void testNewParseContextWithLegacyScriptLanguage() throws Exception {
String defaultLegacyScriptLanguage = randomAsciiOfLength(4);
IndexMetaData.Builder indexMetadata = new IndexMetaData.Builder("index");
indexMetadata.settings(Settings.builder().put("index.version.created", Version.CURRENT)
.put("index.number_of_shards", 1)
.put("index.number_of_replicas", 1)
);
IndicesQueriesRegistry indicesQueriesRegistry = new SearchModule(Settings.EMPTY, false, emptyList()).getQueryParserRegistry();
IndexSettings indexSettings = new IndexSettings(indexMetadata.build(),
Settings.builder().put(ScriptSettings.LEGACY_SCRIPT_SETTING, defaultLegacyScriptLanguage).build());
QueryRewriteContext queryRewriteContext =
new QueryRewriteContext(indexSettings, null, null, indicesQueriesRegistry, null, null, null);;
// verify that the default script language in the query parse context is equal to defaultLegacyScriptLanguage variable:
QueryParseContext queryParseContext =
queryRewriteContext.newParseContextWithLegacyScriptLanguage(XContentHelper.createParser(new BytesArray("{}")));
assertEquals(defaultLegacyScriptLanguage, queryParseContext.getDefaultScriptLanguage());
// verify that the script query's script language is equal to defaultLegacyScriptLanguage variable:
XContentParser parser = XContentHelper.createParser(new BytesArray("{\"script\" : {\"script\": \"return true\"}}"));
queryParseContext = queryRewriteContext.newParseContextWithLegacyScriptLanguage(parser);
ScriptQueryBuilder queryBuilder = (ScriptQueryBuilder) queryParseContext.parseInnerQueryBuilder().get();
assertEquals(defaultLegacyScriptLanguage, queryBuilder.script().getLang());
}
}

View File

@ -45,6 +45,7 @@ import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import static org.hamcrest.CoreMatchers.containsString;
@ -86,7 +87,9 @@ public class ScriptServiceTests extends ESTestCase {
resourceWatcherService = new ResourceWatcherService(baseSettings, null);
scriptEngineService = new TestEngineService();
dangerousScriptEngineService = new TestDangerousEngineService();
scriptEnginesByLangMap = ScriptModesTests.buildScriptEnginesByLangMap(Collections.singleton(scriptEngineService));
TestEngineService defaultScriptServiceEngine = new TestEngineService(Script.DEFAULT_SCRIPT_LANG) {};
scriptEnginesByLangMap = ScriptModesTests.buildScriptEnginesByLangMap(
new HashSet<>(Arrays.asList(scriptEngineService, defaultScriptServiceEngine)));
//randomly register custom script contexts
int randomInt = randomIntBetween(0, 3);
//prevent duplicates using map
@ -103,7 +106,8 @@ public class ScriptServiceTests extends ESTestCase {
String context = plugin + "_" + operation;
contexts.put(context, new ScriptContext.Plugin(plugin, operation));
}
scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(scriptEngineService, dangerousScriptEngineService));
scriptEngineRegistry = new ScriptEngineRegistry(Arrays.asList(scriptEngineService, dangerousScriptEngineService,
defaultScriptServiceEngine));
scriptContextRegistry = new ScriptContextRegistry(contexts.values());
scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
scriptContexts = scriptContextRegistry.scriptContexts().toArray(new ScriptContext[scriptContextRegistry.scriptContexts().size()]);
@ -406,12 +410,11 @@ public class ScriptServiceTests extends ESTestCase {
public void testDefaultLanguage() throws IOException {
Settings.Builder builder = Settings.builder();
builder.put("script.default_lang", "test");
builder.put("script.inline", "true");
buildScriptService(builder.build());
CompiledScript script = scriptService.compile(new Script("1 + 1", ScriptType.INLINE, null, null),
randomFrom(scriptContexts), Collections.emptyMap());
assertEquals(script.lang(), "test");
assertEquals(script.lang(), Script.DEFAULT_SCRIPT_LANG);
}
public void testStoreScript() throws Exception {
@ -509,14 +512,24 @@ public class ScriptServiceTests extends ESTestCase {
public static final String NAME = "test";
private final String name;
public TestEngineService() {
this(NAME);
}
public TestEngineService(String name) {
this.name = name;
}
@Override
public String getType() {
return NAME;
return name;
}
@Override
public String getExtension() {
return NAME;
return name;
}
@Override

View File

@ -34,32 +34,33 @@ import static org.hamcrest.Matchers.equalTo;
public class ScriptSettingsTests extends ESTestCase {
public void testDefaultLanguageIsPainless() {
public void testDefaultLegacyLanguageIsPainless() {
ScriptEngineRegistry scriptEngineRegistry =
new ScriptEngineRegistry(Collections.singletonList(new CustomScriptEngineService()));
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
assertThat(scriptSettings.getDefaultScriptLanguageSetting().get(Settings.EMPTY), equalTo("painless"));
assertThat(scriptSettings.getDefaultLegacyScriptLanguageSetting().get(Settings.EMPTY),
equalTo(ScriptSettings.LEGACY_DEFAULT_LANG));
}
public void testCustomDefaultLanguage() {
public void testCustomLegacyDefaultLanguage() {
ScriptEngineRegistry scriptEngineRegistry =
new ScriptEngineRegistry(Collections.singletonList(new CustomScriptEngineService()));
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
String defaultLanguage = CustomScriptEngineService.NAME;
Settings settings = Settings.builder().put("script.default_lang", defaultLanguage).build();
assertThat(scriptSettings.getDefaultScriptLanguageSetting().get(settings), equalTo(defaultLanguage));
Settings settings = Settings.builder().put(ScriptSettings.LEGACY_SCRIPT_SETTING, defaultLanguage).build();
assertThat(scriptSettings.getDefaultLegacyScriptLanguageSetting().get(settings), equalTo(defaultLanguage));
}
public void testInvalidDefaultLanguage() {
public void testInvalidLegacyDefaultLanguage() {
ScriptEngineRegistry scriptEngineRegistry =
new ScriptEngineRegistry(Collections.singletonList(new CustomScriptEngineService()));
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
Settings settings = Settings.builder().put("script.default_lang", "C++").build();
Settings settings = Settings.builder().put(ScriptSettings.LEGACY_SCRIPT_SETTING, "C++").build();
try {
scriptSettings.getDefaultScriptLanguageSetting().get(settings);
scriptSettings.getDefaultLegacyScriptLanguageSetting().get(settings);
fail("should have seen unregistered default language");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("unregistered default language [C++]"));

View File

@ -20,12 +20,10 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.plugins.Plugin;
@ -49,6 +47,7 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.Signi
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser;
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.support.XContentParseContext;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.search.aggregations.bucket.SharedSignificantTermsTestMethods;
@ -172,7 +171,7 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
@Override
public List<SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser>> getSignificanceHeuristics() {
return singletonList(new SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser>(SimpleHeuristic.NAME,
SimpleHeuristic::new, SimpleHeuristic::parse));
SimpleHeuristic::new, (context) -> SimpleHeuristic.parse(context)));
}
@Override
@ -239,9 +238,9 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
return subsetFreq / subsetSize > supersetFreq / supersetSize ? 2.0 : 1.0;
}
public static SignificanceHeuristic parse(XContentParser parser, ParseFieldMatcher parseFieldMatcher)
public static SignificanceHeuristic parse(XContentParseContext context)
throws IOException, QueryShardException {
parser.nextToken();
context.getParser().nextToken();
return new SimpleHeuristic();
}
}

View File

@ -182,7 +182,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
ScriptSortBuilder builder = ScriptSortBuilder.fromXContent(context, null);
assertEquals("doc['field_name'].value * factor", builder.script().getScript());
assertNull(builder.script().getLang());
assertEquals(Script.DEFAULT_SCRIPT_LANG, builder.script().getLang());
assertEquals(1.1, builder.script().getParams().get("factor"));
assertEquals(ScriptType.INLINE, builder.script().getType());
assertEquals(ScriptSortType.NUMBER, builder.type());
@ -208,7 +208,7 @@ public class ScriptSortBuilderTests extends AbstractSortTestCase<ScriptSortBuild
QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, ParseFieldMatcher.STRICT);
ScriptSortBuilder builder = ScriptSortBuilder.fromXContent(context, null);
assertEquals("doc['field_name'].value", builder.script().getScript());
assertNull(builder.script().getLang());
assertEquals(Script.DEFAULT_SCRIPT_LANG, builder.script().getLang());
assertNull(builder.script().getParams());
assertEquals(ScriptType.INLINE, builder.script().getType());
assertEquals(ScriptSortType.NUMBER, builder.type());

View File

@ -17,7 +17,7 @@ elasticsearch Java `ActionFuture` (in turn a nicer extension to Java own
`Future`) which allows to register listeners (closures) on it for
success and failures, as well as blocking for the response. For example:
[source,js]
[source,groovy]
--------------------------------------------------
def indexR = client.index {
index "test"
@ -38,7 +38,7 @@ println "Indexed $indexR.response.id into $indexR.response.index/$indexR.respons
In the above example, calling `indexR.response` will simply block for
the response. We can also block for the response for a specific timeout:
[source,js]
[source,groovy]
--------------------------------------------------
IndexResponse response = indexR.response "5s" // block for 5 seconds, same as:
response = indexR.response 5, TimeValue.SECONDS //
@ -47,7 +47,7 @@ response = indexR.response 5, TimeValue.SECONDS //
We can also register closures that will be called on success and on
failure:
[source,js]
[source,groovy]
--------------------------------------------------
indexR.success = {IndexResponse response ->
println "Indexed $response.id into $response.index/$response.type"
@ -65,7 +65,7 @@ This option allows to pass the actual instance of the request (instead
of a closure) as a parameter. The rest is similar to the closure as a
parameter option (the `GActionFuture` handling). For example:
[source,js]
[source,groovy]
--------------------------------------------------
def indexR = client.index (new IndexRequest(
index: "test",
@ -90,7 +90,7 @@ The last option is to provide an actual instance of the API request, and
an `ActionListener` for the callback. This is exactly like the Java API
with the added `gexecute` which returns the `GActionFuture`:
[source,js]
[source,groovy]
--------------------------------------------------
def indexR = node.client.prepareIndex("test", "type1", "1").setSource({
test = "value"

View File

@ -13,7 +13,7 @@ within the cluster.
A Node based client is the simplest form to get a `GClient` to start
executing operations against elasticsearch.
[source,js]
[source,groovy]
--------------------------------------------------
import org.elasticsearch.groovy.client.GClient
import org.elasticsearch.groovy.node.GNode
@ -33,7 +33,7 @@ Since elasticsearch allows to configure it using JSON based settings,
the configuration itself can be done using a closure that represent the
JSON:
[source,js]
[source,groovy]
--------------------------------------------------
import org.elasticsearch.groovy.node.GNode
import org.elasticsearch.groovy.node.GNodeBuilder

View File

@ -6,7 +6,7 @@ The delete API is very similar to the
Java delete API, here is an
example:
[source,js]
[source,groovy]
--------------------------------------------------
def deleteF = node.client.delete {
index "test"

View File

@ -7,7 +7,7 @@ Java get API. The main benefit
of using groovy is handling the source content. It can be automatically
converted to a `Map` which means using Groovy to navigate it is simple:
[source,js]
[source,groovy]
--------------------------------------------------
def getF = node.client.get {
index "test"

View File

@ -7,7 +7,7 @@ Java index API. The Groovy
extension to it is the ability to provide the indexed source using a
closure. For example:
[source,js]
[source,groovy]
--------------------------------------------------
def indexR = client.index {
index "test"

View File

@ -7,7 +7,7 @@ Java search API. The Groovy
extension allows to provide the search source to execute as a `Closure`
including the query itself (similar to GORM criteria builder):
[source,js]
[source,groovy]
--------------------------------------------------
def search = node.client.search {
indices "test"
@ -19,7 +19,7 @@ def search = node.client.search {
}
}
search.response.hits.each {SearchHit hit ->
search.response.hits.each {SearchHit hit ->
println "Got hit $hit.id from $hit.index/$hit.type"
}
--------------------------------------------------
@ -27,13 +27,13 @@ search.response.hits.each {SearchHit hit ->
It can also be executed using the "Java API" while still using a closure
for the query:
[source,js]
[source,groovy]
--------------------------------------------------
def search = node.client.prepareSearch("test").setQuery({
term(test: "value")
}).gexecute();
search.response.hits.each {SearchHit hit ->
search.response.hits.each {SearchHit hit ->
println "Got hit $hit.id from $hit.index/$hit.type"
}
--------------------------------------------------
@ -48,7 +48,7 @@ The format of the search `Closure` follows the same JSON syntax as the
Term query where multiple values are provided (see
{ref}/query-dsl-terms-query.html[terms]):
[source,js]
[source,groovy]
--------------------------------------------------
def search = node.client.search {
indices "test"
@ -64,7 +64,7 @@ def search = node.client.search {
Query string (see
{ref}/query-dsl-query-string-query.html[query string]):
[source,js]
[source,groovy]
--------------------------------------------------
def search = node.client.search {
indices "test"
@ -82,7 +82,7 @@ def search = node.client.search {
Pagination (see
{ref}/search-request-from-size.html[from/size]):
[source,js]
[source,groovy]
--------------------------------------------------
def search = node.client.search {
indices "test"
@ -99,7 +99,7 @@ def search = node.client.search {
Sorting (see {ref}/search-request-sort.html[sort]):
[source,js]
[source,groovy]
--------------------------------------------------
def search = node.client.search {
indices "test"

View File

@ -17,7 +17,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install analysis-icu
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -32,7 +31,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove analysis-icu
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -14,7 +14,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install analysis-kuromoji
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -29,7 +28,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove analysis-kuromoji
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -15,7 +15,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install analysis-phonetic
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -30,7 +29,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove analysis-phonetic
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -20,7 +20,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install analysis-smartcn
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -35,7 +34,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove analysis-smartcn
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -17,7 +17,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install analysis-stempel
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -32,7 +31,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove analysis-stempel
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -17,7 +17,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install discovery-azure-classic
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -32,7 +31,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove discovery-azure-classic
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.
@ -359,7 +357,7 @@ ssh azure-elasticsearch-cluster.cloudapp.net
Once connected, install Elasticsearch:
[source,sh]
["source","sh",subs="attributes,callouts"]
----
# Install Latest Java version
# Read http://www.webupd8.org/2012/09/install-oracle-java-8-in-ubuntu-via-ppa.html for details
@ -372,36 +370,43 @@ sudo apt-get install oracle-java8-installer
# sudo apt-get install openjdk-8-jre-headless
# Download Elasticsearch
curl -s https://download.elasticsearch.org/elasticsearch/elasticsearch/elasticsearch-2.0.0.deb -o elasticsearch-2.0.0.deb
curl -s https://download.elasticsearch.org/elasticsearch/elasticsearch/elasticsearch-{version}.deb -o elasticsearch-{version}.deb
# Prepare Elasticsearch installation
sudo dpkg -i elasticsearch-2.0.0.deb
sudo dpkg -i elasticsearch-{version}.deb
----
Check that elasticsearch is running:
[source,sh]
[source,js]
----
curl http://localhost:9200/
GET /
----
// CONSOLE
This command should give you a JSON result:
[source,js]
----
["source","js",subs="attributes,callouts"]
--------------------------------------------
{
"status" : 200,
"name" : "Living Colossus",
"name" : "Cp8oag6",
"cluster_name" : "elasticsearch",
"version" : {
"number" : "2.0.0",
"build_hash" : "a46900e9c72c0a623d71b54016357d5f94c8ea32",
"build_timestamp" : "2014-02-12T16:18:34Z",
"number" : "{version}",
"build_hash" : "f27399d",
"build_date" : "2016-03-30T09:51:41.449Z",
"build_snapshot" : false,
"lucene_version" : "5.1"
"lucene_version" : "{lucene_version}"
},
"tagline" : "You Know, for Search"
}
----
--------------------------------------------
// TESTRESPONSE[s/"name" : "Cp8oag6",/"name" : "$body.name",/]
// TESTRESPONSE[s/"cluster_name" : "elasticsearch",/"cluster_name" : "$body.cluster_name",/]
// TESTRESPONSE[s/"build_hash" : "f27399d",/"build_hash" : "$body.version.build_hash",/]
// TESTRESPONSE[s/"build_date" : "2016-03-30T09:51:41.449Z",/"build_date" : $body.version.build_date,/]
// TESTRESPONSE[s/"build_snapshot" : false,/"build_snapshot" : $body.version.build_snapshot,/]
// So much s/// but at least we test that the layout is close to matching....
[[discovery-azure-classic-long-plugin]]
===== Install elasticsearch cloud azure plugin

View File

@ -15,7 +15,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install discovery-ec2
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -30,7 +29,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove discovery-ec2
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.
@ -231,6 +229,7 @@ Management Console. It should look similar to this.
"Version": "2012-10-17"
}
----
// NOTCONSOLE
[[discovery-ec2-filtering]]
===== Filtering by Tags

View File

@ -13,7 +13,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install discovery-gce
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -28,7 +27,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove discovery-gce
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -1,7 +1,9 @@
= Elasticsearch Plugins and Integrations
:ref: https://www.elastic.co/guide/en/elasticsearch/reference/master
:guide: https://www.elastic.co/guide
:ref: https://www.elastic.co/guide/en/elasticsearch/reference/master
:guide: https://www.elastic.co/guide
:version: 5.0.0-alpha5
:lucene_version: 6.2.0
[[intro]]
== Introduction to plugins
@ -62,5 +64,3 @@ include::integrations.asciidoc[]
include::authors.asciidoc[]
include::redirects.asciidoc[]

View File

@ -21,7 +21,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install ingest-attachment
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -36,7 +35,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove ingest-attachment
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -21,7 +21,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install ingest-geoip
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -36,7 +35,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove ingest-geoip
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -16,7 +16,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install ingest-user-agent
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -31,7 +30,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove ingest-user-agent
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -17,7 +17,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install lang-javascript
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -32,7 +31,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove lang-javascript
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.
@ -147,11 +145,10 @@ JavaScript:
First, save this file as `config/scripts/my_script.js` on every node
in the cluster:
[source,js]
[source,painless]
----
doc["num"].value * factor
----
// NOTCONSOLE
then use the script as follows:

View File

@ -16,7 +16,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install lang-python
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -31,7 +30,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove lang-python
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -19,7 +19,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install mapper-attachments
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -34,7 +33,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove mapper-attachments
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

View File

@ -15,7 +15,6 @@ This plugin can be installed using the plugin manager:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin install mapper-murmur3
----------------------------------------------------------------
// NOTCONSOLE
The plugin must be installed on every node in the cluster, and each node must
be restarted after installation.
@ -30,7 +29,6 @@ The plugin can be removed with the following command:
----------------------------------------------------------------
sudo bin/elasticsearch-plugin remove mapper-murmur3
----------------------------------------------------------------
// NOTCONSOLE
The node must be stopped before removing the plugin.

Some files were not shown because too many files have changed in this diff Show More