Add automatic parallelization support to reindex and friends (#20767)

Adds support for `?slices=N` to reindex which automatically
parallelizes the process using parallel scrolls on `_uid`. Performance
testing sees a 3x performance improvement for simple docs
on decent hardware, maybe 30% performance improvement
for more complex docs. Still compelling, especially because
clusters should be able to get closer to the 3x than the 30%
number.

Closes #20624
This commit is contained in:
Nik Everett 2016-11-04 20:59:15 -04:00 committed by GitHub
parent 555084a226
commit a13a050271
68 changed files with 4263 additions and 1002 deletions

View File

@ -20,6 +20,7 @@
package org.elasticsearch.action.admin.cluster.node.tasks.cancel;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.support.ActionFilters;
@ -111,17 +112,18 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
}
@Override
protected synchronized TaskInfo taskOperation(CancelTasksRequest request, CancellableTask cancellableTask) {
protected synchronized void taskOperation(CancelTasksRequest request, CancellableTask cancellableTask,
ActionListener<TaskInfo> listener) {
final BanLock banLock = new BanLock(nodes -> removeBanOnNodes(cancellableTask, nodes));
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);
listener.onResponse(cancellableTask.taskInfo(clusterService.localNode().getId(), false));
} else {
logger.trace("cancelling task {} with children on nodes [{}]", cancellableTask.getId(), childNodes);
setBanOnNodes(request.getReason(), cancellableTask, childNodes, banLock);
return cancellableTask.taskInfo(clusterService.localNode(), false);
listener.onResponse(cancellableTask.taskInfo(clusterService.localNode().getId(), false));
}
} else {
logger.trace("task {} is already cancelled", cancellableTask.getId());

View File

@ -150,7 +150,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
@Override
protected void doRun() throws Exception {
taskManager.waitForTaskCompletion(runningTask, waitForCompletionTimeout(request.getTimeout()));
waitedForCompletion(thisTask, request, runningTask.taskInfo(clusterService.localNode(), true), listener);
waitedForCompletion(thisTask, request, runningTask.taskInfo(clusterService.localNode().getId(), true), listener);
}
@Override
@ -159,7 +159,7 @@ public class TransportGetTaskAction extends HandledTransportAction<GetTaskReques
}
});
} else {
TaskInfo info = runningTask.taskInfo(clusterService.localNode(), true);
TaskInfo info = runningTask.taskInfo(clusterService.localNode().getId(), true);
listener.onResponse(new GetTaskResponse(new TaskResult(false, info)));
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.admin.cluster.node.tasks.list;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.support.ActionFilters;
@ -69,8 +70,8 @@ public class TransportListTasksAction extends TransportTasksAction<Task, ListTas
}
@Override
protected TaskInfo taskOperation(ListTasksRequest request, Task task) {
return task.taskInfo(clusterService.localNode(), request.getDetailed());
protected void taskOperation(ListTasksRequest request, Task task, ActionListener<TaskInfo> listener) {
listener.onResponse(task.taskInfo(clusterService.localNode().getId(), request.getDetailed()));
}
@Override

View File

@ -19,16 +19,22 @@
package org.elasticsearch.action.support.tasks;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
import static java.util.stream.Collectors.toList;
import static org.elasticsearch.ExceptionsHelper.rethrowAndSuppress;
/**
@ -60,6 +66,17 @@ public class BaseTasksResponse extends ActionResponse {
return nodeFailures;
}
/**
* Rethrow task failures if there are any.
*/
public void rethrowFailures(String operationName) {
rethrowAndSuppress(Stream.concat(
getNodeFailures().stream(),
getTaskFailures().stream().map(f -> new ElasticsearchException(
"{} of [{}] failed", f.getCause(), operationName, new TaskId(f.getNodeId(), f.getTaskId()))))
.collect(toList()));
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);

View File

@ -63,5 +63,14 @@ public class TasksRequestBuilder<
request.setTimeout(timeout);
return (RequestBuilder) this;
}
/**
* Match all children of the provided task.
*/
@SuppressWarnings("unchecked")
public final RequestBuilder setParentTaskId(TaskId taskId) {
request.setParentTaskId(taskId);
return (RequestBuilder) this;
}
}

View File

