Merge remote-tracking branch 'upstream/feature-suggest-refactoring' into suggester-wiring-refactoring

This commit is contained in:
Ali Beyad 2016-03-10 09:56:08 -05:00
commit 807d085218
153 changed files with 1707 additions and 849 deletions

View File

@ -53,12 +53,18 @@ public class CancelTasksRequest extends BaseTasksRequest<CancelTasksRequest> {
return super.match(task) && task instanceof CancellableTask;
}
public CancelTasksRequest reason(String reason) {
/**
* Set the reason for canceling the task.
*/
public CancelTasksRequest setReason(String reason) {
this.reason = reason;
return this;
}
public String reason() {
/**
* The reason for canceling the task.
*/
public String getReason() {
return reason;
}
}

View File

@ -84,21 +84,21 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
}
protected void processTasks(CancelTasksRequest request, Consumer<CancellableTask> operation) {
if (request.taskId().isSet() == false) {
if (request.getTaskId().isSet() == false) {
// we are only checking one task, we can optimize it
CancellableTask task = taskManager.getCancellableTask(request.taskId().getId());
CancellableTask task = taskManager.getCancellableTask(request.getTaskId().getId());
if (task != null) {
if (request.match(task)) {
operation.accept(task);
} else {
throw new IllegalArgumentException("task [" + request.taskId() + "] doesn't support this operation");
throw new IllegalArgumentException("task [" + request.getTaskId() + "] doesn't support this operation");
}
} else {
if (taskManager.getTask(request.taskId().getId()) != null) {
if (taskManager.getTask(request.getTaskId().getId()) != null) {
// The task exists, but doesn't support cancellation
throw new IllegalArgumentException("task [" + request.taskId() + "] doesn't support cancellation");
throw new IllegalArgumentException("task [" + request.getTaskId() + "] doesn't support cancellation");
} else {
throw new ResourceNotFoundException("task [{}] doesn't support cancellation", request.taskId());
throw new ResourceNotFoundException("task [{}] doesn't support cancellation", request.getTaskId());
}
}
} else {
@ -113,14 +113,14 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
@Override
protected synchronized TaskInfo taskOperation(CancelTasksRequest request, CancellableTask cancellableTask) {
final BanLock banLock = new BanLock(nodes -> removeBanOnNodes(cancellableTask, nodes));
Set<String> childNodes = taskManager.cancel(cancellableTask, request.reason(), banLock::onTaskFinished);
Set<String> childNodes = taskManager.cancel(cancellableTask, request.getReason(), banLock::onTaskFinished);
if (childNodes != null) {
if (childNodes.isEmpty()) {
logger.trace("cancelling task {} with no children", cancellableTask.getId());
return cancellableTask.taskInfo(clusterService.localNode(), false);
} else {
logger.trace("cancelling task {} with children on nodes [{}]", cancellableTask.getId(), childNodes);
setBanOnNodes(request.reason(), cancellableTask, childNodes, banLock);
setBanOnNodes(request.getReason(), cancellableTask, childNodes, banLock);
return cancellableTask.taskInfo(clusterService.localNode(), false);
}
} else {

View File

@ -31,31 +31,49 @@ import java.io.IOException;
public class ListTasksRequest extends BaseTasksRequest<ListTasksRequest> {
private boolean detailed = false;
private boolean waitForCompletion = false;
/**
* Should the detailed task information be returned.
*/
public boolean detailed() {
public boolean getDetailed() {
return this.detailed;
}
/**
* Should the detailed task information be returned.
*/
public ListTasksRequest detailed(boolean detailed) {
public ListTasksRequest setDetailed(boolean detailed) {
this.detailed = detailed;
return this;
}
/**
* Should this request wait for all found tasks to complete?
*/
public boolean getWaitForCompletion() {
return waitForCompletion;
}
/**
* Should this request wait for all found tasks to complete?
*/
public ListTasksRequest setWaitForCompletion(boolean waitForCompletion) {
this.waitForCompletion = waitForCompletion;
return this;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
detailed = in.readBoolean();
waitForCompletion = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(detailed);
out.writeBoolean(waitForCompletion);
}
}

View File

@ -35,7 +35,15 @@ public class ListTasksRequestBuilder extends TasksRequestBuilder<ListTasksReques
* Should detailed task information be returned.
*/
public ListTasksRequestBuilder setDetailed(boolean detailed) {
request.detailed(detailed);
request.setDetailed(detailed);
return this;
}
/**
* Should this request wait for all found tasks to complete?
*/
public final ListTasksRequestBuilder setWaitForCompletion(boolean waitForCompletion) {
request.setWaitForCompletion(waitForCompletion);
return this;
}
}

View File

@ -19,6 +19,8 @@
package org.elasticsearch.action.admin.cluster.node.tasks.list;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.support.ActionFilters;
@ -29,18 +31,24 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.function.Consumer;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
/**
*
*/
public class TransportListTasksAction extends TransportTasksAction<Task, ListTasksRequest, ListTasksResponse, TaskInfo> {
private static final TimeValue WAIT_FOR_COMPLETION_POLL = timeValueMillis(100);
private static final TimeValue DEFAULT_WAIT_FOR_COMPLETION_TIMEOUT = timeValueSeconds(30);
@Inject
public TransportListTasksAction(Settings settings, ClusterName clusterName, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
@ -59,7 +67,34 @@ public class TransportListTasksAction extends TransportTasksAction<Task, ListTas
@Override
protected TaskInfo taskOperation(ListTasksRequest request, Task task) {
return task.taskInfo(clusterService.localNode(), request.detailed());
return task.taskInfo(clusterService.localNode(), request.getDetailed());
}
@Override
protected void processTasks(ListTasksRequest request, Consumer<Task> operation) {
if (false == request.getWaitForCompletion()) {
super.processTasks(request, operation);
return;
}
// If we should wait for completion then we have to intercept every found task and wait for it to leave the manager.
TimeValue timeout = request.getTimeout();
if (timeout == null) {
timeout = DEFAULT_WAIT_FOR_COMPLETION_TIMEOUT;
}
long timeoutTime = System.nanoTime() + timeout.nanos();
super.processTasks(request, operation.andThen((Task t) -> {
while (System.nanoTime() - timeoutTime < 0) {
if (taskManager.getTask(t.getId()) == null) {
return;
}
try {
Thread.sleep(WAIT_FOR_COMPLETION_POLL.millis());
} catch (InterruptedException e) {
throw new ElasticsearchException("Interrupted waiting for completion of [{}]", e, t);
}
}
throw new ElasticsearchTimeoutException("Timed out waiting for completion of [{}]", t);
}));
}
@Override

View File

@ -77,7 +77,7 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
@Override
protected EmptyResult shardOperation(ClearIndicesCacheRequest request, ShardRouting shardRouting) {
IndexService service = indicesService.indexService(shardRouting.getIndexName());
IndexService service = indicesService.indexService(shardRouting.index());
if (service != null) {
IndexShard shard = service.getShardOrNull(shardRouting.id());
boolean clearedAtLeastOne = false;

View File

@ -93,7 +93,7 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastByNodeActi
@Override
protected ShardSegments shardOperation(IndicesSegmentsRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(shardRouting.getIndexName());
IndexService indexService = indicesService.indexServiceSafe(shardRouting.index());
IndexShard indexShard = indexService.getShard(shardRouting.id());
return new ShardSegments(indexShard.routingEntry(), indexShard.segments(request.verbose()));
}

View File

@ -47,6 +47,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
@ -104,8 +105,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
@Override
protected Tuple<BulkShardResponse, BulkShardRequest> shardOperationOnPrimary(MetaData metaData, BulkShardRequest request) {
final IndexService indexService = indicesService.indexServiceSafe(request.index());
final IndexShard indexShard = indexService.getShard(request.shardId().id());
ShardId shardId = request.shardId();
final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
final IndexShard indexShard = indexService.getShard(shardId.getId());
long[] preVersions = new long[request.items().length];
VersionType[] preVersionTypes = new VersionType[request.items().length];

View File

@ -32,8 +32,6 @@ import org.elasticsearch.search.SearchShardTarget;
import java.io.IOException;
import static org.elasticsearch.search.SearchShardTarget.readSearchShardTarget;
/**
* Represents a failure to search on a specific shard.
*/
@ -106,7 +104,7 @@ public class ShardSearchFailure implements ShardOperationFailedException {
@Override
public int shardId() {
if (shardTarget != null) {
return shardTarget.shardId();
return shardTarget.shardId().id();
}
return -1;
}
@ -133,7 +131,7 @@ public class ShardSearchFailure implements ShardOperationFailedException {
@Override
public void readFrom(StreamInput in) throws IOException {
if (in.readBoolean()) {
shardTarget = readSearchShardTarget(in);
shardTarget = new SearchShardTarget(in);
}
reason = in.readString();
status = RestStatus.readFrom(in);

View File

@ -53,6 +53,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.shard.IndexShard;
@ -372,18 +373,18 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
private void failReplicaIfNeeded(Throwable t) {
String index = request.shardId().getIndex().getName();
Index index = request.shardId().getIndex();
int shardId = request.shardId().id();
logger.trace("failure on replica [{}][{}], action [{}], request [{}]", t, index, shardId, actionName, request);
if (ignoreReplicaException(t) == false) {
IndexService indexService = indicesService.indexService(index);
if (indexService == null) {
logger.debug("ignoring failed replica [{}][{}] because index was already removed.", index, shardId);
logger.debug("ignoring failed replica {}[{}] because index was already removed.", index, shardId);
return;
}
IndexShard indexShard = indexService.getShardOrNull(shardId);
if (indexShard == null) {
logger.debug("ignoring failed replica [{}][{}] because index was already removed.", index, shardId);
logger.debug("ignoring failed replica {}[{}] because index was already removed.", index, shardId);
return;
}
indexShard.failShard(actionName + " failed on replica", t);

View File

@ -27,6 +27,7 @@ import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
@ -42,8 +43,8 @@ public abstract class InstanceShardOperationRequest<Request extends InstanceShar
protected TimeValue timeout = DEFAULT_TIMEOUT;
protected String index;
// -1 means its not set, allows to explicitly direct a request to a specific shard
protected int shardId = -1;
// null means its not set, allows to explicitly direct a request to a specific shard
protected ShardId shardId = null;
private String concreteIndex;
@ -115,7 +116,11 @@ public abstract class InstanceShardOperationRequest<Request extends InstanceShar
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
index = in.readString();
shardId = in.readInt();
if (in.readBoolean()) {
shardId = ShardId.readShardId(in);
} else {
shardId = null;
}
timeout = TimeValue.readTimeValue(in);
concreteIndex = in.readOptionalString();
}
@ -124,7 +129,7 @@ public abstract class InstanceShardOperationRequest<Request extends InstanceShar
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(index);
out.writeInt(shardId);
out.writeOptionalStreamable(shardId);
timeout.writeTo(out);
out.writeOptionalString(concreteIndex);
}

View File

@ -172,7 +172,7 @@ public abstract class TransportInstanceSingleOperationAction<Request extends Ins
return;
}
request.shardId = shardIt.shardId().id();
request.shardId = shardIt.shardId();
DiscoveryNode node = nodes.get(shard.currentNodeId());
transportService.sendRequest(node, shardActionName, request, transportOptions(), new BaseTransportResponseHandler<Response>() {

View File

@ -71,7 +71,7 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
* Sets the list of action masks for the actions that should be returned
*/
@SuppressWarnings("unchecked")
public final Request actions(String... actions) {
public final Request setActions(String... actions) {
this.actions = actions;
return (Request) this;
}
@ -79,16 +79,16 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
/**
* Return the list of action masks for the actions that should be returned
*/
public String[] actions() {
public String[] getActions() {
return actions;
}
public final String[] nodesIds() {
public final String[] getNodesIds() {
return nodesIds;
}
@SuppressWarnings("unchecked")
public final Request nodesIds(String... nodesIds) {
public final Request setNodesIds(String... nodesIds) {
this.nodesIds = nodesIds;
return (Request) this;
}
@ -98,12 +98,12 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
*
* By default tasks with any ids are returned.
*/
public TaskId taskId() {
public TaskId getTaskId() {
return taskId;
}
@SuppressWarnings("unchecked")
public final Request taskId(TaskId taskId) {
public final Request setTaskId(TaskId taskId) {
this.taskId = taskId;
return (Request) this;
}
@ -112,29 +112,29 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
/**
* Returns the parent task id that tasks should be filtered by
*/
public TaskId parentTaskId() {
public TaskId getParentTaskId() {
return parentTaskId;
}
@SuppressWarnings("unchecked")
public Request parentTaskId(TaskId parentTaskId) {
public Request setParentTaskId(TaskId parentTaskId) {
this.parentTaskId = parentTaskId;
return (Request) this;
}
public TimeValue timeout() {
public TimeValue getTimeout() {
return this.timeout;
}
@SuppressWarnings("unchecked")
public final Request timeout(TimeValue timeout) {
public final Request setTimeout(TimeValue timeout) {
this.timeout = timeout;
return (Request) this;
}
@SuppressWarnings("unchecked")
public final Request timeout(String timeout) {
public final Request setTimeout(String timeout) {
this.timeout = TimeValue.parseTimeValue(timeout, null, getClass().getSimpleName() + ".timeout");
return (Request) this;
}
@ -162,11 +162,11 @@ public class BaseTasksRequest<Request extends BaseTasksRequest<Request>> extends
}
public boolean match(Task task) {
if (actions() != null && actions().length > 0 && Regex.simpleMatch(actions(), task.getAction()) == false) {
if (getActions() != null && getActions().length > 0 && Regex.simpleMatch(getActions(), task.getAction()) == false) {
return false;
}
if (taskId().isSet() == false) {
if(taskId().getId() != task.getId()) {
if (getTaskId().isSet() == false) {
if(getTaskId().getId() != task.getId()) {
return false;
}
}

View File

@ -35,19 +35,19 @@ public class TasksRequestBuilder <Request extends BaseTasksRequest<Request>, Res
@SuppressWarnings("unchecked")
public final RequestBuilder setNodesIds(String... nodesIds) {
request.nodesIds(nodesIds);
request.setNodesIds(nodesIds);
return (RequestBuilder) this;
}
@SuppressWarnings("unchecked")
public final RequestBuilder setActions(String... actions) {
request.actions(actions);
request.setActions(actions);
return (RequestBuilder) this;
}
@SuppressWarnings("unchecked")
public final RequestBuilder setTimeout(TimeValue timeout) {
request.timeout(timeout);
request.setTimeout(timeout);
return (RequestBuilder) this;
}
}

View File

@ -124,25 +124,25 @@ public abstract class TransportTasksAction<
}
protected String[] resolveNodes(TasksRequest request, ClusterState clusterState) {
if (request.taskId().isSet()) {
return clusterState.nodes().resolveNodesIds(request.nodesIds());
if (request.getTaskId().isSet()) {
return clusterState.nodes().resolveNodesIds(request.getNodesIds());
} else {
return new String[]{request.taskId().getNodeId()};
return new String[]{request.getTaskId().getNodeId()};
}
}
protected void processTasks(TasksRequest request, Consumer<OperationTask> operation) {
if (request.taskId().isSet() == false) {
if (request.getTaskId().isSet() == false) {
// we are only checking one task, we can optimize it
Task task = taskManager.getTask(request.taskId().getId());
Task task = taskManager.getTask(request.getTaskId().getId());
if (task != null) {
if (request.match(task)) {
operation.accept((OperationTask) task);
} else {
throw new ResourceNotFoundException("task [{}] doesn't support this operation", request.taskId());
throw new ResourceNotFoundException("task [{}] doesn't support this operation", request.getTaskId());
}
} else {
throw new ResourceNotFoundException("task [{}] is missing", request.taskId());
throw new ResourceNotFoundException("task [{}] is missing", request.getTaskId());
}
} else {
for (Task task : taskManager.getTasks().values()) {
@ -224,8 +224,8 @@ public abstract class TransportTasksAction<
}
} else {
TransportRequestOptions.Builder builder = TransportRequestOptions.builder();
if (request.timeout() != null) {
builder.withTimeout(request.timeout());
if (request.getTimeout() != null) {
builder.withTimeout(request.getTimeout());
}
builder.withCompress(transportCompress());
for (int i = 0; i < nodesIds.length; i++) {

View File

@ -75,12 +75,12 @@ public class TransportShardMultiTermsVectorAction extends TransportSingleShardAc
@Override
protected MultiTermVectorsShardResponse shardOperation(MultiTermVectorsShardRequest request, ShardId shardId) {
MultiTermVectorsShardResponse response = new MultiTermVectorsShardResponse();
final MultiTermVectorsShardResponse response = new MultiTermVectorsShardResponse();
final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
final IndexShard indexShard = indexService.getShard(shardId.id());
for (int i = 0; i < request.locations.size(); i++) {
TermVectorsRequest termVectorsRequest = request.requests.get(i);
try {
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.getShard(shardId.id());
TermVectorsResponse termVectorsResponse = TermVectorsService.getTermVectors(indexShard, termVectorsRequest);
termVectorsResponse.updateTookInMillis(termVectorsRequest.startTime());
response.add(request.locations.get(i), termVectorsResponse);

View File

@ -51,6 +51,7 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
@ -147,8 +148,8 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
@Override
protected ShardIterator shards(ClusterState clusterState, UpdateRequest request) {
if (request.shardId() != -1) {
return clusterState.routingTable().index(request.concreteIndex()).shard(request.shardId()).primaryShardIt();
if (request.getShardId() != null) {
return clusterState.routingTable().index(request.concreteIndex()).shard(request.getShardId().getId()).primaryShardIt();
}
ShardIterator shardIterator = clusterService.operationRouting()
.indexShards(clusterState, request.concreteIndex(), request.type(), request.id(), request.routing());
@ -167,8 +168,9 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
}
protected void shardOperation(final UpdateRequest request, final ActionListener<UpdateResponse> listener, final int retryCount) {
final IndexService indexService = indicesService.indexServiceSafe(request.concreteIndex());
final IndexShard indexShard = indexService.getShard(request.shardId());
final ShardId shardId = request.getShardId();
final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
final IndexShard indexShard = indexService.getShard(shardId.getId());
final UpdateHelper.Result result = updateHelper.prepare(request, indexShard);
switch (result.operation()) {
case UPSERT:
@ -194,7 +196,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
if (e instanceof VersionConflictEngineException) {
if (retryCount < request.retryOnConflict()) {
logger.trace("Retry attempt [{}] of [{}] on version conflict on [{}][{}][{}]",
retryCount + 1, request.retryOnConflict(), request.index(), request.shardId(), request.id());
retryCount + 1, request.retryOnConflict(), request.index(), request.getShardId(), request.id());
threadPool.executor(executor()).execute(new ActionRunnable<UpdateResponse>(listener) {
@Override
protected void doRun() {
@ -267,9 +269,9 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
break;
case NONE:
UpdateResponse update = result.action();
IndexService indexServiceOrNull = indicesService.indexService(request.concreteIndex());
IndexService indexServiceOrNull = indicesService.indexService(shardId.getIndex());
if (indexServiceOrNull != null) {
IndexShard shard = indexService.getShardOrNull(request.shardId());
IndexShard shard = indexService.getShardOrNull(shardId.getId());
if (shard != null) {
shard.noopUpdate(request.type());
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptParameterParser;
import org.elasticsearch.script.ScriptParameterParser.ScriptParameterValue;
@ -88,7 +89,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
}
public UpdateRequest(String index, String type, String id) {
this.index = index;
super(index);
this.type = type;
this.id = id;
}
@ -195,7 +196,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
return parent;
}
int shardId() {
public ShardId getShardId() {
return this.shardId;
}

View File

@ -225,7 +225,7 @@ final class BootstrapCheck {
static class MaxNumberOfThreadsCheck implements Check {
private final long maxNumberOfThreadsThreshold = 1 << 15;
private final long maxNumberOfThreadsThreshold = 1 << 11;
@Override
public boolean check() {

View File

@ -23,6 +23,7 @@ import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.index.Index;
import java.util.ArrayList;
import java.util.Collections;
@ -120,7 +121,7 @@ public class ClusterChangedEvent {
/**
* Returns the indices deleted in this event
*/
public List<String> indicesDeleted() {
public List<Index> indicesDeleted() {
// If the new cluster state has a new cluster UUID, the likely scenario is that a node was elected
// master that has had its data directory wiped out, in which case we don't want to delete the indices and lose data;
// rather we want to import them as dangling indices instead. So we check here if the cluster UUID differs from the previous
@ -131,17 +132,18 @@ public class ClusterChangedEvent {
if (metaDataChanged() == false || isNewCluster()) {
return Collections.emptyList();
}
List<String> deleted = null;
for (ObjectCursor<String> cursor : previousState.metaData().indices().keys()) {
String index = cursor.value;
if (!state.metaData().hasIndex(index)) {
List<Index> deleted = null;
for (ObjectCursor<IndexMetaData> cursor : previousState.metaData().indices().values()) {
IndexMetaData index = cursor.value;
IndexMetaData current = state.metaData().index(index.getIndex().getName());
if (current == null || index.getIndexUUID().equals(current.getIndexUUID()) == false) {
if (deleted == null) {
deleted = new ArrayList<>();
}
deleted.add(index);
deleted.add(index.getIndex());
}
}
return deleted == null ? Collections.<String>emptyList() : deleted;
return deleted == null ? Collections.<Index>emptyList() : deleted;
}
/**

View File

@ -686,7 +686,7 @@ public class IndexNameExpressionResolver extends AbstractComponent {
}
private boolean isEmptyOrTrivialWildcard(List<String> expressions) {
return expressions.isEmpty() || (expressions.size() == 1 && (MetaData.ALL.equals(expressions.get(0))) || Regex.isMatchAllPattern(expressions.get(0)));
return expressions.isEmpty() || (expressions.size() == 1 && (MetaData.ALL.equals(expressions.get(0)) || Regex.isMatchAllPattern(expressions.get(0))));
}
private List<String> resolveEmptyOrTrivialWildcard(IndicesOptions options, MetaData metaData, boolean assertEmpty) {

View File

@ -53,6 +53,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper;
@ -188,7 +189,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
@Override
public ClusterState execute(ClusterState currentState) throws Exception {
boolean indexCreated = false;
Index createdIndex = null;
String removalReason = null;
try {
validate(request, currentState);
@ -308,10 +309,9 @@ public class MetaDataCreateIndexService extends AbstractComponent {
// Set up everything, now locally create the index to see that things are ok, and apply
final IndexMetaData tmpImd = IndexMetaData.builder(request.index()).settings(actualIndexSettings).build();
// create the index here (on the master) to validate it can be created, as well as adding the mapping
indicesService.createIndex(nodeServicesProvider, tmpImd, Collections.emptyList());
indexCreated = true;
final IndexService indexService = indicesService.createIndex(nodeServicesProvider, tmpImd, Collections.emptyList());
createdIndex = indexService.index();
// now add the mappings
IndexService indexService = indicesService.indexServiceSafe(request.index());
MapperService mapperService = indexService.mapperService();
// first, add the default mapping
if (mappings.containsKey(MapperService.DEFAULT_MAPPING)) {
@ -415,9 +415,9 @@ public class MetaDataCreateIndexService extends AbstractComponent {
removalReason = "cleaning up after validating index on master";
return updatedState;
} finally {
if (indexCreated) {
if (createdIndex != null) {
// Index was already partially created - need to clean up
indicesService.removeIndex(request.index(), removalReason != null ? removalReason : "failed to create index");
indicesService.removeIndex(createdIndex, removalReason != null ? removalReason : "failed to create index");
}
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
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.NodeServicesProvider;
@ -74,7 +75,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
@Override
public ClusterState execute(final ClusterState currentState) {
List<String> indicesToClose = new ArrayList<>();
List<Index> indicesToClose = new ArrayList<>();
Map<String, IndexService> indices = new HashMap<>();
try {
for (AliasAction aliasAction : request.actions()) {
@ -112,7 +113,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
logger.warn("[{}] failed to temporary create in order to apply alias action", e, indexMetaData.getIndex());
continue;
}
indicesToClose.add(indexMetaData.getIndex().getName());
indicesToClose.add(indexMetaData.getIndex());
}
indices.put(indexMetaData.getIndex().getName(), indexService);
}
@ -153,7 +154,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
}
return currentState;
} finally {
for (String index : indicesToClose) {
for (Index index : indicesToClose) {
indicesService.removeIndex(index, "created for alias processing");
}
}

View File

@ -19,12 +19,14 @@
package org.elasticsearch.cluster.metadata;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.RestoreInProgress;
import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
@ -37,11 +39,14 @@ import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
/**
* Service responsible for submitting open/close index requests
@ -78,7 +83,7 @@ public class MetaDataIndexStateService extends AbstractComponent {
@Override
public ClusterState execute(ClusterState currentState) {
List<String> indicesToClose = new ArrayList<>();
Set<String> indicesToClose = new HashSet<>();
for (String index : request.indices()) {
IndexMetaData indexMetaData = currentState.metaData().index(index);
if (indexMetaData == null) {
@ -94,6 +99,28 @@ public class MetaDataIndexStateService extends AbstractComponent {
return currentState;
}
// Check if any of the indices to be closed are currently being restored from a snapshot and fail closing if such an index
// is found as closing an index that is being restored makes the index unusable (it cannot be recovered).
RestoreInProgress restore = currentState.custom(RestoreInProgress.TYPE);
if (restore != null) {
Set<String> indicesToFail = null;
for (RestoreInProgress.Entry entry : restore.entries()) {
for (ObjectObjectCursor<ShardId, RestoreInProgress.ShardRestoreStatus> shard : entry.shards()) {
if (!shard.value.state().completed()) {
if (indicesToClose.contains(shard.key.getIndexName())) {
if (indicesToFail == null) {
indicesToFail = new HashSet<>();
}
indicesToFail.add(shard.key.getIndexName());
}
}
}
}
if (indicesToFail != null) {
throw new IllegalArgumentException("Cannot close indices that are being restored: " + indicesToFail);
}
}
logger.info("closing indices [{}]", indicesAsString);
MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());

View File

@ -36,6 +36,7 @@ import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
@ -112,13 +113,13 @@ public class MetaDataMappingService extends AbstractComponent {
MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
for (Map.Entry<String, List<RefreshTask>> entry : tasksPerIndex.entrySet()) {
String index = entry.getKey();
IndexMetaData indexMetaData = mdBuilder.get(index);
IndexMetaData indexMetaData = mdBuilder.get(entry.getKey());
if (indexMetaData == null) {
// index got deleted on us, ignore...
logger.debug("[{}] ignoring tasks - index meta data doesn't exist", index);
logger.debug("[{}] ignoring tasks - index meta data doesn't exist", entry.getKey());
continue;
}
final Index index = indexMetaData.getIndex();
// the tasks lists to iterate over, filled with the list of mapping tasks, trying to keep
// the latest (based on order) update mapping one per node
List<RefreshTask> allIndexTasks = entry.getValue();
@ -127,7 +128,7 @@ public class MetaDataMappingService extends AbstractComponent {
if (indexMetaData.isSameUUID(task.indexUUID)) {
hasTaskWithRightUUID = true;
} else {
logger.debug("[{}] ignoring task [{}] - index meta data doesn't match task uuid", index, task);
logger.debug("{} ignoring task [{}] - index meta data doesn't match task uuid", index, task);
}
}
if (hasTaskWithRightUUID == false) {
@ -136,7 +137,7 @@ public class MetaDataMappingService extends AbstractComponent {
// construct the actual index if needed, and make sure the relevant mappings are there
boolean removeIndex = false;
IndexService indexService = indicesService.indexService(index);
IndexService indexService = indicesService.indexService(indexMetaData.getIndex());
if (indexService == null) {
// we need to create the index here, and add the current mapping to it, so we can merge
indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.emptyList());
@ -208,47 +209,57 @@ public class MetaDataMappingService extends AbstractComponent {
class PutMappingExecutor implements ClusterStateTaskExecutor<PutMappingClusterStateUpdateRequest> {
@Override
public BatchResult<PutMappingClusterStateUpdateRequest> execute(ClusterState currentState, List<PutMappingClusterStateUpdateRequest> tasks) throws Exception {
Set<String> indicesToClose = new HashSet<>();
public BatchResult<PutMappingClusterStateUpdateRequest> execute(ClusterState currentState,
List<PutMappingClusterStateUpdateRequest> tasks) throws Exception {
Set<Index> indicesToClose = new HashSet<>();
BatchResult.Builder<PutMappingClusterStateUpdateRequest> builder = BatchResult.builder();
try {
// precreate incoming indices;
for (PutMappingClusterStateUpdateRequest request : tasks) {
// failures here mean something is broken with our cluster state - fail all tasks by letting exceptions bubble up
final List<Index> indices = new ArrayList<>(request.indices().length);
try {
for (String index : request.indices()) {
final IndexMetaData indexMetaData = currentState.metaData().index(index);
if (indexMetaData != null && indicesService.hasIndex(index) == false) {
// if we don't have the index, we will throw exceptions later;
indicesToClose.add(index);
IndexService indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.emptyList());
if (indexMetaData != null) {
if (indicesService.hasIndex(indexMetaData.getIndex()) == false) {
// if the index does not exists we create it once, add all types to the mapper service and
// close it later once we are done with mapping update
indicesToClose.add(indexMetaData.getIndex());
IndexService indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData,
Collections.emptyList());
// add mappings for all types, we need them for cross-type validation
for (ObjectCursor<MappingMetaData> mapping : indexMetaData.getMappings().values()) {
indexService.mapperService().merge(mapping.value.type(), mapping.value.source(), MapperService.MergeReason.MAPPING_RECOVERY, request.updateAllTypes());
indexService.mapperService().merge(mapping.value.type(), mapping.value.source(),
MapperService.MergeReason.MAPPING_RECOVERY, request.updateAllTypes());
}
}
indices.add(indexMetaData.getIndex());
} else {
// we didn't find the index in the clusterstate - maybe it was deleted
// NOTE: this doesn't fail the entire batch only the current PutMapping request we are processing
throw new IndexNotFoundException(index);
}
}
for (PutMappingClusterStateUpdateRequest request : tasks) {
try {
currentState = applyRequest(currentState, request);
currentState = applyRequest(currentState, request, indices);
builder.success(request);
} catch (Throwable t) {
builder.failure(request, t);
}
}
return builder.build(currentState);
} finally {
for (String index : indicesToClose) {
for (Index index : indicesToClose) {
indicesService.removeIndex(index, "created for mapping processing");
}
}
}
private ClusterState applyRequest(ClusterState currentState, PutMappingClusterStateUpdateRequest request) throws IOException {
private ClusterState applyRequest(ClusterState currentState, PutMappingClusterStateUpdateRequest request,
List<Index> indices) throws IOException {
String mappingType = request.type();
CompressedXContent mappingUpdateSource = new CompressedXContent(request.source());
for (String index : request.indices()) {
final MetaData metaData = currentState.metaData();
for (Index index : indices) {
IndexService indexService = indicesService.indexServiceSafe(index);
// try and parse it (no need to add it here) so we can bail early in case of parsing exception
DocumentMapper newMapper;
@ -270,7 +281,7 @@ public class MetaDataMappingService extends AbstractComponent {
// and a put mapping api call, so we don't which type did exist before.
// Also the order of the mappings may be backwards.
if (newMapper.parentFieldMapper().active()) {
IndexMetaData indexMetaData = currentState.metaData().index(index);
IndexMetaData indexMetaData = metaData.index(index);
for (ObjectCursor<MappingMetaData> mapping : indexMetaData.getMappings().values()) {
if (newMapper.parentFieldMapper().type().equals(mapping.value.type())) {
throw new IllegalArgumentException("can't add a _parent field that points to an already existing type");
@ -290,11 +301,11 @@ public class MetaDataMappingService extends AbstractComponent {
if (!MapperService.DEFAULT_MAPPING.equals(mappingType) && !PercolatorService.TYPE_NAME.equals(mappingType) && mappingType.charAt(0) == '_') {
throw new InvalidTypeNameException("Document mapping type name can't start with '_'");
}
MetaData.Builder builder = MetaData.builder(currentState.metaData());
for (String index : request.indices()) {
MetaData.Builder builder = MetaData.builder(metaData);
for (Index index : indices) {
// do the actual merge here on the master, and update the mapping source
IndexService indexService = indicesService.indexService(index);
if (indexService == null) {
if (indexService == null) { // TODO this seems impossible given we use indexServiceSafe above
continue;
}
@ -326,7 +337,7 @@ public class MetaDataMappingService extends AbstractComponent {
}
}
IndexMetaData indexMetaData = currentState.metaData().index(index);
IndexMetaData indexMetaData = metaData.index(index);
if (indexMetaData == null) {
throw new IndexNotFoundException(index);
}

View File

@ -235,12 +235,8 @@ public class Lucene {
@Override
protected Object doBody(String segmentFileName) throws IOException {
try (IndexInput input = directory.openInput(segmentFileName, IOContext.READ)) {
final int format = input.readInt();
if (format == CodecUtil.CODEC_MAGIC) {
CodecUtil.checksumEntireFile(input);
}
// legacy....
}
return null;
}
}.run();

View File

@ -35,6 +35,7 @@ import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.Store;
@ -44,6 +45,7 @@ import org.elasticsearch.indices.IndicesRequestCache;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.function.Predicate;
@ -133,8 +135,15 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
FsDirectoryService.INDEX_LOCK_FACTOR_SETTING,
EngineConfig.INDEX_CODEC_SETTING,
IndexWarmer.INDEX_NORMS_LOADING_SETTING,
// this sucks but we can't really validate all the analyzers/similarity in here
Setting.groupSetting("index.similarity.", false, Setting.Scope.INDEX), // this allows similarity settings to be passed
// validate that built-in similarities don't get redefined
Setting.groupSetting("index.similarity.", false, Setting.Scope.INDEX, (s) -> {
Map<String, Settings> groups = s.getAsGroups();
for (String key : SimilarityService.BUILT_IN.keySet()) {
if (groups.containsKey(key)) {
throw new IllegalArgumentException("illegal value for [index.similarity."+ key + "] cannot redefine built-in similarity");
}
}
}), // this allows similarity settings to be passed
Setting.groupSetting("index.analysis.", false, Setting.Scope.INDEX) // this allows analysis settings to be passed
)));

View File

@ -20,6 +20,7 @@ package org.elasticsearch.common.settings;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Strings;
@ -30,16 +31,19 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.MemorySizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@ -177,7 +181,7 @@ public class Setting<T> extends ToXContentToBytes {
/**
* Returns <code>true</code> iff this setting is present in the given settings object. Otherwise <code>false</code>
*/
public final boolean exists(Settings settings) {
public boolean exists(Settings settings) {
return settings.get(getKey()) != null;
}
@ -505,17 +509,45 @@ public class Setting<T> extends ToXContentToBytes {
throw new ElasticsearchException(ex);
}
}
public static Setting<Settings> groupSetting(String key, boolean dynamic, Scope scope) {
return groupSetting(key, dynamic, scope, (s) -> {});
}
public static Setting<Settings> groupSetting(String key, boolean dynamic, Scope scope, Consumer<Settings> validator) {
return new Setting<Settings>(new GroupKey(key), (s) -> "", (s) -> null, dynamic, scope) {
@Override
public boolean isGroupSetting() {
return true;
}
@Override
public String getRaw(Settings settings) {
Settings subSettings = get(settings);
try {
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
subSettings.toXContent(builder, EMPTY_PARAMS);
builder.endObject();
return builder.string();
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Override
public Settings get(Settings settings) {
return settings.getByPrefix(getKey());
Settings byPrefix = settings.getByPrefix(getKey());
validator.accept(byPrefix);
return byPrefix;
}
@Override
public boolean exists(Settings settings) {
for (Map.Entry<String, String> entry : settings.getAsMap().entrySet()) {
if (entry.getKey().startsWith(key)) {
return true;
}
}
return false;
}
@Override

View File

@ -26,7 +26,6 @@ import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.node.service.NodeService;
import java.io.IOException;
@ -41,11 +40,6 @@ public interface Discovery extends LifecycleComponent<Discovery> {
String nodeDescription();
/**
* Here as a hack to solve dep injection problem...
*/
void setNodeService(@Nullable NodeService nodeService);
/**
* Another hack to solve dep injection problem..., note, this will be called before
* any start is called.

View File

@ -33,7 +33,6 @@ import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.ClusterSettings;
@ -45,7 +44,6 @@ import org.elasticsearch.discovery.BlockingClusterStatePublishResponseHandler;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.DiscoveryStats;
import org.elasticsearch.node.service.NodeService;
import java.util.HashSet;
import java.util.Queue;
@ -84,11 +82,6 @@ public class LocalDiscovery extends AbstractLifecycleComponent<Discovery> implem
this.discoverySettings = new DiscoverySettings(settings, clusterSettings);
}
@Override
public void setNodeService(@Nullable NodeService nodeService) {
// nothing to do here
}
@Override
public void setRoutingService(RoutingService routingService) {
this.routingService = routingService;

View File

@ -20,8 +20,6 @@
package org.elasticsearch.discovery.zen;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.node.service.NodeService;
/**
*
@ -30,6 +28,4 @@ public interface DiscoveryNodesProvider {
DiscoveryNodes nodes();
@Nullable
NodeService nodeService();
}

View File

@ -60,7 +60,6 @@ import org.elasticsearch.discovery.zen.ping.ZenPing;
import org.elasticsearch.discovery.zen.ping.ZenPingService;
import org.elasticsearch.discovery.zen.publish.PendingClusterStateStats;
import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.EmptyTransportResponseHandler;
import org.elasticsearch.transport.TransportChannel;
@ -137,10 +136,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
/** counts the time this node has joined the cluster or have elected it self as master */
private final AtomicLong clusterJoinsCounter = new AtomicLong();
@Nullable
private NodeService nodeService;
// must initialized in doStart(), when we have the routingService set
private volatile NodeJoinController nodeJoinController;
@ -192,11 +187,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
transportService.registerRequestHandler(DISCOVERY_REJOIN_ACTION_NAME, RejoinClusterRequest::new, ThreadPool.Names.SAME, new RejoinClusterRequestHandler());
}
@Override
public void setNodeService(@Nullable NodeService nodeService) {
this.nodeService = nodeService;
}
@Override
public void setRoutingService(RoutingService routingService) {
this.routingService = routingService;
@ -292,11 +282,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return clusterService.state().nodes();
}
@Override
public NodeService nodeService() {
return this.nodeService;
}
@Override
public boolean nodeHasJoinedClusterOnce() {
return clusterJoinsCounter.get() > 0;

View File

@ -104,6 +104,8 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
&& matchingNodes.isNodeMatchBySyncID(nodeWithHighestMatch) == true) {
// we found a better match that has a full sync id match, the existing allocation is not fully synced
// so we found a better one, cancel this one
logger.debug("cancelling allocation of replica on [{}], sync id match found on node [{}]",
currentNode, nodeWithHighestMatch);
it.moveToUnassigned(new UnassignedInfo(UnassignedInfo.Reason.REALLOCATED_REPLICA,
"existing allocation of replica to [" + currentNode + "] cancelled, sync id match found on node [" + nodeWithHighestMatch + "]",
null, allocation.getCurrentNanoTime(), System.currentTimeMillis()));

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
@ -50,8 +51,15 @@ public class Index implements Writeable<Index> {
@Override
public String toString() {
/*
* If we have a uuid we put it in the toString so it'll show up in logs which is useful as more and more things use the uuid rather
* than the name as the lookup key for the index.
*/
if (ClusterState.UNKNOWN_UUID.equals(uuid)) {
return "[" + name + "]";
}
return "[" + name + "/" + uuid + "]";
}
@Override
public boolean equals(Object o) {

View File

@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit;
/**
*/
public final class IndexingSlowLog implements IndexingOperationListener {
private final Index index;
private boolean reformat;
private long indexWarnThreshold;
private long indexInfoThreshold;
@ -51,7 +52,6 @@ public final class IndexingSlowLog implements IndexingOperationListener {
private SlowLogLevel level;
private final ESLogger indexLogger;
private final ESLogger deleteLogger;
private static final String INDEX_INDEXING_SLOWLOG_PREFIX = "index.indexing.slowlog";
public static final Setting<TimeValue> INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN_SETTING = Setting.timeSetting(INDEX_INDEXING_SLOWLOG_PREFIX +".threshold.index.warn", TimeValue.timeValueNanos(-1), TimeValue.timeValueMillis(-1), true, Setting.Scope.INDEX);
@ -75,16 +75,8 @@ public final class IndexingSlowLog implements IndexingOperationListener {
}, true, Setting.Scope.INDEX);
IndexingSlowLog(IndexSettings indexSettings) {
this(indexSettings, Loggers.getLogger(INDEX_INDEXING_SLOWLOG_PREFIX + ".index"),
Loggers.getLogger(INDEX_INDEXING_SLOWLOG_PREFIX + ".delete"));
}
/**
* Build with the specified loggers. Only used to testing.
*/
IndexingSlowLog(IndexSettings indexSettings, ESLogger indexLogger, ESLogger deleteLogger) {
this.indexLogger = indexLogger;
this.deleteLogger = deleteLogger;
this.indexLogger = Loggers.getLogger(INDEX_INDEXING_SLOWLOG_PREFIX + ".index", indexSettings.getSettings());
this.index = indexSettings.getIndex();
indexSettings.getScopedSettings().addSettingsUpdateConsumer(INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING, this::setReformat);
this.reformat = indexSettings.getValue(INDEX_INDEXING_SLOWLOG_REFORMAT_SETTING);
@ -109,7 +101,6 @@ public final class IndexingSlowLog implements IndexingOperationListener {
private void setLevel(SlowLogLevel level) {
this.level = level;
this.indexLogger.setLevel(level.name());
this.deleteLogger.setLevel(level.name());
}
private void setWarnThreshold(TimeValue warnThreshold) {
@ -141,13 +132,13 @@ public final class IndexingSlowLog implements IndexingOperationListener {
private void postIndexing(ParsedDocument doc, long tookInNanos) {
if (indexWarnThreshold >= 0 && tookInNanos > indexWarnThreshold) {
indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(doc, tookInNanos, reformat, maxSourceCharsToLog));
indexLogger.warn("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog));
} else if (indexInfoThreshold >= 0 && tookInNanos > indexInfoThreshold) {
indexLogger.info("{}", new SlowLogParsedDocumentPrinter(doc, tookInNanos, reformat, maxSourceCharsToLog));
indexLogger.info("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog));
} else if (indexDebugThreshold >= 0 && tookInNanos > indexDebugThreshold) {
indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(doc, tookInNanos, reformat, maxSourceCharsToLog));
indexLogger.debug("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog));
} else if (indexTraceThreshold >= 0 && tookInNanos > indexTraceThreshold) {
indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(doc, tookInNanos, reformat, maxSourceCharsToLog));
indexLogger.trace("{}", new SlowLogParsedDocumentPrinter(index, doc, tookInNanos, reformat, maxSourceCharsToLog));
}
}
@ -156,9 +147,11 @@ public final class IndexingSlowLog implements IndexingOperationListener {
private final long tookInNanos;
private final boolean reformat;
private final int maxSourceCharsToLog;
private final Index index;
SlowLogParsedDocumentPrinter(ParsedDocument doc, long tookInNanos, boolean reformat, int maxSourceCharsToLog) {
SlowLogParsedDocumentPrinter(Index index, ParsedDocument doc, long tookInNanos, boolean reformat, int maxSourceCharsToLog) {
this.doc = doc;
this.index = index;
this.tookInNanos = tookInNanos;
this.reformat = reformat;
this.maxSourceCharsToLog = maxSourceCharsToLog;
@ -167,6 +160,7 @@ public final class IndexingSlowLog implements IndexingOperationListener {
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(index).append(" ");
sb.append("took[").append(TimeValue.timeValueNanos(tookInNanos)).append("], took_millis[").append(TimeUnit.NANOSECONDS.toMillis(tookInNanos)).append("], ");
sb.append("type[").append(doc.type()).append("], ");
sb.append("id[").append(doc.id()).append("], ");

View File

@ -20,10 +20,14 @@
package org.elasticsearch.index.fielddata;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.FieldComparatorSource;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.Weight;
@ -122,11 +126,11 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
public static class Nested {
private final BitSetProducer rootFilter;
private final Weight innerFilter;
private final Query innerQuery;
public Nested(BitSetProducer rootFilter, Weight innerFilter) {
public Nested(BitSetProducer rootFilter, Query innerQuery) {
this.rootFilter = rootFilter;
this.innerFilter = innerFilter;
this.innerQuery = innerQuery;
}
/**
@ -140,7 +144,10 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
* Get a {@link DocIdSet} that matches the inner documents.
*/
public DocIdSetIterator innerDocs(LeafReaderContext ctx) throws IOException {
Scorer s = innerFilter.scorer(ctx);
final IndexReaderContext topLevelCtx = ReaderUtil.getTopLevelContext(ctx);
IndexSearcher indexSearcher = new IndexSearcher(topLevelCtx);
Weight weight = indexSearcher.createNormalizedWeight(innerQuery, false);
Scorer s = weight.scorer(ctx);
return s == null ? null : s.iterator();
}
}

View File

@ -28,6 +28,8 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -56,13 +58,13 @@ import static org.elasticsearch.index.mapper.core.TypeParsers.parseMultiField;
*/
public abstract class BaseGeoPointFieldMapper extends FieldMapper implements ArrayValueMapperParser {
public static final String CONTENT_TYPE = "geo_point";
protected static final DeprecationLogger deprecationLogger = new DeprecationLogger(Loggers.getLogger(BaseGeoPointFieldMapper.class));
public static class Names {
public static final String LAT = "lat";
public static final String LAT_SUFFIX = "." + LAT;
public static final String LON = "lon";
public static final String LON_SUFFIX = "." + LON;
public static final String GEOHASH = "geohash";
public static final String GEOHASH_SUFFIX = "." + GEOHASH;
public static final String IGNORE_MALFORMED = "ignore_malformed";
}
@ -194,9 +196,13 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
String propName = Strings.toUnderscoreCase(entry.getKey());
Object propNode = entry.getValue();
if (propName.equals("lat_lon")) {
deprecationLogger.deprecated(CONTENT_TYPE + " lat_lon parameter is deprecated and will be removed "
+ "in the next major release");
builder.enableLatLon(XContentMapValues.lenientNodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("precision_step")) {
deprecationLogger.deprecated(CONTENT_TYPE + " precision_step parameter is deprecated and will be removed "
+ "in the next major release");
builder.precisionStep(XContentMapValues.nodeIntegerValue(propNode));
iterator.remove();
} else if (propName.equals("geohash")) {

View File

@ -250,7 +250,8 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
GeoPoint luceneTopLeft = new GeoPoint(topLeft);
GeoPoint luceneBottomRight = new GeoPoint(bottomRight);
if (GeoValidationMethod.isCoerce(validationMethod)) {
final Version indexVersionCreated = context.indexVersionCreated();
if (indexVersionCreated.onOrAfter(Version.V_2_2_0) || GeoValidationMethod.isCoerce(validationMethod)) {
// Special case: if the difference between the left and right is 360 and the right is greater than the left, we are asking for
// the complete longitude range so need to set longitude to the complete longitude range
double right = luceneBottomRight.getLon();
@ -265,7 +266,6 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBounding
}
}
final Version indexVersionCreated = context.indexVersionCreated();
if (indexVersionCreated.onOrAfter(Version.V_2_2_0)) {
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
// if index created V_2_3 > use prefix encoded postings format

View File

@ -219,18 +219,18 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
throw new QueryShardException(shardContext, "field [" + fieldName + "] is not a geo_point field");
}
final Version indexVersionCreated = shardContext.indexVersionCreated();
QueryValidationException exception = checkLatLon(shardContext.indexVersionCreated().before(Version.V_2_0_0));
if (exception != null) {
throw new QueryShardException(shardContext, "couldn't validate latitude/ longitude values", exception);
}
if (GeoValidationMethod.isCoerce(validationMethod)) {
if (indexVersionCreated.onOrAfter(Version.V_2_2_0) || GeoValidationMethod.isCoerce(validationMethod)) {
GeoUtils.normalizePoint(center, true, true);
}
double normDistance = geoDistance.normalize(this.distance, DistanceUnit.DEFAULT);
final Version indexVersionCreated = shardContext.indexVersionCreated();
if (indexVersionCreated.before(Version.V_2_2_0)) {
GeoPointFieldMapperLegacy.GeoPointFieldType geoFieldType = ((GeoPointFieldMapperLegacy.GeoPointFieldType) fieldType);
IndexGeoPointFieldData indexFieldData = shardContext.getForField(fieldType);

View File

@ -120,9 +120,6 @@ public class GeoDistanceQueryParser implements QueryParser<GeoDistanceQueryBuild
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.LON_SUFFIX)) {
point.resetLon(parser.doubleValue());
fieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.LON_SUFFIX.length());
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.GEOHASH_SUFFIX)) {
point.resetFromGeoHash(parser.text());
fieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.GEOHASH_SUFFIX.length());
} else if (parseContext.parseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.NAME_FIELD)) {
queryName = parser.text();
} else if (parseContext.parseFieldMatcher().match(currentFieldName, AbstractQueryBuilder.BOOST_FIELD)) {

View File

@ -236,7 +236,7 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
}
GeoPoint point = new GeoPoint(this.point);
if (GeoValidationMethod.isCoerce(validationMethod)) {
if (indexCreatedBeforeV2_2 == false || GeoValidationMethod.isCoerce(validationMethod)) {
GeoUtils.normalizePoint(point, true, true);
}

View File

@ -196,15 +196,6 @@ public class GeoDistanceRangeQueryParser implements QueryParser<GeoDistanceRange
point = new GeoPoint();
}
point.resetLon(parser.doubleValue());
} else if (currentFieldName.endsWith(GeoPointFieldMapper.Names.GEOHASH_SUFFIX)) {
String maybeFieldName = currentFieldName.substring(0, currentFieldName.length() - GeoPointFieldMapper.Names.GEOHASH_SUFFIX.length());
if (fieldName == null || fieldName.equals(maybeFieldName)) {
fieldName = maybeFieldName;
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + GeoDistanceRangeQueryBuilder.NAME +
"] field name already set to [" + fieldName + "] but found [" + currentFieldName + "]");
}
point = GeoPoint.fromGeohash(parser.text());
} else if (parseContext.parseFieldMatcher().match(currentFieldName, NAME_FIELD)) {
queryName = parser.text();
} else if (parseContext.parseFieldMatcher().match(currentFieldName, BOOST_FIELD)) {

View File

@ -131,13 +131,13 @@ public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQuery
}
}
if (GeoValidationMethod.isCoerce(validationMethod)) {
final Version indexVersionCreated = context.indexVersionCreated();
if (indexVersionCreated.onOrAfter(Version.V_2_2_0) || GeoValidationMethod.isCoerce(validationMethod)) {
for (GeoPoint point : shell) {
GeoUtils.normalizePoint(point, true, true);
}
}
final Version indexVersionCreated = context.indexVersionCreated();
if (indexVersionCreated.before(Version.V_2_2_0)) {
IndexGeoPointFieldData indexFieldData = context.getForField(fieldType);
return new GeoPolygonQuery(indexFieldData, shell.toArray(new GeoPoint[shellSize]));

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.QueryShardException;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -61,8 +60,8 @@ public class NestedInnerQueryParseSupport {
protected ObjectMapper nestedObjectMapper;
private ObjectMapper parentObjectMapper;
public NestedInnerQueryParseSupport(XContentParser parser, SearchContext searchContext) {
shardContext = searchContext.getQueryShardContext();
public NestedInnerQueryParseSupport(XContentParser parser, QueryShardContext context) {
shardContext = context;
parseContext = shardContext.parseContext();
shardContext.reset(parser);

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.similarity;
import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.Version;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexModule;
@ -63,6 +64,10 @@ public final class SimilarityService extends AbstractIndexComponent {
Map<String, Settings> similaritySettings = this.indexSettings.getSettings().getGroups(IndexModule.SIMILARITY_SETTINGS_PREFIX);
for (Map.Entry<String, Settings> entry : similaritySettings.entrySet()) {
String name = entry.getKey();
// Starting with v5.0 indices, it should no longer be possible to redefine built-in similarities
if(BUILT_IN.containsKey(name) && indexSettings.getIndexVersionCreated().onOrAfter(Version.V_5_0_0)) {
throw new IllegalArgumentException("Cannot redefine built-in Similarity [" + name + "]");
}
Settings settings = entry.getValue();
String typeName = settings.get("type");
if (typeName == null) {
@ -76,9 +81,16 @@ public final class SimilarityService extends AbstractIndexComponent {
}
providers.put(name, factory.apply(name, settings));
}
addSimilarities(similaritySettings, providers, DEFAULTS);
for (Map.Entry<String, SimilarityProvider> entry : addSimilarities(similaritySettings, DEFAULTS).entrySet()) {
// Avoid overwriting custom providers for indices older that v5.0
if (providers.containsKey(entry.getKey()) && indexSettings.getIndexVersionCreated().before(Version.V_5_0_0)) {
continue;
}
providers.put(entry.getKey(), entry.getValue());
}
this.similarities = providers;
defaultSimilarity = providers.get(SimilarityService.DEFAULT_SIMILARITY).get();
defaultSimilarity = (providers.get("default") != null) ? providers.get("default").get()
: providers.get(SimilarityService.DEFAULT_SIMILARITY).get();
// Expert users can configure the base type as being different to default, but out-of-box we use default.
baseSimilarity = (providers.get("base") != null) ? providers.get("base").get() :
defaultSimilarity;
@ -90,7 +102,9 @@ public final class SimilarityService extends AbstractIndexComponent {
defaultSimilarity;
}
private void addSimilarities(Map<String, Settings> similaritySettings, Map<String, SimilarityProvider> providers, Map<String, BiFunction<String, Settings, SimilarityProvider>> similarities) {
private Map<String, SimilarityProvider> addSimilarities(Map<String, Settings> similaritySettings,
Map<String, BiFunction<String, Settings, SimilarityProvider>> similarities) {
Map<String, SimilarityProvider> providers = new HashMap<>(similarities.size());
for (Map.Entry<String, BiFunction<String, Settings, SimilarityProvider>> entry : similarities.entrySet()) {
String name = entry.getKey();
BiFunction<String, Settings, SimilarityProvider> factory = entry.getValue();
@ -100,12 +114,17 @@ public final class SimilarityService extends AbstractIndexComponent {
}
providers.put(name, factory.apply(name, settings));
}
return providers;
}
public SimilarityProvider getSimilarity(String name) {
return similarities.get(name);
}
public SimilarityProvider getDefaultSimilarity() {
return similarities.get("default");
}
static class PerFieldSimilarity extends PerFieldSimilarityWrapper {
private final Similarity defaultSimilarity;

View File

@ -103,6 +103,7 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Predicate;
import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
@ -185,14 +186,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
ExecutorService indicesStopExecutor = Executors.newFixedThreadPool(5, EsExecutors.daemonThreadFactory("indices_shutdown"));
// Copy indices because we modify it asynchronously in the body of the loop
Set<String> indices = new HashSet<>(this.indices.keySet());
final Set<Index> indices = this.indices.values().stream().map(s -> s.index()).collect(Collectors.toSet());
final CountDownLatch latch = new CountDownLatch(indices.size());
for (final String index : indices) {
for (final Index index : indices) {
indicesStopExecutor.execute(() -> {
try {
removeIndex(index, "shutdown", false);
} catch (Throwable e) {
logger.warn("failed to remove index on stop [" + index + "]", e);
logger.warn("failed to remove index on stop " + index + "", e);
} finally {
latch.countDown();
}
@ -256,7 +257,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
Map<Index, List<IndexShardStats>> statsByShard = new HashMap<>();
for (IndexService indexService : indices.values()) {
for (IndexService indexService : this) {
for (IndexShard indexShard : indexService) {
try {
if (indexShard.routingEntry() == null) {
@ -290,17 +291,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
return indices.values().iterator();
}
public boolean hasIndex(String index) {
return indices.containsKey(index);
}
/**
* Returns an IndexService for the specified index if exists otherwise returns <code>null</code>.
*
*/
@Nullable
public IndexService indexService(String index) {
return indices.get(index);
public boolean hasIndex(Index index) {
return indices.containsKey(index.getUUID());
}
/**
@ -309,33 +301,21 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
*/
@Nullable
public IndexService indexService(Index index) {
return indexService(index.getName());
}
/**
* Returns an IndexService for the specified index if exists otherwise a {@link IndexNotFoundException} is thrown.
*/
public IndexService indexServiceSafe(String index) {
IndexService indexService = indexService(index);
if (indexService == null) {
throw new IndexNotFoundException(index);
}
return indexService;
return indices.get(index.getUUID());
}
/**
* Returns an IndexService for the specified index if exists otherwise a {@link IndexNotFoundException} is thrown.
*/
public IndexService indexServiceSafe(Index index) {
IndexService indexService = indexServiceSafe(index.getName());
if (indexService.indexUUID().equals(index.getUUID()) == false) {
IndexService indexService = indices.get(index.getUUID());
if (indexService == null) {
throw new IndexNotFoundException(index);
}
assert indexService.indexUUID().equals(index.getUUID()) : "uuid mismatch local: " + indexService.indexUUID() + " incoming: " + index.getUUID();
return indexService;
}
/**
* Creates a new {@link IndexService} for the given metadata.
* @param indexMetaData the index metadata to create the index for
@ -346,10 +326,13 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
if (!lifecycle.started()) {
throw new IllegalStateException("Can't create an index [" + indexMetaData.getIndex() + "], node is closed");
}
if (indexMetaData.getIndexUUID().equals(IndexMetaData.INDEX_UUID_NA_VALUE)) {
throw new IllegalArgumentException("index must have a real UUID found value: [" + indexMetaData.getIndexUUID() + "]");
}
final Index index = indexMetaData.getIndex();
final Predicate<String> indexNameMatcher = (indexExpression) -> indexNameExpressionResolver.matchesIndex(index.getName(), indexExpression, clusterService.state());
final IndexSettings idxSettings = new IndexSettings(indexMetaData, this.settings, indexNameMatcher, indexScopeSetting);
if (indices.containsKey(index.getName())) {
if (hasIndex(index)) {
throw new IndexAlreadyExistsException(index);
}
logger.debug("creating Index [{}], shards [{}]/[{}{}]",
@ -378,7 +361,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
try {
assert indexService.getIndexEventListener() == listener;
listener.afterIndexCreated(indexService);
indices = newMapBuilder(indices).put(index.getName(), indexService).immutableMap();
indices = newMapBuilder(indices).put(index.getUUID(), indexService).immutableMap();
success = true;
return indexService;
} finally {
@ -395,22 +378,24 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @param index the index to remove
* @param reason the high level reason causing this removal
*/
public void removeIndex(String index, String reason) {
public void removeIndex(Index index, String reason) {
removeIndex(index, reason, false);
}
private void removeIndex(String index, String reason, boolean delete) {
private void removeIndex(Index index, String reason, boolean delete) {
final String indexName = index.getName();
try {
final IndexService indexService;
final IndexEventListener listener;
synchronized (this) {
if (indices.containsKey(index) == false) {
if (hasIndex(index) == false) {
return;
}
logger.debug("[{}] closing ... (reason [{}])", index, reason);
logger.debug("[{}] closing ... (reason [{}])", indexName, reason);
Map<String, IndexService> newIndices = new HashMap<>(indices);
indexService = newIndices.remove(index);
indexService = newIndices.remove(index.getUUID());
assert indexService != null : "IndexService is null for index: " + index;
indices = unmodifiableMap(newIndices);
listener = indexService.getIndexEventListener();
}
@ -419,9 +404,9 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
if (delete) {
listener.beforeIndexDeleted(indexService);
}
logger.debug("[{}] closing index service (reason [{}])", index, reason);
logger.debug("{} closing index service (reason [{}])", index, reason);
indexService.close(reason, delete);
logger.debug("[{}] closed... (reason [{}])", index, reason);
logger.debug("{} closed... (reason [{}])", index, reason);
listener.afterIndexClosed(indexService.index(), indexService.getIndexSettings().getSettings());
if (delete) {
final IndexSettings indexSettings = indexService.getIndexSettings();
@ -474,12 +459,12 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* Deletes the given index. Persistent parts of the index
* like the shards files, state and transaction logs are removed once all resources are released.
*
* Equivalent to {@link #removeIndex(String, String)} but fires
* Equivalent to {@link #removeIndex(Index, String)} but fires
* different lifecycle events to ensure pending resources of this index are immediately removed.
* @param index the index to delete
* @param reason the high level reason causing this delete
*/
public void deleteIndex(String index, String reason) throws IOException {
public void deleteIndex(Index index, String reason) throws IOException {
removeIndex(index, reason, true);
}
@ -505,16 +490,17 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
public void deleteIndexStore(String reason, IndexMetaData metaData, ClusterState clusterState, boolean closed) throws IOException {
if (nodeEnv.hasNodeFile()) {
synchronized (this) {
String indexName = metaData.getIndex().getName();
if (indices.containsKey(indexName)) {
String localUUid = indices.get(indexName).indexUUID();
throw new IllegalStateException("Can't delete index store for [" + indexName + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]");
Index index = metaData.getIndex();
if (hasIndex(index)) {
String localUUid = indexService(index).indexUUID();
throw new IllegalStateException("Can't delete index store for [" + index.getName() + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]");
}
if (clusterState.metaData().hasIndex(indexName) && (clusterState.nodes().localNode().masterNode() == true)) {
if (clusterState.metaData().hasIndex(index.getName()) && (clusterState.nodes().localNode().masterNode() == true)) {
// we do not delete the store if it is a master eligible node and the index is still in the cluster state
// because we want to keep the meta data for indices around even if no shards are left here
final IndexMetaData index = clusterState.metaData().index(indexName);
throw new IllegalStateException("Can't delete closed index store for [" + indexName + "] - it's still part of the cluster state [" + index.getIndexUUID() + "] [" + metaData.getIndexUUID() + "]");
final IndexMetaData idxMeta = clusterState.metaData().index(index.getName());
throw new IllegalStateException("Can't delete closed index store for [" + index.getName() + "] - it's still part of the cluster state [" + idxMeta.getIndexUUID() + "] [" + metaData.getIndexUUID() + "]");
}
}
final IndexSettings indexSettings = buildIndexSettings(metaData);
@ -607,7 +593,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @return true if the index can be deleted on this node
*/
public boolean canDeleteIndexContents(Index index, IndexSettings indexSettings, boolean closed) {
final IndexService indexService = this.indices.get(index.getName());
final IndexService indexService = indexService(index);
// Closed indices may be deleted, even if they are on a shared
// filesystem. Since it is closed we aren't deleting it for relocation
if (indexSettings.isOnSharedFilesystem() == false || closed) {
@ -634,7 +620,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
*/
public boolean canDeleteShardContent(ShardId shardId, IndexSettings indexSettings) {
assert shardId.getIndex().equals(indexSettings.getIndex());
final IndexService indexService = this.indices.get(shardId.getIndexName());
final IndexService indexService = indexService(shardId.getIndex());
if (indexSettings.isOnSharedFilesystem() == false) {
if (indexService != null && nodeEnv.hasNodeFile()) {
return indexService.hasShard(shardId.id()) == false;

View File

@ -46,6 +46,7 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexShardAlreadyExistsException;
@ -157,13 +158,13 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// TODO: this feels a bit hacky here, a block disables state persistence, and then we clean the allocated shards, maybe another flag in blocks?
if (event.state().blocks().disableStatePersistence()) {
for (IndexService indexService : indicesService) {
String index = indexService.index().getName();
Index index = indexService.index();
for (Integer shardId : indexService.shardIds()) {
logger.debug("[{}][{}] removing shard (disabled block persistence)", index, shardId);
logger.debug("{}[{}] removing shard (disabled block persistence)", index, shardId);
try {
indexService.removeShard(shardId, "removing shard (disabled block persistence)");
} catch (Throwable e) {
logger.warn("[{}] failed to remove shard (disabled block persistence)", e, index);
logger.warn("{} failed to remove shard (disabled block persistence)", e, index);
}
}
removeIndex(index, "cleaning index (disabled block persistence)");
@ -201,10 +202,10 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
}
}
for (IndexService indexService : indicesService) {
String index = indexService.index().getName();
Index index = indexService.index();
if (indexService.shardIds().isEmpty()) {
if (logger.isDebugEnabled()) {
logger.debug("[{}] cleaning index (no shards allocated)", index);
logger.debug("{} cleaning index (no shards allocated)", index);
}
// clean the index
removeIndex(index, "removing index (no shards allocated)");
@ -222,12 +223,12 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (indexMetaData != null) {
if (!indexMetaData.isSameUUID(indexService.indexUUID())) {
logger.debug("[{}] mismatch on index UUIDs between cluster state and local state, cleaning the index so it will be recreated", indexMetaData.getIndex());
deleteIndex(indexMetaData.getIndex().getName(), "mismatch on index UUIDs between cluster state and local state, cleaning the index so it will be recreated");
deleteIndex(indexMetaData.getIndex(), "mismatch on index UUIDs between cluster state and local state, cleaning the index so it will be recreated");
}
}
}
for (String index : event.indicesDeleted()) {
for (Index index : event.indicesDeleted()) {
if (logger.isDebugEnabled()) {
logger.debug("[{}] cleaning index, no longer part of the metadata", index);
}
@ -243,7 +244,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
indicesService.deleteClosedIndex("closed index no longer part of the metadata", metaData, event.state());
}
try {
nodeIndexDeletedAction.nodeIndexDeleted(event.state(), index, indexSettings, localNodeId);
nodeIndexDeletedAction.nodeIndexDeleted(event.state(), index.getName(), indexSettings, localNodeId);
} catch (Throwable e) {
logger.debug("failed to send to master index {} deleted event", e, index);
}
@ -298,7 +299,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
return;
}
for (ShardRouting shard : routingNode) {
if (!indicesService.hasIndex(shard.getIndexName())) {
if (!indicesService.hasIndex(shard.index())) {
final IndexMetaData indexMetaData = event.state().metaData().index(shard.index());
if (logger.isDebugEnabled()) {
logger.debug("[{}] creating index", indexMetaData.getIndex());
@ -317,7 +318,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
return;
}
for (IndexMetaData indexMetaData : event.state().metaData()) {
if (!indicesService.hasIndex(indexMetaData.getIndex().getName())) {
if (!indicesService.hasIndex(indexMetaData.getIndex())) {
// we only create / update here
continue;
}
@ -325,7 +326,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (!event.indexMetaDataChanged(indexMetaData)) {
continue;
}
String index = indexMetaData.getIndex().getName();
Index index = indexMetaData.getIndex();
IndexService indexService = indicesService.indexService(index);
if (indexService == null) {
// already deleted on us, ignore it
@ -339,12 +340,12 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private void applyMappings(ClusterChangedEvent event) {
// go over and update mappings
for (IndexMetaData indexMetaData : event.state().metaData()) {
if (!indicesService.hasIndex(indexMetaData.getIndex().getName())) {
Index index = indexMetaData.getIndex();
if (!indicesService.hasIndex(index)) {
// we only create / update here
continue;
}
boolean requireRefresh = false;
String index = indexMetaData.getIndex().getName();
IndexService indexService = indicesService.indexService(index);
if (indexService == null) {
// got deleted on us, ignore (closing the node)
@ -357,11 +358,11 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
MappingMetaData mappingMd = cursor.value;
String mappingType = mappingMd.type();
CompressedXContent mappingSource = mappingMd.source();
requireRefresh |= processMapping(index, mapperService, mappingType, mappingSource);
requireRefresh |= processMapping(index.getName(), mapperService, mappingType, mappingSource);
}
if (requireRefresh && sendRefreshMapping) {
nodeMappingRefreshAction.nodeMappingRefresh(event.state(),
new NodeMappingRefreshAction.NodeMappingRefreshRequest(index, indexMetaData.getIndexUUID(),
new NodeMappingRefreshAction.NodeMappingRefreshRequest(index.getName(), indexMetaData.getIndexUUID(),
event.state().nodes().localNodeId())
);
}
@ -727,7 +728,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
}
}
private void removeIndex(String index, String reason) {
private void removeIndex(Index index, String reason) {
try {
indicesService.removeIndex(index, reason);
} catch (Throwable e) {
@ -735,7 +736,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
}
}
private void deleteIndex(String index, String reason) {
private void deleteIndex(Index index, String reason) {
try {
indicesService.deleteIndex(index, reason);
} catch (Throwable e) {
@ -774,7 +775,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private class FailedShardHandler implements Callback<IndexShard.ShardFailure> {
@Override
public void handle(final IndexShard.ShardFailure shardFailure) {
final IndexService indexService = indicesService.indexService(shardFailure.routing.shardId().getIndex().getName());
final IndexService indexService = indicesService.indexService(shardFailure.routing.shardId().getIndex());
final ShardRouting shardRouting = shardFailure.routing;
threadPool.generic().execute(() -> {
synchronized (mutex) {

View File

@ -83,7 +83,7 @@ public class RecoverySource extends AbstractComponent implements IndexEventListe
}
private RecoveryResponse recover(final StartRecoveryRequest request) throws IOException {
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex().getName());
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
final IndexShard shard = indexService.getShard(request.shardId().id());
// starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise

View File

@ -137,7 +137,7 @@ public class RecoverySourceHandler {
}
}
logger.trace("snapshot translog for recovery. current size is [{}]", translogView.totalOperations());
logger.trace("{} snapshot translog for recovery. current size is [{}]", shard.shardId(), translogView.totalOperations());
try {
phase2(translogView.snapshot());
} catch (Throwable e) {

View File

@ -348,7 +348,7 @@ public class IndicesStore extends AbstractComponent implements ClusterStateListe
return null;
}
ShardId shardId = request.shardId;
IndexService indexService = indicesService.indexService(shardId.getIndexName());
IndexService indexService = indicesService.indexService(shardId.getIndex());
if (indexService != null && indexService.indexUUID().equals(request.indexUUID)) {
return indexService.getShardOrNull(shardId.id());
}

View File

@ -126,7 +126,7 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesAction<T
@Override
protected NodeStoreFilesMetaData nodeOperation(NodeRequest request) {
if (request.unallocated) {
IndexService indexService = indicesService.indexService(request.shardId.getIndexName());
IndexService indexService = indicesService.indexService(request.shardId.getIndex());
if (indexService == null) {
return new NodeStoreFilesMetaData(clusterService.localNode(), null);
}
@ -150,7 +150,7 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesAction<T
long startTimeNS = System.nanoTime();
boolean exists = false;
try {
IndexService indexService = indicesService.indexService(shardId.getIndexName());
IndexService indexService = indicesService.indexService(shardId.getIndex());
if (indexService != null) {
IndexShard indexShard = indexService.getShardOrNull(shardId.id());
if (indexShard != null) {

View File

@ -84,7 +84,6 @@ public class NodeService extends AbstractComponent implements Closeable {
this.transportService = transportService;
this.indicesService = indicesService;
this.discovery = discovery;
discovery.setNodeService(this);
this.version = version;
this.pluginService = pluginService;
this.circuitBreakerService = circuitBreakerService;

View File

@ -52,10 +52,10 @@ public class RestCancelTasksAction extends BaseRestHandler {
TaskId parentTaskId = new TaskId(request.param("parent_task_id"));
CancelTasksRequest cancelTasksRequest = new CancelTasksRequest();
cancelTasksRequest.taskId(taskId);
cancelTasksRequest.nodesIds(nodesIds);
cancelTasksRequest.actions(actions);
cancelTasksRequest.parentTaskId(parentTaskId);
cancelTasksRequest.setTaskId(taskId);
cancelTasksRequest.setNodesIds(nodesIds);
cancelTasksRequest.setActions(actions);
cancelTasksRequest.setParentTaskId(parentTaskId);
client.admin().cluster().cancelTasks(cancelTasksRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -50,13 +50,15 @@ public class RestListTasksAction extends BaseRestHandler {
TaskId taskId = new TaskId(request.param("taskId"));
String[] actions = Strings.splitStringByCommaToArray(request.param("actions"));
TaskId parentTaskId = new TaskId(request.param("parent_task_id"));
boolean waitForCompletion = request.paramAsBoolean("wait_for_completion", false);
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.taskId(taskId);
listTasksRequest.nodesIds(nodesIds);
listTasksRequest.detailed(detailed);
listTasksRequest.actions(actions);
listTasksRequest.parentTaskId(parentTaskId);
listTasksRequest.setTaskId(taskId);
listTasksRequest.setNodesIds(nodesIds);
listTasksRequest.setDetailed(detailed);
listTasksRequest.setActions(actions);
listTasksRequest.setParentTaskId(parentTaskId);
listTasksRequest.setWaitForCompletion(waitForCompletion);
client.admin().cluster().listTasks(listTasksRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -55,7 +55,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.query.TemplateQueryParser;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.lookup.SearchLookup;
import org.elasticsearch.watcher.FileChangesListener;
import org.elasticsearch.watcher.FileWatcher;
@ -225,6 +224,8 @@ public class ScriptService extends AbstractComponent implements Closeable {
return scriptEngineService;
}
/**
* Checks if a script can be executed and compiles it if needed, or returns the previously compiled and cached script.
*/
@ -516,35 +517,43 @@ public class ScriptService extends AbstractComponent implements Closeable {
private class ScriptChangesListener extends FileChangesListener {
private Tuple<String, String> scriptNameExt(Path file) {
private Tuple<String, String> getScriptNameExt(Path file) {
Path scriptPath = scriptsDirectory.relativize(file);
int extIndex = scriptPath.toString().lastIndexOf('.');
if (extIndex != -1) {
String ext = scriptPath.toString().substring(extIndex + 1);
String scriptName = scriptPath.toString().substring(0, extIndex).replace(scriptPath.getFileSystem().getSeparator(), "_");
return new Tuple<>(scriptName, ext);
} else {
if (extIndex <= 0) {
return null;
}
String ext = scriptPath.toString().substring(extIndex + 1);
if (ext.isEmpty()) {
return null;
}
String scriptName = scriptPath.toString().substring(0, extIndex).replace(scriptPath.getFileSystem().getSeparator(), "_");
return new Tuple<>(scriptName, ext);
}
@Override
public void onFileInit(Path file) {
Tuple<String, String> scriptNameExt = getScriptNameExt(file);
if (scriptNameExt == null) {
logger.debug("Skipped script with invalid extension : [{}]", file);
return;
}
if (logger.isTraceEnabled()) {
logger.trace("Loading script file : [{}]", file);
}
Tuple<String, String> scriptNameExt = scriptNameExt(file);
if (scriptNameExt != null) {
ScriptEngineService engineService = getScriptEngineServiceForFileExt(scriptNameExt.v2());
if (engineService == null) {
logger.warn("no script engine found for [{}]", scriptNameExt.v2());
logger.warn("No script engine found for [{}]", scriptNameExt.v2());
} else {
try {
//we don't know yet what the script will be used for, but if all of the operations for this lang
// with file scripts are disabled, it makes no sense to even compile it and cache it.
if (isAnyScriptContextEnabled(engineService.getTypes().get(0), engineService, ScriptType.FILE)) {
logger.info("compiling script file [{}]", file.toAbsolutePath());
try(InputStreamReader reader = new InputStreamReader(Files.newInputStream(file), StandardCharsets.UTF_8)) {
try (InputStreamReader reader = new InputStreamReader(Files.newInputStream(file), StandardCharsets.UTF_8)) {
String script = Streams.copyToString(reader);
CacheKey cacheKey = new CacheKey(engineService, scriptNameExt.v1(), null, Collections.emptyMap());
staticCache.put(cacheKey, new CompiledScript(ScriptType.FILE, scriptNameExt.v1(), engineService.getTypes().get(0), engineService.compile(script, Collections.emptyMap())));
@ -558,7 +567,6 @@ public class ScriptService extends AbstractComponent implements Closeable {
}
}
}
}
@Override
public void onFileCreated(Path file) {
@ -567,7 +575,7 @@ public class ScriptService extends AbstractComponent implements Closeable {
@Override
public void onFileDeleted(Path file) {
Tuple<String, String> scriptNameExt = scriptNameExt(file);
Tuple<String, String> scriptNameExt = getScriptNameExt(file);
if (scriptNameExt != null) {
ScriptEngineService engineService = getScriptEngineServiceForFileExt(scriptNameExt.v2());
assert engineService != null;

View File

@ -45,7 +45,7 @@ public class SearchException extends ElasticsearchException implements Elasticse
public SearchException(StreamInput in) throws IOException {
super(in);
if (in.readBoolean()) {
shardTarget = SearchShardTarget.readSearchShardTarget(in);
shardTarget = new SearchShardTarget(in);
} else {
shardTarget = null;
}
@ -54,7 +54,12 @@ public class SearchException extends ElasticsearchException implements Elasticse
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeOptionalStreamable(shardTarget);
if (shardTarget == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
shardTarget.writeTo(out);
}
}
public SearchShardTarget shard() {

View File

@ -534,10 +534,9 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
}
final SearchContext createContext(ShardSearchRequest request, @Nullable Engine.Searcher searcher) {
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard indexShard = indexService.getShard(request.shardId());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), indexShard.shardId().getIndex(), request.shardId());
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.getShard(request.shardId().getId());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), indexShard.shardId());
Engine.Searcher engineSearcher = searcher == null ? indexShard.acquireSearcher("search") : searcher;

View File

@ -23,28 +23,38 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
* The target that the search request was executed on.
*/
public class SearchShardTarget implements Streamable, Comparable<SearchShardTarget> {
public class SearchShardTarget implements Writeable<SearchShardTarget>, Comparable<SearchShardTarget> {
private Text nodeId;
private Text index;
private int shardId;
private ShardId shardId;
private SearchShardTarget() {
public SearchShardTarget(StreamInput in) throws IOException {
if (in.readBoolean()) {
nodeId = in.readText();
}
shardId = ShardId.readShardId(in);
index = new Text(shardId.getIndexName());
}
public SearchShardTarget(String nodeId, ShardId shardId) {
this.nodeId = nodeId == null ? null : new Text(nodeId);
this.index = new Text(shardId.getIndexName());
this.shardId = shardId;
}
public SearchShardTarget(String nodeId, Index index, int shardId) {
this.nodeId = nodeId == null ? null : new Text(nodeId);
this.index = new Text(index.getName());
this.shardId = shardId;
this(nodeId, new ShardId(index, shardId));
}
@Nullable
@ -73,36 +83,26 @@ public class SearchShardTarget implements Streamable, Comparable<SearchShardTarg
return this.index;
}
public int shardId() {
public ShardId shardId() {
return shardId;
}
public int getShardId() {
public ShardId getShardId() {
return shardId;
}
public static SearchShardTarget readSearchShardTarget(StreamInput in) throws IOException {
SearchShardTarget result = new SearchShardTarget();
result.readFrom(in);
return result;
}
@Override
public int compareTo(SearchShardTarget o) {
int i = index.string().compareTo(o.index());
if (i == 0) {
i = shardId - o.shardId;
i = shardId.getId() - o.shardId.id();
}
return i;
}
@Override
public void readFrom(StreamInput in) throws IOException {
if (in.readBoolean()) {
nodeId = in.readText();
}
index = in.readText();
shardId = in.readVInt();
public SearchShardTarget readFrom(StreamInput in) throws IOException {
return new SearchShardTarget(in);
}
@Override
@ -113,19 +113,15 @@ public class SearchShardTarget implements Streamable, Comparable<SearchShardTarg
out.writeBoolean(true);
out.writeText(nodeId);
}
out.writeText(index);
out.writeVInt(shardId);
shardId.writeTo(out);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
SearchShardTarget that = (SearchShardTarget) o;
if (shardId != that.shardId) return false;
if (index != null ? !index.equals(that.index) : that.index != null) return false;
if (shardId.equals(that.shardId) == false) return false;
if (nodeId != null ? !nodeId.equals(that.nodeId) : that.nodeId != null) return false;
return true;
@ -135,7 +131,7 @@ public class SearchShardTarget implements Streamable, Comparable<SearchShardTarg
public int hashCode() {
int result = nodeId != null ? nodeId.hashCode() : 0;
result = 31 * result + (index != null ? index.hashCode() : 0);
result = 31 * result + shardId;
result = 31 * result + shardId.hashCode();
return result;
}

View File

@ -76,7 +76,7 @@ public class SearchPhaseController extends AbstractComponent {
public int compare(AtomicArray.Entry<? extends QuerySearchResultProvider> o1, AtomicArray.Entry<? extends QuerySearchResultProvider> o2) {
int i = o1.value.shardTarget().index().compareTo(o2.value.shardTarget().index());
if (i == 0) {
i = o1.value.shardTarget().shardId() - o2.value.shardTarget().shardId();
i = o1.value.shardTarget().shardId().id() - o2.value.shardTarget().shardId().id();
}
return i;
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.transport.TransportResponse;
import java.io.IOException;
import static org.elasticsearch.search.SearchShardTarget.readSearchShardTarget;
import static org.elasticsearch.search.fetch.QueryFetchSearchResult.readQueryFetchSearchResult;
/**
@ -56,7 +55,7 @@ public class ScrollQueryFetchSearchResult extends TransportResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shardTarget = readSearchShardTarget(in);
shardTarget = new SearchShardTarget(in);
result = readQueryFetchSearchResult(in);
result.shardTarget(shardTarget);
}

View File

@ -55,7 +55,6 @@ import static java.util.Collections.singletonMap;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.lucene.Lucene.readExplanation;
import static org.elasticsearch.common.lucene.Lucene.writeExplanation;
import static org.elasticsearch.search.SearchShardTarget.readSearchShardTarget;
import static org.elasticsearch.search.highlight.HighlightField.readHighlightField;
import static org.elasticsearch.search.internal.InternalSearchHitField.readSearchHitField;
@ -638,7 +637,7 @@ public class InternalSearchHit implements SearchHit {
if (context.streamShardTarget() == ShardTargetType.STREAM) {
if (in.readBoolean()) {
shard = readSearchShardTarget(in);
shard = new SearchShardTarget(in);
}
} else if (context.streamShardTarget() == ShardTargetType.LOOKUP) {
int lookupId = in.readVInt();

View File

@ -34,7 +34,6 @@ import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.Map;
import static org.elasticsearch.search.SearchShardTarget.readSearchShardTarget;
import static org.elasticsearch.search.internal.InternalSearchHit.readSearchHit;
/**
@ -216,7 +215,7 @@ public class InternalSearchHits implements SearchHits {
// read the lookup table first
int lookupSize = in.readVInt();
for (int i = 0; i < lookupSize; i++) {
context.handleShardLookup().put(in.readVInt(), readSearchShardTarget(in));
context.handleShardLookup().put(in.readVInt(), new SearchShardTarget(in));
}
}

View File

@ -58,8 +58,7 @@ import static org.elasticsearch.search.Scroll.readScroll;
public class ShardSearchLocalRequest implements ShardSearchRequest {
private String index;
private int shardId;
private ShardId shardId;
private int numberOfShards;
private SearchType searchType;
private Scroll scroll;
@ -97,8 +96,7 @@ public class ShardSearchLocalRequest implements ShardSearchRequest {
public ShardSearchLocalRequest(ShardId shardId, int numberOfShards, SearchType searchType, SearchSourceBuilder source, String[] types,
Boolean requestCache) {
this.index = shardId.getIndexName();
this.shardId = shardId.id();
this.shardId = shardId;
this.numberOfShards = numberOfShards;
this.searchType = searchType;
this.source = source;
@ -106,13 +104,9 @@ public class ShardSearchLocalRequest implements ShardSearchRequest {
this.requestCache = requestCache;
}
@Override
public String index() {
return index;
}
@Override
public int shardId() {
public ShardId shardId() {
return shardId;
}
@ -177,8 +171,7 @@ public class ShardSearchLocalRequest implements ShardSearchRequest {
@SuppressWarnings("unchecked")
protected void innerReadFrom(StreamInput in) throws IOException {
index = in.readString();
shardId = in.readVInt();
shardId = ShardId.readShardId(in);
searchType = SearchType.fromId(in.readByte());
numberOfShards = in.readVInt();
if (in.readBoolean()) {
@ -195,8 +188,7 @@ public class ShardSearchLocalRequest implements ShardSearchRequest {
}
protected void innerWriteTo(StreamOutput out, boolean asKey) throws IOException {
out.writeString(index);
out.writeVInt(shardId);
shardId.writeTo(out);
out.writeByte(searchType.id());
if (!asKey) {
out.writeVInt(numberOfShards);

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.internal;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.script.Template;
import org.elasticsearch.search.Scroll;
import org.elasticsearch.search.builder.SearchSourceBuilder;
@ -34,9 +35,7 @@ import java.io.IOException;
*/
public interface ShardSearchRequest {
String index();
int shardId();
ShardId shardId();
String[] types();

View File

@ -28,6 +28,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.script.Template;
import org.elasticsearch.search.Scroll;
import org.elasticsearch.search.builder.SearchSourceBuilder;
@ -71,13 +72,9 @@ public class ShardSearchTransportRequest extends TransportRequest implements Sha
return originalIndices.indicesOptions();
}
@Override
public String index() {
return shardSearchLocalRequest.index();
}
@Override
public int shardId() {
public ShardId shardId() {
return shardSearchLocalRequest.shardId();
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.transport.TransportResponse;
import java.io.IOException;
import static org.elasticsearch.search.SearchShardTarget.readSearchShardTarget;
import static org.elasticsearch.search.query.QuerySearchResult.readQuerySearchResult;
/**
@ -56,7 +55,7 @@ public class ScrollQuerySearchResult extends TransportResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shardTarget = readSearchShardTarget(in);
shardTarget = new SearchShardTarget(in);
queryResult = readQuerySearchResult(in);
queryResult.shardTarget(shardTarget);
}

View File

@ -27,16 +27,12 @@ import java.io.IOException;
/**
* A sort builder to sort based on a document field.
*/
public class FieldSortBuilder extends SortBuilder {
public class FieldSortBuilder extends SortBuilder<FieldSortBuilder> {
private final String fieldName;
private SortOrder order;
private Object missing;
private Boolean ignoreUnmapped;
private String unmappedType;
private String sortMode;
@ -57,36 +53,15 @@ public class FieldSortBuilder extends SortBuilder {
this.fieldName = fieldName;
}
/**
* The order of sorting. Defaults to {@link SortOrder#ASC}.
*/
@Override
public FieldSortBuilder order(SortOrder order) {
this.order = order;
return this;
}
/**
* Sets the value when a field is missing in a doc. Can also be set to <tt>_last</tt> or
* <tt>_first</tt> to sort missing last or first respectively.
*/
@Override
public FieldSortBuilder missing(Object missing) {
this.missing = missing;
return this;
}
/**
* Sets if the field does not exists in the index, it should be ignored and not sorted by or not. Defaults
* to <tt>false</tt> (not ignoring).
* @deprecated Use {@link #unmappedType(String)} instead.
*/
@Deprecated
public FieldSortBuilder ignoreUnmapped(boolean ignoreUnmapped) {
this.ignoreUnmapped = ignoreUnmapped;
return this;
}
/**
* Set the type to use in case the current field is not mapped in an index.
* Specifying a type tells Elasticsearch what type the sort values should have, which is important
@ -132,15 +107,10 @@ public class FieldSortBuilder extends SortBuilder {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(fieldName);
if (order != null) {
builder.field("order", order.toString());
}
builder.field(ORDER_FIELD.getPreferredName(), order);
if (missing != null) {
builder.field("missing", missing);
}
if (ignoreUnmapped != null) {
builder.field(SortParseElement.IGNORE_UNMAPPED.getPreferredName(), ignoreUnmapped);
}
if (unmappedType != null) {
builder.field(SortParseElement.UNMAPPED_TYPE.getPreferredName(), unmappedType);
}

View File

@ -44,7 +44,7 @@ import java.util.Objects;
/**
* A geo distance based sorting on a geo point like field.
*/
public class GeoDistanceSortBuilder extends SortBuilder
public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
implements ToXContent, NamedWriteable<GeoDistanceSortBuilder>, SortElementParserTemp<GeoDistanceSortBuilder> {
public static final String NAME = "_geo_distance";
public static final boolean DEFAULT_COERCE = false;
@ -57,7 +57,6 @@ public class GeoDistanceSortBuilder extends SortBuilder
private GeoDistance geoDistance = GeoDistance.DEFAULT;
private DistanceUnit unit = DistanceUnit.DEFAULT;
private SortOrder order = SortOrder.ASC;
// TODO there is an enum that covers that parameter which we should be using here
private String sortMode = null;
@ -204,30 +203,6 @@ public class GeoDistanceSortBuilder extends SortBuilder
return this.unit;
}
/**
* The order of sorting. Defaults to {@link SortOrder#ASC}.
*/
@Override
public GeoDistanceSortBuilder order(SortOrder order) {
this.order = order;
return this;
}
/** Returns the order of sorting. */
public SortOrder order() {
return this.order;
}
/**
* Not relevant.
*
* TODO should this throw an exception rather than silently ignore a parameter that is not used?
*/
@Override
public GeoDistanceSortBuilder missing(Object missing) {
return this;
}
/**
* Defines which distance to use for sorting in the case a document contains multiple geo points.
* Possible values: min and max
@ -250,7 +225,7 @@ public class GeoDistanceSortBuilder extends SortBuilder
* Sets the nested filter that the nested objects should match with in order to be taken into account
* for sorting.
*/
public GeoDistanceSortBuilder setNestedFilter(QueryBuilder nestedFilter) {
public GeoDistanceSortBuilder setNestedFilter(QueryBuilder<?> nestedFilter) {
this.nestedFilter = nestedFilter;
return this;
}
@ -259,7 +234,7 @@ public class GeoDistanceSortBuilder extends SortBuilder
* Returns the nested filter that the nested objects should match with in order to be taken into account
* for sorting.
**/
public QueryBuilder getNestedFilter() {
public QueryBuilder<?> getNestedFilter() {
return this.nestedFilter;
}
@ -312,11 +287,7 @@ public class GeoDistanceSortBuilder extends SortBuilder
builder.field("unit", unit);
builder.field("distance_type", geoDistance.name().toLowerCase(Locale.ROOT));
if (order == SortOrder.DESC) {
builder.field("reverse", true);
} else {
builder.field("reverse", false);
}
builder.field(ORDER_FIELD.getPreferredName(), order);
if (sortMode != null) {
builder.field("mode", sortMode);
@ -419,9 +390,9 @@ public class GeoDistanceSortBuilder extends SortBuilder
List<GeoPoint> geoPoints = new ArrayList<>();
DistanceUnit unit = DistanceUnit.DEFAULT;
GeoDistance geoDistance = GeoDistance.DEFAULT;
boolean reverse = false;
SortOrder order = SortOrder.ASC;
MultiValueMode sortMode = null;
QueryBuilder nestedFilter = null;
QueryBuilder<?> nestedFilter = null;
String nestedPath = null;
boolean coerce = GeoDistanceSortBuilder.DEFAULT_COERCE;
@ -451,9 +422,9 @@ public class GeoDistanceSortBuilder extends SortBuilder
}
} else if (token.isValue()) {
if ("reverse".equals(currentName)) {
reverse = parser.booleanValue();
order = parser.booleanValue() ? SortOrder.DESC : SortOrder.ASC;
} else if ("order".equals(currentName)) {
reverse = "desc".equals(parser.text());
order = SortOrder.fromString(parser.text());
} else if ("unit".equals(currentName)) {
unit = DistanceUnit.fromString(parser.text());
} else if ("distance_type".equals(currentName) || "distanceType".equals(currentName)) {
@ -484,11 +455,7 @@ public class GeoDistanceSortBuilder extends SortBuilder
GeoDistanceSortBuilder result = new GeoDistanceSortBuilder(fieldName, geoPoints.toArray(new GeoPoint[geoPoints.size()]));
result.geoDistance(geoDistance);
result.unit(unit);
if (reverse) {
result.order(SortOrder.DESC);
} else {
result.order(SortOrder.ASC);
}
result.order(order);
if (sortMode != null) {
result.sortMode(sortMode.name());
}

View File

@ -43,9 +43,9 @@ import org.elasticsearch.index.fielddata.MultiGeoPointValues;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.ArrayList;
@ -62,7 +62,7 @@ public class GeoDistanceSortParser implements SortParser {
}
@Override
public SortField parse(XContentParser parser, SearchContext context) throws Exception {
public SortField parse(XContentParser parser, QueryShardContext context) throws Exception {
String fieldName = null;
List<GeoPoint> geoPoints = new ArrayList<>();
DistanceUnit unit = DistanceUnit.DEFAULT;
@ -71,7 +71,7 @@ public class GeoDistanceSortParser implements SortParser {
MultiValueMode sortMode = null;
NestedInnerQueryParseSupport nestedHelper = null;
final boolean indexCreatedBeforeV2_0 = context.indexShard().indexSettings().getIndexVersionCreated().before(Version.V_2_0_0);
final boolean indexCreatedBeforeV2_0 = context.indexVersionCreated().before(Version.V_2_0_0);
boolean coerce = GeoDistanceSortBuilder.DEFAULT_COERCE;
boolean ignoreMalformed = GeoDistanceSortBuilder.DEFAULT_IGNORE_MALFORMED;
@ -155,12 +155,12 @@ public class GeoDistanceSortParser implements SortParser {
throw new IllegalArgumentException("sort_mode [sum] isn't supported for sorting by geo distance");
}
MappedFieldType fieldType = context.smartNameFieldType(fieldName);
MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType == null) {
throw new IllegalArgumentException("failed to find mapper for [" + fieldName + "] for geo distance based sort");
}
final MultiValueMode finalSortMode = sortMode; // final reference for use in the anonymous class
final IndexGeoPointFieldData geoIndexFieldData = context.fieldData().getForField(fieldType);
final IndexGeoPointFieldData geoIndexFieldData = context.getForField(fieldType);
final FixedSourceDistance[] distances = new FixedSourceDistance[geoPoints.size()];
for (int i = 0; i< geoPoints.size(); i++) {
distances[i] = geoDistance.fixedSourceDistance(geoPoints.get(i).lat(), geoPoints.get(i).lon(), unit);
@ -168,15 +168,16 @@ public class GeoDistanceSortParser implements SortParser {
final Nested nested;
if (nestedHelper != null && nestedHelper.getPath() != null) {
BitSetProducer rootDocumentsFilter = context.bitsetFilterCache().getBitSetProducer(Queries.newNonNestedFilter());
Query innerDocumentsFilter;
BitSetProducer rootDocumentsFilter = context.bitsetFilter(Queries.newNonNestedFilter());
Query innerDocumentsQuery;
if (nestedHelper.filterFound()) {
// TODO: use queries instead
innerDocumentsFilter = nestedHelper.getInnerFilter();
innerDocumentsQuery = nestedHelper.getInnerFilter();
} else {
innerDocumentsFilter = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
innerDocumentsQuery = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
}
nested = new Nested(rootDocumentsFilter, context.searcher().createNormalizedWeight(innerDocumentsFilter, false));
nested = new Nested(rootDocumentsFilter, innerDocumentsQuery);
} else {
nested = null;
}

View File

@ -19,40 +19,103 @@
package org.elasticsearch.search.sort;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.io.stream.NamedWriteable;
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.QueryParseContext;
import java.io.IOException;
import java.util.Objects;
/**
* A sort builder allowing to sort by score.
*
*
*/
public class ScoreSortBuilder extends SortBuilder {
public class ScoreSortBuilder extends SortBuilder<ScoreSortBuilder> implements NamedWriteable<ScoreSortBuilder>,
SortElementParserTemp<ScoreSortBuilder> {
private SortOrder order;
private static final String NAME = "_score";
static final ScoreSortBuilder PROTOTYPE = new ScoreSortBuilder();
public static final ParseField REVERSE_FIELD = new ParseField("reverse");
public static final ParseField ORDER_FIELD = new ParseField("order");
/**
* The order of sort scoring. By default, its {@link SortOrder#DESC}.
*/
@Override
public ScoreSortBuilder order(SortOrder order) {
this.order = order;
return this;
public ScoreSortBuilder() {
// order defaults to desc when sorting on the _score
order(SortOrder.DESC);
}
@Override
public SortBuilder missing(Object missing) {
return this;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject("_score");
if (order == SortOrder.ASC) {
builder.field("reverse", true);
}
builder.startObject(NAME);
builder.field(ORDER_FIELD.getPreferredName(), order);
builder.endObject();
return builder;
}
@Override
public ScoreSortBuilder fromXContent(QueryParseContext context, String elementName) throws IOException {
XContentParser parser = context.parser();
ParseFieldMatcher matcher = context.parseFieldMatcher();
XContentParser.Token token;
String currentName = parser.currentName();
ScoreSortBuilder result = new ScoreSortBuilder();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentName = parser.currentName();
} else if (token.isValue()) {
if (matcher.match(currentName, REVERSE_FIELD)) {
if (parser.booleanValue()) {
result.order(SortOrder.ASC);
}
// else we keep the default DESC
} else if (matcher.match(currentName, ORDER_FIELD)) {
result.order(SortOrder.fromString(parser.text()));
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + NAME + "] failed to parse field [" + currentName + "]");
}
} else {
throw new ParsingException(parser.getTokenLocation(), "[" + NAME + "] unexpected token [" + token + "]");
}
}
return result;
}
@Override
public boolean equals(Object object) {
if (this == object) {
return true;
}
if (object == null || getClass() != object.getClass()) {
return false;
}
ScoreSortBuilder other = (ScoreSortBuilder) object;
return Objects.equals(order, other.order);
}
@Override
public int hashCode() {
return Objects.hash(this.order);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
order.writeTo(out);
}
@Override
public ScoreSortBuilder readFrom(StreamInput in) throws IOException {
ScoreSortBuilder builder = new ScoreSortBuilder().order(SortOrder.readOrderFrom(in));
return builder;
}
@Override
public String getWriteableName() {
return NAME;
}
}

View File

@ -28,14 +28,12 @@ import java.io.IOException;
/**
* Script sort builder allows to sort based on a custom script expression.
*/
public class ScriptSortBuilder extends SortBuilder {
public class ScriptSortBuilder extends SortBuilder<ScriptSortBuilder> {
private Script script;
private final String type;
private SortOrder order;
private String sortMode;
private QueryBuilder nestedFilter;
@ -53,23 +51,6 @@ public class ScriptSortBuilder extends SortBuilder {
this.type = type;
}
/**
* Sets the sort order.
*/
@Override
public ScriptSortBuilder order(SortOrder order) {
this.order = order;
return this;
}
/**
* Not really relevant.
*/
@Override
public SortBuilder missing(Object missing) {
return this;
}
/**
* Defines which distance to use for sorting in the case a document contains multiple geo points.
* Possible values: min and max
@ -83,7 +64,7 @@ public class ScriptSortBuilder extends SortBuilder {
* Sets the nested filter that the nested objects should match with in order to be taken into account
* for sorting.
*/
public ScriptSortBuilder setNestedFilter(QueryBuilder nestedFilter) {
public ScriptSortBuilder setNestedFilter(QueryBuilder<?> nestedFilter) {
this.nestedFilter = nestedFilter;
return this;
}
@ -102,9 +83,7 @@ public class ScriptSortBuilder extends SortBuilder {
builder.startObject("_script");
builder.field("script", script);
builder.field("type", type);
if (order == SortOrder.DESC) {
builder.field("reverse", true);
}
builder.field(ORDER_FIELD.getPreferredName(), order);
if (sortMode != null) {
builder.field("mode", sortMode);
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.SortField;
import org.apache.lucene.search.join.BitSetProducer;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.FieldData;
@ -37,6 +38,7 @@ import org.elasticsearch.index.fielddata.SortedBinaryDocValues;
import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.index.fielddata.fieldcomparator.BytesRefFieldComparatorSource;
import org.elasticsearch.index.fielddata.fieldcomparator.DoubleValuesComparatorSource;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.script.LeafSearchScript;
import org.elasticsearch.script.Script;
@ -68,7 +70,7 @@ public class ScriptSortParser implements SortParser {
}
@Override
public SortField parse(XContentParser parser, SearchContext context) throws Exception {
public SortField parse(XContentParser parser, QueryShardContext context) throws Exception {
ScriptParameterParser scriptParameterParser = new ScriptParameterParser();
Script script = null;
String type = null;
@ -122,19 +124,20 @@ public class ScriptSortParser implements SortParser {
script = new Script(scriptValue.script(), scriptValue.scriptType(), scriptParameterParser.lang(), params);
}
} else if (params != null) {
throw new SearchParseException(context, "script params must be specified inside script object", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "script params must be specified inside script object");
}
if (script == null) {
throw new SearchParseException(context, "_script sorting requires setting the script to sort by", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "_script sorting requires setting the script to sort by");
}
if (type == null) {
throw new SearchParseException(context, "_script sorting requires setting the type of the script", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "_script sorting requires setting the type of the script");
}
final SearchScript searchScript = context.scriptService().search(context.lookup(), script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
final SearchScript searchScript = context.getScriptService().search(
context.lookup(), script, ScriptContext.Standard.SEARCH, Collections.emptyMap());
if (STRING_SORT_TYPE.equals(type) && (sortMode == MultiValueMode.SUM || sortMode == MultiValueMode.AVG)) {
throw new SearchParseException(context, "type [string] doesn't support mode [" + sortMode + "]", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "type [string] doesn't support mode [" + sortMode + "]");
}
if (sortMode == null) {
@ -144,7 +147,7 @@ public class ScriptSortParser implements SortParser {
// If nested_path is specified, then wrap the `fieldComparatorSource` in a `NestedFieldComparatorSource`
final Nested nested;
if (nestedHelper != null && nestedHelper.getPath() != null) {
BitSetProducer rootDocumentsFilter = context.bitsetFilterCache().getBitSetProducer(Queries.newNonNestedFilter());
BitSetProducer rootDocumentsFilter = context.bitsetFilter(Queries.newNonNestedFilter());
Query innerDocumentsFilter;
if (nestedHelper.filterFound()) {
// TODO: use queries instead
@ -152,7 +155,7 @@ public class ScriptSortParser implements SortParser {
} else {
innerDocumentsFilter = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
}
nested = new Nested(rootDocumentsFilter, context.searcher().createNormalizedWeight(innerDocumentsFilter, false));
nested = new Nested(rootDocumentsFilter, innerDocumentsFilter);
} else {
nested = null;
}
@ -205,7 +208,7 @@ public class ScriptSortParser implements SortParser {
};
break;
default:
throw new SearchParseException(context, "custom script sort type [" + type + "] not supported", parser.getTokenLocation());
throw new ParsingException(parser.getTokenLocation(), "custom script sort type [" + type + "] not supported");
}
return new SortField("_script", fieldComparatorSource, reverse);

View File

@ -20,14 +20,20 @@
package org.elasticsearch.search.sort;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import java.util.Objects;
/**
*
*/
public abstract class SortBuilder implements ToXContent {
public abstract class SortBuilder<T extends SortBuilder<?>> implements ToXContent {
protected SortOrder order = SortOrder.ASC;
public static final ParseField ORDER_FIELD = new ParseField("order");
@Override
public String toString() {
@ -42,13 +48,19 @@ public abstract class SortBuilder implements ToXContent {
}
/**
* The order of sorting. Defaults to {@link SortOrder#ASC}.
* Set the order of sorting.
*/
public abstract SortBuilder order(SortOrder order);
@SuppressWarnings("unchecked")
public T order(SortOrder order) {
Objects.requireNonNull(order, "sort order cannot be null.");
this.order = order;
return (T) this;
}
/**
* Sets the value when a field is missing in a doc. Can also be set to <tt>_last</tt> or
* <tt>_first</tt> to sort missing last or first respectively.
* Return the {@link SortOrder} used for this {@link SortBuilder}.
*/
public abstract SortBuilder missing(Object missing);
public SortOrder order() {
return this.order;
}
}

View File

@ -19,13 +19,12 @@
package org.elasticsearch.search.sort;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.index.query.QueryParseContext;
import java.io.IOException;
// TODO once sort refactoring is done this needs to be merged into SortBuilder
public interface SortElementParserTemp<T extends ToXContent> {
public interface SortElementParserTemp<T extends SortBuilder> {
/**
* Creates a new SortBuilder from the json held by the {@link SortElementParserTemp}
* in {@link org.elasticsearch.common.xcontent.XContent} format

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.core.LongFieldMapper;
import org.elasticsearch.index.query.support.NestedInnerQueryParseSupport;
import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.SearchParseElement;
@ -55,7 +54,6 @@ public class SortParseElement implements SearchParseElement {
private static final SortField SORT_DOC = new SortField(null, SortField.Type.DOC);
private static final SortField SORT_DOC_REVERSE = new SortField(null, SortField.Type.DOC, true);
public static final ParseField IGNORE_UNMAPPED = new ParseField("ignore_unmapped");
public static final ParseField UNMAPPED_TYPE = new ParseField("unmapped_type");
public static final String SCORE_FIELD_NAME = "_score";
@ -140,7 +138,7 @@ public class SortParseElement implements SearchParseElement {
addSortField(context, sortFields, fieldName, reverse, unmappedType, missing, sortMode, nestedFilterParseHelper);
} else {
if (PARSERS.containsKey(fieldName)) {
sortFields.add(PARSERS.get(fieldName).parse(parser, context));
sortFields.add(PARSERS.get(fieldName).parse(parser, context.getQueryShardContext()));
} else {
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
@ -156,19 +154,13 @@ public class SortParseElement implements SearchParseElement {
}
} else if ("missing".equals(innerJsonName)) {
missing = parser.textOrNull();
} else if (context.parseFieldMatcher().match(innerJsonName, IGNORE_UNMAPPED)) {
// backward compatibility: ignore_unmapped has been replaced with unmapped_type
if (unmappedType == null // don't override if unmapped_type has been provided too
&& parser.booleanValue()) {
unmappedType = LongFieldMapper.CONTENT_TYPE;
}
} else if (context.parseFieldMatcher().match(innerJsonName, UNMAPPED_TYPE)) {
unmappedType = parser.textOrNull();
} else if ("mode".equals(innerJsonName)) {
sortMode = MultiValueMode.fromString(parser.text());
} else if ("nested_path".equals(innerJsonName) || "nestedPath".equals(innerJsonName)) {
if (nestedFilterParseHelper == null) {
nestedFilterParseHelper = new NestedInnerQueryParseSupport(parser, context);
nestedFilterParseHelper = new NestedInnerQueryParseSupport(parser, context.getQueryShardContext());
}
nestedFilterParseHelper.setPath(parser.text());
} else {
@ -177,7 +169,7 @@ public class SortParseElement implements SearchParseElement {
} else if (token == XContentParser.Token.START_OBJECT) {
if ("nested_filter".equals(innerJsonName) || "nestedFilter".equals(innerJsonName)) {
if (nestedFilterParseHelper == null) {
nestedFilterParseHelper = new NestedInnerQueryParseSupport(parser, context);
nestedFilterParseHelper = new NestedInnerQueryParseSupport(parser, context.getQueryShardContext());
}
nestedFilterParseHelper.filter();
} else {
@ -239,14 +231,13 @@ public class SortParseElement implements SearchParseElement {
final Nested nested;
if (nestedHelper != null && nestedHelper.getPath() != null) {
BitSetProducer rootDocumentsFilter = context.bitsetFilterCache().getBitSetProducer(Queries.newNonNestedFilter());
Query innerDocumentsFilter;
Query innerDocumentsQuery;
if (nestedHelper.filterFound()) {
// TODO: use queries instead
innerDocumentsFilter = nestedHelper.getInnerFilter();
innerDocumentsQuery = nestedHelper.getInnerFilter();
} else {
innerDocumentsFilter = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
innerDocumentsQuery = nestedHelper.getNestedObjectMapper().nestedTypeFilter();
}
nested = new Nested(rootDocumentsFilter, context.searcher().createNormalizedWeight(innerDocumentsFilter, false));
nested = new Nested(rootDocumentsFilter, innerDocumentsQuery);
} else {
nested = null;
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.search.sort;
import org.apache.lucene.search.SortField;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.index.query.QueryShardContext;
/**
*
@ -30,5 +30,5 @@ public interface SortParser {
String[] names();
SortField parse(XContentParser parser, SearchContext context) throws Exception;
SortField parse(XContentParser parser, QueryShardContext context) throws Exception;
}

View File

@ -222,7 +222,7 @@ public class ThreadPool extends AbstractComponent implements Closeable {
int halfProcMaxAt5 = Math.min(((availableProcessors + 1) / 2), 5);
int halfProcMaxAt10 = Math.min(((availableProcessors + 1) / 2), 10);
Map<String, Settings> defaultExecutorTypeSettings = new HashMap<>();
add(defaultExecutorTypeSettings, new ExecutorSettingsBuilder(Names.GENERIC).keepAlive("30s"));
add(defaultExecutorTypeSettings, new ExecutorSettingsBuilder(Names.GENERIC).size(4 * availableProcessors).keepAlive("30s"));
add(defaultExecutorTypeSettings, new ExecutorSettingsBuilder(Names.INDEX).size(availableProcessors).queueSize(200));
add(defaultExecutorTypeSettings, new ExecutorSettingsBuilder(Names.BULK).size(availableProcessors).queueSize(50));
add(defaultExecutorTypeSettings, new ExecutorSettingsBuilder(Names.GET).size(availableProcessors).queueSize(1000));

View File

@ -41,6 +41,8 @@ import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
@ -52,7 +54,10 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.gateway.GatewayService;
import org.elasticsearch.node.Node;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.TransportSettings;
import java.util.Arrays;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
@ -154,6 +159,15 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
public static final Set<String> TRIBE_SETTING_KEYS = Sets.newHashSet(TRIBE_NAME_SETTING.getKey(), ON_CONFLICT_SETTING.getKey(),
BLOCKS_METADATA_INDICES_SETTING.getKey(), BLOCKS_METADATA_SETTING.getKey(), BLOCKS_READ_INDICES_SETTING.getKey(), BLOCKS_WRITE_INDICES_SETTING.getKey(), BLOCKS_WRITE_SETTING.getKey());
// these settings should be passed through to each tribe client, if they are not set explicitly
private static final List<Setting<?>> PASS_THROUGH_SETTINGS = Arrays.asList(
NetworkService.GLOBAL_NETWORK_HOST_SETTING,
NetworkService.GLOBAL_NETWORK_BINDHOST_SETTING,
NetworkService.GLOBAL_NETWORK_PUBLISHHOST_SETTING,
TransportSettings.HOST,
TransportSettings.BIND_HOST,
TransportSettings.PUBLISH_HOST
);
private final String onConflict;
private final Set<String> droppedIndices = ConcurrentCollections.newConcurrentSet();
@ -167,18 +181,8 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
nodesSettings.remove("blocks"); // remove prefix settings that don't indicate a client
nodesSettings.remove("on_conflict"); // remove prefix settings that don't indicate a client
for (Map.Entry<String, Settings> entry : nodesSettings.entrySet()) {
Settings.Builder sb = Settings.builder().put(entry.getValue());
sb.put("node.name", settings.get("node.name") + "/" + entry.getKey());
sb.put(Environment.PATH_HOME_SETTING.getKey(), Environment.PATH_HOME_SETTING.get(settings)); // pass through ES home dir
if (Environment.PATH_CONF_SETTING.exists(settings)) {
sb.put(Environment.PATH_CONF_SETTING.getKey(), Environment.PATH_CONF_SETTING.get(settings));
}
sb.put(TRIBE_NAME_SETTING.getKey(), entry.getKey());
if (sb.get("http.enabled") == null) {
sb.put("http.enabled", false);
}
sb.put(Node.NODE_CLIENT_SETTING.getKey(), true);
nodes.add(new TribeClientNode(sb.build()));
Settings clientSettings = buildClientSettings(entry.getKey(), settings, entry.getValue());
nodes.add(new TribeClientNode(clientSettings));
}
this.blockIndicesMetadata = BLOCKS_METADATA_INDICES_SETTING.get(settings).toArray(Strings.EMPTY_ARRAY);
@ -197,6 +201,46 @@ public class TribeService extends AbstractLifecycleComponent<TribeService> {
this.onConflict = ON_CONFLICT_SETTING.get(settings);
}
// pkg private for testing
/**
* Builds node settings for a tribe client node from the tribe node's global settings,
* combined with tribe specific settings.
*/
static Settings buildClientSettings(String tribeName, Settings globalSettings, Settings tribeSettings) {
for (String tribeKey : tribeSettings.getAsMap().keySet()) {
if (tribeKey.startsWith("path.")) {
throw new IllegalArgumentException("Setting [" + tribeKey + "] not allowed in tribe client [" + tribeName + "]");
}
}
Settings.Builder sb = Settings.builder().put(tribeSettings);
sb.put("node.name", globalSettings.get("node.name") + "/" + tribeName);
sb.put(Environment.PATH_HOME_SETTING.getKey(), Environment.PATH_HOME_SETTING.get(globalSettings)); // pass through ES home dir
if (Environment.PATH_CONF_SETTING.exists(globalSettings)) {
sb.put(Environment.PATH_CONF_SETTING.getKey(), Environment.PATH_CONF_SETTING.get(globalSettings));
}
if (Environment.PATH_PLUGINS_SETTING.exists(globalSettings)) {
sb.put(Environment.PATH_PLUGINS_SETTING.getKey(), Environment.PATH_PLUGINS_SETTING.get(globalSettings));
}
if (Environment.PATH_LOGS_SETTING.exists(globalSettings)) {
sb.put(Environment.PATH_LOGS_SETTING.getKey(), Environment.PATH_LOGS_SETTING.get(globalSettings));
}
if (Environment.PATH_SCRIPTS_SETTING.exists(globalSettings)) {
sb.put(Environment.PATH_SCRIPTS_SETTING.getKey(), Environment.PATH_SCRIPTS_SETTING.get(globalSettings));
}
for (Setting<?> passthrough : PASS_THROUGH_SETTINGS) {
if (passthrough.exists(tribeSettings) == false && passthrough.exists(globalSettings)) {
sb.put(passthrough.getKey(), globalSettings.get(passthrough.getKey()));
}
}
sb.put(TRIBE_NAME_SETTING.getKey(), tribeName);
if (sb.get(NetworkModule.HTTP_ENABLED.getKey()) == null) {
sb.put(NetworkModule.HTTP_ENABLED.getKey(), false);
}
sb.put(Node.NODE_CLIENT_SETTING.getKey(), true);
return sb.build();
}
@Override
protected void doStart() {
if (nodes.isEmpty() == false) {

View File

@ -237,8 +237,8 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Cancel main task
CancelTasksRequest request = new CancelTasksRequest();
request.reason("Testing Cancellation");
request.taskId(new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()));
request.setReason("Testing Cancellation");
request.setTaskId(new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()));
// And send the cancellation request to a random node
CancelTasksResponse response = testNodes[randomIntBetween(0, testNodes.length - 1)].transportCancelTasksAction.execute(request)
.get();
@ -270,7 +270,7 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Make sure that tasks are no longer running
ListTasksResponse listTasksResponse = testNodes[randomIntBetween(0, testNodes.length - 1)]
.transportListTasksAction.execute(new ListTasksRequest().taskId(
.transportListTasksAction.execute(new ListTasksRequest().setTaskId(
new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()))).get();
assertEquals(0, listTasksResponse.getTasks().size());
@ -313,7 +313,7 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Make sure that tasks are running
ListTasksResponse listTasksResponse = testNodes[randomIntBetween(0, testNodes.length - 1)]
.transportListTasksAction.execute(new ListTasksRequest().parentTaskId(new TaskId(mainNode, mainTask.getId()))).get();
.transportListTasksAction.execute(new ListTasksRequest().setParentTaskId(new TaskId(mainNode, mainTask.getId()))).get();
assertThat(listTasksResponse.getTasks().size(), greaterThanOrEqualTo(blockOnNodes.size()));
// Simulate the coordinating node leaving the cluster
@ -331,8 +331,8 @@ public class CancellableTasksTests extends TaskManagerTestCase {
logger.info("--> Simulate issuing cancel request on the node that is about to leave the cluster");
// Simulate issuing cancel request on the node that is about to leave the cluster
CancelTasksRequest request = new CancelTasksRequest();
request.reason("Testing Cancellation");
request.taskId(new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()));
request.setReason("Testing Cancellation");
request.setTaskId(new TaskId(testNodes[0].discoveryNode.getId(), mainTask.getId()));
// And send the cancellation request to a random node
CancelTasksResponse response = testNodes[0].transportCancelTasksAction.execute(request).get();
logger.info("--> Done simulating issuing cancel request on the node that is about to leave the cluster");
@ -356,7 +356,7 @@ public class CancellableTasksTests extends TaskManagerTestCase {
// Make sure that tasks are no longer running
try {
ListTasksResponse listTasksResponse1 = testNodes[randomIntBetween(1, testNodes.length - 1)]
.transportListTasksAction.execute(new ListTasksRequest().taskId(new TaskId(mainNode, mainTask.getId()))).get();
.transportListTasksAction.execute(new ListTasksRequest().setTaskId(new TaskId(mainNode, mainTask.getId()))).get();
assertEquals(0, listTasksResponse1.getTasks().size());
} catch (InterruptedException ex) {
Thread.currentThread().interrupt();

View File

@ -18,6 +18,8 @@
*/
package org.elasticsearch.action.admin.cluster.node.tasks;
import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.ListenableActionFuture;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
@ -40,6 +42,7 @@ import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.tasks.MockTaskManager;
import org.elasticsearch.test.tasks.MockTaskManagerListener;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
import java.io.IOException;
import java.util.ArrayList;
@ -54,8 +57,11 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantLock;
import java.util.function.Function;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.hamcrest.Matchers.either;
import static org.hamcrest.Matchers.emptyCollectionOf;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.not;
@ -327,6 +333,78 @@ public class TasksIT extends ESIntegTestCase {
assertEquals(0, client().admin().cluster().prepareListTasks().setActions(TestTaskPlugin.TestTaskAction.NAME + "[n]").get().getTasks().size());
}
public void testTasksListWaitForCompletion() throws Exception {
// Start blocking test task
ListenableActionFuture<TestTaskPlugin.NodesResponse> future = TestTaskPlugin.TestTaskAction.INSTANCE.newRequestBuilder(client())
.execute();
ListenableActionFuture<ListTasksResponse> waitResponseFuture;
try {
// Wait for the task to start on all nodes
assertBusy(() -> assertEquals(internalCluster().numDataAndMasterNodes(),
client().admin().cluster().prepareListTasks().setActions(TestTaskPlugin.TestTaskAction.NAME + "[n]").get().getTasks().size()));
// Spin up a request to wait for that task to finish
waitResponseFuture = client().admin().cluster().prepareListTasks()
.setActions(TestTaskPlugin.TestTaskAction.NAME + "[n]").setWaitForCompletion(true).execute();
} finally {
// Unblock the request so the wait for completion request can finish
TestTaskPlugin.UnblockTestTasksAction.INSTANCE.newRequestBuilder(client()).get();
}
// Now that the task is unblocked the list response will come back
ListTasksResponse waitResponse = waitResponseFuture.get();
// If any tasks come back then they are the tasks we asked for - it'd be super weird if this wasn't true
for (TaskInfo task: waitResponse.getTasks()) {
assertEquals(task.getAction(), TestTaskPlugin.TestTaskAction.NAME + "[n]");
}
// See the next test to cover the timeout case
future.get();
}
public void testTasksListWaitForTimeout() throws Exception {
// Start blocking test task
ListenableActionFuture<TestTaskPlugin.NodesResponse> future = TestTaskPlugin.TestTaskAction.INSTANCE.newRequestBuilder(client())
.execute();
try {
// Wait for the task to start on all nodes
assertBusy(() -> assertEquals(internalCluster().numDataAndMasterNodes(),
client().admin().cluster().prepareListTasks().setActions(TestTaskPlugin.TestTaskAction.NAME + "[n]").get().getTasks().size()));
// Spin up a request that should wait for those tasks to finish
// It will timeout because we haven't unblocked the tasks
ListTasksResponse waitResponse = client().admin().cluster().prepareListTasks()
.setActions(TestTaskPlugin.TestTaskAction.NAME + "[n]").setWaitForCompletion(true).setTimeout(timeValueMillis(100))
.get();
assertFalse(waitResponse.getNodeFailures().isEmpty());
for (FailedNodeException failure : waitResponse.getNodeFailures()) {
Throwable timeoutException = failure.getCause();
// The exception sometimes comes back wrapped depending on the client
if (timeoutException.getCause() != null) {
timeoutException = timeoutException.getCause();
}
assertThat(timeoutException,
either(instanceOf(ElasticsearchTimeoutException.class)).or(instanceOf(ReceiveTimeoutTransportException.class)));
}
} finally {
// Now we can unblock those requests
TestTaskPlugin.UnblockTestTasksAction.INSTANCE.newRequestBuilder(client()).get();
}
future.get();
}
public void testTasksListWaitForNoTask() throws Exception {
// Spin up a request to wait for no matching tasks
ListenableActionFuture<ListTasksResponse> waitResponseFuture = client().admin().cluster().prepareListTasks()
.setActions(TestTaskPlugin.TestTaskAction.NAME + "[n]").setWaitForCompletion(true).setTimeout(timeValueMillis(10))
.execute();
// It should finish quickly and without complaint
assertThat(waitResponseFuture.get().getTasks(), emptyCollectionOf(TaskInfo.class));
}
@Override
public void tearDown() throws Exception {
for (Map.Entry<Tuple<String, String>, RecordingTaskManagerListener> entry : listeners.entrySet()) {

View File

@ -345,7 +345,10 @@ public class TestTaskPlugin extends Plugin {
public static class UnblockTestTasksRequest extends BaseTasksRequest<UnblockTestTasksRequest> {
@Override
public boolean match(Task task) {
return task instanceof TestTask && super.match(task);
}
}
public static class UnblockTestTasksResponse extends BaseTasksResponse {

View File

@ -355,7 +355,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
int testNodeNum = randomIntBetween(0, testNodes.length - 1);
TestNode testNode = testNodes[testNodeNum];
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.actions("testAction*"); // pick all test actions
listTasksRequest.setActions("testAction*"); // pick all test actions
logger.info("Listing currently running tasks using node [{}]", testNodeNum);
ListTasksResponse response = testNode.transportListTasksAction.execute(listTasksRequest).get();
logger.info("Checking currently running tasks");
@ -371,7 +371,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Check task counts using transport with filtering
testNode = testNodes[randomIntBetween(0, testNodes.length - 1)];
listTasksRequest = new ListTasksRequest();
listTasksRequest.actions("testAction[n]"); // only pick node actions
listTasksRequest.setActions("testAction[n]"); // only pick node actions
response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getPerNodeTasks().size());
for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
@ -380,7 +380,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
}
// Check task counts using transport with detailed description
listTasksRequest.detailed(true); // same request only with detailed description
listTasksRequest.setDetailed(true); // same request only with detailed description
response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getPerNodeTasks().size());
for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
@ -389,7 +389,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
}
// Make sure that the main task on coordinating node is the task that was returned to us by execute()
listTasksRequest.actions("testAction"); // only pick the main task
listTasksRequest.setActions("testAction"); // only pick the main task
response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(1, response.getTasks().size());
assertEquals(mainTask.getId(), response.getTasks().get(0).getId());
@ -417,7 +417,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Get the parent task
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.actions("testAction");
listTasksRequest.setActions("testAction");
ListTasksResponse response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(1, response.getTasks().size());
String parentNode = response.getTasks().get(0).getNode().getId();
@ -425,7 +425,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Find tasks with common parent
listTasksRequest = new ListTasksRequest();
listTasksRequest.parentTaskId(new TaskId(parentNode, parentTaskId));
listTasksRequest.setParentTaskId(new TaskId(parentNode, parentTaskId));
response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getTasks().size());
for (TaskInfo task : response.getTasks()) {
@ -451,7 +451,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Get the parent task
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.actions("testAction*");
listTasksRequest.setActions("testAction*");
ListTasksResponse response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(0, response.getTasks().size());
@ -472,7 +472,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Check task counts using transport with filtering
TestNode testNode = testNodes[randomIntBetween(0, testNodes.length - 1)];
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.actions("testAction[n]"); // only pick node actions
listTasksRequest.setActions("testAction[n]"); // only pick node actions
ListTasksResponse response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getPerNodeTasks().size());
for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
@ -482,7 +482,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Check task counts using transport with detailed description
long minimalDurationNanos = System.nanoTime() - maximumStartTimeNanos;
listTasksRequest.detailed(true); // same request only with detailed description
listTasksRequest.setDetailed(true); // same request only with detailed description
response = testNode.transportListTasksAction.execute(listTasksRequest).get();
assertEquals(testNodes.length, response.getPerNodeTasks().size());
for (Map.Entry<DiscoveryNode, List<TaskInfo>> entry : response.getPerNodeTasks().entrySet()) {
@ -518,9 +518,9 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Try to cancel main task using action name
CancelTasksRequest request = new CancelTasksRequest();
request.nodesIds(testNodes[0].discoveryNode.getId());
request.reason("Testing Cancellation");
request.actions(actionName);
request.setNodesIds(testNodes[0].discoveryNode.getId());
request.setReason("Testing Cancellation");
request.setActions(actionName);
CancelTasksResponse response = testNodes[randomIntBetween(0, testNodes.length - 1)].transportCancelTasksAction.execute(request)
.get();
@ -532,8 +532,8 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Try to cancel main task using id
request = new CancelTasksRequest();
request.reason("Testing Cancellation");
request.taskId(new TaskId(testNodes[0].discoveryNode.getId(), task.getId()));
request.setReason("Testing Cancellation");
request.setTaskId(new TaskId(testNodes[0].discoveryNode.getId(), task.getId()));
response = testNodes[randomIntBetween(0, testNodes.length - 1)].transportCancelTasksAction.execute(request).get();
// Shouldn't match any tasks since testAction doesn't support cancellation
@ -544,7 +544,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Make sure that task is still running
ListTasksRequest listTasksRequest = new ListTasksRequest();
listTasksRequest.actions(actionName);
listTasksRequest.setActions(actionName);
ListTasksResponse listResponse = testNodes[randomIntBetween(0, testNodes.length - 1)].transportListTasksAction.execute
(listTasksRequest).get();
assertEquals(1, listResponse.getPerNodeTasks().size());
@ -617,7 +617,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
// Run task action on node tasks that are currently running
// should be successful on all nodes except one
TestTasksRequest testTasksRequest = new TestTasksRequest();
testTasksRequest.actions("testAction[n]"); // pick all test actions
testTasksRequest.setActions("testAction[n]"); // pick all test actions
TestTasksResponse response = tasksActions[0].execute(testTasksRequest).get();
// Get successful responses from all nodes except one
assertEquals(testNodes.length - 1, response.tasks.size());

View File

@ -32,6 +32,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.collect.ImmutableOpenIntMap;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
@ -157,6 +158,7 @@ public class IndicesShardStoreRequestIT extends ESIntegTestCase {
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, "5")
.put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false)
));
indexRandomData(index);
ensureGreen(index);
@ -165,9 +167,10 @@ public class IndicesShardStoreRequestIT extends ESIntegTestCase {
logger.info("--> corrupt random shard copies");
Map<Integer, Set<String>> corruptedShardIDMap = new HashMap<>();
Index idx = resolveIndex(index);
for (String node : internalCluster().nodesInclude(index)) {
IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node);
IndexService indexShards = indexServices.indexServiceSafe(index);
IndexService indexShards = indexServices.indexServiceSafe(idx);
for (Integer shardId : indexShards.shardIds()) {
IndexShard shard = indexShards.getShard(shardId);
if (randomBoolean()) {

View File

@ -113,7 +113,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
@Override
protected ShardIterator shards(ClusterState clusterState, Request request) {
return clusterState.routingTable().index(request.concreteIndex()).shard(request.shardId).primaryShardIt();
return clusterState.routingTable().index(request.concreteIndex()).shard(request.shardId.getId()).primaryShardIt();
}
}
@ -178,7 +178,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
public void testBasicRequestWorks() throws InterruptedException, ExecutionException, TimeoutException {
Request request = new Request().index("test");
request.shardId = 0;
request.shardId = new ShardId("test", "_na_", 0);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
clusterService.setState(ClusterStateCreationUtils.state("test", randomBoolean(), ShardRoutingState.STARTED));
action.new AsyncSingleAction(request, listener).start();
@ -189,7 +189,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
public void testFailureWithoutRetry() throws Exception {
Request request = new Request().index("test");
request.shardId = 0;
request.shardId = new ShardId("test", "_na_", 0);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
clusterService.setState(ClusterStateCreationUtils.state("test", randomBoolean(), ShardRoutingState.STARTED));
@ -215,7 +215,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
public void testSuccessAfterRetryWithClusterStateUpdate() throws Exception {
Request request = new Request().index("test");
request.shardId = 0;
request.shardId = new ShardId("test", "_na_", 0);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
boolean local = randomBoolean();
clusterService.setState(ClusterStateCreationUtils.state("test", local, ShardRoutingState.INITIALIZING));
@ -231,7 +231,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
public void testSuccessAfterRetryWithExceptionFromTransport() throws Exception {
Request request = new Request().index("test");
request.shardId = 0;
request.shardId = new ShardId("test", "_na_", 0);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
boolean local = randomBoolean();
clusterService.setState(ClusterStateCreationUtils.state("test", local, ShardRoutingState.STARTED));
@ -250,7 +250,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
public void testRetryOfAnAlreadyTimedOutRequest() throws Exception {
Request request = new Request().index("test").timeout(new TimeValue(0, TimeUnit.MILLISECONDS));
request.shardId = 0;
request.shardId = new ShardId("test", "_na_", 0);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
clusterService.setState(ClusterStateCreationUtils.state("test", randomBoolean(), ShardRoutingState.STARTED));
action.new AsyncSingleAction(request, listener).start();
@ -299,7 +299,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
}
};
Request request = new Request().index("test");
request.shardId = 0;
request.shardId = new ShardId("test", "_na_", 0);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
clusterService.setState(ClusterStateCreationUtils.state("test", randomBoolean(), ShardRoutingState.STARTED));
action.new AsyncSingleAction(request, listener).start();

View File

@ -131,7 +131,7 @@ public class BootstrapCheckTests extends ESTestCase {
}
public void testMaxNumberOfThreadsCheck() {
final int limit = 1 << 15;
final int limit = 1 << 11;
final AtomicLong maxNumberOfThreads = new AtomicLong(randomIntBetween(1, limit - 1));
final BootstrapCheck.MaxNumberOfThreadsCheck check = new BootstrapCheck.MaxNumberOfThreadsCheck() {
@Override

View File

@ -30,6 +30,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.index.Index;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
@ -37,6 +38,7 @@ import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.hamcrest.Matchers.equalTo;
@ -220,7 +222,7 @@ public class ClusterChangedEventTests extends ESTestCase {
final ClusterState newState = nextState(previousState, changeClusterUUID, addedIndices, delIndices, 0);
final ClusterChangedEvent event = new ClusterChangedEvent("_na_", newState, previousState);
final List<String> addsFromEvent = event.indicesCreated();
final List<String> delsFromEvent = event.indicesDeleted();
final List<String> delsFromEvent = event.indicesDeleted().stream().map((s) -> s.getName()).collect(Collectors.toList());
Collections.sort(addsFromEvent);
Collections.sort(delsFromEvent);
assertThat(addsFromEvent, equalTo(addedIndices));

View File

@ -47,6 +47,8 @@ public class WildcardExpressionResolverTests extends ESTestCase {
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("test*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testX*"))), equalTo(newHashSet("testXXX", "testXYY")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("testX*", "kuku"))), equalTo(newHashSet("testXXX", "testXYY", "kuku")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("*"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY", "kuku")));
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("*", "-kuku"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
}
public void testConvertWildcardsTests() {
@ -107,6 +109,18 @@ public class WildcardExpressionResolverTests extends ESTestCase {
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("*Y*X"))).size(), equalTo(0));
}
public void testAll() {
MetaData.Builder mdBuilder = MetaData.builder()
.put(indexBuilder("testXXX"))
.put(indexBuilder("testXYY"))
.put(indexBuilder("testYYY"));
ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(mdBuilder).build();
IndexNameExpressionResolver.WildcardExpressionResolver resolver = new IndexNameExpressionResolver.WildcardExpressionResolver();
IndexNameExpressionResolver.Context context = new IndexNameExpressionResolver.Context(state, IndicesOptions.lenientExpandOpen());
assertThat(newHashSet(resolver.resolve(context, Arrays.asList("_all"))), equalTo(newHashSet("testXXX", "testXYY", "testYYY")));
}
private IndexMetaData.Builder indexBuilder(String index) {
return IndexMetaData.builder(index).settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0));
}

View File

@ -216,6 +216,13 @@ public class ScopedSettingsTests extends ESTestCase {
} catch (IllegalArgumentException e) {
assertEquals("Failed to parse value [true] for setting [index.number_of_replicas]", e.getMessage());
}
try {
settings.validate("index.similarity.classic.type", Settings.builder().put("index.similarity.classic.type", "mine").build());
fail();
} catch (IllegalArgumentException e) {
assertEquals("illegal value for [index.similarity.classic] cannot redefine built-in similarity", e.getMessage());
}
}

View File

@ -177,13 +177,17 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
}
private void configureUnicastCluster(int numberOfNodes, @Nullable int[] unicastHostsOrdinals, int minimumMasterNode) throws ExecutionException, InterruptedException {
configureUnicastCluster(DEFAULT_SETTINGS, numberOfNodes, unicastHostsOrdinals, minimumMasterNode);
}
private void configureUnicastCluster(Settings settings, int numberOfNodes, @Nullable int[] unicastHostsOrdinals, int minimumMasterNode) throws ExecutionException, InterruptedException {
if (minimumMasterNode < 0) {
minimumMasterNode = numberOfNodes / 2 + 1;
}
logger.info("---> configured unicast");
// TODO: Rarely use default settings form some of these
Settings nodeSettings = Settings.builder()
.put(DEFAULT_SETTINGS)
.put(settings)
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minimumMasterNode)
.build();
@ -196,7 +200,6 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
}
}
/**
* Test that no split brain occurs under partial network partition. See https://github.com/elastic/elasticsearch/issues/2488
*/
@ -1075,25 +1078,40 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
* Tests that indices are properly deleted even if there is a master transition in between.
* Test for https://github.com/elastic/elasticsearch/issues/11665
*/
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/16890")
public void testIndicesDeleted() throws Exception {
configureUnicastCluster(3, null, 2);
final Settings settings = Settings.builder()
.put(DEFAULT_SETTINGS)
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "0s") // don't wait on isolated data node
.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "30s") // wait till cluster state is committed
.build();
final String idxName = "test";
configureUnicastCluster(settings, 3, null, 2);
InternalTestCluster.Async<List<String>> masterNodes = internalCluster().startMasterOnlyNodesAsync(2);
InternalTestCluster.Async<String> dataNode = internalCluster().startDataOnlyNodeAsync();
dataNode.get();
masterNodes.get();
final List<String> allMasterEligibleNodes = masterNodes.get();
ensureStableCluster(3);
assertAcked(prepareCreate("test"));
ensureYellow();
String masterNode1 = internalCluster().getMasterName();
final String masterNode1 = internalCluster().getMasterName();
NetworkPartition networkPartition = new NetworkUnresponsivePartition(masterNode1, dataNode.get(), getRandom());
internalCluster().setDisruptionScheme(networkPartition);
networkPartition.startDisrupting();
internalCluster().client(masterNode1).admin().indices().prepareDelete("test").setTimeout("1s").get();
// We know this will time out due to the partition, we check manually below to not proceed until
// the delete has been applied to the master node and the master eligible node.
internalCluster().client(masterNode1).admin().indices().prepareDelete(idxName).setTimeout("0s").get();
// Don't restart the master node until we know the index deletion has taken effect on master and the master eligible node.
assertBusy(() -> {
for (String masterNode : allMasterEligibleNodes) {
final ClusterState masterState = internalCluster().clusterService(masterNode).state();
assertTrue("index not deleted on " + masterNode, masterState.metaData().hasIndex(idxName) == false &&
masterState.status() == ClusterState.ClusterStateStatus.APPLIED);
}
});
internalCluster().restartNode(masterNode1, InternalTestCluster.EMPTY_CALLBACK);
ensureYellow();
assertFalse(client().admin().indices().prepareExists("test").get().isExists());
assertFalse(client().admin().indices().prepareExists(idxName).get().isExists());
}
protected NetworkPartition addRandomPartition() {

View File

@ -33,7 +33,6 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.discovery.zen.ping.PingContextProvider;
import org.elasticsearch.discovery.zen.ping.ZenPing;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -82,11 +81,6 @@ public class UnicastZenPingIT extends ESTestCase {
return DiscoveryNodes.builder().put(nodeA).localNodeId("UZP_A").build();
}
@Override
public NodeService nodeService() {
return null;
}
@Override
public boolean nodeHasJoinedClusterOnce() {
return false;
@ -101,11 +95,6 @@ public class UnicastZenPingIT extends ESTestCase {
return DiscoveryNodes.builder().put(nodeB).localNodeId("UZP_B").build();
}
@Override
public NodeService nodeService() {
return null;
}
@Override
public boolean nodeHasJoinedClusterOnce() {
return true;

View File

@ -43,7 +43,6 @@ import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
@ -134,11 +133,6 @@ public class PublishClusterStateActionTests extends ESTestCase {
return clusterState.nodes();
}
@Override
public NodeService nodeService() {
assert false;
throw new UnsupportedOperationException("Shouldn't be here");
}
}
public MockNode createMockNode(final String name) throws Exception {

View File

@ -0,0 +1,44 @@
/*
* 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;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.Strings;
import org.elasticsearch.test.ESTestCase;
import static org.apache.lucene.util.TestUtil.randomSimpleString;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.not;
public class IndexTests extends ESTestCase {
public void testToString() {
assertEquals("[name/uuid]", new Index("name", "uuid").toString());
assertEquals("[name]", new Index("name", ClusterState.UNKNOWN_UUID).toString());
Index random = new Index(randomSimpleString(random(), 1, 100),
usually() ? Strings.randomBase64UUID(random()) : ClusterState.UNKNOWN_UUID);
assertThat(random.toString(), containsString(random.getName()));
if (ClusterState.UNKNOWN_UUID.equals(random.getUUID())) {
assertThat(random.toString(), not(containsString(random.getUUID())));
} else {
assertThat(random.toString(), containsString(random.getUUID()));
}
}
}

View File

@ -156,10 +156,11 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
ensureGreen();
refresh();
Index index = resolveIndex("foo-copy");
for (IndicesService service : internalCluster().getDataNodeInstances(IndicesService.class)) {
if (service.hasIndex("foo-copy")) {
IndexShard shard = service.indexServiceSafe("foo-copy").getShardOrNull(0);
if (service.hasIndex(index)) {
IndexShard shard = service.indexServiceSafe(index).getShardOrNull(0);
if (shard.routingEntry().primary()) {
assertFalse(shard instanceof ShadowIndexShard);
} else {
@ -201,8 +202,9 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats(IDX).clear().setTranslog(true).get();
assertEquals(2, indicesStatsResponse.getIndex(IDX).getPrimaries().getTranslog().estimatedNumberOfOperations());
assertEquals(2, indicesStatsResponse.getIndex(IDX).getTotal().getTranslog().estimatedNumberOfOperations());
Index index = resolveIndex(IDX);
for (IndicesService service : internalCluster().getInstances(IndicesService.class)) {
IndexService indexService = service.indexService(IDX);
IndexService indexService = service.indexService(index);
if (indexService != null) {
IndexShard shard = indexService.getShard(0);
TranslogStats translogStats = shard.translogStats();

View File

@ -36,24 +36,30 @@ import java.io.IOException;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.startsWith;
public class IndexingSlowLogTests extends ESTestCase {
public void testSlowLogParsedDocumentPrinterSourceToLog() throws IOException {
BytesReference source = JsonXContent.contentBuilder().startObject().field("foo", "bar").endObject().bytes();
ParsedDocument pd = new ParsedDocument(new StringField("uid", "test:id", Store.YES), new LegacyIntField("version", 1, Store.YES), "id",
"test", null, 0, -1, null, source, null);
Index index = new Index("foo", "123");
// Turning off document logging doesn't log source[]
SlowLogParsedDocumentPrinter p = new SlowLogParsedDocumentPrinter(pd, 10, true, 0);
SlowLogParsedDocumentPrinter p = new SlowLogParsedDocumentPrinter(index, pd, 10, true, 0);
assertThat(p.toString(), not(containsString("source[")));
// Turning on document logging logs the whole thing
p = new SlowLogParsedDocumentPrinter(pd, 10, true, Integer.MAX_VALUE);
p = new SlowLogParsedDocumentPrinter(index, pd, 10, true, Integer.MAX_VALUE);
assertThat(p.toString(), containsString("source[{\"foo\":\"bar\"}]"));
// And you can truncate the source
p = new SlowLogParsedDocumentPrinter(pd, 10, true, 3);
p = new SlowLogParsedDocumentPrinter(index, pd, 10, true, 3);
assertThat(p.toString(), containsString("source[{\"f]"));
// And you can truncate the source
p = new SlowLogParsedDocumentPrinter(index, pd, 10, true, 3);
assertThat(p.toString(), containsString("source[{\"f]"));
assertThat(p.toString(), startsWith("[foo/123] took"));
}
public void testReformatSetting() {

View File

@ -168,7 +168,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
protected Nested createNested(IndexSearcher searcher, Query parentFilter, Query childFilter) throws IOException {
BitsetFilterCache s = indexService.cache().bitsetFilterCache();
return new Nested(s.getBitSetProducer(parentFilter), searcher.createNormalizedWeight(childFilter, false));
return new Nested(s.getBitSetProducer(parentFilter), childFilter);
}
public void testEmpty() throws Exception {

View File

@ -140,6 +140,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
protected static final String DATE_FIELD_NAME = "mapped_date";
protected static final String OBJECT_FIELD_NAME = "mapped_object";
protected static final String GEO_POINT_FIELD_NAME = "mapped_geo_point";
protected static final String GEO_POINT_FIELD_MAPPING = "type=geo_point,lat_lon=true,geohash=true,geohash_prefix=true";
protected static final String GEO_SHAPE_FIELD_NAME = "mapped_geo_shape";
protected static final String[] MAPPED_FIELD_NAMES = new String[] { STRING_FIELD_NAME, INT_FIELD_NAME, DOUBLE_FIELD_NAME,
BOOLEAN_FIELD_NAME, DATE_FIELD_NAME, OBJECT_FIELD_NAME, GEO_POINT_FIELD_NAME, GEO_SHAPE_FIELD_NAME };
@ -300,7 +301,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
BOOLEAN_FIELD_NAME, "type=boolean",
DATE_FIELD_NAME, "type=date",
OBJECT_FIELD_NAME, "type=object",
GEO_POINT_FIELD_NAME, "type=geo_point,lat_lon=true,geohash=true,geohash_prefix=true",
GEO_POINT_FIELD_NAME, GEO_POINT_FIELD_MAPPING,
GEO_SHAPE_FIELD_NAME, "type=geo_shape"
).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
// also add mappings for two inner field in the object field

View File

@ -24,10 +24,12 @@ import org.apache.lucene.spatial.geopoint.search.GeoPointDistanceRangeQuery;
import org.apache.lucene.spatial.util.GeoDistanceUtils;
import org.apache.lucene.util.NumericUtils;
import org.elasticsearch.Version;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.geo.GeoDistance;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.search.geo.GeoDistanceRangeQuery;
import org.elasticsearch.test.geo.RandomGeoGenerator;
@ -296,6 +298,36 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
}
}
public void testNestedRangeQuery() throws IOException {
// create a nested geo_point type with a subfield named "geohash" (explicit testing for ISSUE #15179)
MapperService mapperService = queryShardContext().getMapperService();
String nestedMapping =
"{\"nested_doc\" : {\"properties\" : {" +
"\"locations\": {\"properties\": {" +
"\"geohash\": {\"type\": \"geo_point\"}}," +
"\"type\": \"nested\"}" +
"}}}";
mapperService.merge("nested_doc", new CompressedXContent(nestedMapping), MapperService.MergeReason.MAPPING_UPDATE, false);
// create a range query on the nested locations.geohash sub-field
String queryJson =
"{\n" +
" \"nested\": {\n" +
" \"path\": \"locations\",\n" +
" \"query\": {\n" +
" \"geo_distance_range\": {\n" +
" \"from\": \"0.0km\",\n" +
" \"to\" : \"200.0km\",\n" +
" \"locations.geohash\": \"s7ws01wyd7ws\"\n" +
" }\n" +
" }\n" +
" }\n" +
"}\n";
NestedQueryBuilder builder = (NestedQueryBuilder) parseQuery(queryJson);
QueryShardContext context = createShardContext();
builder.toQuery(context);
}
public void testFromJson() throws IOException {
String json =
"{\n" +

View File

@ -63,7 +63,7 @@ public class GeohashCellQueryBuilderTests extends AbstractQueryTestCase<Builder>
assertThat(query, instanceOf(TermQuery.class));
TermQuery termQuery = (TermQuery) query;
Term term = termQuery.getTerm();
assertThat(term.field(), equalTo(queryBuilder.fieldName() + GeoPointFieldMapper.Names.GEOHASH_SUFFIX));
assertThat(term.field(), equalTo(queryBuilder.fieldName() + "." + GeoPointFieldMapper.Names.GEOHASH));
String geohash = queryBuilder.geohash();
if (queryBuilder.precision() != null) {
int len = Math.min(queryBuilder.precision(), geohash.length());

View File

@ -52,6 +52,7 @@ public class NestedQueryBuilderTests extends AbstractQueryTestCase<NestedQueryBu
BOOLEAN_FIELD_NAME, "type=boolean",
DATE_FIELD_NAME, "type=date",
OBJECT_FIELD_NAME, "type=object",
GEO_POINT_FIELD_NAME, GEO_POINT_FIELD_MAPPING,
"nested1", "type=nested"
).string()), MapperService.MergeReason.MAPPING_UPDATE, false);
}

View File

@ -68,7 +68,7 @@ public class CustomQueryParserIT extends ESIntegTestCase {
private static QueryShardContext queryShardContext() {
IndicesService indicesService = internalCluster().getDataNodeInstance(IndicesService.class);
return indicesService.indexServiceSafe("index").newQueryShardContext();
return indicesService.indexServiceSafe(resolveIndex("index")).newQueryShardContext();
}
//see #11120

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