Merge branch 'master' into feature-suggest-refactoring

This commit is contained in:
Christoph Büscher 2016-02-29 14:06:30 +01:00
commit 379bd94c2a
37 changed files with 290 additions and 161 deletions

View File

@ -29,6 +29,7 @@ import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskId;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.TimeUnit;
/** /**
* Information about a currently running task. * Information about a currently running task.
@ -50,17 +51,24 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
private final String description; private final String description;
private final long startTime;
private final long runningTimeNanos;
private final Task.Status status; private final Task.Status status;
private final TaskId parentTaskId; private final TaskId parentTaskId;
public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status, TaskId parentTaskId) { public TaskInfo(DiscoveryNode node, long id, String type, String action, String description, Task.Status status, long startTime,
long runningTimeNanos, TaskId parentTaskId) {
this.node = node; this.node = node;
this.taskId = new TaskId(node.getId(), id); this.taskId = new TaskId(node.getId(), id);
this.type = type; this.type = type;
this.action = action; this.action = action;
this.description = description; this.description = description;
this.status = status; this.status = status;
this.startTime = startTime;
this.runningTimeNanos = runningTimeNanos;
this.parentTaskId = parentTaskId; this.parentTaskId = parentTaskId;
} }
@ -75,6 +83,8 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
} else { } else {
status = null; status = null;
} }
startTime = in.readLong();
runningTimeNanos = in.readLong();
parentTaskId = new TaskId(in); parentTaskId = new TaskId(in);
} }
@ -110,6 +120,23 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
return status; return status;
} }
/**
* Returns the task start time
*/
public long getStartTime() {
return startTime;
}
/**
* Returns the task running time
*/
public long getRunningTimeNanos() {
return runningTimeNanos;
}
/**
* Returns the parent task id
*/
public TaskId getParentTaskId() { public TaskId getParentTaskId() {
return parentTaskId; return parentTaskId;
} }
@ -132,6 +159,8 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
} else { } else {
out.writeBoolean(false); out.writeBoolean(false);
} }
out.writeLong(startTime);
out.writeLong(runningTimeNanos);
parentTaskId.writeTo(out); parentTaskId.writeTo(out);
} }
@ -147,6 +176,8 @@ public class TaskInfo implements Writeable<TaskInfo>, ToXContent {
if (description != null) { if (description != null) {
builder.field("description", description); builder.field("description", description);
} }
builder.dateValueField("start_time_in_millis", "start_time", startTime);
builder.timeValueField("running_time_in_nanos", "running_time", runningTimeNanos, TimeUnit.NANOSECONDS);
if (parentTaskId.isSet() == false) { if (parentTaskId.isSet() == false) {
builder.field("parent_task_id", parentTaskId.toString()); builder.field("parent_task_id", parentTaskId.toString());
} }

View File

@ -30,10 +30,12 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.TransportDeleteAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.AutoCreateIndex;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.update.TransportUpdateAction;
import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -42,12 +44,9 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndexAlreadyExistsException;
@ -197,10 +196,10 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
*/ */
public void executeBulk(final BulkRequest bulkRequest, final ActionListener<BulkResponse> listener) { public void executeBulk(final BulkRequest bulkRequest, final ActionListener<BulkResponse> listener) {
final long startTime = System.currentTimeMillis(); final long startTime = System.currentTimeMillis();
executeBulk(bulkRequest, startTime, listener, new AtomicArray<BulkItemResponse>(bulkRequest.requests.size())); executeBulk(bulkRequest, startTime, listener, new AtomicArray<>(bulkRequest.requests.size()));
} }
private final long buildTookInMillis(long startTime) { private long buildTookInMillis(long startTime) {
// protect ourselves against time going backwards // protect ourselves against time going backwards
return Math.max(1, System.currentTimeMillis() - startTime); return Math.max(1, System.currentTimeMillis() - startTime);
} }
@ -214,33 +213,53 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
MetaData metaData = clusterState.metaData(); MetaData metaData = clusterState.metaData();
for (int i = 0; i < bulkRequest.requests.size(); i++) { for (int i = 0; i < bulkRequest.requests.size(); i++) {
ActionRequest request = bulkRequest.requests.get(i); ActionRequest request = bulkRequest.requests.get(i);
if (request instanceof DocumentRequest) { //the request can only be null because we set it to null in the previous step, so it gets ignored
DocumentRequest req = (DocumentRequest) request; if (request == null) {
continue;
if (addFailureIfIndexIsUnavailable(req, bulkRequest, responses, i, concreteIndices, metaData)) { }
continue; DocumentRequest documentRequest = (DocumentRequest) request;
if (addFailureIfIndexIsUnavailable(documentRequest, bulkRequest, responses, i, concreteIndices, metaData)) {
continue;
}
String concreteIndex = concreteIndices.resolveIfAbsent(documentRequest);
if (request instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) request;
MappingMetaData mappingMd = null;
if (metaData.hasIndex(concreteIndex)) {
mappingMd = metaData.index(concreteIndex).mappingOrDefault(indexRequest.type());
}
try {
indexRequest.process(metaData, mappingMd, allowIdGeneration, concreteIndex);
} catch (ElasticsearchParseException | RoutingMissingException e) {
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(concreteIndex, indexRequest.type(), indexRequest.id(), e);
BulkItemResponse bulkItemResponse = new BulkItemResponse(i, "index", failure);
responses.set(i, bulkItemResponse);
// make sure the request gets never processed again
bulkRequest.requests.set(i, null);
}
} else if (request instanceof DeleteRequest) {
try {
TransportDeleteAction.resolveAndValidateRouting(metaData, concreteIndex, (DeleteRequest)request);
} catch(RoutingMissingException e) {
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(concreteIndex, documentRequest.type(), documentRequest.id(), e);
BulkItemResponse bulkItemResponse = new BulkItemResponse(i, "delete", failure);
responses.set(i, bulkItemResponse);
// make sure the request gets never processed again
bulkRequest.requests.set(i, null);
} }
String concreteIndex = concreteIndices.resolveIfAbsent(req); } else if (request instanceof UpdateRequest) {
if (request instanceof IndexRequest) { try {
IndexRequest indexRequest = (IndexRequest) request; TransportUpdateAction.resolveAndValidateRouting(metaData, concreteIndex, (UpdateRequest)request);
MappingMetaData mappingMd = null; } catch(RoutingMissingException e) {
if (metaData.hasIndex(concreteIndex)) { BulkItemResponse.Failure failure = new BulkItemResponse.Failure(concreteIndex, documentRequest.type(), documentRequest.id(), e);
mappingMd = metaData.index(concreteIndex).mappingOrDefault(indexRequest.type()); BulkItemResponse bulkItemResponse = new BulkItemResponse(i, "update", failure);
} responses.set(i, bulkItemResponse);
try { // make sure the request gets never processed again
indexRequest.process(metaData, mappingMd, allowIdGeneration, concreteIndex); bulkRequest.requests.set(i, null);
} catch (ElasticsearchParseException | RoutingMissingException e) {
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(concreteIndex, indexRequest.type(), indexRequest.id(), e);
BulkItemResponse bulkItemResponse = new BulkItemResponse(i, "index", failure);
responses.set(i, bulkItemResponse);
// make sure the request gets never processed again
bulkRequest.requests.set(i, null);
}
} else {
concreteIndices.resolveIfAbsent(req);
req.routing(clusterState.metaData().resolveIndexRouting(req.parent(), req.routing(), req.index()));
} }
} else {
throw new AssertionError("request type not supported: [" + request.getClass().getName() + "]");
} }
} }
@ -262,37 +281,16 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
} else if (request instanceof DeleteRequest) { } else if (request instanceof DeleteRequest) {
DeleteRequest deleteRequest = (DeleteRequest) request; DeleteRequest deleteRequest = (DeleteRequest) request;
String concreteIndex = concreteIndices.getConcreteIndex(deleteRequest.index()); String concreteIndex = concreteIndices.getConcreteIndex(deleteRequest.index());
MappingMetaData mappingMd = clusterState.metaData().index(concreteIndex).mappingOrDefault(deleteRequest.type()); ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, deleteRequest.type(), deleteRequest.id(), deleteRequest.routing()).shardId();
if (mappingMd != null && mappingMd.routing().required() && deleteRequest.routing() == null) { List<BulkItemRequest> list = requestsByShard.get(shardId);
// if routing is required, and no routing on the delete request, we need to broadcast it.... if (list == null) {
GroupShardsIterator groupShards = clusterService.operationRouting().broadcastDeleteShards(clusterState, concreteIndex); list = new ArrayList<>();
for (ShardIterator shardIt : groupShards) { requestsByShard.put(shardId, list);
List<BulkItemRequest> list = requestsByShard.get(shardIt.shardId());
if (list == null) {
list = new ArrayList<>();
requestsByShard.put(shardIt.shardId(), list);
}
list.add(new BulkItemRequest(i, deleteRequest));
}
} else {
ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, deleteRequest.type(), deleteRequest.id(), deleteRequest.routing()).shardId();
List<BulkItemRequest> list = requestsByShard.get(shardId);
if (list == null) {
list = new ArrayList<>();
requestsByShard.put(shardId, list);
}
list.add(new BulkItemRequest(i, request));
} }
list.add(new BulkItemRequest(i, request));
} else if (request instanceof UpdateRequest) { } else if (request instanceof UpdateRequest) {
UpdateRequest updateRequest = (UpdateRequest) request; UpdateRequest updateRequest = (UpdateRequest) request;
String concreteIndex = concreteIndices.getConcreteIndex(updateRequest.index()); String concreteIndex = concreteIndices.getConcreteIndex(updateRequest.index());
MappingMetaData mappingMd = clusterState.metaData().index(concreteIndex).mappingOrDefault(updateRequest.type());
if (mappingMd != null && mappingMd.routing().required() && updateRequest.routing() == null) {
BulkItemResponse.Failure failure = new BulkItemResponse.Failure(updateRequest.index(), updateRequest.type(),
updateRequest.id(), new IllegalArgumentException("routing is required for this item"));
responses.set(i, new BulkItemResponse(i, updateRequest.type(), failure));
continue;
}
ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, updateRequest.type(), updateRequest.id(), updateRequest.routing()).shardId(); ShardId shardId = clusterService.operationRouting().indexShards(clusterState, concreteIndex, updateRequest.type(), updateRequest.id(), updateRequest.routing()).shardId();
List<BulkItemRequest> list = requestsByShard.get(shardId); List<BulkItemRequest> list = requestsByShard.get(shardId);
if (list == null) { if (list == null) {

View File

@ -96,23 +96,27 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
@Override @Override
protected void resolveRequest(final MetaData metaData, String concreteIndex, DeleteRequest request) { protected void resolveRequest(final MetaData metaData, String concreteIndex, DeleteRequest request) {
resolveAndValidateRouting(metaData, concreteIndex, request);
ShardId shardId = clusterService.operationRouting().shardId(clusterService.state(), concreteIndex, request.id(), request.routing());
request.setShardId(shardId);
}
public static void resolveAndValidateRouting(final MetaData metaData, String concreteIndex, DeleteRequest request) {
request.routing(metaData.resolveIndexRouting(request.parent(), request.routing(), request.index())); request.routing(metaData.resolveIndexRouting(request.parent(), request.routing(), request.index()));
if (metaData.hasIndex(concreteIndex)) { if (metaData.hasIndex(concreteIndex)) {
// check if routing is required, if so, do a broadcast delete // check if routing is required, if so, throw error if routing wasn't specified
MappingMetaData mappingMd = metaData.index(concreteIndex).mappingOrDefault(request.type()); MappingMetaData mappingMd = metaData.index(concreteIndex).mappingOrDefault(request.type());
if (mappingMd != null && mappingMd.routing().required()) { if (mappingMd != null && mappingMd.routing().required()) {
if (request.routing() == null) { if (request.routing() == null) {
if (request.versionType() != VersionType.INTERNAL) { if (request.versionType() != VersionType.INTERNAL) {
// TODO: implement this feature // TODO: implement this feature
throw new IllegalArgumentException("routing value is required for deleting documents of type [" + request.type() throw new IllegalArgumentException("routing value is required for deleting documents of type [" + request.type()
+ "] while using version_type [" + request.versionType() + "]"); + "] while using version_type [" + request.versionType() + "]");
} }
throw new RoutingMissingException(concreteIndex, request.type(), request.id()); throw new RoutingMissingException(concreteIndex, request.type(), request.id());
} }
} }
} }
ShardId shardId = clusterService.operationRouting().shardId(clusterService.state(), concreteIndex, request.id(), request.routing());
request.setShardId(shardId);
} }
private void innerExecute(Task task, final DeleteRequest request, final ActionListener<DeleteResponse> listener) { private void innerExecute(Task task, final DeleteRequest request, final ActionListener<DeleteResponse> listener) {

View File

@ -35,10 +35,8 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.LoggerMessageFormat;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler; import org.elasticsearch.transport.BaseTransportResponseHandler;
@ -91,11 +89,11 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
protected ClusterBlockException checkRequestBlock(ClusterState state, Request request) { protected ClusterBlockException checkRequestBlock(ClusterState state, Request request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.concreteIndex()); return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.concreteIndex());
} }
/** /**
* Resolves the request. If the resolve means a different execution, then return false * Resolves the request. Throws an exception if the request cannot be resolved.
* here to indicate not to continue and execute this request.
*/ */
protected abstract boolean resolveRequest(ClusterState state, Request request, ActionListener<Response> listener); protected abstract void resolveRequest(ClusterState state, Request request);
protected boolean retryOnFailure(Throwable e) { protected boolean retryOnFailure(Throwable e) {
return false; return false;
@ -141,11 +139,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
} }
} }
request.concreteIndex(indexNameExpressionResolver.concreteSingleIndex(observer.observedState(), request)); request.concreteIndex(indexNameExpressionResolver.concreteSingleIndex(observer.observedState(), request));
// check if we need to execute, and if not, return resolveRequest(observer.observedState(), request);
if (!resolveRequest(observer.observedState(), request, listener)) {
listener.onFailure(new IllegalStateException(LoggerMessageFormat.format("[{}][{}] request {} could not be resolved",request.index, request.shardId, actionName)));
return;
}
blockException = checkRequestBlock(observer.observedState(), request); blockException = checkRequestBlock(observer.observedState(), request);
if (blockException != null) { if (blockException != null) {
if (blockException.retryable()) { if (blockException.retryable()) {

View File

@ -38,6 +38,7 @@ import org.elasticsearch.action.support.single.instance.TransportInstanceSingleO
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.PlainShardIterator; import org.elasticsearch.cluster.routing.PlainShardIterator;
import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
@ -99,13 +100,16 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
} }
@Override @Override
protected boolean resolveRequest(ClusterState state, UpdateRequest request, ActionListener<UpdateResponse> listener) { protected void resolveRequest(ClusterState state, UpdateRequest request) {
request.routing((state.metaData().resolveIndexRouting(request.parent(), request.routing(), request.index()))); resolveAndValidateRouting(state.metaData(), request.concreteIndex(), request);
}
public static void resolveAndValidateRouting(MetaData metaData, String concreteIndex, UpdateRequest request) {
request.routing((metaData.resolveIndexRouting(request.parent(), request.routing(), request.index())));
// Fail fast on the node that received the request, rather than failing when translating on the index or delete request. // Fail fast on the node that received the request, rather than failing when translating on the index or delete request.
if (request.routing() == null && state.getMetaData().routingRequired(request.concreteIndex(), request.type())) { if (request.routing() == null && metaData.routingRequired(concreteIndex, request.type())) {
throw new RoutingMissingException(request.concreteIndex(), request.type(), request.id()); throw new RoutingMissingException(concreteIndex, request.type(), request.id());
} }
return true;
} }
@Override @Override

View File

@ -22,6 +22,7 @@ package org.elasticsearch.bootstrap;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper; import org.apache.lucene.util.StringHelper;
import org.elasticsearch.Build;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.common.PidFile; import org.elasticsearch.common.PidFile;
@ -193,21 +194,6 @@ final class Bootstrap {
node = new Node(nodeSettings); node = new Node(nodeSettings);
} }
@SuppressForbidden(reason = "Exception#printStackTrace()")
private static void setupLogging(Settings settings) {
try {
Class.forName("org.apache.log4j.Logger");
LogConfigurator.configure(settings, true);
} catch (ClassNotFoundException e) {
// no log4j
} catch (NoClassDefFoundError e) {
// no log4j
} catch (Exception e) {
sysError("Failed to configure logging...", false);
e.printStackTrace();
}
}
private static Environment initialSettings(boolean foreground) { private static Environment initialSettings(boolean foreground) {
Terminal terminal = foreground ? Terminal.DEFAULT : null; Terminal terminal = foreground ? Terminal.DEFAULT : null;
return InternalSettingsPreparer.prepareEnvironment(EMPTY_SETTINGS, terminal); return InternalSettingsPreparer.prepareEnvironment(EMPTY_SETTINGS, terminal);
@ -254,7 +240,7 @@ final class Bootstrap {
Environment environment = initialSettings(foreground); Environment environment = initialSettings(foreground);
Settings settings = environment.settings(); Settings settings = environment.settings();
setupLogging(settings); LogConfigurator.configure(settings, true);
checkForCustomConfFile(); checkForCustomConfFile();
if (environment.pidFile() != null) { if (environment.pidFile() != null) {
@ -373,6 +359,9 @@ final class Bootstrap {
))); )));
private static boolean enforceLimits(Settings settings) { private static boolean enforceLimits(Settings settings) {
if (Build.CURRENT.isSnapshot()) {
return false;
}
for (Setting setting : ENFORCE_SETTINGS) { for (Setting setting : ENFORCE_SETTINGS) {
if (setting.exists(settings)) { if (setting.exists(settings)) {
return true; return true;

View File

@ -29,7 +29,6 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.math.MathUtils; import org.elasticsearch.common.math.MathUtils;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException; import org.elasticsearch.index.shard.ShardNotFoundException;
@ -67,10 +66,6 @@ public class OperationRouting extends AbstractComponent {
return preferenceActiveShardIterator(indexShard, clusterState.nodes().localNodeId(), clusterState.nodes(), preference); return preferenceActiveShardIterator(indexShard, clusterState.nodes().localNodeId(), clusterState.nodes(), preference);
} }
public GroupShardsIterator broadcastDeleteShards(ClusterState clusterState, String index) {
return indexRoutingTable(clusterState, index).groupByShardsIt();
}
public int searchShardsCount(ClusterState clusterState, String[] concreteIndices, @Nullable Map<String, Set<String>> routing) { public int searchShardsCount(ClusterState clusterState, String[] concreteIndices, @Nullable Map<String, Set<String>> routing) {
final Set<IndexShardRoutingTable> shards = computeTargetedShards(clusterState, concreteIndices, routing); final Set<IndexShardRoutingTable> shards = computeTargetedShards(clusterState, concreteIndices, routing);
return shards.size(); return shards.size();

View File

@ -48,7 +48,7 @@ import java.util.Date;
import java.util.HashMap; import java.util.HashMap;
import java.util.Locale; import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.function.BiConsumer; import java.util.concurrent.TimeUnit;
/** /**
* *
@ -961,6 +961,23 @@ public final class XContentBuilder implements BytesStream, Releasable {
return this; return this;
} }
public XContentBuilder timeValueField(String rawFieldName, String readableFieldName, long rawTime, TimeUnit timeUnit) throws
IOException {
if (humanReadable) {
field(readableFieldName, new TimeValue(rawTime, timeUnit).toString());
}
field(rawFieldName, rawTime);
return this;
}
public XContentBuilder dateValueField(String rawFieldName, String readableFieldName, long rawTimestamp) throws IOException {
if (humanReadable) {
field(readableFieldName, defaultDatePrinter.print(rawTimestamp));
}
field(rawFieldName, rawTimestamp);
return this;
}
public XContentBuilder byteSizeField(XContentBuilderString rawFieldName, XContentBuilderString readableFieldName, ByteSizeValue byteSizeValue) throws IOException { public XContentBuilder byteSizeField(XContentBuilderString rawFieldName, XContentBuilderString readableFieldName, ByteSizeValue byteSizeValue) throws IOException {
if (humanReadable) { if (humanReadable) {
field(readableFieldName, byteSizeValue.toString()); field(readableFieldName, byteSizeValue.toString());

View File

@ -313,7 +313,7 @@ public abstract class MetaDataStateFormat<T> {
} }
return state; return state;
} catch (Throwable e) { } catch (Throwable e) {
exceptions.add(e); exceptions.add(new IOException("failed to read " + pathAndStateId.toString(), e));
logger.debug("{}: failed to read [{}], ignoring...", e, pathAndStateId.file.toAbsolutePath(), prefix); logger.debug("{}: failed to read [{}], ignoring...", e, pathAndStateId.file.toAbsolutePath(), prefix);
} }
} }

View File

@ -40,16 +40,26 @@ public class Task {
private final TaskId parentTask; private final TaskId parentTask;
private final long startTime;
private final long startTimeNanos;
public Task(long id, String type, String action, String description) { public Task(long id, String type, String action, String description) {
this(id, type, action, description, TaskId.EMPTY_TASK_ID); this(id, type, action, description, TaskId.EMPTY_TASK_ID);
} }
public Task(long id, String type, String action, String description, TaskId parentTask) { public Task(long id, String type, String action, String description, TaskId parentTask) {
this(id, type, action, description, parentTask, System.currentTimeMillis(), System.nanoTime());
}
public Task(long id, String type, String action, String description, TaskId parentTask, long startTime, long startTimeNanos) {
this.id = id; this.id = id;
this.type = type; this.type = type;
this.action = action; this.action = action;
this.description = description; this.description = description;
this.parentTask = parentTask; this.parentTask = parentTask;
this.startTime = startTime;
this.startTimeNanos = startTimeNanos;
} }
/** /**
@ -69,7 +79,8 @@ public class Task {
description = getDescription(); description = getDescription();
status = getStatus(); status = getStatus();
} }
return new TaskInfo(node, getId(), getType(), getAction(), description, status, parentTask); return new TaskInfo(node, getId(), getType(), getAction(), description, status, startTime, System.nanoTime() - startTimeNanos,
parentTask);
} }
/** /**
@ -100,6 +111,13 @@ public class Task {
return description; return description;
} }
/**
* Returns the task start time
*/
public long getStartTime() {
return startTime;
}
/** /**
* Returns id of the parent task or NO_PARENT_ID if the task doesn't have any parent tasks * Returns id of the parent task or NO_PARENT_ID if the task doesn't have any parent tasks
*/ */

View File

@ -25,6 +25,9 @@ grant {
// groovy IndyInterface bootstrap requires this property for indy logging // groovy IndyInterface bootstrap requires this property for indy logging
permission java.util.PropertyPermission "groovy.indy.logging", "read"; permission java.util.PropertyPermission "groovy.indy.logging", "read";
// groovy requires this to enable workaround for certain JVMs (https://github.com/apache/groovy/pull/137)
permission java.util.PropertyPermission "java.vm.name", "read";
permission java.util.PropertyPermission "groovy.use.classvalue", "read";
// needed by Rhino engine exception handling // needed by Rhino engine exception handling
permission java.util.PropertyPermission "rhino.stack.style", "read"; permission java.util.PropertyPermission "rhino.stack.style", "read";

View File

@ -63,6 +63,7 @@ import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.action.support.PlainActionFuture.newFuture; import static org.elasticsearch.action.support.PlainActionFuture.newFuture;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.not;
public class TransportTasksActionTests extends TaskManagerTestCase { public class TransportTasksActionTests extends TaskManagerTestCase {
@ -461,10 +462,12 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
} }
public void testTasksDescriptions() throws Exception { public void testTasksDescriptions() throws Exception {
long minimalStartTime = System.currentTimeMillis();
setupTestNodes(Settings.EMPTY); setupTestNodes(Settings.EMPTY);
connectNodes(testNodes); connectNodes(testNodes);
CountDownLatch checkLatch = new CountDownLatch(1); CountDownLatch checkLatch = new CountDownLatch(1);
ActionFuture<NodesResponse> future = startBlockingTestNodesAction(checkLatch); ActionFuture<NodesResponse> future = startBlockingTestNodesAction(checkLatch);
long maximumStartTimeNanos = System.nanoTime();
// Check task counts using transport with filtering // Check task counts using transport with filtering
TestNode testNode = testNodes[randomIntBetween(0, testNodes.length - 1)]; TestNode testNode = testNodes[randomIntBetween(0, testNodes.length - 1)];
@ -478,12 +481,15 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
} }
// Check task counts using transport with detailed description // Check task counts using transport with detailed description
long minimalDurationNanos = System.nanoTime() - maximumStartTimeNanos;
listTasksRequest.detailed(true); // same request only with detailed description listTasksRequest.detailed(true); // same request only with detailed description
response = testNode.transportListTasksAction.execute(listTasksRequest).get(); response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getPerNodeTasks().size()); assertEquals(testNodes.length, response.getPerNodeTasks().size());
for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) { for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
assertEquals(1, entry.getValue().size()); assertEquals(1, entry.getValue().size());
assertEquals("CancellableNodeRequest[Test Request, true]", entry.getValue().get(0).getDescription()); assertEquals("CancellableNodeRequest[Test Request, true]", entry.getValue().get(0).getDescription());
assertThat(entry.getValue().get(0).getStartTime(), greaterThanOrEqualTo(minimalStartTime));
assertThat(entry.getValue().get(0).getRunningTimeNanos(), greaterThanOrEqualTo(minimalDurationNanos));
} }
// Release all tasks and wait for response // Release all tasks and wait for response

View File

@ -108,8 +108,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
} }
@Override @Override
protected boolean resolveRequest(ClusterState state, Request request, ActionListener<Response> listener) { protected void resolveRequest(ClusterState state, Request request) {
return true;
} }
@Override @Override
@ -230,7 +229,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
listener.get(); listener.get();
} }
public void testSuccessAfterRetryWithExcpetionFromTransport() throws Exception { public void testSuccessAfterRetryWithExceptionFromTransport() throws Exception {
Request request = new Request().index("test"); Request request = new Request().index("test");
request.shardId = 0; request.shardId = 0;
PlainActionFuture<Response> listener = new PlainActionFuture<>(); PlainActionFuture<Response> listener = new PlainActionFuture<>();
@ -290,13 +289,13 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
Settings.EMPTY, Settings.EMPTY,
"indices:admin/test_unresolvable", "indices:admin/test_unresolvable",
transportService, transportService,
new ActionFilters(new HashSet<ActionFilter>()), new ActionFilters(new HashSet<>()),
new MyResolver(), new MyResolver(),
Request::new Request::new
) { ) {
@Override @Override
protected boolean resolveRequest(ClusterState state, Request request, ActionListener<Response> listener) { protected void resolveRequest(ClusterState state, Request request) {
return false; throw new IllegalStateException("request cannot be resolved");
} }
}; };
Request request = new Request().index("test"); Request request = new Request().index("test");

View File

@ -21,7 +21,6 @@ package org.elasticsearch.bootstrap;
import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.monitor.os.OsProbe;
import org.elasticsearch.monitor.process.ProcessProbe; import org.elasticsearch.monitor.process.ProcessProbe;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -34,6 +33,7 @@ public class BootstrapSettingsTests extends ESTestCase {
assertTrue(BootstrapSettings.CTRLHANDLER_SETTING.get(Settings.EMPTY)); assertTrue(BootstrapSettings.CTRLHANDLER_SETTING.get(Settings.EMPTY));
} }
@AwaitsFix(bugUrl = "this feature is disabled for snapshot builds, for now - see #16835")
public void testEnforceMaxFileDescriptorLimits() { public void testEnforceMaxFileDescriptorLimits() {
// nothing should happen since we are in OOB mode // nothing should happen since we are in OOB mode
Bootstrap.enforceOrLogLimits(Settings.EMPTY); Bootstrap.enforceOrLogLimits(Settings.EMPTY);

View File

@ -27,6 +27,7 @@ import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.SimpleFSDirectory; import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
@ -59,7 +60,6 @@ import java.util.Set;
import java.util.stream.StreamSupport; import java.util.stream.StreamSupport;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.notNullValue;
@ -377,7 +377,7 @@ public class MetaDataStateFormatTests extends ESTestCase {
format.loadLatestState(logger, dirList.toArray(new Path[0])); format.loadLatestState(logger, dirList.toArray(new Path[0]));
fail("latest version can not be read"); fail("latest version can not be read");
} catch (ElasticsearchException ex) { } catch (ElasticsearchException ex) {
assertThat(ex.getCause(), instanceOf(CorruptStateException.class)); assertThat(ExceptionsHelper.unwrap(ex, CorruptStateException.class), notNullValue());
} }
} }