@ -33,10 +33,12 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.NodeShouldNotConnectException;
@ -57,6 +59,8 @@ import java.util.concurrent.atomic.AtomicReferenceArray;
import java.util.function.Consumer;
import java.util.function.Supplier;
import static java.util.Collections.emptyList;
/**
* The base class for transport actions that are interacting with currently running tasks.
*/
@ -100,21 +104,56 @@ public abstract class TransportTasksAction<
new AsyncAction(task, request, listener).start();
}
private NodeTasksResponse nodeOperation(NodeTaskRequest nodeTaskRequest) {
private void nodeOperation(NodeTaskRequest nodeTaskRequest, ActionListener<NodeTasksResponse> listener) {
TasksRequest request = nodeTaskRequest.tasksRequest;
List<TaskResponse> results = new ArrayList<>();
List<TaskOperationFailure> exceptions = new ArrayList<>();
processTasks(request, task -> {
try {
TaskResponse response = taskOperation(request, task);
if (response != null) {
results.add(response);
List<OperationTask> tasks = new ArrayList<>();
processTasks(request, tasks::add);
if (tasks.isEmpty()) {
listener.onResponse(new NodeTasksResponse(clusterService.localNode().getId(), emptyList(), emptyList()));
return;
}
AtomicArray<Tuple<TaskResponse, Exception>> responses = new AtomicArray<>(tasks.size());
final AtomicInteger counter = new AtomicInteger(tasks.size());
for (int i = 0; i < tasks.size(); i++) {
final int taskIndex = i;
ActionListener<TaskResponse> taskListener = new ActionListener<TaskResponse>() {
@Override
public void onResponse(TaskResponse response) {
responses.setOnce(taskIndex, response == null ? null : new Tuple<>(response, null));
respondIfFinished();
}
} catch (Exception ex) {
exceptions.add(new TaskOperationFailure(clusterService.localNode().getId(), task.getId(), ex));
@Override
public void onFailure(Exception e) {
responses.setOnce(taskIndex, new Tuple<>(null, e));
respondIfFinished();
}
private void respondIfFinished() {
if (counter.decrementAndGet() != 0) {
return;
}
List<TaskResponse> results = new ArrayList<>();
List<TaskOperationFailure> exceptions = new ArrayList<>();
for (AtomicArray.Entry<Tuple<TaskResponse, Exception>> response : responses.asList()) {
if (response.value.v1() == null) {
assert response.value.v2() != null;
exceptions.add(new TaskOperationFailure(clusterService.localNode().getId(), tasks.get(taskIndex).getId(),
response.value.v2()));
} else {
assert response.value.v2() == null;
results.add(response.value.v1());
}
}
listener.onResponse(new NodeTasksResponse(clusterService.localNode().getId(), results, exceptions));
}
};
try {
taskOperation(request, tasks.get(taskIndex), taskListener);
} catch (Exception e) {
taskListener.onFailure(e);
}
});
return new NodeTasksResponse(clusterService.localNode().getId(), results, exceptions);
}
}
protected String[] filterNodeIds(DiscoveryNodes nodes, String[] nodesIds) {
@ -178,7 +217,10 @@ public abstract class TransportTasksAction<
protected abstract TaskResponse readTaskResponse(StreamInput in) throws IOException;
protected abstract TaskResponse taskOperation(TasksRequest request, OperationTask task);
/**
* Perform the required operation on the task. It is OK start an asynchronous operation or to throw an exception but not both.
*/
protected abstract void taskOperation(TasksRequest request, OperationTask task, ActionListener<TaskResponse> listener);
protected boolean transportCompress() {
return false;
@ -305,7 +347,27 @@ public abstract class TransportTasksAction<
@Override
public void messageReceived(final NodeTaskRequest request, final TransportChannel channel) throws Exception {
channel.sendResponse(nodeOperation(request));
nodeOperation(request, new ActionListener<NodeTasksResponse>() {
@Override
public void onResponse(
TransportTasksAction<OperationTask, TasksRequest, TasksResponse, TaskResponse>.NodeTasksResponse response) {
try {
channel.sendResponse(response);
} catch (Exception e) {
onFailure(e);
}
}
@Override
public void onFailure(Exception e) {
try {
channel.sendResponse(e);
} catch (IOException e1) {
e1.addSuppressed(e);
logger.warn("Failed to send failure", e1);
}
}
});
}
}

View File

@ -879,7 +879,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
* infinitely.
*/
public SearchSourceBuilder rewrite(QueryShardContext context) throws IOException {
assert (this.equals(shallowCopy(queryBuilder, postQueryBuilder)));
assert (this.equals(shallowCopy(queryBuilder, postQueryBuilder, sliceBuilder)));
QueryBuilder queryBuilder = null;
if (this.queryBuilder != null) {
queryBuilder = this.queryBuilder.rewrite(context);
@ -890,40 +890,51 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
}
boolean rewritten = queryBuilder != this.queryBuilder || postQueryBuilder != this.postQueryBuilder;
if (rewritten) {
return shallowCopy(queryBuilder, postQueryBuilder);
return shallowCopy(queryBuilder, postQueryBuilder, sliceBuilder);
}
return this;
}
private SearchSourceBuilder shallowCopy(QueryBuilder queryBuilder, QueryBuilder postQueryBuilder) {
SearchSourceBuilder rewrittenBuilder = new SearchSourceBuilder();
rewrittenBuilder.aggregations = aggregations;
rewrittenBuilder.explain = explain;
rewrittenBuilder.extBuilders = extBuilders;
rewrittenBuilder.fetchSourceContext = fetchSourceContext;
rewrittenBuilder.docValueFields = docValueFields;
rewrittenBuilder.storedFieldsContext = storedFieldsContext;
rewrittenBuilder.from = from;
rewrittenBuilder.highlightBuilder = highlightBuilder;
rewrittenBuilder.indexBoost = indexBoost;
rewrittenBuilder.minScore = minScore;
rewrittenBuilder.postQueryBuilder = postQueryBuilder;
rewrittenBuilder.profile = profile;
rewrittenBuilder.queryBuilder = queryBuilder;
rewrittenBuilder.rescoreBuilders = rescoreBuilders;
rewrittenBuilder.scriptFields = scriptFields;
rewrittenBuilder.searchAfterBuilder = searchAfterBuilder;
rewrittenBuilder.sliceBuilder = sliceBuilder;
rewrittenBuilder.size = size;
rewrittenBuilder.sorts = sorts;
rewrittenBuilder.stats = stats;
rewrittenBuilder.suggestBuilder = suggestBuilder;
rewrittenBuilder.terminateAfter = terminateAfter;
rewrittenBuilder.timeout = timeout;
rewrittenBuilder.trackScores = trackScores;
rewrittenBuilder.version = version;
return rewrittenBuilder;
}
/**
* Create a shallow copy of this builder with a new slice configuration.
*/
public SearchSourceBuilder copyWithNewSlice(SliceBuilder slice) {
return shallowCopy(queryBuilder, postQueryBuilder, slice);
}
/**
* Create a shallow copy of this source replaced {@link #queryBuilder}, {@link #postQueryBuilder}, and {@linkplain slice}. Used by
* {@link #rewrite(QueryShardContext)} and {@link #copyWithNewSlice(SliceBuilder)}.
*/
private SearchSourceBuilder shallowCopy(QueryBuilder queryBuilder, QueryBuilder postQueryBuilder, SliceBuilder slice) {
SearchSourceBuilder rewrittenBuilder = new SearchSourceBuilder();
rewrittenBuilder.aggregations = aggregations;
rewrittenBuilder.explain = explain;
rewrittenBuilder.extBuilders = extBuilders;
rewrittenBuilder.fetchSourceContext = fetchSourceContext;
rewrittenBuilder.docValueFields = docValueFields;
rewrittenBuilder.storedFieldsContext = storedFieldsContext;
rewrittenBuilder.from = from;
rewrittenBuilder.highlightBuilder = highlightBuilder;
rewrittenBuilder.indexBoost = indexBoost;
rewrittenBuilder.minScore = minScore;
rewrittenBuilder.postQueryBuilder = postQueryBuilder;
rewrittenBuilder.profile = profile;
rewrittenBuilder.queryBuilder = queryBuilder;
rewrittenBuilder.rescoreBuilders = rescoreBuilders;
rewrittenBuilder.scriptFields = scriptFields;
rewrittenBuilder.searchAfterBuilder = searchAfterBuilder;
rewrittenBuilder.sliceBuilder = slice;
rewrittenBuilder.size = size;
rewrittenBuilder.sorts = sorts;
rewrittenBuilder.stats = stats;
rewrittenBuilder.suggestBuilder = suggestBuilder;
rewrittenBuilder.terminateAfter = terminateAfter;
rewrittenBuilder.timeout = timeout;
rewrittenBuilder.trackScores = trackScores;
rewrittenBuilder.version = version;
return rewrittenBuilder;
}
/**
* Parse some xContent into this SearchSourceBuilder, overwriting any values specified in the xContent. Use this if you need to set up

View File

@ -42,8 +42,14 @@ public class Task {
private final TaskId parentTask;
/**
* The task's start time as a wall clock time since epoch ({@link System#currentTimeMillis()} style).
*/
private final long startTime;
/**
* The task's start time as a relative time ({@link System#nanoTime()} style).
*/
private final long startTimeNanos;
public Task(long id, String type, String action, String description, TaskId parentTask) {
@ -64,20 +70,27 @@ public class Task {
* Build a version of the task status you can throw over the wire and back
* to the user.
*
* @param node
* the node this task is running on
* @param localNodeId
* the id of the node this task is running on
* @param detailed
* should the information include detailed, potentially slow to
* generate data?
*/
public TaskInfo taskInfo(DiscoveryNode node, boolean detailed) {
public final TaskInfo taskInfo(String localNodeId, boolean detailed) {
String description = null;
Task.Status status = null;
if (detailed) {
description = getDescription();
status = getStatus();
}
return new TaskInfo(new TaskId(node.getId(), getId()), getType(), getAction(), description, status, startTime,
return taskInfo(localNodeId, description, status);
}
/**
* Build a proper {@link TaskInfo} for this task.
*/
protected final TaskInfo taskInfo(String localNodeId, String description, Status status) {
return new TaskInfo(new TaskId(localNodeId, getId()), getType(), getAction(), description, status, startTime,
System.nanoTime() - startTimeNanos, this instanceof CancellableTask, parentTask);
}
@ -110,7 +123,7 @@ public class Task {
}
/**
* Returns the task start time
* Returns the task's start time as a wall clock time since epoch ({@link System#currentTimeMillis()} style).
*/
public long getStartTime() {
return startTime;
@ -136,12 +149,12 @@ public class Task {
public interface Status extends ToXContent, NamedWriteable {}
public TaskResult result(DiscoveryNode node, Exception error) throws IOException {
return new TaskResult(taskInfo(node, true), error);
return new TaskResult(taskInfo(node.getId(), true), error);
}
public TaskResult result(DiscoveryNode node, ActionResponse response) throws IOException {
if (response instanceof ToXContent) {
return new TaskResult(taskInfo(node, true), (ToXContent) response);
return new TaskResult(taskInfo(node.getId(), true), (ToXContent) response);
} else {
throw new IllegalStateException("response has to implement ToXContent to be able to store the results");
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.action.admin.cluster.node.tasks;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.tasks.Task;
@ -37,26 +36,26 @@ import java.util.stream.Collectors;
public class RecordingTaskManagerListener implements MockTaskManagerListener {
private String[] actionMasks;
private DiscoveryNode localNode;
private String localNodeId;
private List<Tuple<Boolean, TaskInfo>> events = new ArrayList<>();
public RecordingTaskManagerListener(DiscoveryNode localNode, String... actionMasks) {
public RecordingTaskManagerListener(String localNodeId, String... actionMasks) {
this.actionMasks = actionMasks;
this.localNode = localNode;
this.localNodeId = localNodeId;
}
@Override
public synchronized void onTaskRegistered(Task task) {
if (Regex.simpleMatch(actionMasks, task.getAction())) {
events.add(new Tuple<>(true, task.taskInfo(localNode, true)));
events.add(new Tuple<>(true, task.taskInfo(localNodeId, true)));
}
}
@Override
public synchronized void onTaskUnregistered(Task task) {
if (Regex.simpleMatch(actionMasks, task.getAction())) {
events.add(new Tuple<>(false, task.taskInfo(localNode, true)));
events.add(new Tuple<>(false, task.taskInfo(localNodeId, true)));
}
}

View File

@ -231,7 +231,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
public static RecordingTaskManagerListener[] setupListeners(TestNode[] nodes, String... actionMasks) {
RecordingTaskManagerListener[] listeners = new RecordingTaskManagerListener[nodes.length];
for (int i = 0; i < nodes.length; i++) {
listeners[i] = new RecordingTaskManagerListener(nodes[i].discoveryNode, actionMasks);
listeners[i] = new RecordingTaskManagerListener(nodes[i].discoveryNode.getId(), actionMasks);
((MockTaskManager) (nodes[i].transportService.getTaskManager())).addListener(listeners[i]);
}
return listeners;

View File

@ -58,7 +58,6 @@ 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.MockTcpTransportPlugin;
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
import org.elasticsearch.transport.TransportService;
@ -783,7 +782,7 @@ public class TasksIT extends ESIntegTestCase {
private void registerTaskManageListeners(String actionMasks) {
for (String nodeName : internalCluster().getNodeNames()) {
DiscoveryNode node = internalCluster().getInstance(ClusterService.class, nodeName).localNode();
RecordingTaskManagerListener listener = new RecordingTaskManagerListener(node, actionMasks.split(","));
RecordingTaskManagerListener listener = new RecordingTaskManagerListener(node.getId(), actionMasks.split(","));
((MockTaskManager) internalCluster().getInstance(TransportService.class, nodeName).getTaskManager()).addListener(listener);
RecordingTaskManagerListener oldListener = listeners.put(new Tuple<>(node.getName(), actionMasks), listener);
assertNull(oldListener);

View File

@ -438,9 +438,9 @@ public class TestTaskPlugin extends Plugin implements ActionPlugin {
}
@Override
protected UnblockTestTaskResponse taskOperation(UnblockTestTasksRequest request, Task task) {
protected void taskOperation(UnblockTestTasksRequest request, Task task, ActionListener<UnblockTestTaskResponse> listener) {
((TestTask) task).unblock();
return new UnblockTestTaskResponse();
listener.onResponse(new UnblockTestTaskResponse());
}
@Override

View File

@ -70,6 +70,7 @@ import static org.elasticsearch.action.support.PlainActionFuture.newFuture;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.not;
public class TransportTasksActionTests extends TaskManagerTestCase {
@ -625,13 +626,34 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
tasksActions[i] = new TestTasksAction(CLUSTER_SETTINGS, "testTasksAction", threadPool, testNodes[i].clusterService,
testNodes[i].transportService) {
@Override
protected TestTaskResponse taskOperation(TestTasksRequest request, Task task) {
protected void taskOperation(TestTasksRequest request, Task task, ActionListener<TestTaskResponse> listener) {
logger.info("Task action on node {}", node);
if (failTaskOnNode == node && task.getParentTaskId().isSet()) {
logger.info("Failing on node {}", node);
throw new RuntimeException("Task level failure");
// Fail in a random way to make sure we can handle all these ways
Runnable failureMode = randomFrom(
() -> {
logger.info("Throwing exception from taskOperation");
throw new RuntimeException("Task level failure (direct)");
},
() -> {
logger.info("Calling listener synchronously with exception from taskOperation");
listener.onFailure(new RuntimeException("Task level failure (sync listener)"));
},
() -> {
logger.info("Calling listener asynchronously with exception from taskOperation");
threadPool.generic()
.execute(() -> listener.onFailure(new RuntimeException("Task level failure (async listener)")));
}
);
failureMode.run();
} else {
if (randomBoolean()) {
listener.onResponse(new TestTaskResponse("Success on node (sync)" + node));
} else {
threadPool.generic().execute(() -> listener.onResponse(new TestTaskResponse("Success on node (async)" + node)));
}
}
return new TestTaskResponse("Success on node " + node);
}
};
}
@ -641,10 +663,10 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
TestTasksRequest testTasksRequest = new TestTasksRequest();
testTasksRequest.setActions("testAction[n]"); // pick all test actions
TestTasksResponse response = tasksActions[0].execute(testTasksRequest).get();
assertThat(response.getTaskFailures(), hasSize(1)); // one task failed
assertThat(response.getTaskFailures().get(0).getReason(), containsString("Task level failure"));
// Get successful responses from all nodes except one
assertEquals(testNodes.length - 1, response.tasks.size());
assertEquals(1, response.getTaskFailures().size()); // one task failed
assertThat(response.getTaskFailures().get(0).getReason(), containsString("Task level failure"));
assertEquals(0, response.getNodeFailures().size()); // no nodes failed
// Release all node tasks and wait for response
@ -696,8 +718,12 @@ public class TransportTasksActionTests extends TaskManagerTestCase {
}
@Override
protected TestTaskResponse taskOperation(TestTasksRequest request, Task task) {
return new TestTaskResponse(testNodes[node].getNodeId());
protected void taskOperation(TestTasksRequest request, Task task, ActionListener<TestTaskResponse> listener) {
if (randomBoolean()) {
listener.onResponse(new TestTaskResponse(testNodes[node].getNodeId()));
} else {
threadPool.generic().execute(() -> listener.onResponse(new TestTaskResponse(testNodes[node].getNodeId())));
}
}
};
}

View File

@ -20,37 +20,21 @@
package org.elasticsearch.search;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
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 org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilderTests;
import org.elasticsearch.search.rescore.QueryRescoreBuilderTests;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.slice.SliceBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.search.suggest.SuggestBuilderTests;
import org.elasticsearch.test.AbstractQueryTestCase;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
@ -63,6 +47,7 @@ import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.Function;
import java.util.function.Supplier;
public abstract class AbstractSearchTestCase extends ESTestCase {
@ -84,212 +69,8 @@ public abstract class AbstractSearchTestCase extends ESTestCase {
queriesRegistry = searchModule.getQueryParserRegistry();
}
protected SearchSourceBuilder createSearchSourceBuilder() throws IOException {
SearchSourceBuilder builder = new SearchSourceBuilder();
if (randomBoolean()) {
builder.from(randomIntBetween(0, 10000));
}
if (randomBoolean()) {
builder.size(randomIntBetween(0, 10000));
}
if (randomBoolean()) {
builder.explain(randomBoolean());
}
if (randomBoolean()) {
builder.version(randomBoolean());
}
if (randomBoolean()) {
builder.trackScores(randomBoolean());
}
if (randomBoolean()) {
builder.minScore(randomFloat() * 1000);
}
if (randomBoolean()) {
builder.timeout(TimeValue.parseTimeValue(randomTimeValue(), null, "timeout"));
}
if (randomBoolean()) {
builder.terminateAfter(randomIntBetween(1, 100000));
}
switch(randomInt(2)) {
case 0:
builder.storedFields();
break;
case 1:
builder.storedField("_none_");
break;
case 2:
int fieldsSize = randomInt(25);
List<String> fields = new ArrayList<>(fieldsSize);
for (int i = 0; i < fieldsSize; i++) {
fields.add(randomAsciiOfLengthBetween(5, 50));
}
builder.storedFields(fields);
break;
default:
throw new IllegalStateException();
}
if (randomBoolean()) {
int scriptFieldsSize = randomInt(25);
for (int i = 0; i < scriptFieldsSize; i++) {
if (randomBoolean()) {
builder.scriptField(randomAsciiOfLengthBetween(5, 50), new Script("foo"), randomBoolean());
} else {
builder.scriptField(randomAsciiOfLengthBetween(5, 50), new Script("foo"));
}
}
}
if (randomBoolean()) {
FetchSourceContext fetchSourceContext;
int branch = randomInt(5);
String[] includes = new String[randomIntBetween(0, 20)];
for (int i = 0; i < includes.length; i++) {
includes[i] = randomAsciiOfLengthBetween(5, 20);
}
String[] excludes = new String[randomIntBetween(0, 20)];
for (int i = 0; i < excludes.length; i++) {
excludes[i] = randomAsciiOfLengthBetween(5, 20);
}
switch (branch) {
case 0:
fetchSourceContext = new FetchSourceContext(randomBoolean());
break;
case 1:
fetchSourceContext = new FetchSourceContext(true, includes, excludes);
break;
case 2:
fetchSourceContext = new FetchSourceContext(true, new String[]{randomAsciiOfLengthBetween(5, 20)},
new String[]{randomAsciiOfLengthBetween(5, 20)});
break;
case 3:
fetchSourceContext = new FetchSourceContext(true, includes, excludes);
break;
case 4:
fetchSourceContext = new FetchSourceContext(true, includes, null);
break;
case 5:
fetchSourceContext = new FetchSourceContext(true, new String[] {randomAsciiOfLengthBetween(5, 20)}, null);
break;
default:
throw new IllegalStateException();
}
builder.fetchSource(fetchSourceContext);
}
if (randomBoolean()) {
int size = randomIntBetween(0, 20);
List<String> statsGroups = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
statsGroups.add(randomAsciiOfLengthBetween(5, 20));
}
builder.stats(statsGroups);
}
if (randomBoolean()) {
int indexBoostSize = randomIntBetween(1, 10);
for (int i = 0; i < indexBoostSize; i++) {
builder.indexBoost(randomAsciiOfLengthBetween(5, 20), randomFloat() * 10);
}
}
if (randomBoolean()) {
builder.query(QueryBuilders.termQuery(randomAsciiOfLengthBetween(5, 20), randomAsciiOfLengthBetween(5, 20)));
}
if (randomBoolean()) {
builder.postFilter(QueryBuilders.termQuery(randomAsciiOfLengthBetween(5, 20), randomAsciiOfLengthBetween(5, 20)));
}
if (randomBoolean()) {
int numSorts = randomIntBetween(1, 5);
for (int i = 0; i < numSorts; i++) {
int branch = randomInt(5);
switch (branch) {
case 0:
builder.sort(SortBuilders.fieldSort(randomAsciiOfLengthBetween(5, 20)).order(randomFrom(SortOrder.values())));
break;
case 1:
builder.sort(SortBuilders.geoDistanceSort(randomAsciiOfLengthBetween(5, 20),
AbstractQueryTestCase.randomGeohash(1, 12)).order(randomFrom(SortOrder.values())));
break;
case 2:
builder.sort(SortBuilders.scoreSort().order(randomFrom(SortOrder.values())));
break;
case 3:
builder.sort(SortBuilders.scriptSort(new Script("foo"),
ScriptSortBuilder.ScriptSortType.NUMBER).order(randomFrom(SortOrder.values())));
break;
case 4:
builder.sort(randomAsciiOfLengthBetween(5, 20));
break;
case 5:
builder.sort(randomAsciiOfLengthBetween(5, 20), randomFrom(SortOrder.values()));
break;
}
}
}
if (randomBoolean()) {
int numSearchFrom = randomIntBetween(1, 5);
// We build a json version of the search_from first in order to
// ensure that every number type remain the same before/after xcontent (de)serialization.
// This is not a problem because the final type of each field value is extracted from associated sort field.
// This little trick ensure that equals and hashcode are the same when using the xcontent serialization.
XContentBuilder jsonBuilder = XContentFactory.jsonBuilder();
jsonBuilder.startObject();
jsonBuilder.startArray("search_from");
for (int i = 0; i < numSearchFrom; i++) {
int branch = randomInt(8);
switch (branch) {
case 0:
jsonBuilder.value(randomInt());
break;
case 1:
jsonBuilder.value(randomFloat());
break;
case 2:
jsonBuilder.value(randomLong());
break;
case 3:
jsonBuilder.value(randomDouble());
break;
case 4:
jsonBuilder.value(randomAsciiOfLengthBetween(5, 20));
break;
case 5:
jsonBuilder.value(randomBoolean());
break;
case 6:
jsonBuilder.value(randomByte());
break;
case 7:
jsonBuilder.value(randomShort());
break;
case 8:
jsonBuilder.value(new Text(randomAsciiOfLengthBetween(5, 20)));
break;
}
}
jsonBuilder.endArray();
jsonBuilder.endObject();
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(jsonBuilder.bytes());
parser.nextToken();
parser.nextToken();
parser.nextToken();
builder.searchAfter(SearchAfterBuilder.fromXContent(parser, null).getSortValues());
}
if (randomBoolean()) {
builder.highlighter(HighlightBuilderTests.randomHighlighterBuilder());
}
if (randomBoolean()) {
builder.suggest(SuggestBuilderTests.randomSuggestBuilder());
}
if (randomBoolean()) {
int numRescores = randomIntBetween(1, 5);
for (int i = 0; i < numRescores; i++) {
builder.addRescorer(QueryRescoreBuilderTests.randomRescoreBuilder());
}
}
if (randomBoolean()) {
builder.aggregation(AggregationBuilders.avg(randomAsciiOfLengthBetween(5, 20)));
}
if (randomBoolean()) {
protected SearchSourceBuilder createSearchSourceBuilder() {
Supplier<List<SearchExtBuilder>> randomExtBuilders = () -> {
Set<String> elementNames = new HashSet<>(searchExtPlugin.getSupportedElements().keySet());
int numSearchExts = randomIntBetween(1, elementNames.size());
while(elementNames.size() > numSearchExts) {
@ -299,51 +80,17 @@ public abstract class AbstractSearchTestCase extends ESTestCase {
for (String elementName : elementNames) {
searchExtBuilders.add(searchExtPlugin.getSupportedElements().get(elementName).apply(randomAsciiOfLengthBetween(3, 10)));
}
builder.ext(searchExtBuilders);
}
if (randomBoolean()) {
String field = randomBoolean() ? null : randomAsciiOfLengthBetween(5, 20);
int max = between(2, 1000);
int id = randomInt(max-1);
if (field == null) {
builder.slice(new SliceBuilder(id, max));
} else {
builder.slice(new SliceBuilder(field, id, max));
}
}
return builder;
return searchExtBuilders;
};
return RandomSearchRequestGenerator.randomSearchSourceBuilder(
HighlightBuilderTests::randomHighlighterBuilder,
SuggestBuilderTests::randomSuggestBuilder,
QueryRescoreBuilderTests::randomRescoreBuilder,
randomExtBuilders);
}
protected SearchRequest createSearchRequest() throws IOException {
SearchRequest searchRequest = new SearchRequest();
if (randomBoolean()) {
searchRequest.indices(generateRandomStringArray(10, 10, false, false));
}
if (randomBoolean()) {
searchRequest.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean()));
}
if (randomBoolean()) {
searchRequest.types(generateRandomStringArray(10, 10, false, false));
}
if (randomBoolean()) {
searchRequest.preference(randomAsciiOfLengthBetween(3, 10));
}
if (randomBoolean()) {
searchRequest.requestCache(randomBoolean());
}
if (randomBoolean()) {
searchRequest.routing(randomAsciiOfLengthBetween(3, 10));
}
if (randomBoolean()) {
searchRequest.scroll(randomPositiveTimeValue());
}
if (randomBoolean()) {
searchRequest.searchType(randomFrom(SearchType.values()));
}
if (randomBoolean()) {
searchRequest.source(createSearchSourceBuilder());
}
return searchRequest;
return RandomSearchRequestGenerator.randomSearchRequest(this::createSearchSourceBuilder);
}
private static class TestSearchExtPlugin extends Plugin implements SearchPlugin {

View File

@ -97,13 +97,7 @@ public class SearchRequestTests extends AbstractSearchTestCase {
mutators.add(() -> mutation
.scroll(randomValueOtherThan(searchRequest.scroll(), () -> new Scroll(new TimeValue(randomPositiveLong() % 100000)))));
mutators.add(() -> mutation.searchType(randomValueOtherThan(searchRequest.searchType(), () -> randomFrom(SearchType.values()))));
mutators.add(() -> mutation.source(randomValueOtherThan(searchRequest.source(), () -> {
try {
return createSearchSourceBuilder();
} catch (IOException e) {
throw new RuntimeException(e);
}
})));
mutators.add(() -> mutation.source(randomValueOtherThan(searchRequest.source(), this::createSearchSourceBuilder)));
randomFrom(mutators).run();
return mutation;
}

View File

@ -341,6 +341,8 @@ take effect on after completing the current batch. This prevents scroll
timeouts.
[float]
=== Slicing
=== Manually slicing
Delete-by-query supports <<sliced-scroll>> allowing you to manually parallelize
@ -410,3 +412,93 @@ Which results in a sensible `total` like this one:
}
----------------------------------------------------------------
// TESTRESPONSE
==== Automatic slicing
You can also let delete-by-query automatically parallelize using
<<sliced-scroll>> to slice on `_uid`:
[source,js]
----------------------------------------------------------------
POST twitter/_delete_by_query?refresh&slices=5
{
"query": {
"range": {
"likes": {
"lt": 10
}
}
}
}
----------------------------------------------------------------
// CONSOLE
// TEST[setup:big_twitter]
Which you also can verify works with:
[source,js]
----------------------------------------------------------------
POST twitter/_search?size=0&filter_path=hits.total
{
"query": {
"range": {
"likes": {
"lt": 10
}
}
}
}
----------------------------------------------------------------
// CONSOLE
// TEST[continued]
Which results in a sensible `total` like this one:
[source,js]
----------------------------------------------------------------
{
"hits": {
"total": 0
}
}
----------------------------------------------------------------
// TESTRESPONSE
Adding `slices` to `_delete_by_query` just automates the manual process used in
the section above, creating sub-requests which means it has some quirks:
* You can see these requests in the
<<docs-delete-by-query-task-api,Tasks APIs>>. These sub-requests are "child"
tasks of the task for the request with `slices`.
* Fetching the status of the task for the request with `slices` only contains
the status of completed slices.
* These sub-requests are individually addressable for things like cancellation
and rethrottling.
* Rethrottling the request with `slices` will rethrottle the unfinished
sub-request proportionally.
* Canceling the request with `slices` will cancel each sub-request.
* Due to the nature of `slices` each sub-request won't get a perfectly even
portion of the documents. All documents will be addressed, but some slices may
be larger than others. Expect larger slices to have a more even distribution.
* Parameters like `requests_per_second` and `size` on a request with `slices`
are distributed proportionally to each sub-request. Combine that with the point
above about distribution being uneven and you should conclude that the using
`size` with `slices` might not result in exactly `size` documents being
`_delete_by_query`ed.
* Each sub-requests gets a slightly different snapshot of the source index
though these are all taken at approximately the same time.
==== Picking the number of slices
At this point we have a few recommendations around the number of `slices` to
use (the `max` parameter in the slice API if manually parallelizing):
* Don't use large numbers. `500` creates fairly massive CPU thrash.
* It is more efficient from a query performance standpoint to use some multiple
of the number of shards in the source index.
* Using exactly as many shards as are in the source index is the most efficient
from a query performance standpoint.
* Indexing performance should scale linearly across available resources with
the number of `slices`.
* Whether indexing or query performance dominates that process depends on lots
of factors like the documents being reindexed and the cluster doing the
reindexing.

View File

@ -737,9 +737,10 @@ and it'll look like:
Or you can search by `tag` or whatever you want.
[float]
=== Manually slicing
=== Slicing
Reindex supports <<sliced-scroll>> allowing you to manually parallelize the
==== Manual slicing
Reindex supports <<sliced-scroll>>, allowing you to manually parallelize the
process relatively easily:
[source,js]
@ -796,6 +797,85 @@ Which results in a sensible `total` like this one:
----------------------------------------------------------------
// TESTRESPONSE
==== Automatic slicing
You can also let reindex automatically parallelize using <<sliced-scroll>> to
slice on `_uid`:
[source,js]
----------------------------------------------------------------
POST _reindex?slices=5&refresh
{
"source": {
"index": "twitter"
},
"dest": {
"index": "new_twitter"
}
}
----------------------------------------------------------------
// CONSOLE
// TEST[setup:big_twitter]
Which you also can verify works with:
[source,js]
----------------------------------------------------------------
POST new_twitter/_search?size=0&filter_path=hits.total
----------------------------------------------------------------
// CONSOLE
// TEST[continued]
Which results in a sensible `total` like this one:
[source,js]
----------------------------------------------------------------
{
"hits": {
"total": 120
}
}
----------------------------------------------------------------
// TESTRESPONSE
Adding `slices` to `_reindex` just automates the manual process used in the
section above, creating sub-requests which means it has some quirks:
* You can see these requests in the <<docs-reindex-task-api,Tasks APIs>>. These
sub-requests are "child" tasks of the task for the request with `slices`.
* Fetching the status of the task for the request with `slices` only contains
the status of completed slices.
* These sub-requests are individually addressable for things like cancellation
and rethrottling.
* Rethrottling the request with `slices` will rethrottle the unfinished
sub-request proportionally.
* Canceling the request with `slices` will cancel each sub-request.
* Due to the nature of `slices` each sub-request won't get a perfectly even
portion of the documents. All documents will be addressed, but some slices may
be larger than others. Expect larger slices to have a more even distribution.
* Parameters like `requests_per_second` and `size` on a request with `slices`
are distributed proportionally to each sub-request. Combine that with the point
above about distribution being uneven and you should conclude that the using
`size` with `slices` might not result in exactly `size` documents being
`_reindex`ed.
* Each sub-requests gets a slightly different snapshot of the source index
though these are all taken at approximately the same time.
==== Picking the number of slices
At this point we have a few recommendations around the number of `slices` to
use (the `max` parameter in the slice API if manually parallelizing):
* Don't use large numbers. `500` creates fairly massive CPU thrash.
* It is more efficient from a query performance standpoint to use some multiple
of the number of shards in the source index.
* Using exactly as many shards as are in the source index is the most efficient
from a query performance standpoint.
* Indexing performance should scale linearly across available resources with
the number of `slices`.
* Whether indexing or query performance dominates that process depends on lots
of factors like the documents being reindexed and the cluster doing the
reindexing.
[float]
=== Reindex daily indices

View File

@ -406,8 +406,9 @@ take effect on after completing the current batch. This prevents scroll
timeouts.
[float]
=== Manually slicing
=== Slicing
==== Manual slicing
Update-by-query supports <<sliced-scroll>> allowing you to manually parallelize
the process relatively easily:
@ -459,6 +460,84 @@ Which results in a sensible `total` like this one:
----------------------------------------------------------------
// TESTRESPONSE
==== Automatic slicing
You can also let update-by-query automatically parallelize using
<<sliced-scroll>> to slice on `_uid`:
[source,js]
----------------------------------------------------------------
POST twitter/_update_by_query?refresh&slices=5
{
"script": {
"inline": "ctx._source['extra'] = 'test'"
}
}
----------------------------------------------------------------
// CONSOLE
// TEST[setup:big_twitter]
Which you also can verify works with:
[source,js]
----------------------------------------------------------------
POST twitter/_search?size=0&q=extra:test&filter_path=hits.total
----------------------------------------------------------------
// CONSOLE
// TEST[continued]
Which results in a sensible `total` like this one:
[source,js]
----------------------------------------------------------------
{
"hits": {
"total": 120
}
}
----------------------------------------------------------------
// TESTRESPONSE
Adding `slices` to `_update_by_query` just automates the manual process used in
the section above, creating sub-requests which means it has some quirks:
* You can see these requests in the
<<docs-update-by-query-task-api,Tasks APIs>>. These sub-requests are "child"
tasks of the task for the request with `slices`.
* Fetching the status of the task for the request with `slices` only contains
the status of completed slices.
* These sub-requests are individually addressable for things like cancellation
and rethrottling.
* Rethrottling the request with `slices` will rethrottle the unfinished
sub-request proportionally.
* Canceling the request with `slices` will cancel each sub-request.
* Due to the nature of `slices` each sub-request won't get a perfectly even
portion of the documents. All documents will be addressed, but some slices may
be larger than others. Expect larger slices to have a more even distribution.
* Parameters like `requests_per_second` and `size` on a request with `slices`
are distributed proportionally to each sub-request. Combine that with the point
above about distribution being uneven and you should conclude that the using
`size` with `slices` might not result in exactly `size` documents being
`_update_by_query`ed.
* Each sub-requests gets a slightly different snapshot of the source index
though these are all taken at approximately the same time.
==== Picking the number of slices
At this point we have a few recommendations around the number of `slices` to
use (the `max` parameter in the slice API if manually parallelizing):
* Don't use large numbers. `500` creates fairly massive CPU thrash.
* It is more efficient from a query performance standpoint to use some multiple
of the number of shards in the source index.
* Using exactly as many shards as are in the source index is the most efficient
from a query performance standpoint.
* Indexing performance should scale linearly across available resources with
the number of `slices`.
* Whether indexing or query performance dominates that process depends on lots
of factors like the documents being reindexed and the cluster doing the
reindexing.
[float]
[[picking-up-a-new-property]]
=== Pick up a new property

View File

@ -65,7 +65,7 @@ import static org.elasticsearch.search.sort.SortBuilders.fieldSort;
*/
public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBulkByScrollRequest<Request>> {
protected final Logger logger;
protected final BulkByScrollTask task;
protected final WorkingBulkByScrollTask task;
protected final ThreadPool threadPool;
/**
* The request for this action. Named mainRequest because we create lots of <code>request</code> variables all representing child
@ -81,7 +81,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
private final Retry bulkRetry;
private final ScrollableHitSource scrollSource;
public AbstractAsyncBulkByScrollAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
public AbstractAsyncBulkByScrollAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
ThreadPool threadPool, Request mainRequest, ActionListener<BulkIndexByScrollResponse> listener) {
this.task = task;
this.logger = logger;

View File

@ -71,7 +71,7 @@ public abstract class AbstractAsyncBulkIndexByScrollAction<Request extends Abstr
*/
private final BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> scriptApplier;
public AbstractAsyncBulkIndexByScrollAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
public AbstractAsyncBulkIndexByScrollAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
ThreadPool threadPool, Request mainRequest,
ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
@ -422,7 +422,7 @@ public abstract class AbstractAsyncBulkIndexByScrollAction<Request extends Abstr
*/
public abstract class ScriptApplier implements BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> {
private final BulkByScrollTask task;
private final WorkingBulkByScrollTask task;
private final ScriptService scriptService;
private final Script script;
private final Map<String, Object> params;
@ -430,7 +430,7 @@ public abstract class AbstractAsyncBulkIndexByScrollAction<Request extends Abstr
private ExecutableScript executable;
private Map<String, Object> context;
public ScriptApplier(BulkByScrollTask task, ScriptService scriptService, Script script,
public ScriptApplier(WorkingBulkByScrollTask task, ScriptService scriptService, Script script,
Map<String, Object> params) {
this.task = task;
this.scriptService = scriptService;

View File

@ -26,15 +26,11 @@ import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.BytesRestResponse;
import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.search.SearchRequestParsers;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.suggest.Suggesters;
import org.elasticsearch.tasks.LoggingTaskListener;
import org.elasticsearch.tasks.Task;
@ -101,6 +97,7 @@ public abstract class AbstractBaseReindexRestHandler<
request.setRefresh(restRequest.paramAsBoolean("refresh", request.isRefresh()));
request.setTimeout(restRequest.paramAsTime("timeout", request.getTimeout()));
request.setSlices(restRequest.paramAsInt("slices", request.getSlices()));
String waitForActiveShards = restRequest.param("wait_for_active_shards");
if (waitForActiveShards != null) {

View File

@ -98,16 +98,33 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
*/
private boolean shouldStoreResult;
/**
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
*/
private int slices = 1;
/**
* Constructor for deserialization.
*/
public AbstractBulkByScrollRequest() {
}
public AbstractBulkByScrollRequest(SearchRequest source) {
this.searchRequest = source;
/**
* Constructor for actual use.
*
* @param searchRequest the search request to execute to get the documents to process
* @param setDefaults should this request set the defaults on the search request? Usually set to true but leave it false to support
* request slicing
*/
public AbstractBulkByScrollRequest(SearchRequest searchRequest, boolean setDefaults) {
this.searchRequest = searchRequest;
// Set the defaults which differ from SearchRequest's defaults.
source.scroll(DEFAULT_SCROLL_TIMEOUT);
source.source(new SearchSourceBuilder());
source.source().size(DEFAULT_SCROLL_SIZE);
if (setDefaults) {
searchRequest.scroll(DEFAULT_SCROLL_TIMEOUT);
searchRequest.source(new SearchSourceBuilder());
searchRequest.source().size(DEFAULT_SCROLL_SIZE);
}
}
/**
@ -134,6 +151,9 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
+ size + "]",
e);
}
if (searchRequest.source().slice() != null && slices != 1) {
e = addValidationError("can't specify both slice and workers", e);
}
return e;
}
@ -313,9 +333,59 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
return shouldStoreResult;
}
/**
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
*/
public Self setSlices(int slices) {
if (slices < 1) {
throw new IllegalArgumentException("[slices] must be at least 1");
}
this.slices = slices;
return self();
}
/**
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
*/
public int getSlices() {
return slices;
}
/**
* Build a new request for a slice of the parent request.
*/
abstract Self forSlice(TaskId slicingTask, SearchRequest slice);
/**
* Setup a clone of this request with the information needed to process a slice of it.
*/
protected Self doForSlice(Self request, TaskId slicingTask) {
request.setAbortOnVersionConflict(abortOnVersionConflict).setRefresh(refresh).setTimeout(timeout)
.setWaitForActiveShards(activeShardCount).setRetryBackoffInitialTime(retryBackoffInitialTime).setMaxRetries(maxRetries)
// Parent task will store result
.setShouldStoreResult(false)
// Split requests per second between all slices
.setRequestsPerSecond(requestsPerSecond / slices)
// Size is split between workers. This means the size might round down!
.setSize(size == SIZE_ALL_MATCHES ? SIZE_ALL_MATCHES : size / slices)
// Sub requests don't have workers
.setSlices(1);
// Set the parent task so this task is cancelled if we cancel the parent
request.setParentTask(slicingTask);
// TODO It'd be nice not to refresh on every slice. Instead we should refresh after the sub requests finish.
return request;
}
@Override
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
return new BulkByScrollTask(id, type, action, getDescription(), parentTaskId, requestsPerSecond);
if (slices > 1) {
return new ParentBulkByScrollTask(id, type, action, getDescription(), parentTaskId, slices);
}
/* Extract the slice from the search request so it'll be available in the status. This is potentially useful for users that manually
* slice their search requests so they can keep track of it and **absolutely** useful for automatically sliced reindex requests so
* they can properly track the responses. */
Integer sliceId = searchRequest.source().slice() == null ? null : searchRequest.source().slice().getId();
return new WorkingBulkByScrollTask(id, type, action, getDescription(), parentTaskId, sliceId, requestsPerSecond);
}
@Override
@ -331,6 +401,11 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
retryBackoffInitialTime = new TimeValue(in);
maxRetries = in.readVInt();
requestsPerSecond = in.readFloat();
if (in.getVersion().onOrAfter(BulkByScrollTask.V_5_1_0_UNRELEASED)) {
slices = in.readVInt();
} else {
slices = 1;
}
}
@Override
@ -345,6 +420,14 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
retryBackoffInitialTime.writeTo(out);
out.writeVInt(maxRetries);
out.writeFloat(requestsPerSecond);
if (out.getVersion().onOrAfter(BulkByScrollTask.V_5_1_0_UNRELEASED)) {
out.writeVInt(slices);
} else {
if (slices > 1) {
throw new UnsupportedOperationException("Attempting to send sliced reindex-style request to a node that doesn't support "
+ "it. Version is [" + out.getVersion() + "] but must be [" + BulkByScrollTask.V_5_1_0_UNRELEASED + "]");
}
}
}
/**

View File

@ -141,4 +141,12 @@ public abstract class AbstractBulkByScrollRequestBuilder<
request.setShouldStoreResult(shouldStoreResult);
return self();
}
/**
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
*/
public Self setSlices(int workers) {
request.setSlices(workers);
return self();
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.script.Script;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
@ -34,11 +35,21 @@ public abstract class AbstractBulkIndexByScrollRequest<Self extends AbstractBulk
*/
private Script script;
/**
* Constructor for deserialization.
*/
public AbstractBulkIndexByScrollRequest() {
}
public AbstractBulkIndexByScrollRequest(SearchRequest source) {
super(source);
/**
* Constructor for actual use.
*
* @param searchRequest the search request to execute to get the documents to process
* @param setDefaults should this request set the defaults on the search request? Usually set to true but leave it false to support
* request slicing
*/
protected AbstractBulkIndexByScrollRequest(SearchRequest searchRequest, boolean setDefaults) {
super(searchRequest, setDefaults);
}
/**
@ -56,6 +67,11 @@ public abstract class AbstractBulkIndexByScrollRequest<Self extends AbstractBulk
return self();
}
@Override
protected Self doForSlice(Self request, TaskId slicingTask) {
return super.doForSlice(request, slicingTask).setScript(script);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);

View File

@ -19,99 +19,61 @@
package org.elasticsearch.index.reindex;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.tasks.TaskInfo;
import java.io.IOException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.List;
import java.util.Objects;
import static java.lang.Math.max;
import static java.lang.Math.round;
import static java.lang.Math.min;
import static java.util.Collections.emptyList;
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
/**
* Task storing information about a currently running BulkByScroll request.
*/
public class BulkByScrollTask extends CancellableTask {
private static final Logger logger = ESLoggerFactory.getLogger(BulkByScrollTask.class.getPackage().getName());
public abstract class BulkByScrollTask extends CancellableTask {
static final Version V_5_1_0_UNRELEASED = Version.fromId(5010099);
/**
* The total number of documents this request will process. 0 means we don't yet know or, possibly, there are actually 0 documents
* to process. Its ok that these have the same meaning because any request with 0 actual documents should be quite short lived.
*/
private final AtomicLong total = new AtomicLong(0);
private final AtomicLong updated = new AtomicLong(0);
private final AtomicLong created = new AtomicLong(0);
private final AtomicLong deleted = new AtomicLong(0);
private final AtomicLong noops = new AtomicLong(0);
private final AtomicInteger batch = new AtomicInteger(0);
private final AtomicLong versionConflicts = new AtomicLong(0);
private final AtomicLong bulkRetries = new AtomicLong(0);
private final AtomicLong searchRetries = new AtomicLong(0);
private final AtomicLong throttledNanos = new AtomicLong();
/**
* The number of requests per second to which to throttle the request that this task represents. The other variables are all AtomicXXX
* style variables but there isn't an AtomicFloat so we just use a volatile.
*/
private volatile float requestsPerSecond;
/**
* Reference to any the last delayed prepareBulkRequest call. Used during rethrottling and canceling to reschedule the request.
*/
private final AtomicReference<DelayedPrepareBulkRequest> delayedPrepareBulkRequestReference = new AtomicReference<>();
public BulkByScrollTask(long id, String type, String action, String description, TaskId parentTask, float requestsPerSecond) {
super(id, type, action, description, parentTask);
setRequestsPerSecond(requestsPerSecond);
}
@Override
protected void onCancelled() {
// Drop the throttle to 0, immediately rescheduling all outstanding tasks so the task will wake up and cancel itself.
rethrottle(0);
}
@Override
public Status getStatus() {
return new Status(total.get(), updated.get(), created.get(), deleted.get(), batch.get(), versionConflicts.get(), noops.get(),
bulkRetries.get(), searchRetries.get(), timeValueNanos(throttledNanos.get()), getRequestsPerSecond(), getReasonCancelled(),
throttledUntil());
}
private TimeValue throttledUntil() {
DelayedPrepareBulkRequest delayed = delayedPrepareBulkRequestReference.get();
if (delayed == null) {
return timeValueNanos(0);
}
if (delayed.future == null) {
return timeValueNanos(0);
}
return timeValueNanos(max(0, delayed.future.getDelay(TimeUnit.NANOSECONDS)));
public BulkByScrollTask(long id, String type, String action, String description, TaskId parentTaskId) {
super(id, type, action, description, parentTaskId);
}
/**
* Total number of successfully processed documents.
* The number of sub-slices that are still running. {@link WorkingBulkByScrollTask} will always have 0 and
* {@link ParentBulkByScrollTask} will return the number of waiting tasks. Used by {@link TransportRethrottleAction} to decide how to
* perform the rethrottling.
*/
public long getSuccessfullyProcessed() {
return updated.get() + created.get() + deleted.get();
}
abstract int runningSliceSubTasks();
public static class Status implements Task.Status {
/**
* Apply the {@code newRequestsPerSecond}.
*/
abstract void rethrottle(float newRequestsPerSecond);
/*
* Overridden to force children to return compatible status.
*/
public abstract BulkByScrollTask.Status getStatus();
/**
* Build the status for this task given a snapshot of the information of running slices.
*/
public abstract TaskInfo getInfoGivenSliceInfo(String localNodeId, List<TaskInfo> sliceInfo);
public static class Status implements Task.Status, SuccessfullyProcessed {
public static final String NAME = "bulk-by-scroll";
/**
@ -126,6 +88,7 @@ public class BulkByScrollTask extends CancellableTask {
*/
public static final String INCLUDE_UPDATED = "include_updated";
private final Integer sliceId;
private final long total;
private final long updated;
private final long created;
@ -139,10 +102,12 @@ public class BulkByScrollTask extends CancellableTask {
private final float requestsPerSecond;
private final String reasonCancelled;
private final TimeValue throttledUntil;
private final List<StatusOrException> sliceStatuses;
public Status(long total, long updated, long created, long deleted, int batches, long versionConflicts, long noops,
public Status(Integer sliceId, long total, long updated, long created, long deleted, int batches, long versionConflicts, long noops,
long bulkRetries, long searchRetries, TimeValue throttled, float requestsPerSecond, @Nullable String reasonCancelled,
TimeValue throttledUntil) {
this.sliceId = sliceId == null ? null : checkPositive(sliceId, "sliceId");
this.total = checkPositive(total, "total");
this.updated = checkPositive(updated, "updated");
this.created = checkPositive(created, "created");
@ -156,9 +121,77 @@ public class BulkByScrollTask extends CancellableTask {
this.requestsPerSecond = requestsPerSecond;
this.reasonCancelled = reasonCancelled;
this.throttledUntil = throttledUntil;
this.sliceStatuses = emptyList();
}
/**
* Constructor merging many statuses.
*
* @param sliceStatuses Statuses of sub requests that this task was sliced into.
* @param reasonCancelled Reason that this *this* task was cancelled. Note that each entry in {@code sliceStatuses} can be cancelled
* independently of this task but if this task is cancelled then the workers *should* be cancelled.
*/
public Status(List<StatusOrException> sliceStatuses, @Nullable String reasonCancelled) {
sliceId = null;
this.reasonCancelled = reasonCancelled;
long mergedTotal = 0;
long mergedUpdated = 0;
long mergedCreated = 0;
long mergedDeleted = 0;
int mergedBatches = 0;
long mergedVersionConflicts = 0;
long mergedNoops = 0;
long mergedBulkRetries = 0;
long mergedSearchRetries = 0;
long mergedThrottled = 0;
float mergedRequestsPerSecond = 0;
long mergedThrottledUntil = Long.MAX_VALUE;
for (StatusOrException slice : sliceStatuses) {
if (slice == null) {
// Hasn't returned yet.
continue;
}
if (slice.status == null) {
// This slice failed catastrophically so it doesn't count towards the status
continue;
}
mergedTotal += slice.status.getTotal();
mergedUpdated += slice.status.getUpdated();
mergedCreated += slice.status.getCreated();
mergedDeleted += slice.status.getDeleted();
mergedBatches += slice.status.getBatches();
mergedVersionConflicts += slice.status.getVersionConflicts();
mergedNoops += slice.status.getNoops();
mergedBulkRetries += slice.status.getBulkRetries();
mergedSearchRetries += slice.status.getSearchRetries();
mergedThrottled += slice.status.getThrottled().nanos();
mergedRequestsPerSecond += slice.status.getRequestsPerSecond();
mergedThrottledUntil = min(mergedThrottledUntil, slice.status.getThrottledUntil().nanos());
}
total = mergedTotal;
updated = mergedUpdated;
created = mergedCreated;
deleted = mergedDeleted;
batches = mergedBatches;
versionConflicts = mergedVersionConflicts;
noops = mergedNoops;
bulkRetries = mergedBulkRetries;
searchRetries = mergedSearchRetries;
throttled = timeValueNanos(mergedThrottled);
requestsPerSecond = mergedRequestsPerSecond;
throttledUntil = timeValueNanos(mergedThrottledUntil == Long.MAX_VALUE ? 0 : mergedThrottledUntil);
this.sliceStatuses = sliceStatuses;
}
public Status(StreamInput in) throws IOException {
if (in.getVersion().onOrAfter(V_5_1_0_UNRELEASED)) {
sliceId = in.readOptionalVInt();
} else {
sliceId = null;
}
total = in.readVLong();
updated = in.readVLong();
created = in.readVLong();
@ -172,10 +205,18 @@ public class BulkByScrollTask extends CancellableTask {
requestsPerSecond = in.readFloat();
reasonCancelled = in.readOptionalString();
throttledUntil = new TimeValue(in);
if (in.getVersion().onOrAfter(V_5_1_0_UNRELEASED)) {
sliceStatuses = in.readList(stream -> stream.readOptionalWriteable(StatusOrException::new));
} else {
sliceStatuses = emptyList();
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
if (out.getVersion().onOrAfter(V_5_1_0_UNRELEASED)) {
out.writeOptionalVInt(sliceId);
}
out.writeVLong(total);
out.writeVLong(updated);
out.writeVLong(created);
@ -189,6 +230,17 @@ public class BulkByScrollTask extends CancellableTask {
out.writeFloat(requestsPerSecond);
out.writeOptionalString(reasonCancelled);
throttledUntil.writeTo(out);
if (out.getVersion().onOrAfter(V_5_1_0_UNRELEASED)) {
out.writeVInt(sliceStatuses.size());
for (StatusOrException sliceStatus : sliceStatuses) {
out.writeOptionalWriteable(sliceStatus);
}
}
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
@ -200,6 +252,9 @@ public class BulkByScrollTask extends CancellableTask {
public XContentBuilder innerXContent(XContentBuilder builder, Params params)
throws IOException {
if (sliceId != null) {
builder.field("slice_id", sliceId);
}
builder.field("total", total);
if (params.paramAsBoolean(INCLUDE_UPDATED, true)) {
builder.field("updated", updated);
@ -222,6 +277,17 @@ public class BulkByScrollTask extends CancellableTask {
builder.field("canceled", reasonCancelled);
}
builder.timeValueField("throttled_until_millis", "throttled_until", throttledUntil);
if (false == sliceStatuses.isEmpty()) {
builder.startArray("slices");
for (StatusOrException slice : sliceStatuses) {
if (slice == null) {
builder.nullValue();
} else {
slice.toXContent(builder, params);
}
}
builder.endArray();
}
return builder;
}
@ -234,7 +300,8 @@ public class BulkByScrollTask extends CancellableTask {
}
public void innerToString(StringBuilder builder) {
builder.append("updated=").append(updated);
builder.append("sliceId=").append(sliceId);
builder.append(",updated=").append(updated);
builder.append(",created=").append(created);
builder.append(",deleted=").append(deleted);
builder.append(",batches=").append(batches);
@ -245,11 +312,16 @@ public class BulkByScrollTask extends CancellableTask {
builder.append(",canceled=").append(reasonCancelled);
}
builder.append(",throttledUntil=").append(throttledUntil);
if (false == sliceStatuses.isEmpty()) {
builder.append(",workers=").append(sliceStatuses);
}
}
@Override
public String getWriteableName() {
return NAME;
/**
* The id of the slice that this status is reporting or {@code null} if this isn't the status of a sub-slice.
*/
Integer getSliceId() {
return sliceId;
}
/**
@ -260,23 +332,17 @@ public class BulkByScrollTask extends CancellableTask {
return total;
}
/**
* Count of documents updated.
*/
@Override
public long getUpdated() {
return updated;
}
/**
* Count of documents created.
*/
@Override
public long getCreated() {
return created;
}
/**
* Count of successful delete operations.
*/
@Override
public long getDeleted() {
return deleted;
}
@ -344,6 +410,13 @@ public class BulkByScrollTask extends CancellableTask {
return throttledUntil;
}
/**
* Statuses of the sub requests into which this sub-request was sliced. Empty if this request wasn't sliced into sub-requests.
*/
public List<StatusOrException> getSliceStatuses() {
return sliceStatuses;
}
private int checkPositive(int value, String name) {
if (value < 0) {
throw new IllegalArgumentException(name + " must be greater than 0 but was [" + value + "]");
@ -359,195 +432,84 @@ public class BulkByScrollTask extends CancellableTask {
}
}
void setTotal(long totalHits) {
total.set(totalHits);
}
void countBatch() {
batch.incrementAndGet();
}
void countNoop() {
noops.incrementAndGet();
}
void countCreated() {
created.incrementAndGet();
}
void countUpdated() {
updated.incrementAndGet();
}
void countDeleted() {
deleted.incrementAndGet();
}
void countVersionConflict() {
versionConflicts.incrementAndGet();
}
void countBulkRetry() {
bulkRetries.incrementAndGet();
}
void countSearchRetry() {
searchRetries.incrementAndGet();
}
float getRequestsPerSecond() {
return requestsPerSecond;
}
/**
* Schedule prepareBulkRequestRunnable to run after some delay. This is where throttling plugs into reindexing so the request can be
* rescheduled over and over again.
* The status of a slice of the request. Successful requests store the {@link StatusOrException#status} while failing requests store a
* {@link StatusOrException#exception}.
*/
void delayPrepareBulkRequest(ThreadPool threadPool, TimeValue lastBatchStartTime, int lastBatchSize,
AbstractRunnable prepareBulkRequestRunnable) {
// Synchronize so we are less likely to schedule the same request twice.
synchronized (delayedPrepareBulkRequestReference) {
TimeValue delay = throttleWaitTime(lastBatchStartTime, lastBatchSize);
delayedPrepareBulkRequestReference.set(new DelayedPrepareBulkRequest(threadPool, getRequestsPerSecond(),
delay, new RunOnce(prepareBulkRequestRunnable)));
}
}
public static class StatusOrException implements Writeable, ToXContent {
private final Status status;
private final Exception exception;
TimeValue throttleWaitTime(TimeValue lastBatchStartTime, int lastBatchSize) {
long earliestNextBatchStartTime = lastBatchStartTime.nanos() + (long) perfectlyThrottledBatchTime(lastBatchSize);
return timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime()));
}
/**
* How many nanoseconds should a batch of lastBatchSize have taken if it were perfectly throttled? Package private for testing.
*/
float perfectlyThrottledBatchTime(int lastBatchSize) {
if (requestsPerSecond == Float.POSITIVE_INFINITY) {
return 0;
}
// requests
// ------------------- == seconds
// request per seconds
float targetBatchTimeInSeconds = lastBatchSize / requestsPerSecond;
// nanoseconds per seconds * seconds == nanoseconds
return TimeUnit.SECONDS.toNanos(1) * targetBatchTimeInSeconds;
}
private void setRequestsPerSecond(float requestsPerSecond) {
this.requestsPerSecond = requestsPerSecond;
}
void rethrottle(float newRequestsPerSecond) {
synchronized (delayedPrepareBulkRequestReference) {
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Rethrottling to [{}] requests per second", getId(), newRequestsPerSecond);
}
setRequestsPerSecond(newRequestsPerSecond);
DelayedPrepareBulkRequest delayedPrepareBulkRequest = this.delayedPrepareBulkRequestReference.get();
if (delayedPrepareBulkRequest == null) {
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Skipping rescheduling because there is no scheduled task", getId());
}
// No request has been queued yet so nothing to reschedule.
return;
}
this.delayedPrepareBulkRequestReference.set(delayedPrepareBulkRequest.rethrottle(newRequestsPerSecond));
}
}
class DelayedPrepareBulkRequest {
private final ThreadPool threadPool;
private final AbstractRunnable command;
private final float requestsPerSecond;
private final ScheduledFuture<?> future;
DelayedPrepareBulkRequest(ThreadPool threadPool, float requestsPerSecond, TimeValue delay, AbstractRunnable command) {
this.threadPool = threadPool;
this.requestsPerSecond = requestsPerSecond;
this.command = command;
this.future = threadPool.schedule(delay, ThreadPool.Names.GENERIC, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
throttledNanos.addAndGet(delay.nanos());
command.run();
}
@Override
public void onFailure(Exception e) {
command.onFailure(e);
}
});
public StatusOrException(Status status) {
this.status = status;
exception = null;
}
DelayedPrepareBulkRequest rethrottle(float newRequestsPerSecond) {
if (newRequestsPerSecond != 0 && newRequestsPerSecond < requestsPerSecond) {
/*
* The user is attempting to slow the request down. We'll let the change in throttle take effect the next time we delay
* prepareBulkRequest. We can't just reschedule the request further out in the future the bulk context might time out.
*/
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Skipping rescheduling because the new throttle [{}] is slower than the old one [{}].", getId(),
newRequestsPerSecond, requestsPerSecond);
}
return this;
}
long remainingDelay = future.getDelay(TimeUnit.NANOSECONDS);
// Actually reschedule the task
if (false == FutureUtils.cancel(future)) {
// Couldn't cancel, probably because the task has finished or been scheduled. Either way we have nothing to do here.
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Skipping rescheduling we couldn't cancel the task.", getId());
}
return this;
}
/*
* Strangely enough getting here doesn't mean that you actually cancelled the request, just that you probably did. If you stress
* test it you'll find that requests sneak through. So each request is given a runOnce boolean to prevent that.
*/
TimeValue newDelay = newDelay(remainingDelay, newRequestsPerSecond);
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Rescheduling for [{}] in the future.", getId(), newDelay);
}
return new DelayedPrepareBulkRequest(threadPool, requestsPerSecond, newDelay, command);
public StatusOrException(Exception exception) {
status = null;
this.exception = exception;
}
/**
* Scale back remaining delay to fit the new delay.
* Read from a stream.
*/
TimeValue newDelay(long remainingDelay, float newRequestsPerSecond) {
if (remainingDelay < 0 || newRequestsPerSecond == 0) {
return timeValueNanos(0);
}
return timeValueNanos(round(remainingDelay * requestsPerSecond / newRequestsPerSecond));
}
}
/**
* Runnable that can only be run one time. This is paranoia to prevent furiously rethrottling from running the command multiple times.
* Without it the command would be run multiple times.
*/
private static class RunOnce extends AbstractRunnable {
private final AtomicBoolean hasRun = new AtomicBoolean(false);
private final AbstractRunnable delegate;
public RunOnce(AbstractRunnable delegate) {
this.delegate = delegate;
}
@Override
protected void doRun() throws Exception {
if (hasRun.compareAndSet(false, true)) {
delegate.run();
public StatusOrException(StreamInput in) throws IOException {
if (in.readBoolean()) {
status = new Status(in);
exception = null;
} else {
status = null;
exception = in.readException();
}
}
@Override
public void onFailure(Exception e) {
delegate.onFailure(e);
public void writeTo(StreamOutput out) throws IOException {
if (exception == null) {
out.writeBoolean(true);
status.writeTo(out);
} else {
out.writeBoolean(false);
out.writeException(exception);
}
}
public Status getStatus() {
return status;
}
public Exception getException() {
return exception;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (exception == null) {
status.toXContent(builder, params);
} else {
builder.startObject();
ElasticsearchException.toXContent(builder, params, exception);
builder.endObject();
}
return builder;
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj.getClass() != BulkByScrollTask.StatusOrException.class) {
return false;
}
BulkByScrollTask.StatusOrException other = (StatusOrException) obj;
return Objects.equals(status, other.status)
&& Objects.equals(exception, other.exception);
}
@Override
public int hashCode() {
return Objects.hash(status, exception);
}
}
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.TimeValue;
@ -29,10 +30,13 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.reindex.ScrollableHitSource.SearchFailure;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import static java.lang.Math.max;
import static java.lang.Math.min;
import static java.util.Objects.requireNonNull;
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
/**
* Response used for actions that index many documents using a scroll request.
@ -56,6 +60,22 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
this.timedOut = timedOut;
}
public BulkIndexByScrollResponse(Iterable<BulkIndexByScrollResponse> toMerge, @Nullable String reasonCancelled) {
long mergedTook = 0;
List<BulkByScrollTask.StatusOrException> statuses = new ArrayList<>();
bulkFailures = new ArrayList<>();
searchFailures = new ArrayList<>();
for (BulkIndexByScrollResponse response : toMerge) {
mergedTook = max(mergedTook, response.getTook().nanos());
statuses.add(new BulkByScrollTask.StatusOrException(response.status));
bulkFailures.addAll(response.getBulkFailures());
searchFailures.addAll(response.getSearchFailures());
timedOut |= response.isTimedOut();
}
took = timeValueNanos(mergedTook);
status = new BulkByScrollTask.Status(statuses, reasonCancelled);
}
public TimeValue getTook() {
return took;
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.tasks.TaskId;
import static org.elasticsearch.action.ValidateActions.addValidationError;
@ -49,9 +50,15 @@ public class DeleteByQueryRequest extends AbstractBulkByScrollRequest<DeleteByQu
}
public DeleteByQueryRequest(SearchRequest search) {
super(search);
this(search, true);
}
private DeleteByQueryRequest(SearchRequest search, boolean setDefaults) {
super(search, setDefaults);
// Delete-By-Query does not require the source
search.source().fetchSource(false);
if (setDefaults) {
search.source().fetchSource(false);
}
}
@Override
@ -71,6 +78,11 @@ public class DeleteByQueryRequest extends AbstractBulkByScrollRequest<DeleteByQu
return e;
}
@Override
DeleteByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
return doForSlice(new DeleteByQueryRequest(slice, false), slicingTask);
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();

View File

@ -0,0 +1,141 @@
/*
* 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.reindex;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskInfo;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import static java.util.Collections.unmodifiableList;
/**
* Task for parent bulk by scroll requests that have sub-workers.
*/
class ParentBulkByScrollTask extends BulkByScrollTask {
/**
* Holds the responses as they come back. This uses {@link Tuple} as an "Either" style holder where only the response or the exception
* is set.
*/
private final AtomicArray<Tuple<BulkIndexByScrollResponse, Exception>> results;
private final AtomicInteger counter;
public ParentBulkByScrollTask(long id, String type, String action, String description, TaskId parentTaskId, int slices) {
super(id, type, action, description, parentTaskId);
this.results = new AtomicArray<>(slices);
this.counter = new AtomicInteger(slices);
}
@Override
void rethrottle(float newRequestsPerSecond) {
// Nothing to do because all rethrottling is done on slice sub tasks.
}
@Override
public Status getStatus() {
// We only have access to the statuses of requests that have finished so we return them
List<StatusOrException> statuses = Arrays.asList(new StatusOrException[results.length()]);
addResultsToList(statuses);
return new Status(unmodifiableList(statuses), getReasonCancelled());
}
@Override
int runningSliceSubTasks() {
return counter.get();
}
@Override
public TaskInfo getInfoGivenSliceInfo(String localNodeId, List<TaskInfo> sliceInfo) {
/* Merge the list of finished sub requests with the provided info. If a slice is both finished and in the list then we prefer the
* finished status because we don't expect them to change after the task is finished. */
List<StatusOrException> sliceStatuses = Arrays.asList(new StatusOrException[results.length()]);
for (TaskInfo t : sliceInfo) {
Status status = (Status) t.getStatus();
sliceStatuses.set(status.getSliceId(), new StatusOrException(status));
}
addResultsToList(sliceStatuses);
Status status = new Status(sliceStatuses, getReasonCancelled());
return taskInfo(localNodeId, getDescription(), status);
}
private void addResultsToList(List<StatusOrException> sliceStatuses) {
for (AtomicArray.Entry<Tuple<BulkIndexByScrollResponse, Exception>> t : results.asList()) {
if (t.value != null) {
if (t.value.v1() != null) {
sliceStatuses.set(t.index, new StatusOrException(t.value.v1().getStatus()));
} else {
sliceStatuses.set(t.index, new StatusOrException(t.value.v2()));
}
}
}
}
/**
* Record a response from a slice and respond to the listener if the request is finished.
*/
void onSliceResponse(ActionListener<BulkIndexByScrollResponse> listener, int sliceId, BulkIndexByScrollResponse response) {
results.setOnce(sliceId, new Tuple<>(response, null));
/* If the request isn't finished we could automatically rethrottle the sub-requests here but we would only want to do that if we
* were fairly sure they had a while left to go. */
recordSliceCompletionAndRespondIfAllDone(listener);
}
/**
* Record a failure from a slice and respond to the listener if the request is finished.
*/
void onSliceFailure(ActionListener<BulkIndexByScrollResponse> listener, int sliceId, Exception e) {
results.setOnce(sliceId, new Tuple<>(null, e));
recordSliceCompletionAndRespondIfAllDone(listener);
// TODO cancel when a slice fails?
}
private void recordSliceCompletionAndRespondIfAllDone(ActionListener<BulkIndexByScrollResponse> listener) {
if (counter.decrementAndGet() != 0) {
return;
}
List<BulkIndexByScrollResponse> responses = new ArrayList<>(results.length());
Exception exception = null;
for (AtomicArray.Entry<Tuple<BulkIndexByScrollResponse, Exception>> t : results.asList()) {
if (t.value.v1() == null) {
assert t.value.v2() != null : "exception shouldn't be null if value is null";
if (exception == null) {
exception = t.value.v2();
} else {
exception.addSuppressed(t.value.v2());
}
} else {
assert t.value.v2() == null : "exception should be null if response is not null";
responses.add(t.value.v1());
}
}
if (exception == null) {
listener.onResponse(new BulkIndexByScrollResponse(responses, getReasonCancelled()));
} else {
listener.onFailure(exception);
}
}
}

View File

@ -0,0 +1,87 @@
/*
* 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.reindex;
import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.client.Client;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.slice.SliceBuilder;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskManager;
/**
* Helps parallelize reindex requests using sliced scrolls.
*/
public class ReindexParallelizationHelper {
private ReindexParallelizationHelper() {}
public static <
Request extends AbstractBulkByScrollRequest<Request>
> void startSlices(Client client, TaskManager taskManager, Action<Request, BulkIndexByScrollResponse, ?> action,
String localNodeId, ParentBulkByScrollTask task, Request request, ActionListener<BulkIndexByScrollResponse> listener) {
TaskId parentTaskId = new TaskId(localNodeId, task.getId());
for (final SearchRequest slice : sliceIntoSubRequests(request.getSearchRequest(), UidFieldMapper.NAME, request.getSlices())) {
// TODO move the request to the correct node. maybe here or somehow do it as part of startup for reindex in general....
Request requestForSlice = request.forSlice(parentTaskId, slice);
ActionListener<BulkIndexByScrollResponse> sliceListener = ActionListener.wrap(
r -> task.onSliceResponse(listener, slice.source().slice().getId(), r),
e -> task.onSliceFailure(listener, slice.source().slice().getId(), e));
client.execute(action, requestForSlice, sliceListener);
/* Explicitly tell the task manager that we're running child tasks on the local node so it will cancel them when the parent is
* cancelled. */
taskManager.registerChildTask(task, localNodeId);
}
}
/**
* Slice a search request into {@code times} separate search requests slicing on {@code field}. Note that the slices are *shallow*
* copies of this request so don't change them.
*/
static SearchRequest[] sliceIntoSubRequests(SearchRequest request, String field, int times) {
SearchRequest[] slices = new SearchRequest[times];
for (int slice = 0; slice < times; slice++) {
SliceBuilder sliceBuilder = new SliceBuilder(field, slice, times);
SearchSourceBuilder slicedSource;
if (request.source() == null) {
slicedSource = new SearchSourceBuilder().slice(sliceBuilder);
} else {
if (request.source().slice() != null) {
throw new IllegalStateException("Can't slice a request that already has a slice configuration");
}
slicedSource = request.source().copyWithNewSlice(sliceBuilder);
}
slices[slice] = new SearchRequest()
.source(slicedSource)
.searchType(request.searchType())
.indices(request.indices())
.types(request.types())
.routing(request.routing())
.preference(request.preference())
.requestCache(request.requestCache())
.scroll(request.scroll())
.indicesOptions(request.indicesOptions());
}
return slices;
}
}

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.index.reindex.remote.RemoteInfo;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
import java.util.Arrays;
@ -56,7 +57,11 @@ public class ReindexRequest extends AbstractBulkIndexByScrollRequest<ReindexRequ
}
public ReindexRequest(SearchRequest search, IndexRequest destination) {
super(search);
this(search, destination, true);
}
private ReindexRequest(SearchRequest search, IndexRequest destination, boolean setDefaults) {
super(search, setDefaults);
this.destination = destination;
}
@ -94,8 +99,13 @@ public class ReindexRequest extends AbstractBulkIndexByScrollRequest<ReindexRequ
if (destination.timestamp() != null) {
e = addValidationError("setting timestamp on destination isn't supported. use scripts instead.", e);
}
if (getRemoteInfo() != null && getSearchRequest().source().query() != null) {
e = addValidationError("reindex from remote sources should use RemoteInfo's query instead of source's query", e);
if (getRemoteInfo() != null) {
if (getSearchRequest().source().query() != null) {
e = addValidationError("reindex from remote sources should use RemoteInfo's query instead of source's query", e);
}
if (getSlices() != 1) {
e = addValidationError("reindex from remote sources doesn't support workers > 1 but was [" + getSlices() + "]", e);
}
}
return e;
}
@ -125,6 +135,13 @@ public class ReindexRequest extends AbstractBulkIndexByScrollRequest<ReindexRequ
return remoteInfo;
}
@Override
ReindexRequest forSlice(TaskId slicingTask, SearchRequest slice) {
ReindexRequest sliced = doForSlice(new ReindexRequest(slice, destination, false), slicingTask);
sliced.setRemoteInfo(remoteInfo);
return sliced;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.Script.ScriptField;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.SearchRequestParsers;
@ -39,7 +40,6 @@ import java.util.Map;
import java.util.function.Consumer;
import static org.elasticsearch.rest.RestRequest.Method.POST;
import static org.elasticsearch.script.Script.ScriptField;
public class RestUpdateByQueryAction extends AbstractBulkByQueryRestHandler<UpdateByQueryRequest, UpdateByQueryAction> {

View File

@ -0,0 +1,46 @@
/*
* 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.reindex;
/**
* Implemented by {@link BulkByScrollTask} and {@link BulkByScrollTask.Status} to consistently implement
* {@link #getSuccessfullyProcessed()}.
*/
public interface SuccessfullyProcessed {
/**
* Total number of successfully processed documents.
*/
default long getSuccessfullyProcessed() {
return getUpdated() + getCreated() + getDeleted();
}
/**
* Count of documents updated.
*/
long getUpdated();
/**
* Count of documents created.
*/
long getCreated();
/**
* Count of successful delete operations.
*/
long getDeleted();
}

View File

@ -53,9 +53,15 @@ public class TransportDeleteByQueryAction extends HandledTransportAction<DeleteB
@Override
protected void doExecute(Task task, DeleteByQueryRequest request, ActionListener<BulkIndexByScrollResponse> listener) {
ClusterState state = clusterService.state();
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
new AsyncDeleteBySearchAction((BulkByScrollTask) task, logger, client, threadPool, request, listener, scriptService, state).start();
if (request.getSlices() > 1) {
ReindexParallelizationHelper.startSlices(client, taskManager, DeleteByQueryAction.INSTANCE, clusterService.localNode().getId(),
(ParentBulkByScrollTask) task, request, listener);
} else {
ClusterState state = clusterService.state();
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
new AsyncDeleteBySearchAction((WorkingBulkByScrollTask) task, logger, client, threadPool, request, listener, scriptService,
state).start();
}
}
@Override
@ -68,9 +74,9 @@ public class TransportDeleteByQueryAction extends HandledTransportAction<DeleteB
*/
static class AsyncDeleteBySearchAction extends AbstractAsyncBulkIndexByScrollAction<DeleteByQueryRequest> {
public AsyncDeleteBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client, ThreadPool threadPool,
DeleteByQueryRequest request, ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
public AsyncDeleteBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
ThreadPool threadPool, DeleteByQueryRequest request, ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
super(task, logger, client, threadPool, request, listener, scriptService, clusterState);
}

View File

@ -105,12 +105,18 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
@Override
protected void doExecute(Task task, ReindexRequest request, ActionListener<BulkIndexByScrollResponse> listener) {
checkRemoteWhitelist(remoteWhitelist, request.getRemoteInfo());
ClusterState state = clusterService.state();
validateAgainstAliases(request.getSearchRequest(), request.getDestination(), request.getRemoteInfo(), indexNameExpressionResolver,
autoCreateIndex, state);
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
new AsyncIndexBySearchAction((BulkByScrollTask) task, logger, client, threadPool, request, listener, scriptService, state).start();
if (request.getSlices() > 1) {
ReindexParallelizationHelper.startSlices(client, taskManager, ReindexAction.INSTANCE, clusterService.localNode().getId(),
(ParentBulkByScrollTask) task, request, listener);
} else {
checkRemoteWhitelist(remoteWhitelist, request.getRemoteInfo());
ClusterState state = clusterService.state();
validateAgainstAliases(request.getSearchRequest(), request.getDestination(), request.getRemoteInfo(),
indexNameExpressionResolver, autoCreateIndex, state);
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
new AsyncIndexBySearchAction((WorkingBulkByScrollTask) task, logger, client, threadPool, request, listener, scriptService,
state).start();
}
}
@Override
@ -229,9 +235,9 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
*/
private List<Thread> createdThreads = emptyList();
public AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client, ThreadPool threadPool,
ReindexRequest request, ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
public AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
ThreadPool threadPool, ReindexRequest request, ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
super(task, logger, client, threadPool, request, listener, scriptService, clusterState);
}
@ -350,7 +356,7 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
class ReindexScriptApplier extends ScriptApplier {
ReindexScriptApplier(BulkByScrollTask task, ScriptService scriptService, Script script,
ReindexScriptApplier(WorkingBulkByScrollTask task, ScriptService scriptService, Script script,
Map<String, Object> params) {
super(task, scriptService, script, params);
}

View File

@ -19,16 +19,19 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.tasks.TransportTasksAction;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -37,18 +40,38 @@ import java.io.IOException;
import java.util.List;
public class TransportRethrottleAction extends TransportTasksAction<BulkByScrollTask, RethrottleRequest, ListTasksResponse, TaskInfo> {
private final Client client;
@Inject
public TransportRethrottleAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
Client client) {
super(settings, RethrottleAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
RethrottleRequest::new, ListTasksResponse::new, ThreadPool.Names.MANAGEMENT);
this.client = client;
}
@Override
protected TaskInfo taskOperation(RethrottleRequest request, BulkByScrollTask task) {
// Apply the new throttle and fetch status of the task. The user might not want that status but they likely do and it is cheap.
task.rethrottle(request.getRequestsPerSecond());
return task.taskInfo(clusterService.localNode(), true);
protected void taskOperation(RethrottleRequest request, BulkByScrollTask task, ActionListener<TaskInfo> listener) {
rethrottle(clusterService.localNode().getId(), client, task, request.getRequestsPerSecond(), listener);
}
static void rethrottle(String localNodeId, Client client, BulkByScrollTask task, float newRequestsPerSecond,
ActionListener<TaskInfo> listener) {
int runningSubTasks = task.runningSliceSubTasks();
if (runningSubTasks == 0) {
// Nothing to do, all sub tasks are done
task.rethrottle(newRequestsPerSecond);
listener.onResponse(task.taskInfo(localNodeId, true));
return;
}
RethrottleRequest subRequest = new RethrottleRequest();
subRequest.setRequestsPerSecond(newRequestsPerSecond / runningSubTasks);
subRequest.setParentTaskId(new TaskId(localNodeId, task.getId()));
client.execute(RethrottleAction.INSTANCE, subRequest, ActionListener.wrap(r -> {
r.rethrowFailures("Rethrottle");
listener.onResponse(task.getInfoGivenSliceInfo(localNodeId, r.getTasks()));
}, listener::onFailure));
}
@Override

View File

@ -66,9 +66,15 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
@Override
protected void doExecute(Task task, UpdateByQueryRequest request, ActionListener<BulkIndexByScrollResponse> listener) {
ClusterState state = clusterService.state();
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
new AsyncIndexBySearchAction((BulkByScrollTask) task, logger, client, threadPool, request, listener, scriptService, state).start();
if (request.getSlices() > 1) {
ReindexParallelizationHelper.startSlices(client, taskManager, UpdateByQueryAction.INSTANCE, clusterService.localNode().getId(),
(ParentBulkByScrollTask) task, request, listener);
} else {
ClusterState state = clusterService.state();
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
new AsyncIndexBySearchAction((WorkingBulkByScrollTask) task, logger, client, threadPool, request, listener, scriptService,
state).start();
}
}
@Override
@ -81,9 +87,9 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
*/
static class AsyncIndexBySearchAction extends AbstractAsyncBulkIndexByScrollAction<UpdateByQueryRequest> {
public AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client, ThreadPool threadPool,
UpdateByQueryRequest request, ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
public AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
ThreadPool threadPool, UpdateByQueryRequest request, ActionListener<BulkIndexByScrollResponse> listener,
ScriptService scriptService, ClusterState clusterState) {
super(task, logger, client, threadPool, request, listener, scriptService, clusterState);
}
@ -120,7 +126,7 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
class UpdateByQueryScriptApplier extends ScriptApplier {
UpdateByQueryScriptApplier(BulkByScrollTask task, ScriptService scriptService, Script script,
UpdateByQueryScriptApplier(WorkingBulkByScrollTask task, ScriptService scriptService, Script script,
Map<String, Object> params) {
super(task, scriptService, script, params);
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.tasks.TaskId;
import java.io.IOException;
@ -43,7 +44,11 @@ public class UpdateByQueryRequest extends AbstractBulkIndexByScrollRequest<Updat
}
public UpdateByQueryRequest(SearchRequest search) {
super(search);
this(search, true);
}
private UpdateByQueryRequest(SearchRequest search, boolean setDefaults) {
super(search, setDefaults);
}
/**
@ -65,6 +70,13 @@ public class UpdateByQueryRequest extends AbstractBulkIndexByScrollRequest<Updat
return this;
}
@Override
UpdateByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
UpdateByQueryRequest request = doForSlice(new UpdateByQueryRequest(slice, false), slicingTask);
request.setPipeline(pipeline);
return request;
}
@Override
public String toString() {
StringBuilder b = new StringBuilder();

View File

@ -0,0 +1,325 @@
/*
* 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.reindex;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.List;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import static java.lang.Math.max;
import static java.lang.Math.round;
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
/**
* {@link BulkByScrollTask} subclass for tasks that actually perform the work. Compare to {@link ParentBulkByScrollTask}.
*/
public class WorkingBulkByScrollTask extends BulkByScrollTask implements SuccessfullyProcessed {
private static final Logger logger = ESLoggerFactory.getLogger(BulkByScrollTask.class.getPackage().getName());
/**
* The id of the slice that this task is processing or {@code null} if this task isn't for a sliced request.
*/
private final Integer sliceId;
/**
* The total number of documents this request will process. 0 means we don't yet know or, possibly, there are actually 0 documents
* to process. Its ok that these have the same meaning because any request with 0 actual documents should be quite short lived.
*/
private final AtomicLong total = new AtomicLong(0);
private final AtomicLong updated = new AtomicLong(0);
private final AtomicLong created = new AtomicLong(0);
private final AtomicLong deleted = new AtomicLong(0);
private final AtomicLong noops = new AtomicLong(0);
private final AtomicInteger batch = new AtomicInteger(0);
private final AtomicLong versionConflicts = new AtomicLong(0);
private final AtomicLong bulkRetries = new AtomicLong(0);
private final AtomicLong searchRetries = new AtomicLong(0);
private final AtomicLong throttledNanos = new AtomicLong();
/**
* The number of requests per second to which to throttle the request that this task represents. The other variables are all AtomicXXX
* style variables but there isn't an AtomicFloat so we just use a volatile.
*/
private volatile float requestsPerSecond;
/**
* Reference to any the last delayed prepareBulkRequest call. Used during rethrottling and canceling to reschedule the request.
*/
private final AtomicReference<DelayedPrepareBulkRequest> delayedPrepareBulkRequestReference = new AtomicReference<>();
public WorkingBulkByScrollTask(long id, String type, String action, String description, TaskId parentTask, Integer sliceId,
float requestsPerSecond) {
super(id, type, action, description, parentTask);
this.sliceId = sliceId;
setRequestsPerSecond(requestsPerSecond);
}
@Override
public Status getStatus() {
return new Status(sliceId, total.get(), updated.get(), created.get(), deleted.get(), batch.get(), versionConflicts.get(),
noops.get(), bulkRetries.get(), searchRetries.get(), timeValueNanos(throttledNanos.get()), getRequestsPerSecond(),
getReasonCancelled(), throttledUntil());
}
@Override
protected void onCancelled() {
// Drop the throttle to 0, immediately rescheduling all outstanding tasks so the task will wake up and cancel itself.
rethrottle(0);
}
@Override
int runningSliceSubTasks() {
return 0;
}
@Override
public TaskInfo getInfoGivenSliceInfo(String localNodeId, List<TaskInfo> sliceInfo) {
throw new UnsupportedOperationException("This is only supported by " + ParentBulkByScrollTask.class.getName() + ".");
}
TimeValue throttledUntil() {
DelayedPrepareBulkRequest delayed = delayedPrepareBulkRequestReference.get();
if (delayed == null) {
return timeValueNanos(0);
}
if (delayed.future == null) {
return timeValueNanos(0);
}
return timeValueNanos(max(0, delayed.future.getDelay(TimeUnit.NANOSECONDS)));
}
void setTotal(long totalHits) {
total.set(totalHits);
}
void countBatch() {
batch.incrementAndGet();
}
void countNoop() {
noops.incrementAndGet();
}
@Override
public long getCreated() {
return created.get();
}
void countCreated() {
created.incrementAndGet();
}
@Override
public long getUpdated() {
return updated.get();
}
void countUpdated() {
updated.incrementAndGet();
}
@Override
public long getDeleted() {
return deleted.get();
}
void countDeleted() {
deleted.incrementAndGet();
}
void countVersionConflict() {
versionConflicts.incrementAndGet();
}
void countBulkRetry() {
bulkRetries.incrementAndGet();
}
void countSearchRetry() {
searchRetries.incrementAndGet();
}
float getRequestsPerSecond() {
return requestsPerSecond;
}
/**
* Schedule prepareBulkRequestRunnable to run after some delay. This is where throttling plugs into reindexing so the request can be
* rescheduled over and over again.
*/
void delayPrepareBulkRequest(ThreadPool threadPool, TimeValue lastBatchStartTime, int lastBatchSize,
AbstractRunnable prepareBulkRequestRunnable) {
// Synchronize so we are less likely to schedule the same request twice.
synchronized (delayedPrepareBulkRequestReference) {
TimeValue delay = throttleWaitTime(lastBatchStartTime, lastBatchSize);
delayedPrepareBulkRequestReference.set(new DelayedPrepareBulkRequest(threadPool, getRequestsPerSecond(),
delay, new RunOnce(prepareBulkRequestRunnable)));
}
}
TimeValue throttleWaitTime(TimeValue lastBatchStartTime, int lastBatchSize) {
long earliestNextBatchStartTime = lastBatchStartTime.nanos() + (long) perfectlyThrottledBatchTime(lastBatchSize);
return timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime()));
}
/**
* How many nanoseconds should a batch of lastBatchSize have taken if it were perfectly throttled? Package private for testing.
*/
float perfectlyThrottledBatchTime(int lastBatchSize) {
if (requestsPerSecond == Float.POSITIVE_INFINITY) {
return 0;
}
// requests
// ------------------- == seconds
// request per seconds
float targetBatchTimeInSeconds = lastBatchSize / requestsPerSecond;
// nanoseconds per seconds * seconds == nanoseconds
return TimeUnit.SECONDS.toNanos(1) * targetBatchTimeInSeconds;
}
private void setRequestsPerSecond(float requestsPerSecond) {
this.requestsPerSecond = requestsPerSecond;
}
@Override
void rethrottle(float newRequestsPerSecond) {
synchronized (delayedPrepareBulkRequestReference) {
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Rethrottling to [{}] requests per second", getId(), newRequestsPerSecond);
}
setRequestsPerSecond(newRequestsPerSecond);
DelayedPrepareBulkRequest delayedPrepareBulkRequest = this.delayedPrepareBulkRequestReference.get();
if (delayedPrepareBulkRequest == null) {
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Skipping rescheduling because there is no scheduled task", getId());
}
// No request has been queued yet so nothing to reschedule.
return;
}
this.delayedPrepareBulkRequestReference.set(delayedPrepareBulkRequest.rethrottle(newRequestsPerSecond));
}
}
class DelayedPrepareBulkRequest {
private final ThreadPool threadPool;
private final AbstractRunnable command;
private final float requestsPerSecond;
private final ScheduledFuture<?> future;
DelayedPrepareBulkRequest(ThreadPool threadPool, float requestsPerSecond, TimeValue delay, AbstractRunnable command) {
this.threadPool = threadPool;
this.requestsPerSecond = requestsPerSecond;
this.command = command;
this.future = threadPool.schedule(delay, ThreadPool.Names.GENERIC, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
throttledNanos.addAndGet(delay.nanos());
command.run();
}
@Override
public void onFailure(Exception e) {
command.onFailure(e);
}
});
}
DelayedPrepareBulkRequest rethrottle(float newRequestsPerSecond) {
if (newRequestsPerSecond != 0 && newRequestsPerSecond < requestsPerSecond) {
/*
* The user is attempting to slow the request down. We'll let the change in throttle take effect the next time we delay
* prepareBulkRequest. We can't just reschedule the request further out in the future the bulk context might time out.
*/
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Skipping rescheduling because the new throttle [{}] is slower than the old one [{}].", getId(),
newRequestsPerSecond, requestsPerSecond);
}
return this;
}
long remainingDelay = future.getDelay(TimeUnit.NANOSECONDS);
// Actually reschedule the task
if (false == FutureUtils.cancel(future)) {
// Couldn't cancel, probably because the task has finished or been scheduled. Either way we have nothing to do here.
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Skipping rescheduling we couldn't cancel the task.", getId());
}
return this;
}
/*
* Strangely enough getting here doesn't mean that you actually cancelled the request, just that you probably did. If you stress
* test it you'll find that requests sneak through. So each request is given a runOnce boolean to prevent that.
*/
TimeValue newDelay = newDelay(remainingDelay, newRequestsPerSecond);
if (logger.isDebugEnabled()) {
logger.debug("[{}]: Rescheduling for [{}] in the future.", getId(), newDelay);
}
return new DelayedPrepareBulkRequest(threadPool, requestsPerSecond, newDelay, command);
}
/**
* Scale back remaining delay to fit the new delay.
*/
TimeValue newDelay(long remainingDelay, float newRequestsPerSecond) {
if (remainingDelay < 0 || newRequestsPerSecond == 0) {
return timeValueNanos(0);
}
return timeValueNanos(round(remainingDelay * requestsPerSecond / newRequestsPerSecond));
}
}
/**
* Runnable that can only be run one time. This is paranoia to prevent furiously rethrottling from running the command multiple times.
* Without it the command would be run multiple times.
*/
private static class RunOnce extends AbstractRunnable {
private final AtomicBoolean hasRun = new AtomicBoolean(false);
private final AbstractRunnable delegate;
public RunOnce(AbstractRunnable delegate) {
this.delegate = delegate;
}
@Override
protected void doRun() throws Exception {
if (hasRun.compareAndSet(false, true)) {
delegate.run();
}
}
@Override
public void onFailure(Exception e) {
delegate.onFailure(e);
}
}
}

View File

@ -32,12 +32,12 @@ public abstract class AbstractAsyncBulkIndexByScrollActionTestCase<
Response extends BulkIndexByScrollResponse>
extends ESTestCase {
protected ThreadPool threadPool;
protected BulkByScrollTask task;
protected WorkingBulkByScrollTask task;
@Before
public void setupForTest() {
threadPool = new TestThreadPool(getTestName());
task = new BulkByScrollTask(1, "test", "test", "test", TaskId.EMPTY_TASK_ID, 0);
task = new WorkingBulkByScrollTask(1, "test", "test", "test", TaskId.EMPTY_TASK_ID, null, 0);
}
@After

View File

@ -0,0 +1,70 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.reindex;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase;
import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
/**
* Shared superclass for testing reindex and friends. In particular it makes sure to test the slice features.
*/
public abstract class AbstractBulkByScrollRequestTestCase<R extends AbstractBulkByScrollRequest<R>> extends ESTestCase {
public void testForSlice() {
R original = newRequest();
original.setAbortOnVersionConflict(randomBoolean());
original.setRefresh(randomBoolean());
original.setTimeout(parseTimeValue(randomPositiveTimeValue(), "timeout"));
original.setWaitForActiveShards(
randomFrom(ActiveShardCount.ALL, ActiveShardCount.NONE, ActiveShardCount.ONE, ActiveShardCount.DEFAULT));
original.setRetryBackoffInitialTime(parseTimeValue(randomPositiveTimeValue(), "retry_backoff_initial_time"));
original.setMaxRetries(between(0, 1000));
original.setSlices(between(2, 1000));
original.setRequestsPerSecond(
randomBoolean() ? Float.POSITIVE_INFINITY : randomValueOtherThanMany(r -> r < 0, ESTestCase::randomFloat));
original.setSize(randomBoolean() ? AbstractBulkByScrollRequest.SIZE_ALL_MATCHES : between(0, Integer.MAX_VALUE));
TaskId slicingTask = new TaskId(randomAsciiOfLength(5), randomLong());
SearchRequest sliceRequest = new SearchRequest();
R forSliced = original.forSlice(slicingTask, sliceRequest);
assertEquals(original.isAbortOnVersionConflict(), forSliced.isAbortOnVersionConflict());
assertEquals(original.isRefresh(), forSliced.isRefresh());
assertEquals(original.getTimeout(), forSliced.getTimeout());
assertEquals(original.getWaitForActiveShards(), forSliced.getWaitForActiveShards());
assertEquals(original.getRetryBackoffInitialTime(), forSliced.getRetryBackoffInitialTime());
assertEquals(original.getMaxRetries(), forSliced.getMaxRetries());
assertEquals("only the parent task should store results", false, forSliced.getShouldStoreResult());
assertEquals("slice requests always have a single worker", 1, forSliced.getSlices());
assertEquals("requests_per_second is split between all workers", original.getRequestsPerSecond() / original.getSlices(),
forSliced.getRequestsPerSecond(), Float.MIN_NORMAL);
assertEquals("size is split evenly between all workers", original.getSize() == AbstractBulkByScrollRequest.SIZE_ALL_MATCHES
? AbstractBulkByScrollRequest.SIZE_ALL_MATCHES : original.getSize() / original.getSlices(), forSliced.getSize());
assertEquals(slicingTask, forSliced.getParentTask());
extraForSliceAssertions(original, forSliced);
}
protected abstract R newRequest();
protected abstract void extraRandomizationForSlice(R original);
protected abstract void extraForSliceAssertions(R original, R forSliced);
}

View File

@ -120,7 +120,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
private PlainActionFuture<BulkIndexByScrollResponse> listener;
private String scrollId;
private TaskManager taskManager;
private BulkByScrollTask testTask;
private WorkingBulkByScrollTask testTask;
private Map<String, String> expectedHeaders = new HashMap<>();
private DiscoveryNode localNode;
private TaskId taskId;
@ -134,7 +134,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
listener = new PlainActionFuture<>();
scrollId = null;
taskManager = new TaskManager(Settings.EMPTY);
testTask = (BulkByScrollTask) taskManager.register("don'tcare", "hereeither", testRequest);
testTask = (WorkingBulkByScrollTask) taskManager.register("don'tcare", "hereeither", testRequest);
// Fill the context with something random so we can make sure we inherited it appropriately.
expectedHeaders.clear();
@ -685,7 +685,12 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
private static class DummyAbstractBulkByScrollRequest extends AbstractBulkByScrollRequest<DummyAbstractBulkByScrollRequest> {
public DummyAbstractBulkByScrollRequest(SearchRequest searchRequest) {
super(searchRequest);
super(searchRequest, true);
}
@Override
DummyAbstractBulkByScrollRequest forSlice(TaskId slicingTask, SearchRequest slice) {
throw new UnsupportedOperationException();
}
@Override

View File

@ -20,281 +20,143 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.Before;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Delayed;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.Arrays;
import static java.lang.Math.min;
import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
import static org.hamcrest.Matchers.both;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.hasItem;
import static org.hamcrest.Matchers.not;
public class BulkByScrollTaskTests extends ESTestCase {
private BulkByScrollTask task;
@Before
public void createTask() {
task = new BulkByScrollTask(1, "test_type", "test_action", "test", TaskId.EMPTY_TASK_ID, Float.POSITIVE_INFINITY);
}
public void testBasicData() {
assertEquals(1, task.getId());
assertEquals("test_type", task.getType());
assertEquals("test_action", task.getAction());
}
public void testProgress() {
long created = 0;
long updated = 0;
long deleted = 0;
long versionConflicts = 0;
long noops = 0;
int batch = 0;
BulkByScrollTask.Status status = task.getStatus();
assertEquals(0, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batch, status.getBatches());
assertEquals(noops, status.getNoops());
long totalHits = randomIntBetween(10, 1000);
task.setTotal(totalHits);
for (long p = 0; p < totalHits; p++) {
status = task.getStatus();
assertEquals(totalHits, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batch, status.getBatches());
assertEquals(noops, status.getNoops());
if (randomBoolean()) {
created++;
task.countCreated();
} else if (randomBoolean()) {
updated++;
task.countUpdated();
} else {
deleted++;
task.countDeleted();
}
if (rarely()) {
versionConflicts++;
task.countVersionConflict();
}
if (rarely()) {
batch++;
task.countBatch();
}
if (rarely()) {
noops++;
task.countNoop();
}
}
status = task.getStatus();
assertEquals(totalHits, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batch, status.getBatches());
assertEquals(noops, status.getNoops());
}
public void testStatusHatesNegatives() {
checkStatusNegatives(-1, 0, 0, 0, 0, 0, 0, 0, 0, "total");
checkStatusNegatives(0, -1, 0, 0, 0, 0, 0, 0, 0, "updated");
checkStatusNegatives(0, 0, -1, 0, 0, 0, 0, 0, 0, "created");
checkStatusNegatives(0, 0, 0, -1, 0, 0, 0, 0, 0, "deleted");
checkStatusNegatives(0, 0, 0, 0, -1, 0, 0, 0, 0, "batches");
checkStatusNegatives(0, 0, 0, 0, 0, -1, 0, 0, 0, "versionConflicts");
checkStatusNegatives(0, 0, 0, 0, 0, 0, -1, 0, 0, "noops");
checkStatusNegatives(0, 0, 0, 0, 0, 0, 0, -1, 0, "bulkRetries");
checkStatusNegatives(0, 0, 0, 0, 0, 0, 0, 0, -1, "searchRetries");
checkStatusNegatives(-1 , 0, 0, 0, 0, 0, 0, 0, 0, 0, "sliceId");
checkStatusNegatives(null, -1, 0, 0, 0, 0, 0, 0, 0, 0, "total");
checkStatusNegatives(null, 0, -1, 0, 0, 0, 0, 0, 0, 0, "updated");
checkStatusNegatives(null, 0, 0, -1, 0, 0, 0, 0, 0, 0, "created");
checkStatusNegatives(null, 0, 0, 0, -1, 0, 0, 0, 0, 0, "deleted");
checkStatusNegatives(null, 0, 0, 0, 0, -1, 0, 0, 0, 0, "batches");
checkStatusNegatives(null, 0, 0, 0, 0, 0, -1, 0, 0, 0, "versionConflicts");
checkStatusNegatives(null, 0, 0, 0, 0, 0, 0, -1, 0, 0, "noops");
checkStatusNegatives(null, 0, 0, 0, 0, 0, 0, 0, -1, 0, "bulkRetries");
checkStatusNegatives(null, 0, 0, 0, 0, 0, 0, 0, 0, -1, "searchRetries");
}
/**
* Build a task status with only some values. Used for testing negative values.
*/
private void checkStatusNegatives(long total, long updated, long created, long deleted, int batches, long versionConflicts,
long noops, long bulkRetries, long searchRetries, String fieldName) {
private void checkStatusNegatives(Integer sliceId, long total, long updated, long created, long deleted, int batches,
long versionConflicts, long noops, long bulkRetries, long searchRetries, String fieldName) {
TimeValue throttle = parseTimeValue(randomPositiveTimeValue(), "test");
TimeValue throttledUntil = parseTimeValue(randomPositiveTimeValue(), "test");
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(total, updated, created,
deleted, batches, versionConflicts, noops, bulkRetries, searchRetries, throttle, 0f, null, throttledUntil));
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(sliceId, total, updated,
created, deleted, batches, versionConflicts, noops, bulkRetries, searchRetries, throttle, 0f, null, throttledUntil));
assertEquals(e.getMessage(), fieldName + " must be greater than 0 but was [-1]");
}
/**
* Furiously rethrottles a delayed request to make sure that we never run it twice.
*/
public void testDelayAndRethrottle() throws IOException, InterruptedException {
List<Throwable> errors = new CopyOnWriteArrayList<>();
AtomicBoolean done = new AtomicBoolean();
int threads = between(1, 10);
CyclicBarrier waitForShutdown = new CyclicBarrier(threads);
/*
* We never end up waiting this long because the test rethrottles over and over again, ratcheting down the delay a random amount
* each time.
*/
float originalRequestsPerSecond = (float) randomDoubleBetween(1, 10000, true);
task.rethrottle(originalRequestsPerSecond);
TimeValue maxDelay = timeValueSeconds(between(1, 5));
assertThat(maxDelay.nanos(), greaterThanOrEqualTo(0L));
int batchSizeForMaxDelay = (int) (maxDelay.seconds() * originalRequestsPerSecond);
ThreadPool threadPool = new TestThreadPool(getTestName()) {
@Override
public ScheduledFuture<?> schedule(TimeValue delay, String name, Runnable command) {
assertThat(delay.nanos(), both(greaterThanOrEqualTo(0L)).and(lessThanOrEqualTo(maxDelay.nanos())));
return super.schedule(delay, name, command);
}
};
try {
task.delayPrepareBulkRequest(threadPool, timeValueNanos(System.nanoTime()), batchSizeForMaxDelay, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
boolean oldValue = done.getAndSet(true);
if (oldValue) {
throw new RuntimeException("Ran twice oh no!");
}
}
@Override
public void onFailure(Exception e) {
errors.add(e);
}
});
// Rethrottle on a random number of threads, on of which is this thread.
Runnable test = () -> {
try {
int rethrottles = 0;
while (false == done.get()) {
float requestsPerSecond = (float) randomDoubleBetween(0, originalRequestsPerSecond * 2, true);
task.rethrottle(requestsPerSecond);
rethrottles += 1;
}
logger.info("Rethrottled [{}] times", rethrottles);
waitForShutdown.await();
} catch (Exception e) {
errors.add(e);
}
};
for (int i = 1; i < threads; i++) {
threadPool.generic().execute(test);
}
test.run();
} finally {
// Other threads should finish up quickly as they are checking the same AtomicBoolean.
threadPool.shutdown();
threadPool.awaitTermination(10, TimeUnit.SECONDS);
}
assertThat(errors, empty());
}
public void testDelayNeverNegative() throws IOException {
// Thread pool that returns a ScheduledFuture that claims to have a negative delay
ThreadPool threadPool = new TestThreadPool("test") {
public ScheduledFuture<?> schedule(TimeValue delay, String name, Runnable command) {
return new ScheduledFuture<Void>() {
@Override
public long getDelay(TimeUnit unit) {
return -1;
}
@Override
public int compareTo(Delayed o) {
throw new UnsupportedOperationException();
}
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
throw new UnsupportedOperationException();
}
@Override
public boolean isCancelled() {
throw new UnsupportedOperationException();
}
@Override
public boolean isDone() {
throw new UnsupportedOperationException();
}
@Override
public Void get() throws InterruptedException, ExecutionException {
throw new UnsupportedOperationException();
}
@Override
public Void get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
throw new UnsupportedOperationException();
}
};
}
};
try {
// Have the task use the thread pool to delay a task that does nothing
task.delayPrepareBulkRequest(threadPool, timeValueSeconds(0), 1, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
}
@Override
public void onFailure(Exception e) {
throw new UnsupportedOperationException();
}
});
// Even though the future returns a negative delay we just return 0 because the time is up.
assertEquals(timeValueSeconds(0), task.getStatus().getThrottledUntil());
} finally {
threadPool.shutdown();
}
}
public void testXContentRepresentationOfUnlimitedRequestsPerSecon() throws IOException {
public void testXContentRepresentationOfUnlimitedRequestsPerSecond() throws IOException {
XContentBuilder builder = JsonXContent.contentBuilder();
task.getStatus().toXContent(builder, ToXContent.EMPTY_PARAMS);
BulkByScrollTask.Status status = new BulkByScrollTask.Status(null, 0, 0, 0, 0, 0, 0, 0, 0, 0, timeValueMillis(0),
Float.POSITIVE_INFINITY, null, timeValueMillis(0));
status.toXContent(builder, ToXContent.EMPTY_PARAMS);
assertThat(builder.string(), containsString("\"requests_per_second\":-1"));
}
public void testPerfectlyThrottledBatchTime() {
task.rethrottle(Float.POSITIVE_INFINITY);
assertThat((double) task.perfectlyThrottledBatchTime(randomInt()), closeTo(0f, 0f));
public void testXContentRepresentationOfUnfinishedSlices() throws IOException {
XContentBuilder builder = JsonXContent.contentBuilder();
BulkByScrollTask.Status completedStatus = new BulkByScrollTask.Status(2, 0, 0, 0, 0, 0, 0, 0, 0, 0, timeValueMillis(0),
Float.POSITIVE_INFINITY, null, timeValueMillis(0));
BulkByScrollTask.Status status = new BulkByScrollTask.Status(
Arrays.asList(null, null, new BulkByScrollTask.StatusOrException(completedStatus)), null);
status.toXContent(builder, ToXContent.EMPTY_PARAMS);
assertThat(builder.string(), containsString("\"slices\":[null,null,{\"slice_id\":2"));
}
int total = between(0, 1000000);
task.rethrottle(1);
assertThat((double) task.perfectlyThrottledBatchTime(total),
closeTo(TimeUnit.SECONDS.toNanos(total), TimeUnit.SECONDS.toNanos(1)));
public void testXContentRepresentationOfSliceFailures() throws IOException {
XContentBuilder builder = JsonXContent.contentBuilder();
Exception e = new Exception();
BulkByScrollTask.Status status = new BulkByScrollTask.Status(Arrays.asList(null, null, new BulkByScrollTask.StatusOrException(e)),
null);
status.toXContent(builder, ToXContent.EMPTY_PARAMS);
assertThat(builder.string(), containsString("\"slices\":[null,null,{\"type\":\"exception\""));
}
public void testMergeStatuses() {
BulkByScrollTask.StatusOrException[] statuses = new BulkByScrollTask.StatusOrException[between(2, 100)];
boolean containsNullStatuses = randomBoolean();
int mergedTotal = 0;
int mergedUpdated = 0;
int mergedCreated = 0;
int mergedDeleted = 0;
int mergedBatches = 0;
int mergedVersionConflicts = 0;
int mergedNoops = 0;
int mergedBulkRetries = 0;
int mergedSearchRetries = 0;
TimeValue mergedThrottled = timeValueNanos(0);
float mergedRequestsPerSecond = 0;
TimeValue mergedThrottledUntil = timeValueNanos(Integer.MAX_VALUE);
for (int i = 0; i < statuses.length; i++) {
if (containsNullStatuses && rarely()) {
continue;
}
int total = between(0, 10000);
int updated = between(0, total);
int created = between(0, total - updated);
int deleted = between(0, total - updated - created);
int batches = between(0, 10);
int versionConflicts = between(0, 100);
int noops = total - updated - created - deleted;
int bulkRetries = between(0, 100);
int searchRetries = between(0, 100);
TimeValue throttled = timeValueNanos(between(0, 10000));
float requestsPerSecond = randomValueOtherThanMany(r -> r <= 0, () -> randomFloat());
String reasonCancelled = randomBoolean() ? null : "test";
TimeValue throttledUntil = timeValueNanos(between(0, 1000));
statuses[i] = new BulkByScrollTask.StatusOrException(new BulkByScrollTask.Status(i, total, updated, created, deleted, batches,
versionConflicts, noops, bulkRetries, searchRetries, throttled, requestsPerSecond, reasonCancelled, throttledUntil));
mergedTotal += total;
mergedUpdated += updated;
mergedCreated += created;
mergedDeleted += deleted;
mergedBatches += batches;
mergedVersionConflicts += versionConflicts;
mergedNoops += noops;
mergedBulkRetries += bulkRetries;
mergedSearchRetries += searchRetries;
mergedThrottled = timeValueNanos(mergedThrottled.nanos() + throttled.nanos());
mergedRequestsPerSecond += requestsPerSecond;
mergedThrottledUntil = timeValueNanos(min(mergedThrottledUntil.nanos(), throttledUntil.nanos()));
}
String reasonCancelled = randomBoolean() ? randomAsciiOfLength(10) : null;
BulkByScrollTask.Status merged = new BulkByScrollTask.Status(Arrays.asList(statuses), reasonCancelled);
assertEquals(mergedTotal, merged.getTotal());
assertEquals(mergedUpdated, merged.getUpdated());
assertEquals(mergedCreated, merged.getCreated());
assertEquals(mergedDeleted, merged.getDeleted());
assertEquals(mergedBatches, merged.getBatches());
assertEquals(mergedVersionConflicts, merged.getVersionConflicts());
assertEquals(mergedNoops, merged.getNoops());
assertEquals(mergedBulkRetries, merged.getBulkRetries());
assertEquals(mergedSearchRetries, merged.getSearchRetries());
assertEquals(mergedThrottled, merged.getThrottled());
assertEquals(mergedRequestsPerSecond, merged.getRequestsPerSecond(), 0.0001f);
assertEquals(mergedThrottledUntil, merged.getThrottledUntil());
assertEquals(reasonCancelled, merged.getReasonCancelled());
}
public void testUnknownVersions() {
assertThat("5.1.0 has been defined, remove the temporary constant", VersionUtils.allVersions(),
not(hasItem(BulkByScrollTask.V_5_1_0_UNRELEASED)));
}
}

View File

@ -23,6 +23,9 @@ import org.hamcrest.Description;
import org.hamcrest.Matcher;
import org.hamcrest.TypeSafeMatcher;
import java.util.Collection;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
@ -39,6 +42,7 @@ public class BulkIndexByScrollResponseMatcher extends TypeSafeMatcher<BulkIndexB
private Matcher<Long> versionConflictsMatcher = equalTo(0L);
private Matcher<Integer> failuresMatcher = equalTo(0);
private Matcher<String> reasonCancelledMatcher = nullValue(String.class);
private Matcher<Collection<? extends BulkIndexByScrollResponseMatcher>> slicesMatcher = empty();
public BulkIndexByScrollResponseMatcher created(Matcher<Long> createdMatcher) {
this.createdMatcher = createdMatcher;
@ -117,6 +121,14 @@ public class BulkIndexByScrollResponseMatcher extends TypeSafeMatcher<BulkIndexB
return this;
}
/**
* Set the matcher for the workers portion of the response.
*/
public BulkIndexByScrollResponseMatcher slices(Matcher<Collection<? extends BulkIndexByScrollResponseMatcher>> slicesMatcher) {
this.slicesMatcher = slicesMatcher;
return this;
}
@Override
protected boolean matchesSafely(BulkIndexByScrollResponse item) {
return updatedMatcher.matches(item.getUpdated()) &&
@ -125,7 +137,8 @@ public class BulkIndexByScrollResponseMatcher extends TypeSafeMatcher<BulkIndexB
(batchesMatcher == null || batchesMatcher.matches(item.getBatches())) &&
versionConflictsMatcher.matches(item.getVersionConflicts()) &&
failuresMatcher.matches(item.getBulkFailures().size()) &&
reasonCancelledMatcher.matches(item.getReasonCancelled());
reasonCancelledMatcher.matches(item.getReasonCancelled()) &&
slicesMatcher.matches(item.getStatus().getSliceStatuses());
}
@Override
@ -139,5 +152,6 @@ public class BulkIndexByScrollResponseMatcher extends TypeSafeMatcher<BulkIndexB
description.appendText(" and versionConflicts matches ").appendDescriptionOf(versionConflictsMatcher);
description.appendText(" and failures size matches ").appendDescriptionOf(failuresMatcher);
description.appendText(" and reason cancelled matches ").appendDescriptionOf(reasonCancelledMatcher);
description.appendText(" and slices matches ").appendDescriptionOf(slicesMatcher);
}
}

View File

@ -0,0 +1,73 @@
/*
* 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.reindex;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.reindex.ScrollableHitSource.SearchFailure;
import org.elasticsearch.test.ESTestCase;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static java.util.Collections.emptyList;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
public class BulkIndexByScrollResponseTests extends ESTestCase {
public void testMergeConstructor() {
int mergeCount = between(2, 10);
List<BulkIndexByScrollResponse> responses = new ArrayList<>(mergeCount);
int took = between(1000, 10000);
int tookIndex = between(0, mergeCount - 1);
List<BulkItemResponse.Failure> allBulkFailures = new ArrayList<>();
List<SearchFailure> allSearchFailures = new ArrayList<>();
boolean timedOut = false;
String reasonCancelled = rarely() ? randomAsciiOfLength(5) : null;
for (int i = 0; i < mergeCount; i++) {
// One of the merged responses gets the expected value for took, the others get a smaller value
TimeValue thisTook = timeValueMillis(i == tookIndex ? took : between(0, took));
// The actual status doesn't matter too much - we test merging those elsewhere
String thisReasonCancelled = rarely() ? randomAsciiOfLength(5) : null;
BulkByScrollTask.Status status = new BulkByScrollTask.Status(i, 0, 0, 0, 0, 0, 0, 0, 0, 0, timeValueMillis(0), 0f,
thisReasonCancelled, timeValueMillis(0));
List<BulkItemResponse.Failure> bulkFailures = frequently() ? emptyList()
: IntStream.range(0, between(1, 3)).mapToObj(j -> new BulkItemResponse.Failure("idx", "type", "id", new Exception()))
.collect(Collectors.toList());
allBulkFailures.addAll(bulkFailures);
List<SearchFailure> searchFailures = frequently() ? emptyList()
: IntStream.range(0, between(1, 3)).mapToObj(j -> new SearchFailure(new Exception())).collect(Collectors.toList());
allSearchFailures.addAll(searchFailures);
boolean thisTimedOut = rarely();
timedOut |= thisTimedOut;
responses.add(new BulkIndexByScrollResponse(thisTook, status, bulkFailures, searchFailures, thisTimedOut));
}
BulkIndexByScrollResponse merged = new BulkIndexByScrollResponse(responses, reasonCancelled);
assertEquals(timeValueMillis(took), merged.getTook());
assertEquals(allBulkFailures, merged.getBulkFailures());
assertEquals(allSearchFailures, merged.getSearchFailures());
assertEquals(timedOut, merged.isTimedOut());
assertEquals(reasonCancelled, merged.getReasonCancelled());
}
}

View File

@ -32,11 +32,10 @@ import org.elasticsearch.index.shard.IndexingOperationListener;
import org.elasticsearch.ingest.IngestTestPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.tasks.TaskInfo;
import org.junit.BeforeClass;
import org.junit.Before;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
@ -45,7 +44,6 @@ import java.util.stream.IntStream;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.emptyIterable;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
@ -60,9 +58,6 @@ public class CancelTests extends ReindexTestCase {
protected static final String INDEX = "reindex-cancel-index";
protected static final String TYPE = "reindex-cancel-type";
private static final int MIN_OPERATIONS = 2;
private static final int BLOCKING_OPERATIONS = 1;
// Semaphore used to allow & block indexing operations during the test
private static final Semaphore ALLOWED_OPERATIONS = new Semaphore(0);
@ -74,8 +69,8 @@ public class CancelTests extends ReindexTestCase {
return plugins;
}
@BeforeClass
public static void clearAllowedOperations() {
@Before
public void clearAllowedOperations() {
ALLOWED_OPERATIONS.drainPermits();
}
@ -85,8 +80,8 @@ public class CancelTests extends ReindexTestCase {
private void testCancel(String action, AbstractBulkByScrollRequestBuilder<?, ?> builder, CancelAssertion assertion) throws Exception {
createIndex(INDEX);
// Total number of documents created for this test (~10 per primary shard)
int numDocs = getNumShards(INDEX).numPrimaries * 10;
// Total number of documents created for this test (~10 per primary shard per shard)
int numDocs = getNumShards(INDEX).numPrimaries * 10 * builder.request().getSlices();
ALLOWED_OPERATIONS.release(numDocs);
indexRandom(true, false, true, IntStream.range(0, numDocs)
@ -100,43 +95,59 @@ public class CancelTests extends ReindexTestCase {
// Scroll by 1 so that cancellation is easier to control
builder.source().setSize(1);
// Allow a random number of the documents minus 1
// to be modified by the reindex action
int numModifiedDocs = randomIntBetween(MIN_OPERATIONS, numDocs);
ALLOWED_OPERATIONS.release(numModifiedDocs - BLOCKING_OPERATIONS);
/* Allow a random number of the documents less the number of workers to be modified by the reindex action. That way at least one
* worker is blocked. */
int numModifiedDocs = randomIntBetween(builder.request().getSlices() * 2, numDocs);
ALLOWED_OPERATIONS.release(numModifiedDocs - builder.request().getSlices());
// Now execute the reindex action...
ListenableActionFuture<? extends BulkIndexByScrollResponse> future = builder.execute();
// ... and waits for the indexing operation listeners to block
/* ... and waits for the indexing operation listeners to block. It is important to realize that some of the workers might have
* exhausted their slice while others might have quite a bit left to work on. We can't control that. */
awaitBusy(() -> ALLOWED_OPERATIONS.hasQueuedThreads() && ALLOWED_OPERATIONS.availablePermits() == 0);
// Status should show the task running
ListTasksResponse tasksList = client().admin().cluster().prepareListTasks().setActions(action).setDetailed(true).get();
assertThat(tasksList.getNodeFailures(), empty());
assertThat(tasksList.getTaskFailures(), empty());
assertThat(tasksList.getTasks(), hasSize(1));
BulkByScrollTask.Status status = (BulkByScrollTask.Status) tasksList.getTasks().get(0).getStatus();
TaskInfo mainTask = findTaskToCancel(action, builder.request().getSlices());
BulkByScrollTask.Status status = (BulkByScrollTask.Status) mainTask.getStatus();
assertNull(status.getReasonCancelled());
// Cancel the request while the reindex action is blocked by the indexing operation listeners.
// This will prevent further requests from being sent.
List<TaskInfo> cancelledTasks = client().admin().cluster().prepareCancelTasks().setActions(action).get().getTasks();
assertThat(cancelledTasks, hasSize(1));
ListTasksResponse cancelTasksResponse = client().admin().cluster().prepareCancelTasks().setTaskId(mainTask.getTaskId()).get();
cancelTasksResponse.rethrowFailures("Cancel");
assertThat(cancelTasksResponse.getTasks(), hasSize(1));
// The status should now show canceled. The request will still be in the list because it is still blocked.
tasksList = client().admin().cluster().prepareListTasks().setActions(action).setDetailed(true).get();
assertThat(tasksList.getNodeFailures(), empty());
assertThat(tasksList.getTaskFailures(), empty());
assertThat(tasksList.getTasks(), hasSize(1));
status = (BulkByScrollTask.Status) tasksList.getTasks().get(0).getStatus();
// The status should now show canceled. The request will still be in the list because it is (or its children are) still blocked.
mainTask = client().admin().cluster().prepareGetTask(mainTask.getTaskId()).get().getTask().getTask();
status = (BulkByScrollTask.Status) mainTask.getStatus();
assertEquals(CancelTasksRequest.DEFAULT_REASON, status.getReasonCancelled());
if (builder.request().getSlices() > 1) {
boolean foundCancelled = false;
ListTasksResponse sliceList = client().admin().cluster().prepareListTasks().setParentTaskId(mainTask.getTaskId())
.setDetailed(true).get();
sliceList.rethrowFailures("Fetch slice tasks");
for (TaskInfo slice: sliceList.getTasks()) {
BulkByScrollTask.Status sliceStatus = (BulkByScrollTask.Status) slice.getStatus();
if (sliceStatus.getReasonCancelled() == null) continue;
assertEquals(CancelTasksRequest.DEFAULT_REASON, sliceStatus.getReasonCancelled());
foundCancelled = true;
}
assertTrue("Didn't find at least one sub task that was cancelled", foundCancelled);
}
// Unblock the last operation
ALLOWED_OPERATIONS.release(BLOCKING_OPERATIONS);
// Unblock the last operations
ALLOWED_OPERATIONS.release(builder.request().getSlices());
// Checks that no more operations are executed
assertBusy(() -> assertTrue(ALLOWED_OPERATIONS.availablePermits() == 0 && ALLOWED_OPERATIONS.getQueueLength() == 0));
assertBusy(() -> {
if (builder.request().getSlices() == 1) {
/* We can only be sure that we've drained all the permits if we only use a single worker. Otherwise some worker may have
* exhausted all of its documents before we blocked. */
assertEquals(0, ALLOWED_OPERATIONS.availablePermits());
}
assertEquals(0, ALLOWED_OPERATIONS.getQueueLength());
});
// And check the status of the response
BulkIndexByScrollResponse response = future.get();
@ -144,10 +155,30 @@ public class CancelTests extends ReindexTestCase {
assertThat(response.getBulkFailures(), emptyIterable());
assertThat(response.getSearchFailures(), emptyIterable());
if (builder.request().getSlices() >= 1) {
// If we have more than one worker we might not have made all the modifications
numModifiedDocs -= ALLOWED_OPERATIONS.availablePermits();
}
flushAndRefresh(INDEX);
assertion.assertThat(response, numDocs, numModifiedDocs);
}
private TaskInfo findTaskToCancel(String actionName, int workerCount) {
ListTasksResponse tasks;
long start = System.nanoTime();
do {
tasks = client().admin().cluster().prepareListTasks().setActions(actionName).setDetailed(true).get();
tasks.rethrowFailures("Find tasks to cancel");
for (TaskInfo taskInfo : tasks.getTasks()) {
// Skip tasks with a parent because those are children of the task we want to cancel
if (false == taskInfo.getParentTaskId().isSet()) {
return taskInfo;
}
}
} while (System.nanoTime() - start < TimeUnit.SECONDS.toNanos(10));
throw new AssertionError("Couldn't find task to rethrottle after waiting tasks=" + tasks.getTasks());
}
public void testReindexCancel() throws Exception {
testCancel(ReindexAction.NAME, reindex().source(INDEX).destination("dest", TYPE), (response, total, modified) -> {
assertThat(response, matcher().created(modified).reasonCancelled(equalTo("by user request")));
@ -164,14 +195,14 @@ public class CancelTests extends ReindexTestCase {
" \"test\" : {}\n" +
" } ]\n" +
"}");
assertAcked(client().admin().cluster().preparePutPipeline("set-foo", pipeline).get());
assertAcked(client().admin().cluster().preparePutPipeline("set-processed", pipeline).get());
testCancel(UpdateByQueryAction.NAME, updateByQuery().setPipeline("set-foo").source(INDEX), (response, total, modified) -> {
testCancel(UpdateByQueryAction.NAME, updateByQuery().setPipeline("set-processed").source(INDEX), (response, total, modified) -> {
assertThat(response, matcher().updated(modified).reasonCancelled(equalTo("by user request")));
assertHitCount(client().prepareSearch(INDEX).setSize(0).setQuery(termQuery("processed", true)).get(), modified);
});
assertAcked(client().admin().cluster().deletePipeline(new DeletePipelineRequest("set-foo")).get());
assertAcked(client().admin().cluster().deletePipeline(new DeletePipelineRequest("set-processed")).get());
}
public void testDeleteByQueryCancel() throws Exception {
@ -181,8 +212,42 @@ public class CancelTests extends ReindexTestCase {
});
}
public void testReindexCancelWithWorkers() throws Exception {
testCancel(ReindexAction.NAME, reindex().source(INDEX).destination("dest", TYPE).setSlices(5), (response, total, modified) -> {
assertThat(response, matcher().created(modified).reasonCancelled(equalTo("by user request")).slices(hasSize(5)));
refresh("dest");
assertHitCount(client().prepareSearch("dest").setTypes(TYPE).setSize(0).get(), modified);
});
}
public void testUpdateByQueryCancelWithWorkers() throws Exception {
BytesReference pipeline = new BytesArray("{\n" +
" \"description\" : \"sets processed to true\",\n" +
" \"processors\" : [ {\n" +
" \"test\" : {}\n" +
" } ]\n" +
"}");
assertAcked(client().admin().cluster().preparePutPipeline("set-processed", pipeline).get());
testCancel(UpdateByQueryAction.NAME, updateByQuery().setPipeline("set-processed").source(INDEX).setSlices(5),
(response, total, modified) -> {
assertThat(response, matcher().updated(modified).reasonCancelled(equalTo("by user request")).slices(hasSize(5)));
assertHitCount(client().prepareSearch(INDEX).setSize(0).setQuery(termQuery("processed", true)).get(), modified);
});
assertAcked(client().admin().cluster().deletePipeline(new DeletePipelineRequest("set-processed")).get());
}
public void testDeleteByQueryCancelWithWorkers() throws Exception {
testCancel(DeleteByQueryAction.NAME, deleteByQuery().source(INDEX).setSlices(5), (response, total, modified) -> {
assertThat(response, matcher().deleted(modified).reasonCancelled(equalTo("by user request")).slices(hasSize(5)));
assertHitCount(client().prepareSearch(INDEX).setSize(0).get(), total - modified);
});
}
/**
* {@link CancelAssertion} is used to check the result of the cancel test.
* Used to check the result of the cancel test.
*/
private interface CancelAssertion {
void assertThat(BulkIndexByScrollResponse response, int total, int modified);

View File

@ -34,6 +34,7 @@ import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.hasSize;
public class DeleteByQueryBasicTests extends ReindexTestCase {
@ -208,4 +209,27 @@ public class DeleteByQueryBasicTests extends ReindexTestCase {
assertHitCount(client().prepareSearch("test").setSize(0).get(), docs);
}
public void testWorkers() throws Exception {
indexRandom(true,
client().prepareIndex("test", "test", "1").setSource("foo", "a"),
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
client().prepareIndex("test", "test", "3").setSource("foo", "b"),
client().prepareIndex("test", "test", "4").setSource("foo", "c"),
client().prepareIndex("test", "test", "5").setSource("foo", "d"),
client().prepareIndex("test", "test", "6").setSource("foo", "e"),
client().prepareIndex("test", "test", "7").setSource("foo", "f")
);
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 7);
// Deletes the two docs that matches "foo:a"
assertThat(deleteByQuery().source("test").filter(termQuery("foo", "a")).refresh(true).setSlices(5).get(),
matcher().deleted(2).slices(hasSize(5)));
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 5);
// Delete remaining docs
DeleteByQueryRequestBuilder request = deleteByQuery().source("test").refresh(true).setSlices(5);
assertThat(request.get(), matcher().deleted(5).slices(hasSize(5)));
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 0);
}
}

View File

@ -21,11 +21,10 @@ package org.elasticsearch.index.reindex;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.test.ESTestCase;
import static org.apache.lucene.util.TestUtil.randomSimpleString;
public class DeleteByQueryRequestTests extends ESTestCase {
public class DeleteByQueryRequestTests extends AbstractBulkByScrollRequestTestCase<DeleteByQueryRequest> {
public void testDeleteteByQueryRequestImplementsIndicesRequestReplaceable() {
int numIndices = between(1, 100);
String[] indices = new String[numIndices];
@ -58,4 +57,19 @@ public class DeleteByQueryRequestTests extends ESTestCase {
assertEquals(newIndices[i], request.getSearchRequest().indices()[i]);
}
}
@Override
protected DeleteByQueryRequest newRequest() {
return new DeleteByQueryRequest(new SearchRequest(randomAsciiOfLength(5)));
}
@Override
protected void extraRandomizationForSlice(DeleteByQueryRequest original) {
// Nothing else to randomize
}
@Override
protected void extraForSliceAssertions(DeleteByQueryRequest original, DeleteByQueryRequest forSliced) {
// No extra assertions needed
}
}

View File

@ -0,0 +1,121 @@
/*
* 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.reindex;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.test.ESTestCase;
import org.junit.Before;
import java.util.Arrays;
import java.util.List;
import static java.util.Collections.emptyList;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.elasticsearch.index.reindex.TransportRethrottleActionTests.captureResponse;
import static org.elasticsearch.index.reindex.TransportRethrottleActionTests.neverCalled;
import static org.mockito.Mockito.mock;
public class ParentBulkByScrollTaskTests extends ESTestCase {
private int slices;
private ParentBulkByScrollTask task;
@Before
public void createTask() {
slices = between(2, 50);
task = new ParentBulkByScrollTask(1, "test_type", "test_action", "test", null, slices);
}
public void testBasicData() {
assertEquals(1, task.getId());
assertEquals("test_type", task.getType());
assertEquals("test_action", task.getAction());
assertEquals("test", task.getDescription());
}
public void testProgress() {
long total = 0;
long created = 0;
long updated = 0;
long deleted = 0;
long noops = 0;
long versionConflicts = 0;
int batches = 0;
List<BulkByScrollTask.StatusOrException> sliceStatuses = Arrays.asList(new BulkByScrollTask.StatusOrException[slices]);
BulkByScrollTask.Status status = task.getStatus();
assertEquals(total, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(noops, status.getNoops());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batches, status.getBatches());
assertEquals(sliceStatuses, status.getSliceStatuses());
for (int slice = 0; slice < slices; slice++) {
int thisTotal = between(10, 10000);
int thisCreated = between(0, thisTotal);
int thisUpdated = between(0, thisTotal - thisCreated);
int thisDeleted = between(0, thisTotal - thisCreated - thisUpdated);
int thisNoops = thisTotal - thisCreated - thisUpdated - thisDeleted;
int thisVersionConflicts = between(0, 1000);
int thisBatches = between(1, 100);
BulkByScrollTask.Status sliceStatus = new BulkByScrollTask.Status(slice, thisTotal, thisUpdated, thisCreated, thisDeleted,
thisBatches, thisVersionConflicts, thisNoops, 0, 0, timeValueMillis(0), 0, null, timeValueMillis(0));
total += thisTotal;
created += thisCreated;
updated += thisUpdated;
deleted += thisDeleted;
noops += thisNoops;
versionConflicts += thisVersionConflicts;
batches += thisBatches;
sliceStatuses.set(slice, new BulkByScrollTask.StatusOrException(sliceStatus));
@SuppressWarnings("unchecked")
ActionListener<BulkIndexByScrollResponse> listener = slice < slices - 1 ? neverCalled() : mock(ActionListener.class);
task.onSliceResponse(listener, slice,
new BulkIndexByScrollResponse(timeValueMillis(10), sliceStatus, emptyList(), emptyList(), false));
status = task.getStatus();
assertEquals(total, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batches, status.getBatches());
assertEquals(noops, status.getNoops());
assertEquals(sliceStatuses, status.getSliceStatuses());
if (slice == slices - 1) {
// The whole thing succeeded so we should have got the success
status = captureResponse(BulkIndexByScrollResponse.class, listener).getStatus();
assertEquals(total, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batches, status.getBatches());
assertEquals(noops, status.getNoops());
assertEquals(sliceStatuses, status.getSliceStatuses());
}
}
}
}

View File

@ -26,6 +26,9 @@ import java.util.List;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
public class ReindexBasicTests extends ReindexTestCase {
public void testFiltering() throws Exception {
@ -82,4 +85,34 @@ public class ReindexBasicTests extends ReindexTestCase {
assertThat(copy.get(), matcher().created(half).batches(half, 5));
assertHitCount(client().prepareSearch("dest").setTypes("half").setSize(0).get(), half);
}
public void testCopyManyWithSlices() throws Exception {
int workers = between(2, 10);
List<IndexRequestBuilder> docs = new ArrayList<>();
int max = between(150, 500);
for (int i = 0; i < max; i++) {
docs.add(client().prepareIndex("source", "test", Integer.toString(i)).setSource("foo", "a"));
}
indexRandom(true, docs);
assertHitCount(client().prepareSearch("source").setSize(0).get(), max);
// Copy all the docs
ReindexRequestBuilder copy = reindex().source("source").destination("dest", "all").refresh(true).setSlices(workers);
// Use a small batch size so we have to use more than one batch
copy.source().setSize(5);
assertThat(copy.get(), matcher().created(max).batches(greaterThanOrEqualTo(max / 5)).slices(hasSize(workers)));
assertHitCount(client().prepareSearch("dest").setTypes("all").setSize(0).get(), max);
// Copy some of the docs
int half = max / 2;
copy = reindex().source("source").destination("dest", "half").refresh(true).setSlices(workers);
// Use a small batch size so we have to use more than one batch
copy.source().setSize(5);
copy.size(half); // The real "size" of the request.
BulkIndexByScrollResponse response = copy.get();
assertThat(response, matcher().created(lessThanOrEqualTo((long) half)).slices(hasSize(workers)));
assertHitCount(client().prepareSearch("dest").setTypes("half").setSize(0).get(), response.getCreated());
}
}

View File

@ -0,0 +1,63 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.reindex;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.index.mapper.UidFieldMapper;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
import static java.util.Collections.emptyList;
import static org.elasticsearch.index.reindex.ReindexParallelizationHelper.sliceIntoSubRequests;
import static org.elasticsearch.search.RandomSearchRequestGenerator.randomSearchRequest;
import static org.elasticsearch.search.RandomSearchRequestGenerator.randomSearchSourceBuilder;
public class ReindexParallelizationHelperTests extends ESTestCase {
public void testSliceIntoSubRequests() throws IOException {
SearchRequest searchRequest = randomSearchRequest(() -> randomSearchSourceBuilder(
() -> null,
() -> null,
() -> null,
() -> emptyList()));
if (searchRequest.source() != null) {
// Clear the slice builder if there is one set. We can't call sliceIntoSubRequests if it is.
searchRequest.source().slice(null);
}
int times = between(2, 100);
String field = randomBoolean() ? UidFieldMapper.NAME : randomAsciiOfLength(5);
int currentSliceId = 0;
for (SearchRequest slice : sliceIntoSubRequests(searchRequest, field, times)) {
assertEquals(field, slice.source().slice().getField());
assertEquals(currentSliceId, slice.source().slice().getId());
assertEquals(times, slice.source().slice().getMax());
// If you clear the slice then the slice should be the same request as the parent request
slice.source().slice(null);
if (searchRequest.source() == null) {
// Except that adding the slice might have added an empty builder
searchRequest.source(new SearchSourceBuilder());
}
assertEquals(searchRequest, slice);
currentSliceId++;
}
}
}

View File

@ -24,7 +24,8 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.index.reindex.remote.RemoteInfo;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.slice.SliceBuilder;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
@ -32,9 +33,9 @@ import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
/**
* Tests some of the validation of {@linkplain ReindexRequest}. See reindex's rest tests for much more.
*/
public class ReindexRequestTests extends ESTestCase {
public class ReindexRequestTests extends AbstractBulkByScrollRequestTestCase<ReindexRequest> {
public void testTimestampAndTtlNotAllowed() {
ReindexRequest reindex = request();
ReindexRequest reindex = newRequest();
reindex.getDestination().ttl("1s").timestamp("now");
ActionRequestValidationException e = reindex.validate();
assertEquals("Validation Failed: 1: setting ttl on destination isn't supported. use scripts instead.;"
@ -43,7 +44,7 @@ public class ReindexRequestTests extends ESTestCase {
}
public void testReindexFromRemoteDoesNotSupportSearchQuery() {
ReindexRequest reindex = request();
ReindexRequest reindex = newRequest();
reindex.setRemoteInfo(new RemoteInfo(randomAsciiOfLength(5), randomAsciiOfLength(5), between(1, Integer.MAX_VALUE),
new BytesArray("real_query"), null, null, emptyMap()));
reindex.getSearchRequest().source().query(matchAllQuery()); // Unsupported place to put query
@ -52,7 +53,45 @@ public class ReindexRequestTests extends ESTestCase {
e.getMessage());
}
private ReindexRequest request() {
public void testReindexFromRemoteDoesNotSupportWorkers() {
ReindexRequest reindex = newRequest();
reindex.setRemoteInfo(new RemoteInfo(randomAsciiOfLength(5), randomAsciiOfLength(5), between(1, Integer.MAX_VALUE),
new BytesArray("real_query"), null, null, emptyMap()));
reindex.setSlices(between(2, Integer.MAX_VALUE));
ActionRequestValidationException e = reindex.validate();
assertEquals(
"Validation Failed: 1: reindex from remote sources doesn't support workers > 1 but was [" + reindex.getSlices() + "];",
e.getMessage());
}
public void testNoSliceWithWorkers() {
ReindexRequest reindex = newRequest();
reindex.getSearchRequest().source().slice(new SliceBuilder(0, 4));
reindex.setSlices(between(2, Integer.MAX_VALUE));
ActionRequestValidationException e = reindex.validate();
assertEquals("Validation Failed: 1: can't specify both slice and workers;", e.getMessage());
}
@Override
protected void extraRandomizationForSlice(ReindexRequest original) {
if (randomBoolean()) {
original.setScript(new Script(randomAsciiOfLength(5)));
}
if (randomBoolean()) {
original.setRemoteInfo(new RemoteInfo(randomAsciiOfLength(5), randomAsciiOfLength(5), between(1, 10000),
new BytesArray(randomAsciiOfLength(5)), null, null, emptyMap()));
}
}
@Override
protected void extraForSliceAssertions(ReindexRequest original, ReindexRequest forSliced) {
assertEquals(original.getScript(), forSliced.getScript());
assertEquals(original.getDestination(), forSliced.getDestination());
assertEquals(original.getRemoteInfo(), forSliced.getRemoteInfo());
}
@Override
protected ReindexRequest newRequest() {
ReindexRequest reindex = new ReindexRequest(new SearchRequest(), new IndexRequest());
reindex.getSearchRequest().indices("source");
reindex.getDestination().index("dest");

View File

@ -21,8 +21,18 @@ package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ListenableActionFuture;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
import org.elasticsearch.action.admin.cluster.node.tasks.list.TaskGroup;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.tasks.TaskId;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
/**
* Tests that you can set requests_per_second over the Java API and that you can rethrottle running requests. There are REST tests for this
@ -43,31 +53,107 @@ public class RethrottleTests extends ReindexTestCase {
testCase(deleteByQuery().source("test"), DeleteByQueryAction.NAME);
}
private void testCase(AbstractBulkByScrollRequestBuilder<?, ?> request, String actionName)
throws Exception {
// Use a single shard so the reindex has to happen in multiple batches
client().admin().indices().prepareCreate("test").setSettings("index.number_of_shards", 1).get();
indexRandom(true,
client().prepareIndex("test", "test", "1").setSource("foo", "bar"),
client().prepareIndex("test", "test", "2").setSource("foo", "bar"),
client().prepareIndex("test", "test", "3").setSource("foo", "bar"));
public void testReindexWithWorkers() throws Exception {
testCase(reindex().source("test").destination("dest").setSlices(between(2, 10)), ReindexAction.NAME);
}
public void testUpdateByQueryWithWorkers() throws Exception {
testCase(updateByQuery().source("test").setSlices(between(2, 10)), UpdateByQueryAction.NAME);
}
public void testDeleteByQueryWithWorkers() throws Exception {
testCase(deleteByQuery().source("test").setSlices(between(2, 10)), DeleteByQueryAction.NAME);
}
private void testCase(AbstractBulkByScrollRequestBuilder<?, ?> request, String actionName) throws Exception {
logger.info("Starting test for [{}] with [{}] slices", actionName, request.request().getSlices());
/* Add ten documents per slice so most slices will have many documents to process, having to go to multiple batches.
* we can't rely on all of them doing so, but
*/
List<IndexRequestBuilder> docs = new ArrayList<>();
for (int i = 0; i < request.request().getSlices() * 10; i++) {
docs.add(client().prepareIndex("test", "test", Integer.toString(i)).setSource("foo", "bar"));
}
indexRandom(true, docs);
// Start a request that will never finish unless we rethrottle it
request.setRequestsPerSecond(.000001f); // Throttle "forever"
request.source().setSize(1); // Make sure we use multiple batches
ListenableActionFuture<? extends BulkIndexByScrollResponse> responseListener = request.execute();
// Wait for the task to start
assertBusy(() -> assertEquals(1, client().admin().cluster().prepareListTasks().setActions(actionName).get().getTasks().size()));
TaskId taskToRethrottle = findTaskToRethrottle(actionName, request.request().getSlices());
// Now rethrottle it so it'll finish
ListTasksResponse rethrottleResponse = rethrottle().setActions(actionName).setRequestsPerSecond(Float.POSITIVE_INFINITY).get();
float newRequestsPerSecond = randomBoolean() ? Float.POSITIVE_INFINITY : between(1, 1000) * 100000; // No throttle or "very fast"
ListTasksResponse rethrottleResponse = rethrottle().setTaskId(taskToRethrottle).setRequestsPerSecond(newRequestsPerSecond).get();
rethrottleResponse.rethrowFailures("Rethrottle");
assertThat(rethrottleResponse.getTasks(), hasSize(1));
BulkByScrollTask.Status status = (BulkByScrollTask.Status) rethrottleResponse.getTasks().get(0).getStatus();
assertEquals(Float.POSITIVE_INFINITY, status.getRequestsPerSecond(), Float.MIN_NORMAL);
// Now check the resulting requests per second.
if (request.request().getSlices() == 1) {
// If there is a single slice it should match perfectly
assertEquals(newRequestsPerSecond, status.getRequestsPerSecond(), Float.MIN_NORMAL);
} else {
/* Check that at least one slice was rethrottled. We won't always rethrottle all of them because they might have completed.
* With multiple slices these numbers might not add up perfectly, thus the 0.0001f. */
float expectedSliceRequestsPerSecond = newRequestsPerSecond == Float.POSITIVE_INFINITY ? Float.POSITIVE_INFINITY
: newRequestsPerSecond / request.request().getSlices();
boolean oneSliceRethrottled = false;
float totalRequestsPerSecond = 0;
for (BulkByScrollTask.StatusOrException statusOrException : status.getSliceStatuses()) {
if (statusOrException == null) {
/* The slice can be null here because it was completed but hadn't reported its success back to the task when the
* rethrottle request came through. */
continue;
}
assertNull(statusOrException.getException());
BulkByScrollTask.Status slice = statusOrException.getStatus();
if (slice.getTotal() > slice.getSuccessfullyProcessed()) {
assertEquals(expectedSliceRequestsPerSecond, slice.getRequestsPerSecond(), expectedSliceRequestsPerSecond * 0.0001f);
}
if (Math.abs(expectedSliceRequestsPerSecond - slice.getRequestsPerSecond()) <= expectedSliceRequestsPerSecond * 0.0001f
|| expectedSliceRequestsPerSecond == slice.getRequestsPerSecond()) {
oneSliceRethrottled = true;
}
totalRequestsPerSecond += slice.getRequestsPerSecond();
}
assertTrue("At least one slice must be rethrottled", oneSliceRethrottled);
/* Now assert that the parent request has the total requests per second. This is a much weaker assertion than that the parent
* actually has the newRequestsPerSecond. For the most part it will. Sometimes it'll be greater because only unfinished requests
* are rethrottled, the finished ones just keep whatever requests per second they had while they were running. But it might
* also be less than newRequestsPerSecond because the newRequestsPerSecond is divided among running sub-requests and then the
* requests are rethrottled. If one request finishes in between the division and the application of the new throttle then it
* won't be rethrottled, thus only contributing its lower total. */
assertEquals(totalRequestsPerSecond, status.getRequestsPerSecond(), totalRequestsPerSecond * 0.0001f);
}
// Now the response should come back quickly because we've rethrottled the request
BulkIndexByScrollResponse response = responseListener.get();
assertEquals("Batches didn't match, this may invalidate the test as throttling is done between batches", 3, response.getBatches());
assertThat("Entire request completed in a single batch. This may invalidate the test as throttling is done between batches.",
response.getBatches(), greaterThanOrEqualTo(request.request().getSlices()));
}
private TaskId findTaskToRethrottle(String actionName, int sliceCount) {
ListTasksResponse tasks;
long start = System.nanoTime();
do {
tasks = client().admin().cluster().prepareListTasks().setActions(actionName).setDetailed(true).get();
tasks.rethrowFailures("Finding tasks to rethrottle");
for (TaskGroup taskGroup : tasks.getTaskGroups()) {
if (sliceCount == 1) {
assertThat(taskGroup.getChildTasks(), empty());
} else {
if (taskGroup.getChildTasks().stream().noneMatch(t ->
((BulkByScrollTask.Status) t.getTaskInfo().getStatus()).getTotal() > 0)) {
// Need to wait until a child is running that is non-empty so we can rethrottle it
continue;
}
assertThat(taskGroup.getChildTasks(), hasSize(lessThanOrEqualTo(sliceCount)));
}
return taskGroup.getTaskInfo().getTaskId();
}
} while (System.nanoTime() - start < TimeUnit.SECONDS.toNanos(10));
throw new AssertionError("Couldn't find task to rethrottle after waiting tasks=" + tasks.getTasks());
}
}

View File

@ -20,12 +20,14 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.bulk.BulkItemResponse.Failure;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.unit.TimeValue;
@ -40,14 +42,17 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.IntStream;
import static java.lang.Math.abs;
import static java.util.Collections.emptyList;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonList;
import static java.util.stream.Collectors.toList;
import static org.apache.lucene.util.TestUtil.randomSimpleString;
import static org.elasticsearch.common.unit.TimeValue.parseTimeValue;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.hamcrest.Matchers.hasSize;
/**
* Round trip tests for all Streamable things declared in this plugin.
@ -73,19 +78,18 @@ public class RoundTripTests extends ESTestCase {
ReindexRequest tripped = new ReindexRequest();
roundTrip(reindex, tripped);
assertRequestEquals(reindex, tripped);
assertEquals(reindex.getDestination().version(), tripped.getDestination().version());
assertEquals(reindex.getDestination().index(), tripped.getDestination().index());
if (reindex.getRemoteInfo() == null) {
assertNull(tripped.getRemoteInfo());
} else {
assertNotNull(tripped.getRemoteInfo());
assertEquals(reindex.getRemoteInfo().getScheme(), tripped.getRemoteInfo().getScheme());
assertEquals(reindex.getRemoteInfo().getHost(), tripped.getRemoteInfo().getHost());
assertEquals(reindex.getRemoteInfo().getQuery(), tripped.getRemoteInfo().getQuery());
assertEquals(reindex.getRemoteInfo().getUsername(), tripped.getRemoteInfo().getUsername());
assertEquals(reindex.getRemoteInfo().getPassword(), tripped.getRemoteInfo().getPassword());
assertEquals(reindex.getRemoteInfo().getHeaders(), tripped.getRemoteInfo().getHeaders());
}
// Try slices with a version that doesn't support slices. That should fail.
reindex.setSlices(between(2, 1000));
Exception e = expectThrows(UnsupportedOperationException.class, () -> roundTrip(Version.V_5_0_0_rc1, reindex, null));
assertEquals("Attempting to send sliced reindex-style request to a node that doesn't support it. "
+ "Version is [5.0.0-rc1] but must be [5.1.0]", e.getMessage());
// Try without slices with a version that doesn't support slices. That should work.
tripped = new ReindexRequest();
reindex.setSlices(1);
roundTrip(Version.V_5_0_0_rc1, reindex, tripped);
assertRequestEquals(reindex, tripped);
}
public void testUpdateByQueryRequest() throws IOException {
@ -98,9 +102,42 @@ public class RoundTripTests extends ESTestCase {
roundTrip(update, tripped);
assertRequestEquals(update, tripped);
assertEquals(update.getPipeline(), tripped.getPipeline());
// Try slices with a version that doesn't support slices. That should fail.
update.setSlices(between(2, 1000));
Exception e = expectThrows(UnsupportedOperationException.class, () -> roundTrip(Version.V_5_0_0_rc1, update, null));
assertEquals("Attempting to send sliced reindex-style request to a node that doesn't support it. "
+ "Version is [5.0.0-rc1] but must be [5.1.0]", e.getMessage());
// Try without slices with a version that doesn't support slices. That should work.
tripped = new UpdateByQueryRequest();
update.setSlices(1);
roundTrip(Version.V_5_0_0_rc1, update, tripped);
assertRequestEquals(update, tripped);
assertEquals(update.getPipeline(), tripped.getPipeline());
}
private void randomRequest(AbstractBulkIndexByScrollRequest<?> request) {
public void testDeleteByQueryRequest() throws IOException {
DeleteByQueryRequest delete = new DeleteByQueryRequest(new SearchRequest());
randomRequest(delete);
DeleteByQueryRequest tripped = new DeleteByQueryRequest();
roundTrip(delete, tripped);
assertRequestEquals(delete, tripped);
// Try slices with a version that doesn't support slices. That should fail.
delete.setSlices(between(2, 1000));
Exception e = expectThrows(UnsupportedOperationException.class, () -> roundTrip(Version.V_5_0_0_rc1, delete, null));
assertEquals("Attempting to send sliced reindex-style request to a node that doesn't support it. "
+ "Version is [5.0.0-rc1] but must be [5.1.0]", e.getMessage());
// Try without slices with a version that doesn't support slices. That should work.
tripped = new DeleteByQueryRequest();
delete.setSlices(1);
roundTrip(Version.V_5_0_0_rc1, delete, tripped);
assertRequestEquals(delete, tripped);
}
private void randomRequest(AbstractBulkByScrollRequest<?> request) {
request.getSearchRequest().indices("test");
request.getSearchRequest().source().size(between(1, 1000));
request.setSize(random().nextBoolean() ? between(1, Integer.MAX_VALUE) : -1);
@ -108,19 +145,45 @@ public class RoundTripTests extends ESTestCase {
request.setRefresh(rarely());
request.setTimeout(TimeValue.parseTimeValue(randomTimeValue(), null, "test"));
request.setWaitForActiveShards(randomIntBetween(0, 10));
request.setScript(random().nextBoolean() ? null : randomScript());
request.setRequestsPerSecond(between(0, Integer.MAX_VALUE));
request.setSlices(between(1, Integer.MAX_VALUE));
}
private void randomRequest(AbstractBulkIndexByScrollRequest<?> request) {
randomRequest((AbstractBulkByScrollRequest<?>) request);
request.setScript(random().nextBoolean() ? null : randomScript());
}
private void assertRequestEquals(ReindexRequest request, ReindexRequest tripped) {
assertRequestEquals((AbstractBulkIndexByScrollRequest<?>) request, (AbstractBulkIndexByScrollRequest<?>) tripped);
assertEquals(request.getDestination().version(), tripped.getDestination().version());
assertEquals(request.getDestination().index(), tripped.getDestination().index());
if (request.getRemoteInfo() == null) {
assertNull(tripped.getRemoteInfo());
} else {
assertNotNull(tripped.getRemoteInfo());
assertEquals(request.getRemoteInfo().getScheme(), tripped.getRemoteInfo().getScheme());
assertEquals(request.getRemoteInfo().getHost(), tripped.getRemoteInfo().getHost());
assertEquals(request.getRemoteInfo().getQuery(), tripped.getRemoteInfo().getQuery());
assertEquals(request.getRemoteInfo().getUsername(), tripped.getRemoteInfo().getUsername());
assertEquals(request.getRemoteInfo().getPassword(), tripped.getRemoteInfo().getPassword());
assertEquals(request.getRemoteInfo().getHeaders(), tripped.getRemoteInfo().getHeaders());
}
}
private void assertRequestEquals(AbstractBulkIndexByScrollRequest<?> request,
AbstractBulkIndexByScrollRequest<?> tripped) {
assertRequestEquals((AbstractBulkByScrollRequest<?>) request, (AbstractBulkByScrollRequest<?>) tripped);
assertEquals(request.getScript(), tripped.getScript());
}
private void assertRequestEquals(AbstractBulkByScrollRequest<?> request, AbstractBulkByScrollRequest<?> tripped) {
assertArrayEquals(request.getSearchRequest().indices(), tripped.getSearchRequest().indices());
assertEquals(request.getSearchRequest().source().size(), tripped.getSearchRequest().source().size());
assertEquals(request.isAbortOnVersionConflict(), tripped.isAbortOnVersionConflict());
assertEquals(request.isRefresh(), tripped.isRefresh());
assertEquals(request.getTimeout(), tripped.getTimeout());
assertEquals(request.getWaitForActiveShards(), tripped.getWaitForActiveShards());
assertEquals(request.getScript(), tripped.getScript());
assertEquals(request.getRetryBackoffInitialTime(), tripped.getRetryBackoffInitialTime());
assertEquals(request.getMaxRetries(), tripped.getMaxRetries());
assertEquals(request.getRequestsPerSecond(), tripped.getRequestsPerSecond(), 0d);
@ -131,7 +194,16 @@ public class RoundTripTests extends ESTestCase {
BytesStreamOutput out = new BytesStreamOutput();
status.writeTo(out);
BulkByScrollTask.Status tripped = new BulkByScrollTask.Status(out.bytes().streamInput());
assertTaskStatusEquals(status, tripped);
assertTaskStatusEquals(out.getVersion(), status, tripped);
// Also check round tripping pre-5.1 which is the first version to support parallelized scroll
out = new BytesStreamOutput();
out.setVersion(Version.V_5_0_0_rc1); // This can be V_5_0_0
status.writeTo(out);
StreamInput in = out.bytes().streamInput();
in.setVersion(Version.V_5_0_0_rc1);
tripped = new BulkByScrollTask.Status(in);
assertTaskStatusEquals(Version.V_5_0_0_rc1, status, tripped);
}
public void testReindexResponse() throws IOException {
@ -166,10 +238,38 @@ public class RoundTripTests extends ESTestCase {
}
private BulkByScrollTask.Status randomStatus() {
return new BulkByScrollTask.Status(randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(),
randomInt(Integer.MAX_VALUE), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(),
parseTimeValue(randomPositiveTimeValue(), "test"), abs(random().nextFloat()),
random().nextBoolean() ? null : randomSimpleString(random()), parseTimeValue(randomPositiveTimeValue(), "test"));
if (randomBoolean()) {
return randomWorkingStatus(null);
}
boolean canHaveNullStatues = randomBoolean();
List<BulkByScrollTask.StatusOrException> statuses = IntStream.range(0, between(0, 10))
.mapToObj(i -> {
if (canHaveNullStatues && rarely()) {
return null;
}
if (randomBoolean()) {
return new BulkByScrollTask.StatusOrException(new ElasticsearchException(randomAsciiOfLength(5)));
}
return new BulkByScrollTask.StatusOrException(randomWorkingStatus(i));
})
.collect(toList());
return new BulkByScrollTask.Status(statuses, randomBoolean() ? "test" : null);
}
private BulkByScrollTask.Status randomWorkingStatus(Integer sliceId) {
// These all should be believably small because we sum them if we have multiple workers
int total = between(0, 10000000);
int updated = between(0, total);
int created = between(0, total - updated);
int deleted = between(0, total - updated - created);
int noops = total - updated - created - deleted;
int batches = between(0, 10000);
long versionConflicts = between(0, total);
long bulkRetries = between(0, 10000000);
long searchRetries = between(0, 100000);
return new BulkByScrollTask.Status(sliceId, total, updated, created, deleted, batches, versionConflicts, noops, bulkRetries,
searchRetries, parseTimeValue(randomPositiveTimeValue(), "test"), abs(random().nextFloat()),
randomBoolean() ? null : randomSimpleString(random()), parseTimeValue(randomPositiveTimeValue(), "test"));
}
private List<Failure> randomIndexingFailures() {
@ -194,9 +294,16 @@ public class RoundTripTests extends ESTestCase {
}
private void roundTrip(Streamable example, Streamable empty) throws IOException {
roundTrip(Version.CURRENT, example, empty);
}
private void roundTrip(Version version, Streamable example, Streamable empty) throws IOException {
BytesStreamOutput out = new BytesStreamOutput();
out.setVersion(version);
example.writeTo(out);
empty.readFrom(out.bytes().streamInput());
StreamInput in = out.bytes().streamInput();
in.setVersion(version);
empty.readFrom(in);
}
private Script randomScript() {
@ -208,7 +315,7 @@ public class RoundTripTests extends ESTestCase {
private void assertResponseEquals(BulkIndexByScrollResponse expected, BulkIndexByScrollResponse actual) {
assertEquals(expected.getTook(), actual.getTook());
assertTaskStatusEquals(expected.getStatus(), actual.getStatus());
assertTaskStatusEquals(Version.CURRENT, expected.getStatus(), actual.getStatus());
assertEquals(expected.getBulkFailures().size(), actual.getBulkFailures().size());
for (int i = 0; i < expected.getBulkFailures().size(); i++) {
Failure expectedFailure = expected.getBulkFailures().get(i);
@ -232,7 +339,8 @@ public class RoundTripTests extends ESTestCase {
}
private void assertTaskStatusEquals(BulkByScrollTask.Status expected, BulkByScrollTask.Status actual) {
private void assertTaskStatusEquals(Version version, BulkByScrollTask.Status expected, BulkByScrollTask.Status actual) {
assertEquals(expected.getTotal(), actual.getTotal());
assertEquals(expected.getUpdated(), actual.getUpdated());
assertEquals(expected.getCreated(), actual.getCreated());
assertEquals(expected.getDeleted(), actual.getDeleted());
@ -245,5 +353,23 @@ public class RoundTripTests extends ESTestCase {
assertEquals(expected.getRequestsPerSecond(), actual.getRequestsPerSecond(), 0f);
assertEquals(expected.getReasonCancelled(), actual.getReasonCancelled());
assertEquals(expected.getThrottledUntil(), actual.getThrottledUntil());
if (version.onOrAfter(BulkByScrollTask.V_5_1_0_UNRELEASED)) {
assertThat(actual.getSliceStatuses(), hasSize(expected.getSliceStatuses().size()));
for (int i = 0; i < expected.getSliceStatuses().size(); i++) {
BulkByScrollTask.StatusOrException sliceStatus = expected.getSliceStatuses().get(i);
if (sliceStatus == null) {
assertNull(actual.getSliceStatuses().get(i));
} else if (sliceStatus.getException() == null) {
assertNull(actual.getSliceStatuses().get(i).getException());
assertTaskStatusEquals(version, sliceStatus.getStatus(), actual.getSliceStatuses().get(i).getStatus());
} else {
assertNull(actual.getSliceStatuses().get(i).getStatus());
// Just check the message because we're not testing exception serialization in general here.
assertEquals(sliceStatus.getException().getMessage(), actual.getSliceStatuses().get(i).getException().getMessage());
}
}
} else {
assertEquals(emptyList(), actual.getSliceStatuses());
}
}
}

View File

@ -0,0 +1,210 @@
/*
* 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.reindex;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.tasks.TaskInfo;
import org.elasticsearch.test.ESTestCase;
import org.hamcrest.Matcher;
import org.junit.Before;
import org.mockito.ArgumentCaptor;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Consumer;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.hasToString;
import static org.hamcrest.Matchers.theInstance;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.atMost;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.verify;
public class TransportRethrottleActionTests extends ESTestCase {
private int slices;
private ParentBulkByScrollTask task;
@Before
public void createTask() {
slices = between(2, 50);
task = new ParentBulkByScrollTask(1, "test_type", "test_action", "test", null, slices);
}
/**
* Test rethrottling.
* @param runningSlices the number of slices still running
* @param simulator simulate a response from the sub-request to rethrottle the child requests
* @param verifier verify the resulting response
*/
private void rethrottleTestCase(int runningSlices, Consumer<ActionListener<ListTasksResponse>> simulator,
Consumer<ActionListener<TaskInfo>> verifier) {
Client client = mock(Client.class);
String localNodeId = randomAsciiOfLength(5);
float newRequestsPerSecond = randomValueOtherThanMany(f -> f <= 0, () -> randomFloat());
@SuppressWarnings("unchecked")
ActionListener<TaskInfo> listener = mock(ActionListener.class);
TransportRethrottleAction.rethrottle(localNodeId, client, task, newRequestsPerSecond, listener);
// Capture the sub request and the listener so we can verify they are sane
ArgumentCaptor<RethrottleRequest> subRequest = ArgumentCaptor.forClass(RethrottleRequest.class);
@SuppressWarnings({ "unchecked", "rawtypes" }) // Magical generics incantation.....
ArgumentCaptor<ActionListener<ListTasksResponse>> subListener = ArgumentCaptor.forClass((Class) ActionListener.class);
if (runningSlices > 0) {
verify(client).execute(eq(RethrottleAction.INSTANCE), subRequest.capture(), subListener.capture());
assertEquals(new TaskId(localNodeId, task.getId()), subRequest.getValue().getParentTaskId());
assertEquals(newRequestsPerSecond / runningSlices, subRequest.getValue().getRequestsPerSecond(), 0.00001f);
simulator.accept(subListener.getValue());
}
verifier.accept(listener);
}
private Consumer<ActionListener<TaskInfo>> expectSuccessfulRethrottleWithStatuses(
List<BulkByScrollTask.StatusOrException> sliceStatuses) {
return listener -> {
TaskInfo taskInfo = captureResponse(TaskInfo.class, listener);
assertEquals(sliceStatuses, ((BulkByScrollTask.Status) taskInfo.getStatus()).getSliceStatuses());
};
}
public void testRethrottleSuccessfulResponse() {
List<TaskInfo> tasks = new ArrayList<>();
List<BulkByScrollTask.StatusOrException> sliceStatuses = new ArrayList<>(slices);
for (int i = 0; i < slices; i++) {
BulkByScrollTask.Status status = believeableInProgressStatus(i);
tasks.add(new TaskInfo(new TaskId("test", 123), "test", "test", "test", status, 0, 0, true, new TaskId("test", task.getId())));
sliceStatuses.add(new BulkByScrollTask.StatusOrException(status));
}
rethrottleTestCase(slices,
listener -> listener.onResponse(new ListTasksResponse(tasks, emptyList(), emptyList())),
expectSuccessfulRethrottleWithStatuses(sliceStatuses));
}
public void testRethrottleWithSomeSucceeded() {
int succeeded = between(1, slices - 1);
List<BulkByScrollTask.StatusOrException> sliceStatuses = new ArrayList<>(slices);
for (int i = 0; i < succeeded; i++) {
BulkByScrollTask.Status status = believeableCompletedStatus(i);
task.onSliceResponse(neverCalled(), i,
new BulkIndexByScrollResponse(timeValueMillis(10), status, emptyList(), emptyList(), false));
sliceStatuses.add(new BulkByScrollTask.StatusOrException(status));
}
List<TaskInfo> tasks = new ArrayList<>();
for (int i = succeeded; i < slices; i++) {
BulkByScrollTask.Status status = believeableInProgressStatus(i);
tasks.add(new TaskInfo(new TaskId("test", 123), "test", "test", "test", status, 0, 0, true, new TaskId("test", task.getId())));
sliceStatuses.add(new BulkByScrollTask.StatusOrException(status));
}
rethrottleTestCase(slices - succeeded,
listener -> listener.onResponse(new ListTasksResponse(tasks, emptyList(), emptyList())),
expectSuccessfulRethrottleWithStatuses(sliceStatuses));
}
public void testRethrottleWithAllSucceeded() {
List<BulkByScrollTask.StatusOrException> sliceStatuses = new ArrayList<>(slices);
for (int i = 0; i < slices; i++) {
@SuppressWarnings("unchecked")
ActionListener<BulkIndexByScrollResponse> listener = i < slices - 1 ? neverCalled() : mock(ActionListener.class);
BulkByScrollTask.Status status = believeableCompletedStatus(i);
task.onSliceResponse(listener, i, new BulkIndexByScrollResponse(timeValueMillis(10), status, emptyList(), emptyList(), false));
if (i == slices - 1) {
// The whole thing succeeded so we should have got the success
captureResponse(BulkIndexByScrollResponse.class, listener).getStatus();
}
sliceStatuses.add(new BulkByScrollTask.StatusOrException(status));
}
rethrottleTestCase(0,
listener -> { /* There are no async tasks to simulate because the listener is called for us. */},
expectSuccessfulRethrottleWithStatuses(sliceStatuses));
}
private Consumer<ActionListener<TaskInfo>> expectException(Matcher<Exception> exceptionMatcher) {
return listener -> {
ArgumentCaptor<Exception> failure = ArgumentCaptor.forClass(Exception.class);
verify(listener).onFailure(failure.capture());
assertThat(failure.getValue(), exceptionMatcher);
};
}
public void testRethrottleCatastrophicFailures() {
Exception e = new Exception();
rethrottleTestCase(slices, listener -> listener.onFailure(e), expectException(theInstance(e)));
}
public void testRethrottleTaskOperationFailure() {
Exception e = new Exception();
TaskOperationFailure failure = new TaskOperationFailure("test", 123, e);
rethrottleTestCase(slices,
listener -> listener.onResponse(new ListTasksResponse(emptyList(), singletonList(failure), emptyList())),
expectException(hasToString(containsString("Rethrottle of [test:123] failed"))));
}
public void testRethrottleNodeFailure() {
FailedNodeException e = new FailedNodeException("test", "test", new Exception());
rethrottleTestCase(slices,
listener -> listener.onResponse(new ListTasksResponse(emptyList(), emptyList(), singletonList(e))),
expectException(theInstance(e)));
}
private BulkByScrollTask.Status believeableInProgressStatus(Integer sliceId) {
return new BulkByScrollTask.Status(sliceId, 10, 0, 0, 0, 0, 0, 0, 0, 0, timeValueMillis(0), 0, null, timeValueMillis(0));
}
private BulkByScrollTask.Status believeableCompletedStatus(Integer sliceId) {
return new BulkByScrollTask.Status(sliceId, 10, 10, 0, 0, 0, 0, 0, 0, 0, timeValueMillis(0), 0, null, timeValueMillis(0));
}
static <T> ActionListener<T> neverCalled() {
return new ActionListener<T>() {
@Override
public void onResponse(T response) {
throw new RuntimeException("Expected no interactions but got [" + response + "]");
}
@Override
public void onFailure(Exception e) {
throw new RuntimeException("Expected no interations but was received a failure", e);
}
};
}
static <T> T captureResponse(Class<T> responseClass, ActionListener<T> listener) {
ArgumentCaptor<Exception> failure = ArgumentCaptor.forClass(Exception.class);
// Rethrow any failures just so we get a nice exception if there were any. We don't expect any though.
verify(listener, atMost(1)).onFailure(failure.capture());
if (false == failure.getAllValues().isEmpty()) {
throw new AssertionError(failure.getValue());
}
ArgumentCaptor<T> response = ArgumentCaptor.forClass(responseClass);
verify(listener).onResponse(response.capture());
return response.getValue();
}
}

View File

@ -23,6 +23,7 @@ import org.elasticsearch.search.sort.SortOrder;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.hasSize;
public class UpdateByQueryBasicTests extends ReindexTestCase {
public void testBasics() throws Exception {
@ -61,4 +62,33 @@ public class UpdateByQueryBasicTests extends ReindexTestCase {
assertEquals(3, client().prepareGet("test", "test", "3").get().getVersion());
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
}
public void testWorkers() throws Exception {
indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "a"),
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
client().prepareIndex("test", "test", "3").setSource("foo", "b"),
client().prepareIndex("test", "test", "4").setSource("foo", "c"));
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 4);
assertEquals(1, client().prepareGet("test", "test", "1").get().getVersion());
assertEquals(1, client().prepareGet("test", "test", "4").get().getVersion());
// Reindex all the docs
assertThat(updateByQuery().source("test").refresh(true).setSlices(5).get(), matcher().updated(4).slices(hasSize(5)));
assertEquals(2, client().prepareGet("test", "test", "1").get().getVersion());
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
// Now none of them
assertThat(updateByQuery().source("test").filter(termQuery("foo", "no_match")).setSlices(5).refresh(true).get(),
matcher().updated(0).slices(hasSize(5)));
assertEquals(2, client().prepareGet("test", "test", "1").get().getVersion());
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
// Now half of them
assertThat(updateByQuery().source("test").filter(termQuery("foo", "a")).refresh(true).setSlices(5).get(),
matcher().updated(2).slices(hasSize(5)));
assertEquals(3, client().prepareGet("test", "test", "1").get().getVersion());
assertEquals(3, client().prepareGet("test", "test", "2").get().getVersion());
assertEquals(2, client().prepareGet("test", "test", "3").get().getVersion());
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
}
}

View File

@ -21,11 +21,11 @@ package org.elasticsearch.index.reindex;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.script.Script;
import static org.apache.lucene.util.TestUtil.randomSimpleString;
public class UpdateByQueryRequestTests extends ESTestCase {
public class UpdateByQueryRequestTests extends AbstractBulkByScrollRequestTestCase<UpdateByQueryRequest> {
public void testUpdateByQueryRequestImplementsIndicesRequestReplaceable() {
int numIndices = between(1, 100);
String[] indices = new String[numIndices];
@ -58,4 +58,25 @@ public class UpdateByQueryRequestTests extends ESTestCase {
assertEquals(newIndices[i], request.getSearchRequest().indices()[i]);
}
}
@Override
protected UpdateByQueryRequest newRequest() {
return new UpdateByQueryRequest(new SearchRequest(randomAsciiOfLength(5)));
}
@Override
protected void extraRandomizationForSlice(UpdateByQueryRequest original) {
if (randomBoolean()) {
original.setScript(new Script(randomAsciiOfLength(5)));
}
if (randomBoolean()) {
original.setPipeline(randomAsciiOfLength(5));
}
}
@Override
protected void extraForSliceAssertions(UpdateByQueryRequest original, UpdateByQueryRequest forSliced) {
assertEquals(original.getScript(), forSliced.getScript());
assertEquals(original.getPipeline(), forSliced.getPipeline());
}
}

View File

@ -0,0 +1,265 @@
/*
* 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.reindex;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.tasks.TaskId;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.Before;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.Delayed;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
import static org.hamcrest.Matchers.both;
import static org.hamcrest.Matchers.closeTo;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
public class WorkingBulkByScrollTaskTests extends ESTestCase {
private WorkingBulkByScrollTask task;
@Before
public void createTask() {
task = new WorkingBulkByScrollTask(1, "test_type", "test_action", "test", TaskId.EMPTY_TASK_ID, null, Float.POSITIVE_INFINITY);
}
public void testBasicData() {
assertEquals(1, task.getId());
assertEquals("test_type", task.getType());
assertEquals("test_action", task.getAction());
assertEquals("test", task.getDescription());
}
public void testProgress() {
long created = 0;
long updated = 0;
long deleted = 0;
long versionConflicts = 0;
long noops = 0;
int batch = 0;
BulkByScrollTask.Status status = task.getStatus();
assertEquals(0, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batch, status.getBatches());
assertEquals(noops, status.getNoops());
long totalHits = randomIntBetween(10, 1000);
task.setTotal(totalHits);
for (long p = 0; p < totalHits; p++) {
status = task.getStatus();
assertEquals(totalHits, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batch, status.getBatches());
assertEquals(noops, status.getNoops());
if (randomBoolean()) {
created++;
task.countCreated();
} else if (randomBoolean()) {
updated++;
task.countUpdated();
} else {
deleted++;
task.countDeleted();
}
if (rarely()) {
versionConflicts++;
task.countVersionConflict();
}
if (rarely()) {
batch++;
task.countBatch();
}
if (rarely()) {
noops++;
task.countNoop();
}
}
status = task.getStatus();
assertEquals(totalHits, status.getTotal());
assertEquals(created, status.getCreated());
assertEquals(updated, status.getUpdated());
assertEquals(deleted, status.getDeleted());
assertEquals(versionConflicts, status.getVersionConflicts());
assertEquals(batch, status.getBatches());
assertEquals(noops, status.getNoops());
}
/**
* Furiously rethrottles a delayed request to make sure that we never run it twice.
*/
public void testDelayAndRethrottle() throws IOException, InterruptedException {
List<Throwable> errors = new CopyOnWriteArrayList<>();
AtomicBoolean done = new AtomicBoolean();
int threads = between(1, 10);
CyclicBarrier waitForShutdown = new CyclicBarrier(threads);
/*
* We never end up waiting this long because the test rethrottles over and over again, ratcheting down the delay a random amount
* each time.
*/
float originalRequestsPerSecond = (float) randomDoubleBetween(1, 10000, true);
task.rethrottle(originalRequestsPerSecond);
TimeValue maxDelay = timeValueSeconds(between(1, 5));
assertThat(maxDelay.nanos(), greaterThanOrEqualTo(0L));
int batchSizeForMaxDelay = (int) (maxDelay.seconds() * originalRequestsPerSecond);
ThreadPool threadPool = new TestThreadPool(getTestName()) {
@Override
public ScheduledFuture<?> schedule(TimeValue delay, String name, Runnable command) {
assertThat(delay.nanos(), both(greaterThanOrEqualTo(0L)).and(lessThanOrEqualTo(maxDelay.nanos())));
return super.schedule(delay, name, command);
}
};
try {
task.delayPrepareBulkRequest(threadPool, timeValueNanos(System.nanoTime()), batchSizeForMaxDelay, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
boolean oldValue = done.getAndSet(true);
if (oldValue) {
throw new RuntimeException("Ran twice oh no!");
}
}
@Override
public void onFailure(Exception e) {
errors.add(e);
}
});
// Rethrottle on a random number of threads, on of which is this thread.
Runnable test = () -> {
try {
int rethrottles = 0;
while (false == done.get()) {
float requestsPerSecond = (float) randomDoubleBetween(0, originalRequestsPerSecond * 2, true);
task.rethrottle(requestsPerSecond);
rethrottles += 1;
}
logger.info("Rethrottled [{}] times", rethrottles);
waitForShutdown.await();
} catch (Exception e) {
errors.add(e);
}
};
for (int i = 1; i < threads; i++) {
threadPool.generic().execute(test);
}
test.run();
} finally {
// Other threads should finish up quickly as they are checking the same AtomicBoolean.
threadPool.shutdown();
threadPool.awaitTermination(10, TimeUnit.SECONDS);
}
assertThat(errors, empty());
}
public void testDelayNeverNegative() throws IOException {
// Thread pool that returns a ScheduledFuture that claims to have a negative delay
ThreadPool threadPool = new TestThreadPool("test") {
public ScheduledFuture<?> schedule(TimeValue delay, String name, Runnable command) {
return new ScheduledFuture<Void>() {
@Override
public long getDelay(TimeUnit unit) {
return -1;
}
@Override
public int compareTo(Delayed o) {
throw new UnsupportedOperationException();
}
@Override
public boolean cancel(boolean mayInterruptIfRunning) {
throw new UnsupportedOperationException();
}
@Override
public boolean isCancelled() {
throw new UnsupportedOperationException();
}
@Override
public boolean isDone() {
throw new UnsupportedOperationException();
}
@Override
public Void get() throws InterruptedException, ExecutionException {
throw new UnsupportedOperationException();
}
@Override
public Void get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
throw new UnsupportedOperationException();
}
};
}
};
try {
// Have the task use the thread pool to delay a task that does nothing
task.delayPrepareBulkRequest(threadPool, timeValueSeconds(0), 1, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
}
@Override
public void onFailure(Exception e) {
throw new UnsupportedOperationException();
}
});
// Even though the future returns a negative delay we just return 0 because the time is up.
assertEquals(timeValueSeconds(0), task.getStatus().getThrottledUntil());
} finally {
threadPool.shutdown();
}
}
public void testPerfectlyThrottledBatchTime() {
task.rethrottle(Float.POSITIVE_INFINITY);
assertThat((double) task.perfectlyThrottledBatchTime(randomInt()), closeTo(0f, 0f));
int total = between(0, 1000000);
task.rethrottle(1);
assertThat((double) task.perfectlyThrottledBatchTime(total),
closeTo(TimeUnit.SECONDS.toNanos(total), TimeUnit.SECONDS.toNanos(1)));
}
}

View File

@ -109,3 +109,24 @@
body:
query:
match_all: {}
---
"junk in slices fails":
- do:
catch: /Failed to parse int parameter \[slices\] with value \[junk\]/
delete_by_query:
slices: junk
index: test
body:
query:
match_all: {}
---
"zero slices fails":
- do:
catch: /\[slices\] must be at least 1/
delete_by_query:
slices: 0
index: test
body:
query:
match_all: {}

View File

@ -0,0 +1,279 @@
---
"Multiple slices":
- do:
index:
index: test
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 4
body: { "text": "test" }
- do:
indices.refresh: {}
- do:
delete_by_query:
index: test
slices: 5
body:
query:
match_all: {}
- is_false: timed_out
- match: {deleted: 4}
- is_false: created
- is_false: updated
- match: {version_conflicts: 0}
- match: {failures: []}
- match: {noops: 0}
- match: {throttled_millis: 0}
- gte: { took: 0 }
- is_false: task
- match: {slices.0.version_conflicts: 0}
- match: {slices.0.throttled_millis: 0}
- match: {slices.1.version_conflicts: 0}
- match: {slices.1.throttled_millis: 0}
- match: {slices.2.version_conflicts: 0}
- match: {slices.2.throttled_millis: 0}
- match: {slices.3.version_conflicts: 0}
- match: {slices.3.throttled_millis: 0}
- match: {slices.4.version_conflicts: 0}
- match: {slices.4.throttled_millis: 0}
- do:
indices.refresh: {}
- do:
count:
index: test
- match: {count: 0}
---
"Multiple slices with wait_for_completion=false":
- do:
index:
index: test
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 4
body: { "text": "test" }
- do:
indices.refresh: {}
- do:
delete_by_query:
index: test
wait_for_completion: false
slices: 5
body:
query:
match_all: {}
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
- is_false: version_conflicts
- is_false: batches
- is_false: failures
- is_false: noops
- is_false: took
- is_false: throttled_millis
- is_false: created
- is_false: deleted
- do:
tasks.get:
wait_for_completion: true
task_id: $task
- is_false: node_failures
# The task will be in the response even if it finished before we got here
# because of task persistence.
- is_true: task
- match: {response.total: 4}
- match: {response.deleted: 4}
- match: {response.version_conflicts: 0}
- match: {response.failures: []}
- match: {response.throttled_millis: 0}
- gte: { response.took: 0 }
- is_false: response.task
- match: {response.slices.0.version_conflicts: 0}
- match: {response.slices.0.throttled_millis: 0}
- match: {response.slices.1.version_conflicts: 0}
- match: {response.slices.1.throttled_millis: 0}
- match: {response.slices.2.version_conflicts: 0}
- match: {response.slices.2.throttled_millis: 0}
- match: {response.slices.3.version_conflicts: 0}
- match: {response.slices.3.throttled_millis: 0}
- match: {response.slices.4.version_conflicts: 0}
- match: {response.slices.4.throttled_millis: 0}
- match: {task.status.total: 4}
- match: {task.status.deleted: 4}
- match: {task.status.version_conflicts: 0}
- match: {task.status.throttled_millis: 0}
- match: {task.status.slices.0.version_conflicts: 0}
- match: {task.status.slices.0.throttled_millis: 0}
- match: {task.status.slices.1.version_conflicts: 0}
- match: {task.status.slices.1.throttled_millis: 0}
- match: {task.status.slices.2.version_conflicts: 0}
- match: {task.status.slices.2.throttled_millis: 0}
- match: {task.status.slices.3.version_conflicts: 0}
- match: {task.status.slices.3.throttled_millis: 0}
- match: {task.status.slices.4.version_conflicts: 0}
- match: {task.status.slices.4.throttled_millis: 0}
# Only the "parent" reindex task wrote its status to the tasks index though
- do:
indices.refresh: {}
- do:
search:
index: .tasks
- match: { hits.total: 1 }
- do:
count:
index: test
- match: {count: 0}
---
"Multiple slices with rethrottle":
- do:
index:
index: test
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 4
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 5
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 6
body: { "text": "test" }
- do:
indices.refresh: {}
# Start the task with a requests_per_second that should make it take a very long time
- do:
delete_by_query:
index: test
wait_for_completion: false
slices: 2
requests_per_second: 0.0001
body:
query:
match_all: {}
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
- is_false: version_conflicts
- is_false: batches
- is_false: failures
- is_false: noops
- is_false: took
- is_false: throttled_millis
- is_false: created
- is_false: deleted
# Allow the task to complete
- do:
reindex_rethrottle:
requests_per_second: -1
task_id: $task
- do:
tasks.get:
wait_for_completion: true
task_id: $task
- is_false: node_failures
# The task will be in the response even if it finished before we got here
# because of task persistence.
- is_true: task
- match: {response.total: 6}
- match: {response.deleted: 6}
- match: {response.version_conflicts: 0}
- match: {response.failures: []}
- match: {response.throttled_millis: 0}
- gte: { response.took: 0 }
- is_false: response.task
- match: {response.slices.0.version_conflicts: 0}
- match: {response.slices.0.throttled_millis: 0}
- match: {response.slices.1.version_conflicts: 0}
- match: {response.slices.1.throttled_millis: 0}
- match: {task.status.total: 6}
- match: {task.status.deleted: 6}
- match: {task.status.version_conflicts: 0}
- match: {task.status.throttled_millis: 0}
- match: {task.status.slices.0.version_conflicts: 0}
- match: {task.status.slices.0.throttled_millis: 0}
- match: {task.status.slices.1.version_conflicts: 0}
- match: {task.status.slices.1.throttled_millis: 0}
# Only the "parent" reindex task wrote its status to the tasks index though
- do:
indices.refresh: {}
- do:
search:
index: .tasks
- match: { hits.total: 1 }
- do:
count:
index: test
- match: {count: 0}

View File

@ -279,3 +279,27 @@
index: test
dest:
index: dest
---
"junk in slices fails":
- do:
catch: /Failed to parse int parameter \[slices\] with value \[junk\]/
reindex:
slices: junk
body:
source:
index: test
dest:
index: dest
---
"zero slices fails":
- do:
catch: /\[slices\] must be at least 1/
reindex:
slices: 0
body:
source:
index: test
dest:
index: dest

View File

@ -0,0 +1,287 @@
---
"Multiple slices":
- do:
index:
index: source
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 4
body: { "text": "test" }
- do:
indices.refresh: {}
- do:
reindex:
slices: 5
body:
source:
index: source
dest:
index: dest
- match: {created: 4}
- match: {updated: 0}
- match: {version_conflicts: 0}
- match: {failures: []}
- match: {throttled_millis: 0}
- gte: { took: 0 }
- is_false: task
- is_false: deleted
- match: {slices.0.updated: 0}
- match: {slices.0.version_conflicts: 0}
- match: {slices.0.throttled_millis: 0}
- match: {slices.1.updated: 0}
- match: {slices.1.version_conflicts: 0}
- match: {slices.1.throttled_millis: 0}
- match: {slices.2.updated: 0}
- match: {slices.2.version_conflicts: 0}
- match: {slices.2.throttled_millis: 0}
- match: {slices.3.updated: 0}
- match: {slices.3.version_conflicts: 0}
- match: {slices.3.throttled_millis: 0}
- match: {slices.4.updated: 0}
- match: {slices.4.version_conflicts: 0}
- match: {slices.4.throttled_millis: 0}
---
"Multiple slices with wait_for_completion=false":
- do:
index:
index: source
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 4
body: { "text": "test" }
- do:
indices.refresh: {}
- do:
reindex:
slices: 5
wait_for_completion: false
body:
source:
index: source
dest:
index: dest
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
- is_false: version_conflicts
- is_false: batches
- is_false: failures
- is_false: noops
- is_false: took
- is_false: throttled_millis
- is_false: created
- is_false: deleted
- do:
tasks.get:
wait_for_completion: true
task_id: $task
- is_false: node_failures
# The task will be in the response even if it finished before we got here
# because of task persistence.
- is_true: task
- match: {response.total: 4}
- match: {response.created: 4}
- match: {response.updated: 0}
- match: {response.version_conflicts: 0}
- match: {response.failures: []}
- match: {response.throttled_millis: 0}
- gte: { response.took: 0 }
- is_false: response.task
- is_false: response.deleted
- match: {response.slices.0.updated: 0}
- match: {response.slices.0.version_conflicts: 0}
- match: {response.slices.0.throttled_millis: 0}
- match: {response.slices.1.updated: 0}
- match: {response.slices.1.version_conflicts: 0}
- match: {response.slices.1.throttled_millis: 0}
- match: {response.slices.2.updated: 0}
- match: {response.slices.2.version_conflicts: 0}
- match: {response.slices.2.throttled_millis: 0}
- match: {response.slices.3.updated: 0}
- match: {response.slices.3.version_conflicts: 0}
- match: {response.slices.3.throttled_millis: 0}
- match: {response.slices.4.updated: 0}
- match: {response.slices.4.version_conflicts: 0}
- match: {response.slices.4.throttled_millis: 0}
- match: {task.status.total: 4}
- match: {task.status.created: 4}
- match: {task.status.updated: 0}
- match: {task.status.version_conflicts: 0}
- match: {task.status.throttled_millis: 0}
- match: {task.status.slices.0.updated: 0}
- match: {task.status.slices.0.version_conflicts: 0}
- match: {task.status.slices.0.throttled_millis: 0}
- match: {task.status.slices.1.updated: 0}
- match: {task.status.slices.1.version_conflicts: 0}
- match: {task.status.slices.1.throttled_millis: 0}
- match: {task.status.slices.2.updated: 0}
- match: {task.status.slices.2.version_conflicts: 0}
- match: {task.status.slices.2.throttled_millis: 0}
- match: {task.status.slices.3.updated: 0}
- match: {task.status.slices.3.version_conflicts: 0}
- match: {task.status.slices.3.throttled_millis: 0}
- match: {task.status.slices.4.updated: 0}
- match: {task.status.slices.4.version_conflicts: 0}
- match: {task.status.slices.4.throttled_millis: 0}
# Only the "parent" reindex task wrote its status to the tasks index though
- do:
indices.refresh: {}
- do:
search:
index: .tasks
- match: { hits.total: 1 }
---
"Multiple slices with rethrottle":
- do:
index:
index: source
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 4
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 5
body: { "text": "test" }
- do:
index:
index: source
type: foo
id: 6
body: { "text": "test" }
- do:
indices.refresh: {}
# Start the task with a requests_per_second that should make it take a very long time
- do:
reindex:
slices: 2
wait_for_completion: false
requests_per_second: 0.0001
body:
source:
index: source
dest:
index: dest
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
- is_false: version_conflicts
- is_false: batches
- is_false: failures
- is_false: noops
- is_false: took
- is_false: throttled_millis
- is_false: created
- is_false: deleted
# Allow the task to complete
- do:
reindex_rethrottle:
requests_per_second: -1
task_id: $task
- do:
tasks.get:
wait_for_completion: true
task_id: $task
- is_false: node_failures
# The task will be in the response even if it finished before we got here
# because of task persistence.
- is_true: task
- match: {response.total: 6}
- match: {response.created: 6}
- match: {response.updated: 0}
- match: {response.version_conflicts: 0}
- match: {response.failures: []}
- match: {response.throttled_millis: 0}
- gte: { response.took: 0 }
- is_false: response.task
- is_false: response.deleted
- match: {response.slices.0.updated: 0}
- match: {response.slices.0.version_conflicts: 0}
- match: {response.slices.0.throttled_millis: 0}
- match: {response.slices.1.updated: 0}
- match: {response.slices.1.version_conflicts: 0}
- match: {response.slices.1.throttled_millis: 0}
- match: {task.status.total: 6}
- match: {task.status.created: 6}
- match: {task.status.updated: 0}
- match: {task.status.version_conflicts: 0}
- match: {task.status.throttled_millis: 0}
- match: {task.status.slices.0.updated: 0}
- match: {task.status.slices.0.version_conflicts: 0}
- match: {task.status.slices.0.throttled_millis: 0}
- match: {task.status.slices.1.updated: 0}
- match: {task.status.slices.1.version_conflicts: 0}
- match: {task.status.slices.1.throttled_millis: 0}
# Only the "parent" reindex task wrote its status to the tasks index though
- do:
indices.refresh: {}
- do:
search:
index: .tasks
- match: { hits.total: 1 }

View File

@ -104,3 +104,19 @@
update_by_query:
requests_per_second: 0
index: test
---
"junk in slices fails":
- do:
catch: /Failed to parse int parameter \[slices\] with value \[junk\]/
update_by_query:
slices: junk
index: test
---
"zero slices fails":
- do:
catch: /\[slices\] must be at least 1/
update_by_query:
slices: 0
index: test

View File

@ -0,0 +1,261 @@
---
"Multiple slices":
- do:
index:
index: test
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 4
body: { "text": "test" }
- do:
indices.refresh: {}
- do:
update_by_query:
index: test
slices: 5
body:
query:
match_all: {}
- is_false: timed_out
- match: {updated: 4}
- is_false: created
- match: {version_conflicts: 0}
- match: {failures: []}
- match: {noops: 0}
- match: {throttled_millis: 0}
- gte: { took: 0 }
- is_false: task
- match: {slices.0.version_conflicts: 0}
- match: {slices.0.throttled_millis: 0}
- match: {slices.1.version_conflicts: 0}
- match: {slices.1.throttled_millis: 0}
- match: {slices.2.version_conflicts: 0}
- match: {slices.2.throttled_millis: 0}
- match: {slices.3.version_conflicts: 0}
- match: {slices.3.throttled_millis: 0}
- match: {slices.4.version_conflicts: 0}
- match: {slices.4.throttled_millis: 0}
---
"Multiple slices with wait_for_completion=false":
- do:
index:
index: test
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 4
body: { "text": "test" }
- do:
indices.refresh: {}
- do:
update_by_query:
index: test
wait_for_completion: false
slices: 5
body:
query:
match_all: {}
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
- is_false: version_conflicts
- is_false: batches
- is_false: failures
- is_false: noops
- is_false: took
- is_false: throttled_millis
- is_false: created
- is_false: deleted
- do:
tasks.get:
wait_for_completion: true
task_id: $task
- is_false: node_failures
# The task will be in the response even if it finished before we got here
# because of task persistence.
- is_true: task
- match: {response.total: 4}
- match: {response.updated: 4}
- match: {response.version_conflicts: 0}
- match: {response.failures: []}
- match: {response.throttled_millis: 0}
- gte: { response.took: 0 }
- is_false: response.task
- match: {response.slices.0.version_conflicts: 0}
- match: {response.slices.0.throttled_millis: 0}
- match: {response.slices.1.version_conflicts: 0}
- match: {response.slices.1.throttled_millis: 0}
- match: {response.slices.2.version_conflicts: 0}
- match: {response.slices.2.throttled_millis: 0}
- match: {response.slices.3.version_conflicts: 0}
- match: {response.slices.3.throttled_millis: 0}
- match: {response.slices.4.version_conflicts: 0}
- match: {response.slices.4.throttled_millis: 0}
- match: {task.status.total: 4}
- match: {task.status.updated: 4}
- match: {task.status.version_conflicts: 0}
- match: {task.status.throttled_millis: 0}
- match: {task.status.slices.0.version_conflicts: 0}
- match: {task.status.slices.0.throttled_millis: 0}
- match: {task.status.slices.1.version_conflicts: 0}
- match: {task.status.slices.1.throttled_millis: 0}
- match: {task.status.slices.2.version_conflicts: 0}
- match: {task.status.slices.2.throttled_millis: 0}
- match: {task.status.slices.3.version_conflicts: 0}
- match: {task.status.slices.3.throttled_millis: 0}
- match: {task.status.slices.4.version_conflicts: 0}
- match: {task.status.slices.4.throttled_millis: 0}
# Only the "parent" reindex task wrote its status to the tasks index though
- do:
indices.refresh: {}
- do:
search:
index: .tasks
- match: { hits.total: 1 }
---
"Multiple slices with rethrottle":
- do:
index:
index: test
type: foo
id: 1
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 2
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 3
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 4
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 5
body: { "text": "test" }
- do:
index:
index: test
type: foo
id: 6
body: { "text": "test" }
- do:
indices.refresh: {}
# Start the task with a requests_per_second that should make it take a very long time
- do:
update_by_query:
index: test
wait_for_completion: false
requests_per_second: 0.0001
slices: 2
body:
query:
match_all: {}
- is_false: timed_out
- match: {task: '/.+:\d+/'}
- set: {task: task}
- is_false: updated
- is_false: version_conflicts
- is_false: batches
- is_false: failures
- is_false: noops
- is_false: took
- is_false: throttled_millis
- is_false: created
- is_false: deleted
# Allow the task to complete
- do:
reindex_rethrottle:
requests_per_second: -1
task_id: $task
- do:
tasks.get:
wait_for_completion: true
task_id: $task
- is_false: node_failures
# The task will be in the response even if it finished before we got here
# because of task persistence.
- is_true: task
- match: {response.total: 6}
- match: {response.updated: 6}
- match: {response.version_conflicts: 0}
- match: {response.failures: []}
- match: {response.throttled_millis: 0}
- gte: { response.took: 0 }
- is_false: response.task
- match: {response.slices.0.version_conflicts: 0}
- match: {response.slices.0.throttled_millis: 0}
- match: {response.slices.1.version_conflicts: 0}
- match: {response.slices.1.throttled_millis: 0}
- match: {task.status.total: 6}
- match: {task.status.updated: 6}
- match: {task.status.version_conflicts: 0}
- match: {task.status.throttled_millis: 0}
- match: {task.status.slices.0.version_conflicts: 0}
- match: {task.status.slices.0.throttled_millis: 0}
- match: {task.status.slices.1.version_conflicts: 0}
- match: {task.status.slices.1.throttled_millis: 0}
# Only the "parent" reindex task wrote its status to the tasks index though
- do:
indices.refresh: {}
- do:
search:
index: .tasks
- match: { hits.total: 1 }

View File

@ -152,7 +152,12 @@
"requests_per_second": {
"type": "number",
"default": 0,
"description": "The throttle to set on this request in sub-requests per second. -1 means set no throttle as does \"unlimited\" which is the only non-float this accepts."
"description": "The throttle for this request in sub-requests per second. -1 means no throttle."
},
"slices": {
"type": "integer",
"default": 1,
"description": "The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks."
}
}
},

View File

@ -28,7 +28,12 @@
"requests_per_second": {
"type": "number",
"default": 0,
"description": "The throttle to set on this request in sub-requests per second. -1 means set no throttle as does \"unlimited\" which is the only non-float this accepts."
"description": "The throttle to set on this request in sub-requests per second. -1 means no throttle."
},
"slices": {
"type": "integer",
"default": 1,
"description": "The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks."
}
}
},

View File

@ -160,7 +160,12 @@
"requests_per_second": {
"type": "number",
"default": 0,
"description": "The throttle to set on this request in sub-requests per second. -1 means set no throttle as does \"unlimited\" which is the only non-float this accepts."
"description": "The throttle to set on this request in sub-requests per second. -1 means no throttle."
},
"slices": {
"type": "integer",
"default": 1,
"description": "The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks."
}
}
},

View File

@ -0,0 +1,338 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.text.Text;
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 org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.rescore.RescoreBuilder;
import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.slice.SliceBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder;
import org.elasticsearch.search.sort.SortBuilders;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.search.suggest.SuggestBuilder;
import org.elasticsearch.test.AbstractQueryTestCase;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Supplier;
import static org.elasticsearch.test.ESTestCase.between;
import static org.elasticsearch.test.ESTestCase.generateRandomStringArray;
import static org.elasticsearch.test.ESTestCase.randomAsciiOfLengthBetween;
import static org.elasticsearch.test.ESTestCase.randomBoolean;
import static org.elasticsearch.test.ESTestCase.randomByte;
import static org.elasticsearch.test.ESTestCase.randomDouble;
import static org.elasticsearch.test.ESTestCase.randomFloat;
import static org.elasticsearch.test.ESTestCase.randomFrom;
import static org.elasticsearch.test.ESTestCase.randomInt;
import static org.elasticsearch.test.ESTestCase.randomIntBetween;
import static org.elasticsearch.test.ESTestCase.randomLong;
import static org.elasticsearch.test.ESTestCase.randomPositiveTimeValue;
import static org.elasticsearch.test.ESTestCase.randomShort;
import static org.elasticsearch.test.ESTestCase.randomTimeValue;
/**
* Builds random search requests.
*/
public class RandomSearchRequestGenerator {
private RandomSearchRequestGenerator() {}
/**
* Build a random search request.
*
* @param randomSearchSourceBuilder builds a random {@link SearchSourceBuilder}. You can use
* {@link #randomSearchSourceBuilder(Supplier, Supplier, Supplier, Supplier)}.
*/
public static SearchRequest randomSearchRequest(Supplier<SearchSourceBuilder> randomSearchSourceBuilder) throws IOException {
SearchRequest searchRequest = new SearchRequest();
if (randomBoolean()) {
searchRequest.indices(generateRandomStringArray(10, 10, false, false));
}
if (randomBoolean()) {
searchRequest.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean()));
}
if (randomBoolean()) {
searchRequest.types(generateRandomStringArray(10, 10, false, false));
}
if (randomBoolean()) {
searchRequest.preference(randomAsciiOfLengthBetween(3, 10));
}
if (randomBoolean()) {
searchRequest.requestCache(randomBoolean());
}
if (randomBoolean()) {
searchRequest.routing(randomAsciiOfLengthBetween(3, 10));
}
if (randomBoolean()) {
searchRequest.scroll(randomPositiveTimeValue());
}
if (randomBoolean()) {
searchRequest.searchType(randomFrom(SearchType.values()));
}
if (randomBoolean()) {
searchRequest.source(randomSearchSourceBuilder.get());
}
return searchRequest;
}
public static SearchSourceBuilder randomSearchSourceBuilder(
Supplier<HighlightBuilder> randomHighlightBuilder,
Supplier<SuggestBuilder> randomSuggestBuilder,
Supplier<RescoreBuilder<?>> randomRescoreBuilder,
Supplier<List<SearchExtBuilder>> randomExtBuilders) {
SearchSourceBuilder builder = new SearchSourceBuilder();
if (randomBoolean()) {
builder.from(randomIntBetween(0, 10000));
}
if (randomBoolean()) {
builder.size(randomIntBetween(0, 10000));
}
if (randomBoolean()) {
builder.explain(randomBoolean());
}
if (randomBoolean()) {
builder.version(randomBoolean());
}
if (randomBoolean()) {
builder.trackScores(randomBoolean());
}
if (randomBoolean()) {
builder.minScore(randomFloat() * 1000);
}
if (randomBoolean()) {
builder.timeout(TimeValue.parseTimeValue(randomTimeValue(), null, "timeout"));
}
if (randomBoolean()) {
builder.terminateAfter(randomIntBetween(1, 100000));
}
switch(randomInt(2)) {
case 0:
builder.storedFields();
break;
case 1:
builder.storedField("_none_");
break;
case 2:
int fieldsSize = randomInt(25);
List<String> fields = new ArrayList<>(fieldsSize);
for (int i = 0; i < fieldsSize; i++) {
fields.add(randomAsciiOfLengthBetween(5, 50));
}
builder.storedFields(fields);
break;
default:
throw new IllegalStateException();
}
if (randomBoolean()) {
int scriptFieldsSize = randomInt(25);
for (int i = 0; i < scriptFieldsSize; i++) {
if (randomBoolean()) {
builder.scriptField(randomAsciiOfLengthBetween(5, 50), new Script("foo"), randomBoolean());
} else {
builder.scriptField(randomAsciiOfLengthBetween(5, 50), new Script("foo"));
}
}
}
if (randomBoolean()) {
FetchSourceContext fetchSourceContext;
int branch = randomInt(5);
String[] includes = new String[randomIntBetween(0, 20)];
for (int i = 0; i < includes.length; i++) {
includes[i] = randomAsciiOfLengthBetween(5, 20);
}
String[] excludes = new String[randomIntBetween(0, 20)];
for (int i = 0; i < excludes.length; i++) {
excludes[i] = randomAsciiOfLengthBetween(5, 20);
}
switch (branch) {
case 0:
fetchSourceContext = new FetchSourceContext(randomBoolean());
break;
case 1:
fetchSourceContext = new FetchSourceContext(true, includes, excludes);
break;
case 2:
fetchSourceContext = new FetchSourceContext(true, new String[]{randomAsciiOfLengthBetween(5, 20)},
new String[]{randomAsciiOfLengthBetween(5, 20)});
break;
case 3:
fetchSourceContext = new FetchSourceContext(true, includes, excludes);
break;
case 4:
fetchSourceContext = new FetchSourceContext(true, includes, null);
break;
case 5:
fetchSourceContext = new FetchSourceContext(true, new String[] {randomAsciiOfLengthBetween(5, 20)}, null);
break;
default:
throw new IllegalStateException();
}
builder.fetchSource(fetchSourceContext);
}
if (randomBoolean()) {
int size = randomIntBetween(0, 20);
List<String> statsGroups = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
statsGroups.add(randomAsciiOfLengthBetween(5, 20));
}
builder.stats(statsGroups);
}
if (randomBoolean()) {
int indexBoostSize = randomIntBetween(1, 10);
for (int i = 0; i < indexBoostSize; i++) {
builder.indexBoost(randomAsciiOfLengthBetween(5, 20), randomFloat() * 10);
}
}
if (randomBoolean()) {
builder.query(QueryBuilders.termQuery(randomAsciiOfLengthBetween(5, 20), randomAsciiOfLengthBetween(5, 20)));
}
if (randomBoolean()) {
builder.postFilter(QueryBuilders.termQuery(randomAsciiOfLengthBetween(5, 20), randomAsciiOfLengthBetween(5, 20)));
}
if (randomBoolean()) {
int numSorts = randomIntBetween(1, 5);
for (int i = 0; i < numSorts; i++) {
int branch = randomInt(5);
switch (branch) {
case 0:
builder.sort(SortBuilders.fieldSort(randomAsciiOfLengthBetween(5, 20)).order(randomFrom(SortOrder.values())));
break;
case 1:
builder.sort(SortBuilders.geoDistanceSort(randomAsciiOfLengthBetween(5, 20),
AbstractQueryTestCase.randomGeohash(1, 12)).order(randomFrom(SortOrder.values())));
break;
case 2:
builder.sort(SortBuilders.scoreSort().order(randomFrom(SortOrder.values())));
break;
case 3:
builder.sort(SortBuilders.scriptSort(new Script("foo"),
ScriptSortBuilder.ScriptSortType.NUMBER).order(randomFrom(SortOrder.values())));
break;
case 4:
builder.sort(randomAsciiOfLengthBetween(5, 20));
break;
case 5:
builder.sort(randomAsciiOfLengthBetween(5, 20), randomFrom(SortOrder.values()));
break;
}
}
}
if (randomBoolean()) {
int numSearchFrom = randomIntBetween(1, 5);
try {
// We build a json version of the search_from first in order to
// ensure that every number type remain the same before/after xcontent (de)serialization.
// This is not a problem because the final type of each field value is extracted from associated sort field.
// This little trick ensure that equals and hashcode are the same when using the xcontent serialization.
XContentBuilder jsonBuilder = XContentFactory.jsonBuilder();
jsonBuilder.startObject();
jsonBuilder.startArray("search_from");
for (int i = 0; i < numSearchFrom; i++) {
int branch = randomInt(8);
switch (branch) {
case 0:
jsonBuilder.value(randomInt());
break;
case 1:
jsonBuilder.value(randomFloat());
break;
case 2:
jsonBuilder.value(randomLong());
break;
case 3:
jsonBuilder.value(randomDouble());
break;
case 4:
jsonBuilder.value(randomAsciiOfLengthBetween(5, 20));
break;
case 5:
jsonBuilder.value(randomBoolean());
break;
case 6:
jsonBuilder.value(randomByte());
break;
case 7:
jsonBuilder.value(randomShort());
break;
case 8:
jsonBuilder.value(new Text(randomAsciiOfLengthBetween(5, 20)));
break;
}
}
jsonBuilder.endArray();
jsonBuilder.endObject();
XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(jsonBuilder.bytes());
parser.nextToken();
parser.nextToken();
parser.nextToken();
builder.searchAfter(SearchAfterBuilder.fromXContent(parser, null).getSortValues());
} catch (IOException e) {
throw new RuntimeException("Error building search_from", e);
}
}
if (randomBoolean()) {
builder.highlighter(randomHighlightBuilder.get());
}
if (randomBoolean()) {
builder.suggest(randomSuggestBuilder.get());
}
if (randomBoolean()) {
int numRescores = randomIntBetween(1, 5);
for (int i = 0; i < numRescores; i++) {
builder.addRescorer(randomRescoreBuilder.get());
}
}
if (randomBoolean()) {
builder.aggregation(AggregationBuilders.avg(randomAsciiOfLengthBetween(5, 20)));
}
if (randomBoolean()) {
builder.ext(randomExtBuilders.get());
}
if (randomBoolean()) {
String field = randomBoolean() ? null : randomAsciiOfLengthBetween(5, 20);
int max = between(2, 1000);
int id = randomInt(max-1);
if (field == null) {
builder.slice(new SliceBuilder(id, max));
} else {
builder.slice(new SliceBuilder(field, id, max));
}
}
return builder;
}
}

View File

@ -24,6 +24,7 @@ import com.carrotsearch.randomizedtesting.SysGlobals;
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.carrotsearch.randomizedtesting.generators.RandomStrings;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.store.StoreRateLimiting;
import org.apache.lucene.util.IOUtils;
@ -1066,7 +1067,7 @@ public final class InternalTestCluster extends TestCluster {
DiscoveryNode localNode = getInstance(ClusterService.class, nodeAndClient.name).localNode();
List<TaskInfo> taskInfos = taskManager.getTasks().values().stream()
.filter(task -> task instanceof ReplicationTask)
.map(task -> task.taskInfo(localNode, true))
.map(task -> task.taskInfo(localNode.getId(), true))
.collect(Collectors.toList());
ListTasksResponse response = new ListTasksResponse(taskInfos, Collections.emptyList(), Collections.emptyList());
XContentBuilder builder = XContentFactory.jsonBuilder()