View File

@ -20,24 +20,26 @@
package org.elasticsearch.routing; package org.elasticsearch.routing;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.explain.ExplainResponse; import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetRequest;
import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.get.MultiGetResponse;
import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse;
import org.elasticsearch.action.termvectors.TermVectorsRequest; import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.client.Requests; import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.nullValue;
@ -156,8 +158,7 @@ public class SimpleRoutingIT extends ESIntegTestCase {
} }
} }
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/16645") public void testRequiredRoutingCrudApis() throws Exception {
public void testRequiredRoutingMapping() throws Exception {
client().admin().indices().prepareCreate("test").addAlias(new Alias("alias")) client().admin().indices().prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject()) .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject())
.execute().actionGet(); .execute().actionGet();
@ -199,13 +200,31 @@ public class SimpleRoutingIT extends ESIntegTestCase {
assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true));
} }
logger.info("--> indexing with id [1], and routing [0]"); try {
client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); client().prepareUpdate(indexOrAlias(), "type1", "1").setDoc("field", "value2").execute().actionGet();
logger.info("--> verifying get with no routing, should not find anything"); fail("update with missing routing when routing is required should fail");
} catch(ElasticsearchException e) {
assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class));
}
logger.info("--> bulk deleting with no routing, should broadcast the delete since _routing is required"); client().prepareUpdate(indexOrAlias(), "type1", "1").setRouting("0").setDoc("field", "value2").execute().actionGet();
client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1")).execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet(); client().admin().indices().prepareRefresh().execute().actionGet();
for (int i = 0; i < 5; i++) {
try {
client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists();
fail();
} catch (RoutingMissingException e) {
assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
assertThat(e.getMessage(), equalTo("routing is required for [test]/[type1]/[1]"));
}
GetResponse getResponse = client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet();
assertThat(getResponse.isExists(), equalTo(true));
assertThat(getResponse.getSourceAsMap().get("field"), equalTo("value2"));
}
client().prepareDelete(indexOrAlias(), "type1", "1").setRouting("0").setRefresh(true).execute().actionGet();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
try { try {
client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists(); client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists();
@ -227,28 +246,72 @@ public class SimpleRoutingIT extends ESIntegTestCase {
.execute().actionGet(); .execute().actionGet();
ensureGreen(); ensureGreen();
logger.info("--> indexing with id [1], and routing [0]"); {
client().prepareBulk().add( BulkResponse bulkResponse = client().prepareBulk().add(Requests.indexRequest(indexOrAlias()).type("type1").id("1")
client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1")).execute().actionGet(); .source("field", "value")).execute().actionGet();
client().admin().indices().prepareRefresh().execute().actionGet(); assertThat(bulkResponse.getItems().length, equalTo(1));
assertThat(bulkResponse.hasFailures(), equalTo(true));
logger.info("--> verifying get with no routing, should fail"); for (BulkItemResponse bulkItemResponse : bulkResponse) {
for (int i = 0; i < 5; i++) { assertThat(bulkItemResponse.isFailed(), equalTo(true));
try { assertThat(bulkItemResponse.getOpType(), equalTo("index"));
client().prepareGet(indexOrAlias(), "type1", "1").execute().actionGet().isExists(); assertThat(bulkItemResponse.getFailure().getStatus(), equalTo(RestStatus.BAD_REQUEST));
fail(); assertThat(bulkItemResponse.getFailure().getCause(), instanceOf(RoutingMissingException.class));
} catch (RoutingMissingException e) { assertThat(bulkItemResponse.getFailureMessage(), containsString("routing is required for [test]/[type1]/[1]"));
assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST));
assertThat(e.getMessage(), equalTo("routing is required for [test]/[type1]/[1]"));
} }
} }
logger.info("--> verifying get with routing, should find");
for (int i = 0; i < 5; i++) { {
assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); BulkResponse bulkResponse = client().prepareBulk().add(Requests.indexRequest(indexOrAlias()).type("type1").id("1").routing("0")
.source("field", "value")).execute().actionGet();
assertThat(bulkResponse.hasFailures(), equalTo(false));
}
{
BulkResponse bulkResponse = client().prepareBulk().add(new UpdateRequest(indexOrAlias(), "type1", "1").doc("field", "value2"))
.execute().actionGet();
assertThat(bulkResponse.getItems().length, equalTo(1));
assertThat(bulkResponse.hasFailures(), equalTo(true));
for (BulkItemResponse bulkItemResponse : bulkResponse) {
assertThat(bulkItemResponse.isFailed(), equalTo(true));
assertThat(bulkItemResponse.getOpType(), equalTo("update"));
assertThat(bulkItemResponse.getFailure().getStatus(), equalTo(RestStatus.BAD_REQUEST));
assertThat(bulkItemResponse.getFailure().getCause(), instanceOf(RoutingMissingException.class));
assertThat(bulkItemResponse.getFailureMessage(), containsString("routing is required for [test]/[type1]/[1]"));
}
}
{
BulkResponse bulkResponse = client().prepareBulk().add(new UpdateRequest(indexOrAlias(), "type1", "1").doc("field", "value2")
.routing("0")).execute().actionGet();
assertThat(bulkResponse.hasFailures(), equalTo(false));
}
{
BulkResponse bulkResponse = client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1"))
.execute().actionGet();
assertThat(bulkResponse.getItems().length, equalTo(1));
assertThat(bulkResponse.hasFailures(), equalTo(true));
for (BulkItemResponse bulkItemResponse : bulkResponse) {
assertThat(bulkItemResponse.isFailed(), equalTo(true));
assertThat(bulkItemResponse.getOpType(), equalTo("delete"));
assertThat(bulkItemResponse.getFailure().getStatus(), equalTo(RestStatus.BAD_REQUEST));
assertThat(bulkItemResponse.getFailure().getCause(), instanceOf(RoutingMissingException.class));
assertThat(bulkItemResponse.getFailureMessage(), containsString("routing is required for [test]/[type1]/[1]"));
}
}
{
BulkResponse bulkResponse = client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1")
.routing("0")).execute().actionGet();
assertThat(bulkResponse.getItems().length, equalTo(1));
assertThat(bulkResponse.hasFailures(), equalTo(false));
} }
} }
public void testRequiredRoutingMapping_variousAPIs() throws Exception { public void testRequiredRoutingMappingVariousAPIs() throws Exception {
client().admin().indices().prepareCreate("test").addAlias(new Alias("alias")) client().admin().indices().prepareCreate("test").addAlias(new Alias("alias"))
.addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject()) .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject())
.execute().actionGet(); .execute().actionGet();

View File

@ -167,8 +167,6 @@ Client client = TransportClient.builder().settings(settings).build();
//Add transport addresses and do something with the client... //Add transport addresses and do something with the client...
-------------------------------------------------- --------------------------------------------------
Or using `elasticsearch.yml` file as shown in <<node-client>>
The client allows sniffing the rest of the cluster, which adds data nodes The client allows sniffing the rest of the cluster, which adds data nodes
into its list of machines to use. In this case, note that the IP addresses into its list of machines to use. In this case, note that the IP addresses
used will be the ones that the other nodes were started with (the used will be the ones that the other nodes were started with (the

View File

@ -205,9 +205,9 @@ as valid network host settings:
|`_ec2:privateDns_` |The private host of the machine. |`_ec2:privateDns_` |The private host of the machine.
|`_ec2:publicIpv4_` |The public IP address (ipv4) of the machine. |`_ec2:publicIpv4_` |The public IP address (ipv4) of the machine.
|`_ec2:publicDns_` |The public host of the machine. |`_ec2:publicDns_` |The public host of the machine.
|`_ec2:privateIp_` |equivalent to _ec2:privateIpv4_. |`_ec2:privateIp_` |equivalent to `_ec2:privateIpv4_`.
|`_ec2:publicIp_` |equivalent to _ec2:publicIpv4_. |`_ec2:publicIp_` |equivalent to `_ec2:publicIpv4_`.
|`_ec2_` |equivalent to _ec2:privateIpv4_. |`_ec2_` |equivalent to `_ec2:privateIpv4_`.
|================================================================== |==================================================================
[[discovery-ec2-permissions]] [[discovery-ec2-permissions]]

View File

@ -141,10 +141,10 @@ GET test/_search
=== File scripts === File scripts
You can save your scripts to a file in the `config/scripts/` directory on You can save your scripts to a file in the `config/scripts/` directory on
every node. The `.python` file suffix identifies the script as containing every node. The `.py` file suffix identifies the script as containing
Python: Python:
First, save this file as `config/scripts/my_script.python` on every node First, save this file as `config/scripts/my_script.py` on every node
in the cluster: in the cluster:
[source,python] [source,python]
@ -188,5 +188,5 @@ GET test/_search
---- ----
// AUTOSENSE // AUTOSENSE
<1> The function score query retrieves the script with filename `my_script.python`. <1> The function score query retrieves the script with filename `my_script.py`.

View File

@ -8,7 +8,7 @@ Transport plugins offer alternatives to HTTP.
The core transport plugins are: The core transport plugins are:
https://github.com/elasticsearch/elasticsearch-transport-wares::[Servlet transport]:: https://github.com/elastic/elasticsearch-transport-wares[Servlet transport]::
Use the REST interface over servlets. Use the REST interface over servlets.

View File

@ -76,7 +76,7 @@ A `serial_diff` aggregation looks like this in isolation:
}, },
"thirtieth_difference": { "thirtieth_difference": {
"serial_diff": { <3> "serial_diff": { <3>
"buckets_path": "lemmings", "buckets_path": "the_sum",
"lag" : 30 "lag" : 30
} }
} }

View File

@ -9,7 +9,7 @@ of documents that contain the term.
Sorting, aggregations, and access to field values in scripts requires a Sorting, aggregations, and access to field values in scripts requires a
different data access pattern. Instead of lookup up the term and finding different data access pattern. Instead of lookup up the term and finding
documents, we need to be able to look up the document and find the terms that documents, we need to be able to look up the document and find the terms that
is has in a field. it has in a field.
Most fields can use index-time, on-disk <<doc-values,`doc_values`>> to support Most fields can use index-time, on-disk <<doc-values,`doc_values`>> to support
this type of data access pattern, but `analyzed` string fields do not support this type of data access pattern, but `analyzed` string fields do not support

View File

@ -421,6 +421,17 @@ be highlighted regardless of whether the query matched specifically on them.
The default behaviour is `true`, meaning that only fields that hold a query The default behaviour is `true`, meaning that only fields that hold a query
match will be highlighted. match will be highlighted.
[source,js]
--------------------------------------------------
{
"query" : {...},
"highlight" : {
"require_field_match": false
"fields" : {...}
}
}
--------------------------------------------------
[[boundary-characters]] [[boundary-characters]]
==== Boundary Characters ==== Boundary Characters

View File

@ -23,7 +23,7 @@ esplugin {
} }
dependencies { dependencies {
compile 'org.codehaus.groovy:groovy:2.4.4:indy' compile 'org.codehaus.groovy:groovy:2.4.6:indy'
} }
integTest { integTest {

View File

@ -1 +0,0 @@
139af316ac35534120c53f05393ce46d60d6da48

View File

@ -0,0 +1 @@
af78e80fab591a6dcf2d6ccb8bf34d1e888291be

View File

@ -1,5 +1,5 @@
Apache Commons CLI Apache Groovy
Copyright 2001-2009 The Apache Software Foundation Copyright 2003-2016 The Apache Software Foundation
This product includes software developed by This product includes software developed at
The Apache Software Foundation (http://www.apache.org/). The Apache Software Foundation (http://www.apache.org/).

View File

@ -25,7 +25,6 @@ grant {
// needed by groovy engine // needed by groovy engine
permission java.lang.RuntimePermission "accessDeclaredMembers"; permission java.lang.RuntimePermission "accessDeclaredMembers";
permission java.lang.RuntimePermission "accessClassInPackage.sun.reflect"; permission java.lang.RuntimePermission "accessClassInPackage.sun.reflect";
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
// needed by GroovyScriptEngineService to close its classloader (why?) // needed by GroovyScriptEngineService to close its classloader (why?)
permission java.lang.RuntimePermission "closeClassLoader"; permission java.lang.RuntimePermission "closeClassLoader";
// Allow executing groovy scripts with codesource of /untrusted // Allow executing groovy scripts with codesource of /untrusted

View File

@ -23,11 +23,11 @@ esplugin {
} }
dependencies { dependencies {
compile ('com.maxmind.geoip2:geoip2:2.4.0') compile ('com.maxmind.geoip2:geoip2:2.6.0')
// geoip2 dependencies: // geoip2 dependencies:
compile('com.fasterxml.jackson.core:jackson-annotations:2.5.0') compile('com.fasterxml.jackson.core:jackson-annotations:2.7.1')
compile('com.fasterxml.jackson.core:jackson-databind:2.5.3') compile('com.fasterxml.jackson.core:jackson-databind:2.7.1')
compile('com.maxmind.db:maxmind-db:1.0.1') compile('com.maxmind.db:maxmind-db:1.2.0')
testCompile 'org.elasticsearch:geolite2-databases:20151029' testCompile 'org.elasticsearch:geolite2-databases:20151029'
} }
@ -60,4 +60,4 @@ thirdPartyAudit.excludes = [
'com.google.api.client.http.HttpResponseException', 'com.google.api.client.http.HttpResponseException',
'com.google.api.client.http.javanet.NetHttpTransport', 'com.google.api.client.http.javanet.NetHttpTransport',
'com.google.api.client.http.javanet.NetHttpTransport', 'com.google.api.client.http.javanet.NetHttpTransport',
] ]

View File

@ -1 +0,0 @@
ad40667ae87138e0aed075d2c15884497fa64acc

View File

@ -0,0 +1 @@
2574c8b878f1cd39709559f1b96f1b5f0cdd69d3

View File

@ -1 +0,0 @@
a2a55a3375bc1cef830ca426d68d2ea22961190e

View File

@ -0,0 +1 @@
8b93f301823b79033fcbe873779b3d84f9730fc1

View File

@ -1 +0,0 @@
c37875ff66127d93e5f672708cb2dcc14c8232ab

View File

@ -0,0 +1 @@
14d88822bca655de7aa6ed3e4c498d115505710a

View File

@ -1 +0,0 @@
305429b84dbcd1cc3d393686f412cdcaec9cdbe6

View File

@ -0,0 +1 @@
b842823f24555f5d26608fef8122898365b3cd63