reindex: automatically choose the number of slices (#26030)
In reindex APIs, when using the `slices` parameter to choose the number of slices, adds the option to specify `slices` as "auto" which will choose a reasonable number of slices. It uses the number of shards in the source index, up to a ceiling. If there is more than one source index, it uses the smallest number of shards among them. This gives users an easy way to use slicing in these APIs without having to make decisions about how to configure it, as it provides a good-enough configuration for them out of the box. This may become the default behavior for these APIs in the future.
This commit is contained in:
parent
73e936a065
commit
7e3cd6a019
|
@ -45,6 +45,10 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
private static final TimeValue DEFAULT_SCROLL_TIMEOUT = timeValueMinutes(5);
|
private static final TimeValue DEFAULT_SCROLL_TIMEOUT = timeValueMinutes(5);
|
||||||
private static final int DEFAULT_SCROLL_SIZE = 1000;
|
private static final int DEFAULT_SCROLL_SIZE = 1000;
|
||||||
|
|
||||||
|
public static final int AUTO_SLICES = 0;
|
||||||
|
public static final String AUTO_SLICES_VALUE = "auto";
|
||||||
|
private static final int DEFAULT_SLICES = 1;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The search to be executed.
|
* The search to be executed.
|
||||||
*/
|
*/
|
||||||
|
@ -102,7 +106,7 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
/**
|
/**
|
||||||
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
|
* 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;
|
private int slices = DEFAULT_SLICES;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor for deserialization.
|
* Constructor for deserialization.
|
||||||
|
@ -152,8 +156,8 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
+ size + "]",
|
+ size + "]",
|
||||||
e);
|
e);
|
||||||
}
|
}
|
||||||
if (searchRequest.source().slice() != null && slices != 1) {
|
if (searchRequest.source().slice() != null && slices != DEFAULT_SLICES) {
|
||||||
e = addValidationError("can't specify both slice and workers", e);
|
e = addValidationError("can't specify both manual and automatic slicing at the same time", e);
|
||||||
}
|
}
|
||||||
return e;
|
return e;
|
||||||
}
|
}
|
||||||
|
@ -341,8 +345,8 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
|
* 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) {
|
public Self setSlices(int slices) {
|
||||||
if (slices < 1) {
|
if (slices < 0) {
|
||||||
throw new IllegalArgumentException("[slices] must be at least 1");
|
throw new IllegalArgumentException("[slices] must be at least 0 but was [" + slices + "]");
|
||||||
}
|
}
|
||||||
this.slices = slices;
|
this.slices = slices;
|
||||||
return self();
|
return self();
|
||||||
|
@ -358,24 +362,28 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
/**
|
/**
|
||||||
* Build a new request for a slice of the parent request.
|
* Build a new request for a slice of the parent request.
|
||||||
*/
|
*/
|
||||||
public abstract Self forSlice(TaskId slicingTask, SearchRequest slice);
|
public abstract Self forSlice(TaskId slicingTask, SearchRequest slice, int totalSlices);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Setup a clone of this request with the information needed to process a slice of it.
|
* Setup a clone of this request with the information needed to process a slice of it.
|
||||||
*/
|
*/
|
||||||
protected Self doForSlice(Self request, TaskId slicingTask) {
|
protected Self doForSlice(Self request, TaskId slicingTask, int totalSlices) {
|
||||||
|
if (totalSlices < 1) {
|
||||||
|
throw new IllegalArgumentException("Number of total slices must be at least 1 but was [" + totalSlices + "]");
|
||||||
|
}
|
||||||
|
|
||||||
request.setAbortOnVersionConflict(abortOnVersionConflict).setRefresh(refresh).setTimeout(timeout)
|
request.setAbortOnVersionConflict(abortOnVersionConflict).setRefresh(refresh).setTimeout(timeout)
|
||||||
.setWaitForActiveShards(activeShardCount).setRetryBackoffInitialTime(retryBackoffInitialTime).setMaxRetries(maxRetries)
|
.setWaitForActiveShards(activeShardCount).setRetryBackoffInitialTime(retryBackoffInitialTime).setMaxRetries(maxRetries)
|
||||||
// Parent task will store result
|
// Parent task will store result
|
||||||
.setShouldStoreResult(false)
|
.setShouldStoreResult(false)
|
||||||
// Split requests per second between all slices
|
// Split requests per second between all slices
|
||||||
.setRequestsPerSecond(requestsPerSecond / slices)
|
.setRequestsPerSecond(requestsPerSecond / totalSlices)
|
||||||
// Sub requests don't have workers
|
// Sub requests don't have workers
|
||||||
.setSlices(1);
|
.setSlices(1);
|
||||||
if (size != -1) {
|
if (size != -1) {
|
||||||
// Size is split between workers. This means the size might round
|
// Size is split between workers. This means the size might round
|
||||||
// down!
|
// down!
|
||||||
request.setSize(size == SIZE_ALL_MATCHES ? SIZE_ALL_MATCHES : size / slices);
|
request.setSize(size == SIZE_ALL_MATCHES ? SIZE_ALL_MATCHES : size / totalSlices);
|
||||||
}
|
}
|
||||||
// Set the parent task so this task is cancelled if we cancel the parent
|
// Set the parent task so this task is cancelled if we cancel the parent
|
||||||
request.setParentTask(slicingTask);
|
request.setParentTask(slicingTask);
|
||||||
|
@ -385,14 +393,7 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
|
public Task createTask(long id, String type, String action, TaskId parentTaskId) {
|
||||||
if (slices > 1) {
|
return new BulkByScrollTask(id, type, action, getDescription(), parentTaskId);
|
||||||
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
|
@Override
|
||||||
|
@ -408,11 +409,7 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
retryBackoffInitialTime = new TimeValue(in);
|
retryBackoffInitialTime = new TimeValue(in);
|
||||||
maxRetries = in.readVInt();
|
maxRetries = in.readVInt();
|
||||||
requestsPerSecond = in.readFloat();
|
requestsPerSecond = in.readFloat();
|
||||||
if (in.getVersion().onOrAfter(Version.V_5_1_1)) {
|
slices = in.readVInt();
|
||||||
slices = in.readVInt();
|
|
||||||
} else {
|
|
||||||
slices = 1;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -427,13 +424,11 @@ public abstract class AbstractBulkByScrollRequest<Self extends AbstractBulkByScr
|
||||||
retryBackoffInitialTime.writeTo(out);
|
retryBackoffInitialTime.writeTo(out);
|
||||||
out.writeVInt(maxRetries);
|
out.writeVInt(maxRetries);
|
||||||
out.writeFloat(requestsPerSecond);
|
out.writeFloat(requestsPerSecond);
|
||||||
if (out.getVersion().onOrAfter(Version.V_5_1_1)) {
|
if (out.getVersion().before(Version.V_6_1_0) && slices == AUTO_SLICES) {
|
||||||
out.writeVInt(slices);
|
throw new IllegalArgumentException("Slices set as \"auto\" are not supported before version [" + Version.V_6_1_0 + "]. " +
|
||||||
|
"Found version [" + out.getVersion() + "]");
|
||||||
} else {
|
} else {
|
||||||
if (slices > 1) {
|
out.writeVInt(slices);
|
||||||
throw new IllegalArgumentException("Attempting to send sliced reindex-style request to a node that doesn't support "
|
|
||||||
+ "it. Version is [" + out.getVersion() + "] but must be [" + Version.V_5_1_1 + "]");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -145,8 +145,8 @@ public abstract class AbstractBulkByScrollRequestBuilder<
|
||||||
/**
|
/**
|
||||||
* The number of slices this task should be divided into. Defaults to 1 meaning the task isn't sliced into subtasks.
|
* 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) {
|
public Self setSlices(int slices) {
|
||||||
request.setSlices(workers);
|
request.setSlices(slices);
|
||||||
return self();
|
return self();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -68,8 +68,8 @@ public abstract class AbstractBulkIndexByScrollRequest<Self extends AbstractBulk
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Self doForSlice(Self request, TaskId slicingTask) {
|
protected Self doForSlice(Self request, TaskId slicingTask, int totalSlices) {
|
||||||
return super.doForSlice(request, slicingTask).setScript(script);
|
return super.doForSlice(request, slicingTask, totalSlices).setScript(script);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -190,7 +190,7 @@ public class BulkByScrollResponse extends ActionResponse implements ToXContentFr
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
StringBuilder builder = new StringBuilder();
|
StringBuilder builder = new StringBuilder();
|
||||||
builder.append("BulkIndexByScrollResponse[");
|
builder.append(getClass().getSimpleName()).append("[");
|
||||||
builder.append("took=").append(took).append(',');
|
builder.append("took=").append(took).append(',');
|
||||||
builder.append("timed_out=").append(timedOut).append(',');
|
builder.append("timed_out=").append(timedOut).append(',');
|
||||||
status.innerToString(builder);
|
status.innerToString(builder);
|
||||||
|
|
|
@ -34,6 +34,8 @@ import org.elasticsearch.tasks.TaskId;
|
||||||
import org.elasticsearch.tasks.TaskInfo;
|
import org.elasticsearch.tasks.TaskInfo;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
|
||||||
|
@ -43,32 +45,138 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Task storing information about a currently running BulkByScroll request.
|
* Task storing information about a currently running BulkByScroll request.
|
||||||
|
*
|
||||||
|
* When the request is not sliced, this task is the only task created, and starts an action to perform search requests.
|
||||||
|
*
|
||||||
|
* When the request is sliced, this task can either represent a coordinating task (using
|
||||||
|
* {@link BulkByScrollTask#setWorkerCount(int)}) or a worker task that performs search queries (using
|
||||||
|
* {@link BulkByScrollTask#setWorker(float, Integer)}).
|
||||||
|
*
|
||||||
|
* We don't always know if this task will be a leader or worker task when it's created, because if slices is set to "auto" it may
|
||||||
|
* be either depending on the number of shards in the source indices. We figure that out when the request is handled and set it on this
|
||||||
|
* class with {@link #setWorkerCount(int)} or {@link #setWorker(float, Integer)}.
|
||||||
*/
|
*/
|
||||||
public abstract class BulkByScrollTask extends CancellableTask {
|
public class BulkByScrollTask extends CancellableTask {
|
||||||
|
|
||||||
|
private LeaderBulkByScrollTaskState leaderState;
|
||||||
|
private WorkerBulkByScrollTaskState workerState;
|
||||||
|
|
||||||
public BulkByScrollTask(long id, String type, String action, String description, TaskId parentTaskId) {
|
public BulkByScrollTask(long id, String type, String action, String description, TaskId parentTaskId) {
|
||||||
super(id, type, action, description, parentTaskId);
|
super(id, type, action, description, parentTaskId);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* The number of sub-slices that are still running. {@link WorkingBulkByScrollTask} will always have 0 and
|
public BulkByScrollTask.Status getStatus() {
|
||||||
* {@link ParentBulkByScrollTask} will return the number of waiting tasks. Used to decide how to perform rethrottling.
|
if (isLeader()) {
|
||||||
*/
|
return leaderState.getStatus();
|
||||||
public abstract int runningSliceSubTasks();
|
}
|
||||||
|
|
||||||
|
if (isWorker()) {
|
||||||
|
return workerState.getStatus();
|
||||||
|
}
|
||||||
|
|
||||||
|
return emptyStatus();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Apply the {@code newRequestsPerSecond}.
|
* Build the status for this task given a snapshot of the information of running slices. This is only supported if the task is
|
||||||
|
* set as a leader for slice subtasks
|
||||||
*/
|
*/
|
||||||
public abstract void rethrottle(float newRequestsPerSecond);
|
public TaskInfo taskInfoGivenSubtaskInfo(String localNodeId, List<TaskInfo> sliceInfo) {
|
||||||
|
if (isLeader() == false) {
|
||||||
|
throw new IllegalStateException("This task is not set to be a leader of other slice subtasks");
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
List<BulkByScrollTask.StatusOrException> sliceStatuses = Arrays.asList(
|
||||||
* Overridden to force children to return compatible status.
|
new BulkByScrollTask.StatusOrException[leaderState.getSlices()]);
|
||||||
*/
|
for (TaskInfo t : sliceInfo) {
|
||||||
public abstract BulkByScrollTask.Status getStatus();
|
BulkByScrollTask.Status status = (BulkByScrollTask.Status) t.getStatus();
|
||||||
|
sliceStatuses.set(status.getSliceId(), new BulkByScrollTask.StatusOrException(status));
|
||||||
|
}
|
||||||
|
Status status = leaderState.getStatus(sliceStatuses);
|
||||||
|
return taskInfo(localNodeId, getDescription(), status);
|
||||||
|
}
|
||||||
|
|
||||||
|
private BulkByScrollTask.Status emptyStatus() {
|
||||||
|
return new Status(Collections.emptyList(), getReasonCancelled());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Build the status for this task given a snapshot of the information of running slices.
|
* Returns true if this task is a leader for other slice subtasks
|
||||||
*/
|
*/
|
||||||
public abstract TaskInfo getInfoGivenSliceInfo(String localNodeId, List<TaskInfo> sliceInfo);
|
public boolean isLeader() {
|
||||||
|
return leaderState != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets this task to be a leader task for {@code slices} sliced subtasks
|
||||||
|
*/
|
||||||
|
public void setWorkerCount(int slices) {
|
||||||
|
if (isLeader()) {
|
||||||
|
throw new IllegalStateException("This task is already a leader for other slice subtasks");
|
||||||
|
}
|
||||||
|
if (isWorker()) {
|
||||||
|
throw new IllegalStateException("This task is already a worker");
|
||||||
|
}
|
||||||
|
|
||||||
|
leaderState = new LeaderBulkByScrollTaskState(this, slices);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the object that tracks the state of sliced subtasks. Throws IllegalStateException if this task is not set to be
|
||||||
|
* a leader task.
|
||||||
|
*/
|
||||||
|
public LeaderBulkByScrollTaskState getLeaderState() {
|
||||||
|
if (!isLeader()) {
|
||||||
|
throw new IllegalStateException("This task is not set to be a leader for other slice subtasks");
|
||||||
|
}
|
||||||
|
return leaderState;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if this task is a worker task that performs search requests. False otherwise
|
||||||
|
*/
|
||||||
|
public boolean isWorker() {
|
||||||
|
return workerState != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets this task to be a worker task that performs search requests
|
||||||
|
* @param requestsPerSecond How many search requests per second this task should make
|
||||||
|
* @param sliceId If this is is a sliced task, which slice number this task corresponds to. Null if not sliced.
|
||||||
|
*/
|
||||||
|
public void setWorker(float requestsPerSecond, @Nullable Integer sliceId) {
|
||||||
|
if (isWorker()) {
|
||||||
|
throw new IllegalStateException("This task is already a worker");
|
||||||
|
}
|
||||||
|
if (isLeader()) {
|
||||||
|
throw new IllegalStateException("This task is already a leader for other slice subtasks");
|
||||||
|
}
|
||||||
|
|
||||||
|
workerState = new WorkerBulkByScrollTaskState(this, sliceId, requestsPerSecond);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the object that manages sending search requests. Throws IllegalStateException if this task is not set to be a
|
||||||
|
* worker task.
|
||||||
|
*/
|
||||||
|
public WorkerBulkByScrollTaskState getWorkerState() {
|
||||||
|
if (!isWorker()) {
|
||||||
|
throw new IllegalStateException("This task is not set to be a worker");
|
||||||
|
}
|
||||||
|
return workerState;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onCancelled() {
|
||||||
|
if (isLeader()) {
|
||||||
|
// The task cancellation task automatically finds children and cancels them, nothing extra to do
|
||||||
|
} else if (isWorker()) {
|
||||||
|
workerState.handleCancel();
|
||||||
|
} else {
|
||||||
|
throw new IllegalStateException("This task has not had its sliced state initialized and doesn't know how to cancel itself");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean shouldCancelChildrenOnCancellation() {
|
public boolean shouldCancelChildrenOnCancellation() {
|
||||||
|
|
|
@ -81,8 +81,8 @@ public class DeleteByQueryRequest extends AbstractBulkByScrollRequest<DeleteByQu
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DeleteByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
public DeleteByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice, int totalSlices) {
|
||||||
return doForSlice(new DeleteByQueryRequest(slice, false), slicingTask);
|
return doForSlice(new DeleteByQueryRequest(slice, false), slicingTask, totalSlices);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -20,9 +20,7 @@
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.common.collect.Tuple;
|
|
||||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||||
import org.elasticsearch.tasks.TaskId;
|
|
||||||
import org.elasticsearch.tasks.TaskInfo;
|
import org.elasticsearch.tasks.TaskInfo;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -33,60 +31,68 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import static java.util.Collections.unmodifiableList;
|
import static java.util.Collections.unmodifiableList;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Task for parent bulk by scroll requests that have sub-workers.
|
* Tracks the state of sliced subtasks and provides unified status information for a sliced BulkByScrollRequest.
|
||||||
*/
|
*/
|
||||||
public class ParentBulkByScrollTask extends BulkByScrollTask {
|
public class LeaderBulkByScrollTaskState {
|
||||||
|
|
||||||
|
private final BulkByScrollTask task;
|
||||||
|
|
||||||
|
private final int slices;
|
||||||
/**
|
/**
|
||||||
* Holds the responses as they come back. This uses {@link Tuple} as an "Either" style holder where only the response or the exception
|
* Holds the responses of slice workers as they come in
|
||||||
* is set.
|
|
||||||
*/
|
*/
|
||||||
private final AtomicArray<Result> results;
|
private final AtomicArray<Result> results;
|
||||||
private final AtomicInteger counter;
|
/**
|
||||||
|
* How many subtasks are still running
|
||||||
|
*/
|
||||||
|
private final AtomicInteger runningSubtasks;
|
||||||
|
|
||||||
public ParentBulkByScrollTask(long id, String type, String action, String description, TaskId parentTaskId, int slices) {
|
public LeaderBulkByScrollTaskState(BulkByScrollTask task, int slices) {
|
||||||
super(id, type, action, description, parentTaskId);
|
this.task = task;
|
||||||
this.results = new AtomicArray<>(slices);
|
this.slices = slices;
|
||||||
this.counter = new AtomicInteger(slices);
|
results = new AtomicArray<>(slices);
|
||||||
|
runningSubtasks = new AtomicInteger(slices);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
public void rethrottle(float newRequestsPerSecond) {
|
* Returns the number of slices this BulkByScrollRequest will use
|
||||||
// Nothing to do because all rethrottling is done on slice sub tasks.
|
*/
|
||||||
|
public int getSlices() {
|
||||||
|
return slices;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
public Status getStatus() {
|
* Get the combined statuses of slice subtasks, merged with the given list of statuses
|
||||||
|
*/
|
||||||
|
public BulkByScrollTask.Status getStatus(List<BulkByScrollTask.StatusOrException> statuses) {
|
||||||
// We only have access to the statuses of requests that have finished so we return them
|
// 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()]);
|
if (statuses.size() != results.length()) {
|
||||||
addResultsToList(statuses);
|
throw new IllegalArgumentException("Given number of statuses does not match amount of expected results");
|
||||||
return new Status(unmodifiableList(statuses), getReasonCancelled());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public 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);
|
addResultsToList(statuses);
|
||||||
Status status = new Status(sliceStatuses, getReasonCancelled());
|
return new BulkByScrollTask.Status(unmodifiableList(statuses), task.getReasonCancelled());
|
||||||
return taskInfo(localNodeId, getDescription(), status);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addResultsToList(List<StatusOrException> sliceStatuses) {
|
/**
|
||||||
|
* Get the combined statuses of sliced subtasks
|
||||||
|
*/
|
||||||
|
public BulkByScrollTask.Status getStatus() {
|
||||||
|
return getStatus(Arrays.asList(new BulkByScrollTask.StatusOrException[results.length()]));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The number of sliced subtasks that are still running
|
||||||
|
*/
|
||||||
|
public int runningSliceSubTasks() {
|
||||||
|
return runningSubtasks.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void addResultsToList(List<BulkByScrollTask.StatusOrException> sliceStatuses) {
|
||||||
for (Result t : results.asList()) {
|
for (Result t : results.asList()) {
|
||||||
if (t.response != null) {
|
if (t.response != null) {
|
||||||
sliceStatuses.set(t.sliceId, new StatusOrException(t.response.getStatus()));
|
sliceStatuses.set(t.sliceId, new BulkByScrollTask.StatusOrException(t.response.getStatus()));
|
||||||
} else {
|
} else {
|
||||||
sliceStatuses.set(t.sliceId, new StatusOrException(t.failure));
|
sliceStatuses.set(t.sliceId, new BulkByScrollTask.StatusOrException(t.failure));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -111,7 +117,7 @@ public class ParentBulkByScrollTask extends BulkByScrollTask {
|
||||||
}
|
}
|
||||||
|
|
||||||
private void recordSliceCompletionAndRespondIfAllDone(ActionListener<BulkByScrollResponse> listener) {
|
private void recordSliceCompletionAndRespondIfAllDone(ActionListener<BulkByScrollResponse> listener) {
|
||||||
if (counter.decrementAndGet() != 0) {
|
if (runningSubtasks.decrementAndGet() != 0) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
List<BulkByScrollResponse> responses = new ArrayList<>(results.length());
|
List<BulkByScrollResponse> responses = new ArrayList<>(results.length());
|
||||||
|
@ -130,7 +136,7 @@ public class ParentBulkByScrollTask extends BulkByScrollTask {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (exception == null) {
|
if (exception == null) {
|
||||||
listener.onResponse(new BulkByScrollResponse(responses, getReasonCancelled()));
|
listener.onResponse(new BulkByScrollResponse(responses, task.getReasonCancelled()));
|
||||||
} else {
|
} else {
|
||||||
listener.onFailure(exception);
|
listener.onFailure(exception);
|
||||||
}
|
}
|
|
@ -94,8 +94,8 @@ public class ReindexRequest extends AbstractBulkIndexByScrollRequest<ReindexRequ
|
||||||
if (getSearchRequest().source().query() != null) {
|
if (getSearchRequest().source().query() != null) {
|
||||||
e = addValidationError("reindex from remote sources should use RemoteInfo's query instead of source's query", e);
|
e = addValidationError("reindex from remote sources should use RemoteInfo's query instead of source's query", e);
|
||||||
}
|
}
|
||||||
if (getSlices() != 1) {
|
if (getSlices() == AbstractBulkByScrollRequest.AUTO_SLICES || getSlices() > 1) {
|
||||||
e = addValidationError("reindex from remote sources doesn't support workers > 1 but was [" + getSlices() + "]", e);
|
e = addValidationError("reindex from remote sources doesn't support slices > 1 but was [" + getSlices() + "]", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return e;
|
return e;
|
||||||
|
@ -127,8 +127,8 @@ public class ReindexRequest extends AbstractBulkIndexByScrollRequest<ReindexRequ
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ReindexRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
public ReindexRequest forSlice(TaskId slicingTask, SearchRequest slice, int totalSlices) {
|
||||||
ReindexRequest sliced = doForSlice(new ReindexRequest(slice, destination, false), slicingTask);
|
ReindexRequest sliced = doForSlice(new ReindexRequest(slice, destination, false), slicingTask, totalSlices);
|
||||||
sliced.setRemoteInfo(remoteInfo);
|
sliced.setRemoteInfo(remoteInfo);
|
||||||
return sliced;
|
return sliced;
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,7 +20,7 @@
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implemented by {@link BulkByScrollTask} and {@link BulkByScrollTask.Status} to consistently implement
|
* Implemented by {@link WorkerBulkByScrollTaskState} and {@link BulkByScrollTask.Status} to consistently implement
|
||||||
* {@link #getSuccessfullyProcessed()}.
|
* {@link #getSuccessfullyProcessed()}.
|
||||||
*/
|
*/
|
||||||
public interface SuccessfullyProcessed {
|
public interface SuccessfullyProcessed {
|
||||||
|
|
|
@ -71,8 +71,8 @@ public class UpdateByQueryRequest extends AbstractBulkIndexByScrollRequest<Updat
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public UpdateByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
public UpdateByQueryRequest forSlice(TaskId slicingTask, SearchRequest slice, int totalSlices) {
|
||||||
UpdateByQueryRequest request = doForSlice(new UpdateByQueryRequest(slice, false), slicingTask);
|
UpdateByQueryRequest request = doForSlice(new UpdateByQueryRequest(slice, false), slicingTask, totalSlices);
|
||||||
request.setPipeline(pipeline);
|
request.setPipeline(pipeline);
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
|
@ -20,15 +20,12 @@
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||||
import org.elasticsearch.tasks.TaskId;
|
|
||||||
import org.elasticsearch.tasks.TaskInfo;
|
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ScheduledFuture;
|
import java.util.concurrent.ScheduledFuture;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
@ -41,15 +38,19 @@ import static java.lang.Math.round;
|
||||||
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
|
import static org.elasticsearch.common.unit.TimeValue.timeValueNanos;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* {@link BulkByScrollTask} subclass for tasks that actually perform the work. Compare to {@link ParentBulkByScrollTask}.
|
* Task behavior for {@link BulkByScrollTask} that does the actual work of querying and indexing
|
||||||
*/
|
*/
|
||||||
public class WorkingBulkByScrollTask extends BulkByScrollTask implements SuccessfullyProcessed {
|
public class WorkerBulkByScrollTaskState implements SuccessfullyProcessed {
|
||||||
private static final Logger logger = ESLoggerFactory.getLogger(BulkByScrollTask.class.getPackage().getName());
|
|
||||||
|
private static final Logger logger = Loggers.getLogger(WorkerBulkByScrollTaskState.class);
|
||||||
|
|
||||||
|
private final BulkByScrollTask task;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The id of the slice that this task is processing or {@code null} if this task isn't for a sliced request.
|
* The id of the slice that this worker is processing or {@code null} if this task isn't for a sliced request.
|
||||||
*/
|
*/
|
||||||
private final Integer sliceId;
|
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
|
* 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.
|
* to process. Its ok that these have the same meaning because any request with 0 actual documents should be quite short lived.
|
||||||
|
@ -64,58 +65,47 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
private final AtomicLong bulkRetries = new AtomicLong(0);
|
private final AtomicLong bulkRetries = new AtomicLong(0);
|
||||||
private final AtomicLong searchRetries = new AtomicLong(0);
|
private final AtomicLong searchRetries = new AtomicLong(0);
|
||||||
private final AtomicLong throttledNanos = new AtomicLong();
|
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
|
* 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.
|
* style variables but there isn't an AtomicFloat so we just use a volatile.
|
||||||
*/
|
*/
|
||||||
private volatile float requestsPerSecond;
|
private volatile float requestsPerSecond;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reference to any the last delayed prepareBulkRequest call. Used during rethrottling and canceling to reschedule the request.
|
* Reference to any the last delayed prepareBulkRequest call. Used during rethrottling and canceling to reschedule the request.
|
||||||
*/
|
*/
|
||||||
private final AtomicReference<DelayedPrepareBulkRequest> delayedPrepareBulkRequestReference = new AtomicReference<>();
|
private final AtomicReference<DelayedPrepareBulkRequest> delayedPrepareBulkRequestReference = new AtomicReference<>();
|
||||||
|
|
||||||
public WorkingBulkByScrollTask(long id, String type, String action, String description, TaskId parentTask, Integer sliceId,
|
public WorkerBulkByScrollTaskState(BulkByScrollTask task, Integer sliceId, float requestsPerSecond) {
|
||||||
float requestsPerSecond) {
|
this.task = task;
|
||||||
super(id, type, action, description, parentTask);
|
|
||||||
this.sliceId = sliceId;
|
this.sliceId = sliceId;
|
||||||
setRequestsPerSecond(requestsPerSecond);
|
setRequestsPerSecond(requestsPerSecond);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public BulkByScrollTask.Status getStatus() {
|
||||||
public Status getStatus() {
|
return new BulkByScrollTask.Status(
|
||||||
return new Status(sliceId, total.get(), updated.get(), created.get(), deleted.get(), batch.get(), versionConflicts.get(),
|
sliceId,
|
||||||
noops.get(), bulkRetries.get(), searchRetries.get(), timeValueNanos(throttledNanos.get()), getRequestsPerSecond(),
|
total.get(),
|
||||||
getReasonCancelled(), throttledUntil());
|
updated.get(),
|
||||||
|
created.get(),
|
||||||
|
deleted.get(),
|
||||||
|
batch.get(),
|
||||||
|
versionConflicts.get(),
|
||||||
|
noops.get(),
|
||||||
|
bulkRetries.get(),
|
||||||
|
searchRetries.get(),
|
||||||
|
timeValueNanos(throttledNanos.get()),
|
||||||
|
getRequestsPerSecond(),
|
||||||
|
task.getReasonCancelled(),
|
||||||
|
throttledUntil());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public void handleCancel() {
|
||||||
protected void onCancelled() {
|
// Drop the throttle to 0, immediately rescheduling any throttle operation so it will wake up and cancel itself.
|
||||||
/* Drop the throttle to 0, immediately rescheduling any throttled
|
|
||||||
* operation so it will wake up and cancel itself. */
|
|
||||||
rethrottle(Float.POSITIVE_INFINITY);
|
rethrottle(Float.POSITIVE_INFINITY);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public 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)));
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setTotal(long totalHits) {
|
public void setTotal(long totalHits) {
|
||||||
total.set(totalHits);
|
total.set(totalHits);
|
||||||
}
|
}
|
||||||
|
@ -171,6 +161,17 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
return requestsPerSecond;
|
return requestsPerSecond;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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)));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Schedule prepareBulkRequestRunnable to run after some delay. This is where throttling plugs into reindexing so the request can be
|
* Schedule prepareBulkRequestRunnable to run after some delay. This is where throttling plugs into reindexing so the request can be
|
||||||
* rescheduled over and over again.
|
* rescheduled over and over again.
|
||||||
|
@ -180,9 +181,9 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
// Synchronize so we are less likely to schedule the same request twice.
|
// Synchronize so we are less likely to schedule the same request twice.
|
||||||
synchronized (delayedPrepareBulkRequestReference) {
|
synchronized (delayedPrepareBulkRequestReference) {
|
||||||
TimeValue delay = throttleWaitTime(lastBatchStartTime, timeValueNanos(System.nanoTime()), lastBatchSize);
|
TimeValue delay = throttleWaitTime(lastBatchStartTime, timeValueNanos(System.nanoTime()), lastBatchSize);
|
||||||
logger.debug("[{}]: preparing bulk request for [{}]", getId(), delay);
|
logger.debug("[{}]: preparing bulk request for [{}]", task.getId(), delay);
|
||||||
delayedPrepareBulkRequestReference.set(new DelayedPrepareBulkRequest(threadPool, getRequestsPerSecond(),
|
delayedPrepareBulkRequestReference.set(new DelayedPrepareBulkRequest(threadPool, getRequestsPerSecond(),
|
||||||
delay, new RunOnce(prepareBulkRequestRunnable)));
|
delay, new RunOnce(prepareBulkRequestRunnable)));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -213,16 +214,18 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
this.requestsPerSecond = requestsPerSecond;
|
this.requestsPerSecond = requestsPerSecond;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
/**
|
||||||
|
* Apply {@code newRequestsPerSecond} as the new rate limit for this task's search requests
|
||||||
|
*/
|
||||||
public void rethrottle(float newRequestsPerSecond) {
|
public void rethrottle(float newRequestsPerSecond) {
|
||||||
synchronized (delayedPrepareBulkRequestReference) {
|
synchronized (delayedPrepareBulkRequestReference) {
|
||||||
logger.debug("[{}]: rethrottling to [{}] requests per second", getId(), newRequestsPerSecond);
|
logger.debug("[{}]: rethrottling to [{}] requests per second", task.getId(), newRequestsPerSecond);
|
||||||
setRequestsPerSecond(newRequestsPerSecond);
|
setRequestsPerSecond(newRequestsPerSecond);
|
||||||
|
|
||||||
DelayedPrepareBulkRequest delayedPrepareBulkRequest = this.delayedPrepareBulkRequestReference.get();
|
DelayedPrepareBulkRequest delayedPrepareBulkRequest = this.delayedPrepareBulkRequestReference.get();
|
||||||
if (delayedPrepareBulkRequest == null) {
|
if (delayedPrepareBulkRequest == null) {
|
||||||
// No request has been queued so nothing to reschedule.
|
// No request has been queued so nothing to reschedule.
|
||||||
logger.debug("[{}]: skipping rescheduling because there is no scheduled task", getId());
|
logger.debug("[{}]: skipping rescheduling because there is no scheduled task", task.getId());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -260,8 +263,8 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
* change in throttle take effect the next time we delay
|
* change in throttle take effect the next time we delay
|
||||||
* prepareBulkRequest. We can't just reschedule the request further
|
* prepareBulkRequest. We can't just reschedule the request further
|
||||||
* out in the future because the bulk context might time out. */
|
* out in the future because the bulk context might time out. */
|
||||||
logger.debug("[{}]: skipping rescheduling because the new throttle [{}] is slower than the old one [{}]", getId(),
|
logger.debug("[{}]: skipping rescheduling because the new throttle [{}] is slower than the old one [{}]", task.getId(),
|
||||||
newRequestsPerSecond, requestsPerSecond);
|
newRequestsPerSecond, requestsPerSecond);
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -269,7 +272,7 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
// Actually reschedule the task
|
// Actually reschedule the task
|
||||||
if (false == FutureUtils.cancel(future)) {
|
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.
|
// Couldn't cancel, probably because the task has finished or been scheduled. Either way we have nothing to do here.
|
||||||
logger.debug("[{}]: skipping rescheduling because we couldn't cancel the task", getId());
|
logger.debug("[{}]: skipping rescheduling because we couldn't cancel the task", task.getId());
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -278,7 +281,7 @@ public class WorkingBulkByScrollTask extends BulkByScrollTask implements Success
|
||||||
* test it you'll find that requests sneak through. So each request
|
* test it you'll find that requests sneak through. So each request
|
||||||
* is given a runOnce boolean to prevent that. */
|
* is given a runOnce boolean to prevent that. */
|
||||||
TimeValue newDelay = newDelay(remainingDelay, newRequestsPerSecond);
|
TimeValue newDelay = newDelay(remainingDelay, newRequestsPerSecond);
|
||||||
logger.debug("[{}]: rescheduling for [{}] in the future", getId(), newDelay);
|
logger.debug("[{}]: rescheduling for [{}] in the future", task.getId(), newDelay);
|
||||||
return new DelayedPrepareBulkRequest(threadPool, requestsPerSecond, newDelay, command);
|
return new DelayedPrepareBulkRequest(threadPool, requestsPerSecond, newDelay, command);
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,16 +39,21 @@ public abstract class AbstractBulkByScrollRequestTestCase<R extends AbstractBulk
|
||||||
randomFrom(ActiveShardCount.ALL, ActiveShardCount.NONE, ActiveShardCount.ONE, ActiveShardCount.DEFAULT));
|
randomFrom(ActiveShardCount.ALL, ActiveShardCount.NONE, ActiveShardCount.ONE, ActiveShardCount.DEFAULT));
|
||||||
original.setRetryBackoffInitialTime(parseTimeValue(randomPositiveTimeValue(), "retry_backoff_initial_time"));
|
original.setRetryBackoffInitialTime(parseTimeValue(randomPositiveTimeValue(), "retry_backoff_initial_time"));
|
||||||
original.setMaxRetries(between(0, 1000));
|
original.setMaxRetries(between(0, 1000));
|
||||||
original.setSlices(between(2, 1000));
|
|
||||||
original.setRequestsPerSecond(
|
original.setRequestsPerSecond(
|
||||||
randomBoolean() ? Float.POSITIVE_INFINITY : randomValueOtherThanMany(r -> r < 0, ESTestCase::randomFloat));
|
randomBoolean() ? Float.POSITIVE_INFINITY : randomValueOtherThanMany(r -> r < 0, ESTestCase::randomFloat));
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
original.setSize(between(0, Integer.MAX_VALUE));
|
original.setSize(between(0, Integer.MAX_VALUE));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// it's not important how many slices there are, we just need a number for forSlice
|
||||||
|
int actualSlices = between(2, 1000);
|
||||||
|
original.setSlices(randomBoolean()
|
||||||
|
? actualSlices
|
||||||
|
: AbstractBulkByScrollRequest.AUTO_SLICES);
|
||||||
|
|
||||||
TaskId slicingTask = new TaskId(randomAlphaOfLength(5), randomLong());
|
TaskId slicingTask = new TaskId(randomAlphaOfLength(5), randomLong());
|
||||||
SearchRequest sliceRequest = new SearchRequest();
|
SearchRequest sliceRequest = new SearchRequest();
|
||||||
R forSliced = original.forSlice(slicingTask, sliceRequest);
|
R forSliced = original.forSlice(slicingTask, sliceRequest, actualSlices);
|
||||||
assertEquals(original.isAbortOnVersionConflict(), forSliced.isAbortOnVersionConflict());
|
assertEquals(original.isAbortOnVersionConflict(), forSliced.isAbortOnVersionConflict());
|
||||||
assertEquals(original.isRefresh(), forSliced.isRefresh());
|
assertEquals(original.isRefresh(), forSliced.isRefresh());
|
||||||
assertEquals(original.getTimeout(), forSliced.getTimeout());
|
assertEquals(original.getTimeout(), forSliced.getTimeout());
|
||||||
|
@ -57,10 +62,10 @@ public abstract class AbstractBulkByScrollRequestTestCase<R extends AbstractBulk
|
||||||
assertEquals(original.getMaxRetries(), forSliced.getMaxRetries());
|
assertEquals(original.getMaxRetries(), forSliced.getMaxRetries());
|
||||||
assertEquals("only the parent task should store results", false, forSliced.getShouldStoreResult());
|
assertEquals("only the parent task should store results", false, forSliced.getShouldStoreResult());
|
||||||
assertEquals("slice requests always have a single worker", 1, forSliced.getSlices());
|
assertEquals("slice requests always have a single worker", 1, forSliced.getSlices());
|
||||||
assertEquals("requests_per_second is split between all workers", original.getRequestsPerSecond() / original.getSlices(),
|
assertEquals("requests_per_second is split between all workers", original.getRequestsPerSecond() / actualSlices,
|
||||||
forSliced.getRequestsPerSecond(), Float.MIN_NORMAL);
|
forSliced.getRequestsPerSecond(), Float.MIN_NORMAL);
|
||||||
assertEquals("size is split evenly between all workers", original.getSize() == AbstractBulkByScrollRequest.SIZE_ALL_MATCHES
|
assertEquals("size is split evenly between all workers", original.getSize() == AbstractBulkByScrollRequest.SIZE_ALL_MATCHES
|
||||||
? AbstractBulkByScrollRequest.SIZE_ALL_MATCHES : original.getSize() / original.getSlices(), forSliced.getSize());
|
? AbstractBulkByScrollRequest.SIZE_ALL_MATCHES : original.getSize() / actualSlices, forSliced.getSize());
|
||||||
assertEquals(slicingTask, forSliced.getParentTask());
|
assertEquals(slicingTask, forSliced.getParentTask());
|
||||||
|
|
||||||
extraForSliceAssertions(original, forSliced);
|
extraForSliceAssertions(original, forSliced);
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.tasks.TaskId;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.mockito.ArgumentCaptor;
|
import org.mockito.ArgumentCaptor;
|
||||||
|
@ -33,14 +34,17 @@ import static org.mockito.Mockito.atMost;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
public class ParentBulkByScrollTaskTests extends ESTestCase {
|
public class LeaderBulkByScrollTaskStateTests extends ESTestCase {
|
||||||
private int slices;
|
private int slices;
|
||||||
private ParentBulkByScrollTask task;
|
private BulkByScrollTask task;
|
||||||
|
private LeaderBulkByScrollTaskState taskState;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void createTask() {
|
public void createTask() {
|
||||||
slices = between(2, 50);
|
slices = between(2, 50);
|
||||||
task = new ParentBulkByScrollTask(1, "test_type", "test_action", "test", null, slices);
|
task = new BulkByScrollTask(1, "test_type", "test_action", "test", TaskId.EMPTY_TASK_ID);
|
||||||
|
task.setWorkerCount(slices);
|
||||||
|
taskState = task.getLeaderState();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testBasicData() {
|
public void testBasicData() {
|
||||||
|
@ -90,7 +94,7 @@ public class ParentBulkByScrollTaskTests extends ESTestCase {
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
ActionListener<BulkByScrollResponse> listener = slice < slices - 1 ? neverCalled() : mock(ActionListener.class);
|
ActionListener<BulkByScrollResponse> listener = slice < slices - 1 ? neverCalled() : mock(ActionListener.class);
|
||||||
task.onSliceResponse(listener, slice,
|
taskState.onSliceResponse(listener, slice,
|
||||||
new BulkByScrollResponse(timeValueMillis(10), sliceStatus, emptyList(), emptyList(), false));
|
new BulkByScrollResponse(timeValueMillis(10), sliceStatus, emptyList(), emptyList(), false));
|
||||||
|
|
||||||
status = task.getStatus();
|
status = task.getStatus();
|
|
@ -45,7 +45,7 @@ public class ReindexRequestTests extends AbstractBulkByScrollRequestTestCase<Rei
|
||||||
e.getMessage());
|
e.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testReindexFromRemoteDoesNotSupportWorkers() {
|
public void testReindexFromRemoteDoesNotSupportSlices() {
|
||||||
ReindexRequest reindex = newRequest();
|
ReindexRequest reindex = newRequest();
|
||||||
reindex.setRemoteInfo(
|
reindex.setRemoteInfo(
|
||||||
new RemoteInfo(randomAlphaOfLength(5), randomAlphaOfLength(5), between(1, Integer.MAX_VALUE), new BytesArray("real_query"),
|
new RemoteInfo(randomAlphaOfLength(5), randomAlphaOfLength(5), between(1, Integer.MAX_VALUE), new BytesArray("real_query"),
|
||||||
|
@ -53,16 +53,16 @@ public class ReindexRequestTests extends AbstractBulkByScrollRequestTestCase<Rei
|
||||||
reindex.setSlices(between(2, Integer.MAX_VALUE));
|
reindex.setSlices(between(2, Integer.MAX_VALUE));
|
||||||
ActionRequestValidationException e = reindex.validate();
|
ActionRequestValidationException e = reindex.validate();
|
||||||
assertEquals(
|
assertEquals(
|
||||||
"Validation Failed: 1: reindex from remote sources doesn't support workers > 1 but was [" + reindex.getSlices() + "];",
|
"Validation Failed: 1: reindex from remote sources doesn't support slices > 1 but was [" + reindex.getSlices() + "];",
|
||||||
e.getMessage());
|
e.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testNoSliceWithWorkers() {
|
public void testNoSliceBuilderSetWithSlicedRequest() {
|
||||||
ReindexRequest reindex = newRequest();
|
ReindexRequest reindex = newRequest();
|
||||||
reindex.getSearchRequest().source().slice(new SliceBuilder(0, 4));
|
reindex.getSearchRequest().source().slice(new SliceBuilder(0, 4));
|
||||||
reindex.setSlices(between(2, Integer.MAX_VALUE));
|
reindex.setSlices(between(2, Integer.MAX_VALUE));
|
||||||
ActionRequestValidationException e = reindex.validate();
|
ActionRequestValidationException e = reindex.validate();
|
||||||
assertEquals("Validation Failed: 1: can't specify both slice and workers;", e.getMessage());
|
assertEquals("Validation Failed: 1: can't specify both manual and automatic slicing at the same time;", e.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -46,12 +46,15 @@ import static org.hamcrest.Matchers.empty;
|
||||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||||
|
|
||||||
public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
public class WorkerBulkByScrollTaskStateTests extends ESTestCase {
|
||||||
private WorkingBulkByScrollTask task;
|
private BulkByScrollTask task;
|
||||||
|
private WorkerBulkByScrollTaskState workerState;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void createTask() {
|
public void createTask() {
|
||||||
task = new WorkingBulkByScrollTask(1, "test_type", "test_action", "test", TaskId.EMPTY_TASK_ID, null, Float.POSITIVE_INFINITY);
|
task = new BulkByScrollTask(1, "test_type", "test_action", "test", TaskId.EMPTY_TASK_ID);
|
||||||
|
task.setWorker(Float.POSITIVE_INFINITY, null);
|
||||||
|
workerState = task.getWorkerState();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testBasicData() {
|
public void testBasicData() {
|
||||||
|
@ -78,7 +81,7 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
assertEquals(noops, status.getNoops());
|
assertEquals(noops, status.getNoops());
|
||||||
|
|
||||||
long totalHits = randomIntBetween(10, 1000);
|
long totalHits = randomIntBetween(10, 1000);
|
||||||
task.setTotal(totalHits);
|
workerState.setTotal(totalHits);
|
||||||
for (long p = 0; p < totalHits; p++) {
|
for (long p = 0; p < totalHits; p++) {
|
||||||
status = task.getStatus();
|
status = task.getStatus();
|
||||||
assertEquals(totalHits, status.getTotal());
|
assertEquals(totalHits, status.getTotal());
|
||||||
|
@ -91,28 +94,28 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
|
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
created++;
|
created++;
|
||||||
task.countCreated();
|
workerState.countCreated();
|
||||||
} else if (randomBoolean()) {
|
} else if (randomBoolean()) {
|
||||||
updated++;
|
updated++;
|
||||||
task.countUpdated();
|
workerState.countUpdated();
|
||||||
} else {
|
} else {
|
||||||
deleted++;
|
deleted++;
|
||||||
task.countDeleted();
|
workerState.countDeleted();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rarely()) {
|
if (rarely()) {
|
||||||
versionConflicts++;
|
versionConflicts++;
|
||||||
task.countVersionConflict();
|
workerState.countVersionConflict();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rarely()) {
|
if (rarely()) {
|
||||||
batch++;
|
batch++;
|
||||||
task.countBatch();
|
workerState.countBatch();
|
||||||
}
|
}
|
||||||
|
|
||||||
if (rarely()) {
|
if (rarely()) {
|
||||||
noops++;
|
noops++;
|
||||||
task.countNoop();
|
workerState.countNoop();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
status = task.getStatus();
|
status = task.getStatus();
|
||||||
|
@ -139,7 +142,7 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
* each time.
|
* each time.
|
||||||
*/
|
*/
|
||||||
float originalRequestsPerSecond = (float) randomDoubleBetween(1, 10000, true);
|
float originalRequestsPerSecond = (float) randomDoubleBetween(1, 10000, true);
|
||||||
task.rethrottle(originalRequestsPerSecond);
|
workerState.rethrottle(originalRequestsPerSecond);
|
||||||
TimeValue maxDelay = timeValueSeconds(between(1, 5));
|
TimeValue maxDelay = timeValueSeconds(between(1, 5));
|
||||||
assertThat(maxDelay.nanos(), greaterThanOrEqualTo(0L));
|
assertThat(maxDelay.nanos(), greaterThanOrEqualTo(0L));
|
||||||
int batchSizeForMaxDelay = (int) (maxDelay.seconds() * originalRequestsPerSecond);
|
int batchSizeForMaxDelay = (int) (maxDelay.seconds() * originalRequestsPerSecond);
|
||||||
|
@ -151,20 +154,22 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
try {
|
try {
|
||||||
task.delayPrepareBulkRequest(threadPool, timeValueNanos(System.nanoTime()), batchSizeForMaxDelay, new AbstractRunnable() {
|
workerState.delayPrepareBulkRequest(threadPool, timeValueNanos(System.nanoTime()), batchSizeForMaxDelay,
|
||||||
@Override
|
new AbstractRunnable() {
|
||||||
protected void doRun() throws Exception {
|
@Override
|
||||||
boolean oldValue = done.getAndSet(true);
|
protected void doRun() throws Exception {
|
||||||
if (oldValue) {
|
boolean oldValue = done.getAndSet(true);
|
||||||
throw new RuntimeException("Ran twice oh no!");
|
if (oldValue) {
|
||||||
|
throw new RuntimeException("Ran twice oh no!");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(Exception e) {
|
||||||
|
errors.add(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
);
|
||||||
@Override
|
|
||||||
public void onFailure(Exception e) {
|
|
||||||
errors.add(e);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
// Rethrottle on a random number of threads, one of which is this thread.
|
// Rethrottle on a random number of threads, one of which is this thread.
|
||||||
Runnable test = () -> {
|
Runnable test = () -> {
|
||||||
|
@ -172,7 +177,7 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
int rethrottles = 0;
|
int rethrottles = 0;
|
||||||
while (false == done.get()) {
|
while (false == done.get()) {
|
||||||
float requestsPerSecond = (float) randomDoubleBetween(0, originalRequestsPerSecond * 2, true);
|
float requestsPerSecond = (float) randomDoubleBetween(0, originalRequestsPerSecond * 2, true);
|
||||||
task.rethrottle(requestsPerSecond);
|
workerState.rethrottle(requestsPerSecond);
|
||||||
rethrottles += 1;
|
rethrottles += 1;
|
||||||
}
|
}
|
||||||
logger.info("Rethrottled [{}] times", rethrottles);
|
logger.info("Rethrottled [{}] times", rethrottles);
|
||||||
|
@ -237,7 +242,7 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
};
|
};
|
||||||
try {
|
try {
|
||||||
// Have the task use the thread pool to delay a task that does nothing
|
// Have the task use the thread pool to delay a task that does nothing
|
||||||
task.delayPrepareBulkRequest(threadPool, timeValueSeconds(0), 1, new AbstractRunnable() {
|
workerState.delayPrepareBulkRequest(threadPool, timeValueSeconds(0), 1, new AbstractRunnable() {
|
||||||
@Override
|
@Override
|
||||||
protected void doRun() throws Exception {
|
protected void doRun() throws Exception {
|
||||||
}
|
}
|
||||||
|
@ -254,12 +259,12 @@ public class WorkingBulkByScrollTaskTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testPerfectlyThrottledBatchTime() {
|
public void testPerfectlyThrottledBatchTime() {
|
||||||
task.rethrottle(Float.POSITIVE_INFINITY);
|
workerState.rethrottle(Float.POSITIVE_INFINITY);
|
||||||
assertThat((double) task.perfectlyThrottledBatchTime(randomInt()), closeTo(0f, 0f));
|
assertThat((double) workerState.perfectlyThrottledBatchTime(randomInt()), closeTo(0f, 0f));
|
||||||
|
|
||||||
int total = between(0, 1000000);
|
int total = between(0, 1000000);
|
||||||
task.rethrottle(1);
|
workerState.rethrottle(1);
|
||||||
assertThat((double) task.perfectlyThrottledBatchTime(total),
|
assertThat((double) workerState.perfectlyThrottledBatchTime(total),
|
||||||
closeTo(TimeUnit.SECONDS.toNanos(total), TimeUnit.SECONDS.toNanos(1)));
|
closeTo(TimeUnit.SECONDS.toNanos(total), TimeUnit.SECONDS.toNanos(1)));
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -114,21 +114,21 @@ public class SliceBuilderTests extends ESTestCase {
|
||||||
|
|
||||||
public void testInvalidArguments() throws Exception {
|
public void testInvalidArguments() throws Exception {
|
||||||
Exception e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", -1, 10));
|
Exception e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", -1, 10));
|
||||||
assertEquals(e.getMessage(), "id must be greater than or equal to 0");
|
assertEquals("id must be greater than or equal to 0", e.getMessage());
|
||||||
|
|
||||||
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, -1));
|
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, -1));
|
||||||
assertEquals(e.getMessage(), "max must be greater than 1");
|
assertEquals("max must be greater than 1", e.getMessage());
|
||||||
|
|
||||||
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, 0));
|
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, 0));
|
||||||
assertEquals(e.getMessage(), "max must be greater than 1");
|
assertEquals("max must be greater than 1", e.getMessage());
|
||||||
|
|
||||||
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, 5));
|
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, 5));
|
||||||
assertEquals(e.getMessage(), "max must be greater than id");
|
assertEquals("max must be greater than id", e.getMessage());
|
||||||
|
|
||||||
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 1000, 1000));
|
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 1000, 1000));
|
||||||
assertEquals(e.getMessage(), "max must be greater than id");
|
assertEquals("max must be greater than id", e.getMessage());
|
||||||
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 1001, 1000));
|
e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 1001, 1000));
|
||||||
assertEquals(e.getMessage(), "max must be greater than id");
|
assertEquals("max must be greater than id", e.getMessage());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testToFilter() throws IOException {
|
public void testToFilter() throws IOException {
|
||||||
|
|
|
@ -339,11 +339,19 @@ take effect on after completing the current batch. This prevents scroll
|
||||||
timeouts.
|
timeouts.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-delete-by-query-manual-slice]]
|
[[docs-delete-by-query-slice]]
|
||||||
=== Manually slicing
|
=== Slicing
|
||||||
|
|
||||||
Delete-by-query supports <<sliced-scroll>> allowing you to manually parallelize
|
Delete-by-query supports <<sliced-scroll>> to parallelize the deleting process.
|
||||||
the process relatively easily:
|
This parallelization can improve efficiency and provide a convenient way to
|
||||||
|
break the request down into smaller parts.
|
||||||
|
|
||||||
|
[float]
|
||||||
|
[[docs-delete-by-query-manual-slice]]
|
||||||
|
==== Manually slicing
|
||||||
|
|
||||||
|
Slice a delete-by-query manually by providing a slice id and total number of
|
||||||
|
slices to each request:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
|
@ -412,10 +420,11 @@ Which results in a sensible `total` like this one:
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-delete-by-query-automatic-slice]]
|
[[docs-delete-by-query-automatic-slice]]
|
||||||
=== Automatic slicing
|
==== Automatic slicing
|
||||||
|
|
||||||
You can also let delete-by-query automatically parallelize using
|
You can also let delete-by-query automatically parallelize using
|
||||||
<<sliced-scroll>> to slice on `_uid`:
|
<<sliced-scroll>> to slice on `_uid`. Use `slices` to specify the number of
|
||||||
|
slices to use:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
|
@ -463,6 +472,11 @@ Which results in a sensible `total` like this one:
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
// TESTRESPONSE
|
// TESTRESPONSE
|
||||||
|
|
||||||
|
Setting `slices` to `auto` will let Elasticsearch choose the number of slices
|
||||||
|
to use. This setting will use one slice per shard, up to a certain limit. If
|
||||||
|
there are multiple source indices, it will choose the number of slices based
|
||||||
|
on the index with the smallest number of shards.
|
||||||
|
|
||||||
Adding `slices` to `_delete_by_query` just automates the manual process used in
|
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:
|
the section above, creating sub-requests which means it has some quirks:
|
||||||
|
|
||||||
|
@ -489,18 +503,20 @@ though these are all taken at approximately the same time.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-delete-by-query-picking-slices]]
|
[[docs-delete-by-query-picking-slices]]
|
||||||
=== Picking the number of slices
|
===== Picking the number of slices
|
||||||
|
|
||||||
At this point we have a few recommendations around the number of `slices` to
|
If slicing automatically, setting `slices` to `auto` will choose a reasonable
|
||||||
use (the `max` parameter in the slice API if manually parallelizing):
|
number for most indices. If you're slicing manually or otherwise tuning
|
||||||
|
automatic slicing, use these guidelines.
|
||||||
|
|
||||||
* Don't use large numbers. `500` creates fairly massive CPU thrash.
|
Query performance is most efficient when the number of `slices` is equal to the
|
||||||
* It is more efficient from a query performance standpoint to use some multiple
|
number of shards in the index. If that number is large, (for example,
|
||||||
of the number of shards in the source index.
|
500) choose a lower number as too many `slices` will hurt performance. Setting
|
||||||
* Using exactly as many shards as are in the source index is the most efficient
|
`slices` higher than the number of shards generally does not improve efficiency
|
||||||
from a query performance standpoint.
|
and adds overhead.
|
||||||
* Indexing performance should scale linearly across available resources with
|
|
||||||
the number of `slices`.
|
Delete performance scales linearly across available resources with the
|
||||||
* Whether indexing or query performance dominates that process depends on lots
|
number of slices.
|
||||||
of factors like the documents being reindexed and the cluster doing the
|
|
||||||
reindexing.
|
Whether query or delete performance dominates the runtime depends on the
|
||||||
|
documents being reindexed and cluster resources.
|
||||||
|
|
|
@ -787,11 +787,19 @@ and it'll look like:
|
||||||
|
|
||||||
Or you can search by `tag` or whatever you want.
|
Or you can search by `tag` or whatever you want.
|
||||||
|
|
||||||
|
[float]
|
||||||
|
[[docs-reindex-slice]]
|
||||||
|
=== Slicing
|
||||||
|
|
||||||
|
Reindex supports <<sliced-scroll>> to parallelize the reindexing process.
|
||||||
|
This parallelization can improve efficiency and provide a convenient way to
|
||||||
|
break the request down into smaller parts.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-reindex-manual-slice]]
|
[[docs-reindex-manual-slice]]
|
||||||
==== Manual slicing
|
==== Manual slicing
|
||||||
Reindex supports <<sliced-scroll>>, allowing you to manually parallelize the
|
Slice a reindex request manually by providing a slice id and total number of
|
||||||
process relatively easily:
|
slices to each request:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
|
@ -849,10 +857,10 @@ Which results in a sensible `total` like this one:
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-reindex-automatic-slice]]
|
[[docs-reindex-automatic-slice]]
|
||||||
=== Automatic slicing
|
==== Automatic slicing
|
||||||
|
|
||||||
You can also let reindex automatically parallelize using <<sliced-scroll>> to
|
You can also let reindex automatically parallelize using <<sliced-scroll>> to
|
||||||
slice on `_uid`:
|
slice on `_uid`. Use `slices` to specify the number of slices to use:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
|
@ -890,6 +898,11 @@ Which results in a sensible `total` like this one:
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
// TESTRESPONSE
|
// TESTRESPONSE
|
||||||
|
|
||||||
|
Setting `slices` to `auto` will let Elasticsearch choose the number of slices
|
||||||
|
to use. This setting will use one slice per shard, up to a certain limit. If
|
||||||
|
there are multiple source indices, it will choose the number of slices based
|
||||||
|
on the index with the smallest number of shards.
|
||||||
|
|
||||||
Adding `slices` to `_reindex` just automates the manual process used in the
|
Adding `slices` to `_reindex` just automates the manual process used in the
|
||||||
section above, creating sub-requests which means it has some quirks:
|
section above, creating sub-requests which means it has some quirks:
|
||||||
|
|
||||||
|
@ -915,21 +928,23 @@ though these are all taken at approximately the same time.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-reindex-picking-slices]]
|
[[docs-reindex-picking-slices]]
|
||||||
=== Picking the number of slices
|
===== Picking the number of slices
|
||||||
|
|
||||||
At this point we have a few recommendations around the number of `slices` to
|
If slicing automatically, setting `slices` to `auto` will choose a reasonable
|
||||||
use (the `max` parameter in the slice API if manually parallelizing):
|
number for most indices. If you're slicing manually or otherwise tuning
|
||||||
|
automatic slicing, use these guidelines.
|
||||||
|
|
||||||
* Don't use large numbers. `500` creates fairly massive CPU thrash.
|
Query performance is most efficient when the number of `slices` is equal to the
|
||||||
* It is more efficient from a query performance standpoint to use some multiple
|
number of shards in the index. If that number is large, (for example,
|
||||||
of the number of shards in the source index.
|
500) choose a lower number as too many `slices` will hurt performance. Setting
|
||||||
* Using exactly as many shards as are in the source index is the most efficient
|
`slices` higher than the number of shards generally does not improve efficiency
|
||||||
from a query performance standpoint.
|
and adds overhead.
|
||||||
* Indexing performance should scale linearly across available resources with
|
|
||||||
the number of `slices`.
|
Indexing performance scales linearly across available resources with the
|
||||||
* Whether indexing or query performance dominates that process depends on lots
|
number of slices.
|
||||||
of factors like the documents being reindexed and the cluster doing the
|
|
||||||
reindexing.
|
Whether query or indexing performance dominates the runtime depends on the
|
||||||
|
documents being reindexed and cluster resources.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
=== Reindex daily indices
|
=== Reindex daily indices
|
||||||
|
|
|
@ -403,11 +403,19 @@ query takes effect immediately but rethrotting that slows down the query will
|
||||||
take effect on after completing the current batch. This prevents scroll
|
take effect on after completing the current batch. This prevents scroll
|
||||||
timeouts.
|
timeouts.
|
||||||
|
|
||||||
|
[float]
|
||||||
|
[[docs-update-by-query-slice]]
|
||||||
|
=== Slicing
|
||||||
|
|
||||||
|
Update-by-query supports <<sliced-scroll>> to parallelize the updating process.
|
||||||
|
This parallelization can improve efficiency and provide a convenient way to
|
||||||
|
break the request down into smaller parts.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-update-by-query-manual-slice]]
|
[[docs-update-by-query-manual-slice]]
|
||||||
==== Manual slicing
|
==== Manual slicing
|
||||||
Update-by-query supports <<sliced-scroll>> allowing you to manually parallelize
|
Slice an update-by-query manually by providing a slice id and total number of
|
||||||
the process relatively easily:
|
slices to each request:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
|
@ -459,10 +467,11 @@ Which results in a sensible `total` like this one:
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-update-by-query-automatic-slice]]
|
[[docs-update-by-query-automatic-slice]]
|
||||||
=== Automatic slicing
|
==== Automatic slicing
|
||||||
|
|
||||||
You can also let update-by-query automatically parallelize using
|
You can also let update-by-query automatically parallelize using
|
||||||
<<sliced-scroll>> to slice on `_uid`:
|
<<sliced-scroll>> to slice on `_uid`. Use `slices` to specify the number of
|
||||||
|
slices to use:
|
||||||
|
|
||||||
[source,js]
|
[source,js]
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
|
@ -497,6 +506,11 @@ Which results in a sensible `total` like this one:
|
||||||
----------------------------------------------------------------
|
----------------------------------------------------------------
|
||||||
// TESTRESPONSE
|
// TESTRESPONSE
|
||||||
|
|
||||||
|
Setting `slices` to `auto` will let Elasticsearch choose the number of slices
|
||||||
|
to use. This setting will use one slice per shard, up to a certain limit. If
|
||||||
|
there are multiple source indices, it will choose the number of slices based
|
||||||
|
on the index with the smallest number of shards.
|
||||||
|
|
||||||
Adding `slices` to `_update_by_query` just automates the manual process used in
|
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:
|
the section above, creating sub-requests which means it has some quirks:
|
||||||
|
|
||||||
|
@ -523,22 +537,23 @@ though these are all taken at approximately the same time.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[docs-update-by-query-picking-slices]]
|
[[docs-update-by-query-picking-slices]]
|
||||||
=== Picking the number of slices
|
===== Picking the number of slices
|
||||||
|
|
||||||
At this point we have a few recommendations around the number of `slices` to
|
If slicing automatically, setting `slices` to `auto` will choose a reasonable
|
||||||
use (the `max` parameter in the slice API if manually parallelizing):
|
number for most indices. If you're slicing manually or otherwise tuning
|
||||||
|
automatic slicing, use these guidelines.
|
||||||
|
|
||||||
* Don't use large numbers. `500` creates fairly massive CPU thrash.
|
Query performance is most efficient when the number of `slices` is equal to the
|
||||||
* It is more efficient from a query performance standpoint to use some multiple
|
number of shards in the index. If that number is large, (for example,
|
||||||
of the number of shards in the source index.
|
500) choose a lower number as too many `slices` will hurt performance. Setting
|
||||||
* Using exactly as many shards as are in the source index is the most efficient
|
`slices` higher than the number of shards generally does not improve efficiency
|
||||||
from a query performance standpoint.
|
and adds overhead.
|
||||||
* 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.
|
|
||||||
|
|
||||||
|
Update performance scales linearly across available resources with the
|
||||||
|
number of slices.
|
||||||
|
|
||||||
|
Whether query or update performance dominates the runtime depends on the
|
||||||
|
documents being reindexed and cluster resources.
|
||||||
|
|
||||||
[float]
|
[float]
|
||||||
[[picking-up-a-new-property]]
|
[[picking-up-a-new-property]]
|
||||||
|
|
|
@ -87,7 +87,8 @@ import static org.elasticsearch.search.sort.SortBuilders.fieldSort;
|
||||||
*/
|
*/
|
||||||
public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBulkByScrollRequest<Request>> {
|
public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBulkByScrollRequest<Request>> {
|
||||||
protected final Logger logger;
|
protected final Logger logger;
|
||||||
protected final WorkingBulkByScrollTask task;
|
protected final BulkByScrollTask task;
|
||||||
|
protected final WorkerBulkByScrollTaskState worker;
|
||||||
protected final ThreadPool threadPool;
|
protected final ThreadPool threadPool;
|
||||||
protected final ScriptService scriptService;
|
protected final ScriptService scriptService;
|
||||||
protected final ClusterState clusterState;
|
protected final ClusterState clusterState;
|
||||||
|
@ -114,16 +115,22 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
*/
|
*/
|
||||||
private final BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> scriptApplier;
|
private final BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> scriptApplier;
|
||||||
|
|
||||||
public AbstractAsyncBulkByScrollAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
public AbstractAsyncBulkByScrollAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, Request mainRequest, ScriptService scriptService,
|
ThreadPool threadPool, Request mainRequest, ScriptService scriptService,
|
||||||
ClusterState clusterState, ActionListener<BulkByScrollResponse> listener) {
|
ClusterState clusterState, ActionListener<BulkByScrollResponse> listener) {
|
||||||
this(task, logger, client, threadPool, mainRequest, scriptService, clusterState, listener, client.settings());
|
this(task, logger, client, threadPool, mainRequest, scriptService, clusterState, listener, client.settings());
|
||||||
}
|
}
|
||||||
|
|
||||||
public AbstractAsyncBulkByScrollAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
public AbstractAsyncBulkByScrollAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, Request mainRequest, ScriptService scriptService, ClusterState clusterState,
|
ThreadPool threadPool, Request mainRequest, ScriptService scriptService, ClusterState clusterState,
|
||||||
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
||||||
|
|
||||||
this.task = task;
|
this.task = task;
|
||||||
|
if (!task.isWorker()) {
|
||||||
|
throw new IllegalArgumentException("Given task [" + task.getId() + "] must have a child worker");
|
||||||
|
}
|
||||||
|
this.worker = task.getWorkerState();
|
||||||
|
|
||||||
this.logger = logger;
|
this.logger = logger;
|
||||||
this.client = client;
|
this.client = client;
|
||||||
this.settings = settings;
|
this.settings = settings;
|
||||||
|
@ -133,7 +140,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
this.mainRequest = mainRequest;
|
this.mainRequest = mainRequest;
|
||||||
this.listener = listener;
|
this.listener = listener;
|
||||||
BackoffPolicy backoffPolicy = buildBackoffPolicy();
|
BackoffPolicy backoffPolicy = buildBackoffPolicy();
|
||||||
bulkRetry = new Retry(EsRejectedExecutionException.class, BackoffPolicy.wrap(backoffPolicy, task::countBulkRetry), threadPool);
|
bulkRetry = new Retry(EsRejectedExecutionException.class, BackoffPolicy.wrap(backoffPolicy, worker::countBulkRetry), threadPool);
|
||||||
scrollSource = buildScrollableResultSource(backoffPolicy);
|
scrollSource = buildScrollableResultSource(backoffPolicy);
|
||||||
scriptApplier = Objects.requireNonNull(buildScriptApplier(), "script applier must not be null");
|
scriptApplier = Objects.requireNonNull(buildScriptApplier(), "script applier must not be null");
|
||||||
/*
|
/*
|
||||||
|
@ -217,7 +224,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
}
|
}
|
||||||
|
|
||||||
protected ScrollableHitSource buildScrollableResultSource(BackoffPolicy backoffPolicy) {
|
protected ScrollableHitSource buildScrollableResultSource(BackoffPolicy backoffPolicy) {
|
||||||
return new ClientScrollableHitSource(logger, backoffPolicy, threadPool, task::countSearchRetry, this::finishHim, client,
|
return new ClientScrollableHitSource(logger, backoffPolicy, threadPool, worker::countSearchRetry, this::finishHim, client,
|
||||||
mainRequest.getSearchRequest());
|
mainRequest.getSearchRequest());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -272,7 +279,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
if (mainRequest.getSize() > 0) {
|
if (mainRequest.getSize() > 0) {
|
||||||
total = min(total, mainRequest.getSize());
|
total = min(total, mainRequest.getSize());
|
||||||
}
|
}
|
||||||
task.setTotal(total);
|
worker.setTotal(total);
|
||||||
AbstractRunnable prepareBulkRequestRunnable = new AbstractRunnable() {
|
AbstractRunnable prepareBulkRequestRunnable = new AbstractRunnable() {
|
||||||
@Override
|
@Override
|
||||||
protected void doRun() throws Exception {
|
protected void doRun() throws Exception {
|
||||||
|
@ -289,7 +296,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
prepareBulkRequestRunnable = (AbstractRunnable) threadPool.getThreadContext().preserveContext(prepareBulkRequestRunnable);
|
prepareBulkRequestRunnable = (AbstractRunnable) threadPool.getThreadContext().preserveContext(prepareBulkRequestRunnable);
|
||||||
task.delayPrepareBulkRequest(threadPool, lastBatchStartTime, lastBatchSize, prepareBulkRequestRunnable);
|
worker.delayPrepareBulkRequest(threadPool, lastBatchStartTime, lastBatchSize, prepareBulkRequestRunnable);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -308,11 +315,11 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
refreshAndFinish(emptyList(), emptyList(), false);
|
refreshAndFinish(emptyList(), emptyList(), false);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
task.countBatch();
|
worker.countBatch();
|
||||||
List<? extends ScrollableHitSource.Hit> hits = response.getHits();
|
List<? extends ScrollableHitSource.Hit> hits = response.getHits();
|
||||||
if (mainRequest.getSize() != SIZE_ALL_MATCHES) {
|
if (mainRequest.getSize() != SIZE_ALL_MATCHES) {
|
||||||
// Truncate the hits if we have more than the request size
|
// Truncate the hits if we have more than the request size
|
||||||
long remaining = max(0, mainRequest.getSize() - task.getSuccessfullyProcessed());
|
long remaining = max(0, mainRequest.getSize() - worker.getSuccessfullyProcessed());
|
||||||
if (remaining < hits.size()) {
|
if (remaining < hits.size()) {
|
||||||
hits = hits.subList(0, (int) remaining);
|
hits = hits.subList(0, (int) remaining);
|
||||||
}
|
}
|
||||||
|
@ -372,16 +379,16 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
case CREATE:
|
case CREATE:
|
||||||
case INDEX:
|
case INDEX:
|
||||||
if (item.getResponse().getResult() == DocWriteResponse.Result.CREATED) {
|
if (item.getResponse().getResult() == DocWriteResponse.Result.CREATED) {
|
||||||
task.countCreated();
|
worker.countCreated();
|
||||||
} else {
|
} else {
|
||||||
task.countUpdated();
|
worker.countUpdated();
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case UPDATE:
|
case UPDATE:
|
||||||
task.countUpdated();
|
worker.countUpdated();
|
||||||
break;
|
break;
|
||||||
case DELETE:
|
case DELETE:
|
||||||
task.countDeleted();
|
worker.countDeleted();
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
// Track the indexes we've seen so we can refresh them if requested
|
// Track the indexes we've seen so we can refresh them if requested
|
||||||
|
@ -401,7 +408,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (mainRequest.getSize() != SIZE_ALL_MATCHES && task.getSuccessfullyProcessed() >= mainRequest.getSize()) {
|
if (mainRequest.getSize() != SIZE_ALL_MATCHES && worker.getSuccessfullyProcessed() >= mainRequest.getSize()) {
|
||||||
// We've processed all the requested docs.
|
// We've processed all the requested docs.
|
||||||
refreshAndFinish(emptyList(), emptyList(), false);
|
refreshAndFinish(emptyList(), emptyList(), false);
|
||||||
return;
|
return;
|
||||||
|
@ -425,7 +432,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
finishHim(null);
|
finishHim(null);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
TimeValue extraKeepAlive = task.throttleWaitTime(lastBatchStartTime, now, lastBatchSize);
|
TimeValue extraKeepAlive = worker.throttleWaitTime(lastBatchStartTime, now, lastBatchSize);
|
||||||
scrollSource.startNextScroll(extraKeepAlive, response -> {
|
scrollSource.startNextScroll(extraKeepAlive, response -> {
|
||||||
onScrollResponse(lastBatchStartTime, lastBatchSize, response);
|
onScrollResponse(lastBatchStartTime, lastBatchSize, response);
|
||||||
});
|
});
|
||||||
|
@ -433,7 +440,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
|
|
||||||
private void recordFailure(Failure failure, List<Failure> failures) {
|
private void recordFailure(Failure failure, List<Failure> failures) {
|
||||||
if (failure.getStatus() == CONFLICT) {
|
if (failure.getStatus() == CONFLICT) {
|
||||||
task.countVersionConflict();
|
worker.countVersionConflict();
|
||||||
if (false == mainRequest.isAbortOnVersionConflict()) {
|
if (false == mainRequest.isAbortOnVersionConflict()) {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -759,9 +766,9 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
/**
|
/**
|
||||||
* Apply a {@link Script} to a {@link RequestWrapper}
|
* Apply a {@link Script} to a {@link RequestWrapper}
|
||||||
*/
|
*/
|
||||||
public abstract class ScriptApplier implements BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> {
|
public abstract static class ScriptApplier implements BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> {
|
||||||
|
|
||||||
private final WorkingBulkByScrollTask task;
|
private final WorkerBulkByScrollTaskState taskWorker;
|
||||||
private final ScriptService scriptService;
|
private final ScriptService scriptService;
|
||||||
private final Script script;
|
private final Script script;
|
||||||
private final Map<String, Object> params;
|
private final Map<String, Object> params;
|
||||||
|
@ -769,9 +776,11 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
private ExecutableScript executable;
|
private ExecutableScript executable;
|
||||||
private Map<String, Object> context;
|
private Map<String, Object> context;
|
||||||
|
|
||||||
public ScriptApplier(WorkingBulkByScrollTask task, ScriptService scriptService, Script script,
|
public ScriptApplier(WorkerBulkByScrollTaskState taskWorker,
|
||||||
|
ScriptService scriptService,
|
||||||
|
Script script,
|
||||||
Map<String, Object> params) {
|
Map<String, Object> params) {
|
||||||
this.task = task;
|
this.taskWorker = taskWorker;
|
||||||
this.scriptService = scriptService;
|
this.scriptService = scriptService;
|
||||||
this.script = script;
|
this.script = script;
|
||||||
this.params = params;
|
this.params = params;
|
||||||
|
@ -864,7 +873,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
protected RequestWrapper<?> scriptChangedOpType(RequestWrapper<?> request, OpType oldOpType, OpType newOpType) {
|
protected RequestWrapper<?> scriptChangedOpType(RequestWrapper<?> request, OpType oldOpType, OpType newOpType) {
|
||||||
switch (newOpType) {
|
switch (newOpType) {
|
||||||
case NOOP:
|
case NOOP:
|
||||||
task.countNoop();
|
taskWorker.countNoop();
|
||||||
return null;
|
return null;
|
||||||
case DELETE:
|
case DELETE:
|
||||||
RequestWrapper<DeleteRequest> delete = wrap(new DeleteRequest(request.getIndex(), request.getType(), request.getId()));
|
RequestWrapper<DeleteRequest> delete = wrap(new DeleteRequest(request.getIndex(), request.getType(), request.getId()));
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.elasticsearch.rest.BaseRestHandler;
|
||||||
import org.elasticsearch.rest.BytesRestResponse;
|
import org.elasticsearch.rest.BytesRestResponse;
|
||||||
import org.elasticsearch.rest.RestRequest;
|
import org.elasticsearch.rest.RestRequest;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||||
import org.elasticsearch.tasks.LoggingTaskListener;
|
import org.elasticsearch.tasks.LoggingTaskListener;
|
||||||
import org.elasticsearch.tasks.Task;
|
import org.elasticsearch.tasks.Task;
|
||||||
|
|
||||||
|
@ -90,7 +91,11 @@ public abstract class AbstractBaseReindexRestHandler<
|
||||||
|
|
||||||
request.setRefresh(restRequest.paramAsBoolean("refresh", request.isRefresh()));
|
request.setRefresh(restRequest.paramAsBoolean("refresh", request.isRefresh()));
|
||||||
request.setTimeout(restRequest.paramAsTime("timeout", request.getTimeout()));
|
request.setTimeout(restRequest.paramAsTime("timeout", request.getTimeout()));
|
||||||
request.setSlices(restRequest.paramAsInt("slices", request.getSlices()));
|
|
||||||
|
Integer slices = parseSlices(restRequest);
|
||||||
|
if (slices != null) {
|
||||||
|
request.setSlices(slices);
|
||||||
|
}
|
||||||
|
|
||||||
String waitForActiveShards = restRequest.param("wait_for_active_shards");
|
String waitForActiveShards = restRequest.param("wait_for_active_shards");
|
||||||
if (waitForActiveShards != null) {
|
if (waitForActiveShards != null) {
|
||||||
|
@ -115,6 +120,32 @@ public abstract class AbstractBaseReindexRestHandler<
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static Integer parseSlices(RestRequest request) {
|
||||||
|
String slicesString = request.param("slices");
|
||||||
|
if (slicesString == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (slicesString.equals(AbstractBulkByScrollRequest.AUTO_SLICES_VALUE)) {
|
||||||
|
return AbstractBulkByScrollRequest.AUTO_SLICES;
|
||||||
|
}
|
||||||
|
|
||||||
|
int slices;
|
||||||
|
try {
|
||||||
|
slices = Integer.parseInt(slicesString);
|
||||||
|
} catch (NumberFormatException e) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"[slices] must be a positive integer or the string \"auto\", but was [" + slicesString + "]", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (slices < 1) {
|
||||||
|
throw new IllegalArgumentException(
|
||||||
|
"[slices] must be a positive integer or the string \"auto\", but was [" + slicesString + "]");
|
||||||
|
}
|
||||||
|
|
||||||
|
return slices;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return requests_per_second from the request as a float if it was on the request, null otherwise
|
* @return requests_per_second from the request as a float if it was on the request, null otherwise
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -31,7 +31,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
||||||
* Implementation of delete-by-query using scrolling and bulk.
|
* Implementation of delete-by-query using scrolling and bulk.
|
||||||
*/
|
*/
|
||||||
public class AsyncDeleteByQueryAction extends AbstractAsyncBulkByScrollAction<DeleteByQueryRequest> {
|
public class AsyncDeleteByQueryAction extends AbstractAsyncBulkByScrollAction<DeleteByQueryRequest> {
|
||||||
public AsyncDeleteByQueryAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
public AsyncDeleteByQueryAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, DeleteByQueryRequest request, ScriptService scriptService,
|
ThreadPool threadPool, DeleteByQueryRequest request, ScriptService scriptService,
|
||||||
ClusterState clusterState, ActionListener<BulkByScrollResponse> listener) {
|
ClusterState clusterState, ActionListener<BulkByScrollResponse> listener) {
|
||||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener);
|
super(task, logger, client, threadPool, request, scriptService, clusterState, listener);
|
||||||
|
|
|
@ -21,31 +21,118 @@ package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.elasticsearch.action.Action;
|
import org.elasticsearch.action.Action;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest;
|
||||||
|
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse;
|
||||||
import org.elasticsearch.action.search.SearchRequest;
|
import org.elasticsearch.action.search.SearchRequest;
|
||||||
import org.elasticsearch.client.Client;
|
import org.elasticsearch.client.Client;
|
||||||
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.index.Index;
|
||||||
import org.elasticsearch.index.mapper.UidFieldMapper;
|
import org.elasticsearch.index.mapper.UidFieldMapper;
|
||||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||||
import org.elasticsearch.search.slice.SliceBuilder;
|
import org.elasticsearch.search.slice.SliceBuilder;
|
||||||
import org.elasticsearch.tasks.TaskId;
|
import org.elasticsearch.tasks.TaskId;
|
||||||
import org.elasticsearch.tasks.TaskManager;
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Helps parallelize reindex requests using sliced scrolls.
|
* Helps parallelize reindex requests using sliced scrolls.
|
||||||
*/
|
*/
|
||||||
class BulkByScrollParallelizationHelper {
|
class BulkByScrollParallelizationHelper {
|
||||||
|
|
||||||
|
static final int AUTO_SLICE_CEILING = 20;
|
||||||
|
|
||||||
private BulkByScrollParallelizationHelper() {}
|
private BulkByScrollParallelizationHelper() {}
|
||||||
|
|
||||||
public static <Request extends AbstractBulkByScrollRequest<Request>> void startSlices(Client client, TaskManager taskManager,
|
/**
|
||||||
Action<Request, BulkByScrollResponse, ?> action,
|
* Takes an action created by a {@link BulkByScrollTask} and runs it with regard to whether the request is sliced or not.
|
||||||
String localNodeId, ParentBulkByScrollTask task, Request request,
|
*
|
||||||
ActionListener<BulkByScrollResponse> listener) {
|
* If the request is not sliced (i.e. the number of slices is 1), the worker action in the given {@link Runnable} will be started on
|
||||||
|
* the local node. If the request is sliced (i.e. the number of slices is more than 1), then a subrequest will be created for each
|
||||||
|
* slice and sent.
|
||||||
|
*
|
||||||
|
* If slices are set as {@code "auto"}, this class will resolve that to a specific number based on characteristics of the source
|
||||||
|
* indices. A request with {@code "auto"} slices may end up being sliced or unsliced.
|
||||||
|
*/
|
||||||
|
static <Request extends AbstractBulkByScrollRequest<Request>> void startSlicedAction(
|
||||||
|
Request request,
|
||||||
|
BulkByScrollTask task,
|
||||||
|
Action<Request, BulkByScrollResponse, ?> action,
|
||||||
|
ActionListener<BulkByScrollResponse> listener,
|
||||||
|
Client client,
|
||||||
|
DiscoveryNode node,
|
||||||
|
Runnable workerAction) {
|
||||||
|
|
||||||
|
if (request.getSlices() == AbstractBulkByScrollRequest.AUTO_SLICES) {
|
||||||
|
ClusterSearchShardsRequest shardsRequest = new ClusterSearchShardsRequest();
|
||||||
|
shardsRequest.indices(request.getSearchRequest().indices());
|
||||||
|
client.admin().cluster().searchShards(shardsRequest, ActionListener.wrap(
|
||||||
|
response -> {
|
||||||
|
int actualNumSlices = countSlicesBasedOnShards(response);
|
||||||
|
sliceConditionally(request, task, action, listener, client, node, workerAction, actualNumSlices);
|
||||||
|
},
|
||||||
|
listener::onFailure
|
||||||
|
));
|
||||||
|
} else {
|
||||||
|
sliceConditionally(request, task, action, listener, client, node, workerAction, request.getSlices());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <Request extends AbstractBulkByScrollRequest<Request>> void sliceConditionally(
|
||||||
|
Request request,
|
||||||
|
BulkByScrollTask task,
|
||||||
|
Action<Request, BulkByScrollResponse, ?> action,
|
||||||
|
ActionListener<BulkByScrollResponse> listener,
|
||||||
|
Client client,
|
||||||
|
DiscoveryNode node,
|
||||||
|
Runnable workerAction,
|
||||||
|
int slices) {
|
||||||
|
|
||||||
|
if (slices > 1) {
|
||||||
|
task.setWorkerCount(slices);
|
||||||
|
sendSubRequests(client, action, node.getId(), task, request, listener);
|
||||||
|
} else {
|
||||||
|
SliceBuilder sliceBuilder = request.getSearchRequest().source().slice();
|
||||||
|
Integer sliceId = sliceBuilder == null
|
||||||
|
? null
|
||||||
|
: sliceBuilder.getId();
|
||||||
|
task.setWorker(request.getRequestsPerSecond(), sliceId);
|
||||||
|
workerAction.run();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int countSlicesBasedOnShards(ClusterSearchShardsResponse response) {
|
||||||
|
Map<Index, Integer> countsByIndex = Arrays.stream(response.getGroups()).collect(Collectors.toMap(
|
||||||
|
group -> group.getShardId().getIndex(),
|
||||||
|
group -> 1,
|
||||||
|
(sum, term) -> sum + term
|
||||||
|
));
|
||||||
|
Set<Integer> counts = new HashSet<>(countsByIndex.values());
|
||||||
|
int leastShards = Collections.min(counts);
|
||||||
|
return Math.min(leastShards, AUTO_SLICE_CEILING);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static <Request extends AbstractBulkByScrollRequest<Request>> void sendSubRequests(
|
||||||
|
Client client,
|
||||||
|
Action<Request, BulkByScrollResponse, ?> action,
|
||||||
|
String localNodeId,
|
||||||
|
BulkByScrollTask task,
|
||||||
|
Request request,
|
||||||
|
ActionListener<BulkByScrollResponse> listener) {
|
||||||
|
|
||||||
|
LeaderBulkByScrollTaskState worker = task.getLeaderState();
|
||||||
|
int totalSlices = worker.getSlices();
|
||||||
TaskId parentTaskId = new TaskId(localNodeId, task.getId());
|
TaskId parentTaskId = new TaskId(localNodeId, task.getId());
|
||||||
for (final SearchRequest slice : sliceIntoSubRequests(request.getSearchRequest(), UidFieldMapper.NAME, request.getSlices())) {
|
for (final SearchRequest slice : sliceIntoSubRequests(request.getSearchRequest(), UidFieldMapper.NAME, totalSlices)) {
|
||||||
// TODO move the request to the correct node. maybe here or somehow do it as part of startup for reindex in general....
|
// 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);
|
Request requestForSlice = request.forSlice(parentTaskId, slice, totalSlices);
|
||||||
ActionListener<BulkByScrollResponse> sliceListener = ActionListener.wrap(
|
ActionListener<BulkByScrollResponse> sliceListener = ActionListener.wrap(
|
||||||
r -> task.onSliceResponse(listener, slice.source().slice().getId(), r),
|
r -> worker.onSliceResponse(listener, slice.source().slice().getId(), r),
|
||||||
e -> task.onSliceFailure(listener, slice.source().slice().getId(), e));
|
e -> worker.onSliceFailure(listener, slice.source().slice().getId(), e));
|
||||||
client.execute(action, requestForSlice, sliceListener);
|
client.execute(action, requestForSlice, sliceListener);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -80,5 +167,4 @@ class BulkByScrollParallelizationHelper {
|
||||||
}
|
}
|
||||||
return slices;
|
return slices;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,15 +51,17 @@ public class TransportDeleteByQueryAction extends HandledTransportAction<DeleteB
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void doExecute(Task task, DeleteByQueryRequest request, ActionListener<BulkByScrollResponse> listener) {
|
public void doExecute(Task task, DeleteByQueryRequest request, ActionListener<BulkByScrollResponse> listener) {
|
||||||
if (request.getSlices() > 1) {
|
BulkByScrollTask bulkByScrollTask = (BulkByScrollTask) task;
|
||||||
BulkByScrollParallelizationHelper.startSlices(client, taskManager, DeleteByQueryAction.INSTANCE,
|
BulkByScrollParallelizationHelper.startSlicedAction(request, bulkByScrollTask, DeleteByQueryAction.INSTANCE, listener, client,
|
||||||
clusterService.localNode().getId(), (ParentBulkByScrollTask) task, request, listener);
|
clusterService.localNode(),
|
||||||
} else {
|
() -> {
|
||||||
ClusterState state = clusterService.state();
|
ClusterState state = clusterService.state();
|
||||||
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
|
ParentTaskAssigningClient assigningClient = new ParentTaskAssigningClient(client, clusterService.localNode(),
|
||||||
new AsyncDeleteByQueryAction((WorkingBulkByScrollTask) task, logger, client, threadPool, request, scriptService, state,
|
bulkByScrollTask);
|
||||||
|
new AsyncDeleteByQueryAction(bulkByScrollTask, logger, assigningClient, threadPool, request, scriptService, state,
|
||||||
listener).start();
|
listener).start();
|
||||||
}
|
}
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -109,18 +109,22 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doExecute(Task task, ReindexRequest request, ActionListener<BulkByScrollResponse> listener) {
|
protected void doExecute(Task task, ReindexRequest request, ActionListener<BulkByScrollResponse> listener) {
|
||||||
if (request.getSlices() > 1) {
|
checkRemoteWhitelist(remoteWhitelist, request.getRemoteInfo());
|
||||||
BulkByScrollParallelizationHelper.startSlices(client, taskManager, ReindexAction.INSTANCE, clusterService.localNode().getId(),
|
ClusterState state = clusterService.state();
|
||||||
(ParentBulkByScrollTask) task, request, listener);
|
validateAgainstAliases(request.getSearchRequest(), request.getDestination(), request.getRemoteInfo(),
|
||||||
} else {
|
indexNameExpressionResolver, autoCreateIndex, state);
|
||||||
checkRemoteWhitelist(remoteWhitelist, request.getRemoteInfo());
|
|
||||||
ClusterState state = clusterService.state();
|
BulkByScrollTask bulkByScrollTask = (BulkByScrollTask) task;
|
||||||
validateAgainstAliases(request.getSearchRequest(), request.getDestination(), request.getRemoteInfo(),
|
|
||||||
indexNameExpressionResolver, autoCreateIndex, state);
|
BulkByScrollParallelizationHelper.startSlicedAction(request, bulkByScrollTask, ReindexAction.INSTANCE, listener, client,
|
||||||
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
|
clusterService.localNode(),
|
||||||
new AsyncIndexBySearchAction((WorkingBulkByScrollTask) task, logger, client, threadPool, request, scriptService, state,
|
() -> {
|
||||||
|
ParentTaskAssigningClient assigningClient = new ParentTaskAssigningClient(client, clusterService.localNode(),
|
||||||
|
bulkByScrollTask);
|
||||||
|
new AsyncIndexBySearchAction(bulkByScrollTask, logger, assigningClient, threadPool, request, scriptService, state,
|
||||||
listener).start();
|
listener).start();
|
||||||
}
|
}
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -244,13 +248,13 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||||
*/
|
*/
|
||||||
private List<Thread> createdThreads = emptyList();
|
private List<Thread> createdThreads = emptyList();
|
||||||
|
|
||||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, ReindexRequest request, ScriptService scriptService, ClusterState clusterState,
|
ThreadPool threadPool, ReindexRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||||
ActionListener<BulkByScrollResponse> listener) {
|
ActionListener<BulkByScrollResponse> listener) {
|
||||||
this(task, logger, client, threadPool, request, scriptService, clusterState, listener, client.settings());
|
this(task, logger, client, threadPool, request, scriptService, clusterState, listener, client.settings());
|
||||||
}
|
}
|
||||||
|
|
||||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, ReindexRequest request, ScriptService scriptService, ClusterState clusterState,
|
ThreadPool threadPool, ReindexRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||||
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
||||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener, settings);
|
super(task, logger, client, threadPool, request, scriptService, clusterState, listener, settings);
|
||||||
|
@ -271,8 +275,8 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||||
RemoteInfo remoteInfo = mainRequest.getRemoteInfo();
|
RemoteInfo remoteInfo = mainRequest.getRemoteInfo();
|
||||||
createdThreads = synchronizedList(new ArrayList<>());
|
createdThreads = synchronizedList(new ArrayList<>());
|
||||||
RestClient restClient = buildRestClient(remoteInfo, task.getId(), createdThreads);
|
RestClient restClient = buildRestClient(remoteInfo, task.getId(), createdThreads);
|
||||||
return new RemoteScrollableHitSource(logger, backoffPolicy, threadPool, task::countSearchRetry, this::finishHim, restClient,
|
return new RemoteScrollableHitSource(logger, backoffPolicy, threadPool, worker::countSearchRetry, this::finishHim,
|
||||||
remoteInfo.getQuery(), mainRequest.getSearchRequest());
|
restClient, remoteInfo.getQuery(), mainRequest.getSearchRequest());
|
||||||
}
|
}
|
||||||
return super.buildScrollableResultSource(backoffPolicy);
|
return super.buildScrollableResultSource(backoffPolicy);
|
||||||
}
|
}
|
||||||
|
@ -293,7 +297,7 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||||
public BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> buildScriptApplier() {
|
public BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> buildScriptApplier() {
|
||||||
Script script = mainRequest.getScript();
|
Script script = mainRequest.getScript();
|
||||||
if (script != null) {
|
if (script != null) {
|
||||||
return new ReindexScriptApplier(task, scriptService, script, script.getParams());
|
return new ReindexScriptApplier(worker, scriptService, script, script.getParams());
|
||||||
}
|
}
|
||||||
return super.buildScriptApplier();
|
return super.buildScriptApplier();
|
||||||
}
|
}
|
||||||
|
@ -385,9 +389,9 @@ public class TransportReindexAction extends HandledTransportAction<ReindexReques
|
||||||
|
|
||||||
class ReindexScriptApplier extends ScriptApplier {
|
class ReindexScriptApplier extends ScriptApplier {
|
||||||
|
|
||||||
ReindexScriptApplier(WorkingBulkByScrollTask task, ScriptService scriptService, Script script,
|
ReindexScriptApplier(WorkerBulkByScrollTaskState taskWorker, ScriptService scriptService, Script script,
|
||||||
Map<String, Object> params) {
|
Map<String, Object> params) {
|
||||||
super(task, scriptService, script, params);
|
super(taskWorker, scriptService, script, params);
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -59,21 +59,48 @@ public class TransportRethrottleAction extends TransportTasksAction<BulkByScroll
|
||||||
|
|
||||||
static void rethrottle(Logger logger, String localNodeId, Client client, BulkByScrollTask task, float newRequestsPerSecond,
|
static void rethrottle(Logger logger, String localNodeId, Client client, BulkByScrollTask task, float newRequestsPerSecond,
|
||||||
ActionListener<TaskInfo> listener) {
|
ActionListener<TaskInfo> listener) {
|
||||||
int runningSubTasks = task.runningSliceSubTasks();
|
|
||||||
if (runningSubTasks == 0) {
|
if (task.isWorker()) {
|
||||||
logger.debug("rethrottling local task [{}] to [{}] requests per second", task.getId(), newRequestsPerSecond);
|
rethrottleChildTask(logger, localNodeId, task, newRequestsPerSecond, listener);
|
||||||
task.rethrottle(newRequestsPerSecond);
|
|
||||||
listener.onResponse(task.taskInfo(localNodeId, true));
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
RethrottleRequest subRequest = new RethrottleRequest();
|
|
||||||
subRequest.setRequestsPerSecond(newRequestsPerSecond / runningSubTasks);
|
if (task.isLeader()) {
|
||||||
subRequest.setParentTaskId(new TaskId(localNodeId, task.getId()));
|
rethrottleParentTask(logger, localNodeId, client, task, newRequestsPerSecond, listener);
|
||||||
logger.debug("rethrottling children of task [{}] to [{}] requests per second", task.getId(), subRequest.getRequestsPerSecond());
|
return;
|
||||||
client.execute(RethrottleAction.INSTANCE, subRequest, ActionListener.wrap(r -> {
|
}
|
||||||
r.rethrowFailures("Rethrottle");
|
|
||||||
listener.onResponse(task.getInfoGivenSliceInfo(localNodeId, r.getTasks()));
|
throw new IllegalArgumentException("task [" + task.getId() + "] must be set as a child or parent");
|
||||||
}, listener::onFailure));
|
}
|
||||||
|
|
||||||
|
private static void rethrottleParentTask(Logger logger, String localNodeId, Client client, BulkByScrollTask task,
|
||||||
|
float newRequestsPerSecond, ActionListener<TaskInfo> listener) {
|
||||||
|
final LeaderBulkByScrollTaskState leaderState = task.getLeaderState();
|
||||||
|
final int runningSubtasks = leaderState.runningSliceSubTasks();
|
||||||
|
|
||||||
|
if (runningSubtasks > 0) {
|
||||||
|
RethrottleRequest subRequest = new RethrottleRequest();
|
||||||
|
subRequest.setRequestsPerSecond(newRequestsPerSecond / runningSubtasks);
|
||||||
|
subRequest.setParentTaskId(new TaskId(localNodeId, task.getId()));
|
||||||
|
logger.debug("rethrottling children of task [{}] to [{}] requests per second", task.getId(),
|
||||||
|
subRequest.getRequestsPerSecond());
|
||||||
|
client.execute(RethrottleAction.INSTANCE, subRequest, ActionListener.wrap(
|
||||||
|
r -> {
|
||||||
|
r.rethrowFailures("Rethrottle");
|
||||||
|
listener.onResponse(task.taskInfoGivenSubtaskInfo(localNodeId, r.getTasks()));
|
||||||
|
},
|
||||||
|
listener::onFailure));
|
||||||
|
} else {
|
||||||
|
logger.debug("children of task [{}] are already finished, nothing to rethrottle", task.getId());
|
||||||
|
listener.onResponse(task.taskInfo(localNodeId, true));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void rethrottleChildTask(Logger logger, String localNodeId, BulkByScrollTask task, float newRequestsPerSecond,
|
||||||
|
ActionListener<TaskInfo> listener) {
|
||||||
|
logger.debug("rethrottling local task [{}] to [{}] requests per second", task.getId(), newRequestsPerSecond);
|
||||||
|
task.getWorkerState().rethrottle(newRequestsPerSecond);
|
||||||
|
listener.onResponse(task.taskInfo(localNodeId, true));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -64,15 +64,17 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void doExecute(Task task, UpdateByQueryRequest request, ActionListener<BulkByScrollResponse> listener) {
|
protected void doExecute(Task task, UpdateByQueryRequest request, ActionListener<BulkByScrollResponse> listener) {
|
||||||
if (request.getSlices() > 1) {
|
BulkByScrollTask bulkByScrollTask = (BulkByScrollTask) task;
|
||||||
BulkByScrollParallelizationHelper.startSlices(client, taskManager, UpdateByQueryAction.INSTANCE,
|
BulkByScrollParallelizationHelper.startSlicedAction(request, bulkByScrollTask, UpdateByQueryAction.INSTANCE, listener, client,
|
||||||
clusterService.localNode().getId(), (ParentBulkByScrollTask) task, request, listener);
|
clusterService.localNode(),
|
||||||
} else {
|
() -> {
|
||||||
ClusterState state = clusterService.state();
|
ClusterState state = clusterService.state();
|
||||||
ParentTaskAssigningClient client = new ParentTaskAssigningClient(this.client, clusterService.localNode(), task);
|
ParentTaskAssigningClient assigningClient = new ParentTaskAssigningClient(client, clusterService.localNode(),
|
||||||
new AsyncIndexBySearchAction((WorkingBulkByScrollTask) task, logger, client, threadPool, request, scriptService, state,
|
bulkByScrollTask);
|
||||||
|
new AsyncIndexBySearchAction(bulkByScrollTask, logger, assigningClient, threadPool, request, scriptService, state,
|
||||||
listener).start();
|
listener).start();
|
||||||
}
|
}
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -84,13 +86,13 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
|
||||||
* Simple implementation of update-by-query using scrolling and bulk.
|
* Simple implementation of update-by-query using scrolling and bulk.
|
||||||
*/
|
*/
|
||||||
static class AsyncIndexBySearchAction extends AbstractAsyncBulkByScrollAction<UpdateByQueryRequest> {
|
static class AsyncIndexBySearchAction extends AbstractAsyncBulkByScrollAction<UpdateByQueryRequest> {
|
||||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, UpdateByQueryRequest request, ScriptService scriptService, ClusterState clusterState,
|
ThreadPool threadPool, UpdateByQueryRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||||
ActionListener<BulkByScrollResponse> listener) {
|
ActionListener<BulkByScrollResponse> listener) {
|
||||||
this(task, logger, client, threadPool, request, scriptService, clusterState, listener, client.settings());
|
this(task, logger, client, threadPool, request, scriptService, clusterState, listener, client.settings());
|
||||||
}
|
}
|
||||||
|
|
||||||
AsyncIndexBySearchAction(WorkingBulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
AsyncIndexBySearchAction(BulkByScrollTask task, Logger logger, ParentTaskAssigningClient client,
|
||||||
ThreadPool threadPool, UpdateByQueryRequest request, ScriptService scriptService, ClusterState clusterState,
|
ThreadPool threadPool, UpdateByQueryRequest request, ScriptService scriptService, ClusterState clusterState,
|
||||||
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
ActionListener<BulkByScrollResponse> listener, Settings settings) {
|
||||||
super(task, logger, client, threadPool, request, scriptService, clusterState, listener, settings);
|
super(task, logger, client, threadPool, request, scriptService, clusterState, listener, settings);
|
||||||
|
@ -109,7 +111,7 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
|
||||||
public BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> buildScriptApplier() {
|
public BiFunction<RequestWrapper<?>, ScrollableHitSource.Hit, RequestWrapper<?>> buildScriptApplier() {
|
||||||
Script script = mainRequest.getScript();
|
Script script = mainRequest.getScript();
|
||||||
if (script != null) {
|
if (script != null) {
|
||||||
return new UpdateByQueryScriptApplier(task, scriptService, script, script.getParams());
|
return new UpdateByQueryScriptApplier(worker, scriptService, script, script.getParams());
|
||||||
}
|
}
|
||||||
return super.buildScriptApplier();
|
return super.buildScriptApplier();
|
||||||
}
|
}
|
||||||
|
@ -129,9 +131,9 @@ public class TransportUpdateByQueryAction extends HandledTransportAction<UpdateB
|
||||||
|
|
||||||
class UpdateByQueryScriptApplier extends ScriptApplier {
|
class UpdateByQueryScriptApplier extends ScriptApplier {
|
||||||
|
|
||||||
UpdateByQueryScriptApplier(WorkingBulkByScrollTask task, ScriptService scriptService, Script script,
|
UpdateByQueryScriptApplier(WorkerBulkByScrollTaskState taskWorker, ScriptService scriptService, Script script,
|
||||||
Map<String, Object> params) {
|
Map<String, Object> params) {
|
||||||
super(task, scriptService, script, params);
|
super(taskWorker, scriptService, script, params);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -124,7 +124,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
private PlainActionFuture<BulkByScrollResponse> listener;
|
private PlainActionFuture<BulkByScrollResponse> listener;
|
||||||
private String scrollId;
|
private String scrollId;
|
||||||
private TaskManager taskManager;
|
private TaskManager taskManager;
|
||||||
private WorkingBulkByScrollTask testTask;
|
private BulkByScrollTask testTask;
|
||||||
|
private WorkerBulkByScrollTaskState worker;
|
||||||
private Map<String, String> expectedHeaders = new HashMap<>();
|
private Map<String, String> expectedHeaders = new HashMap<>();
|
||||||
private DiscoveryNode localNode;
|
private DiscoveryNode localNode;
|
||||||
private TaskId taskId;
|
private TaskId taskId;
|
||||||
|
@ -141,7 +142,9 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
listener = new PlainActionFuture<>();
|
listener = new PlainActionFuture<>();
|
||||||
scrollId = null;
|
scrollId = null;
|
||||||
taskManager = new TaskManager(Settings.EMPTY);
|
taskManager = new TaskManager(Settings.EMPTY);
|
||||||
testTask = (WorkingBulkByScrollTask) taskManager.register("don'tcare", "hereeither", testRequest);
|
testTask = (BulkByScrollTask) taskManager.register("don'tcare", "hereeither", testRequest);
|
||||||
|
testTask.setWorker(testRequest.getRequestsPerSecond(), null);
|
||||||
|
worker = testTask.getWorkerState();
|
||||||
|
|
||||||
localNode = new DiscoveryNode("thenode", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
|
localNode = new DiscoveryNode("thenode", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
|
||||||
taskId = new TaskId(localNode.getId(), testTask.getId());
|
taskId = new TaskId(localNode.getId(), testTask.getId());
|
||||||
|
@ -309,7 +312,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
* Mimicks a ThreadPool rejecting execution of the task.
|
* Mimicks a ThreadPool rejecting execution of the task.
|
||||||
*/
|
*/
|
||||||
public void testThreadPoolRejectionsAbortRequest() throws Exception {
|
public void testThreadPoolRejectionsAbortRequest() throws Exception {
|
||||||
testTask.rethrottle(1);
|
worker.rethrottle(1);
|
||||||
setupClient(new TestThreadPool(getTestName()) {
|
setupClient(new TestThreadPool(getTestName()) {
|
||||||
@Override
|
@Override
|
||||||
public ScheduledFuture<?> schedule(TimeValue delay, String name, Runnable command) {
|
public ScheduledFuture<?> schedule(TimeValue delay, String name, Runnable command) {
|
||||||
|
@ -439,7 +442,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
firstSearchRequest.scroll(timeValueSeconds(10));
|
firstSearchRequest.scroll(timeValueSeconds(10));
|
||||||
|
|
||||||
// Set throttle to 1 request per second to make the math simpler
|
// Set throttle to 1 request per second to make the math simpler
|
||||||
testTask.rethrottle(1f);
|
worker.rethrottle(1f);
|
||||||
// Make the last batch look nearly instant but have 100 documents
|
// Make the last batch look nearly instant but have 100 documents
|
||||||
TimeValue lastBatchStartTime = timeValueNanos(System.nanoTime());
|
TimeValue lastBatchStartTime = timeValueNanos(System.nanoTime());
|
||||||
TimeValue now = timeValueNanos(lastBatchStartTime.nanos() + 1);
|
TimeValue now = timeValueNanos(lastBatchStartTime.nanos() + 1);
|
||||||
|
@ -459,7 +462,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
assertEquals(99, capturedDelay.get().seconds());
|
assertEquals(99, capturedDelay.get().seconds());
|
||||||
} else {
|
} else {
|
||||||
// Let's rethrottle between the starting the scroll and getting the response
|
// Let's rethrottle between the starting the scroll and getting the response
|
||||||
testTask.rethrottle(10f);
|
worker.rethrottle(10f);
|
||||||
client.lastScroll.get().listener.onResponse(searchResponse);
|
client.lastScroll.get().listener.onResponse(searchResponse);
|
||||||
// The delay uses the new throttle
|
// The delay uses the new throttle
|
||||||
assertEquals(9, capturedDelay.get().seconds());
|
assertEquals(9, capturedDelay.get().seconds());
|
||||||
|
@ -624,7 +627,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
long total = randomIntBetween(0, Integer.MAX_VALUE);
|
long total = randomIntBetween(0, Integer.MAX_VALUE);
|
||||||
ScrollableHitSource.Response response = new ScrollableHitSource.Response(false, emptyList(), total, emptyList(), null);
|
ScrollableHitSource.Response response = new ScrollableHitSource.Response(false, emptyList(), total, emptyList(), null);
|
||||||
// Use a long delay here so the test will time out if the cancellation doesn't reschedule the throttled task
|
// Use a long delay here so the test will time out if the cancellation doesn't reschedule the throttled task
|
||||||
testTask.rethrottle(1);
|
worker.rethrottle(1);
|
||||||
simulateScrollResponse(action, timeValueNanos(System.nanoTime()), 1000, response);
|
simulateScrollResponse(action, timeValueNanos(System.nanoTime()), 1000, response);
|
||||||
|
|
||||||
// Now that we've got our cancel we'll just verify that it all came through all right
|
// Now that we've got our cancel we'll just verify that it all came through all right
|
||||||
|
@ -694,7 +697,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DummyAbstractBulkByScrollRequest forSlice(TaskId slicingTask, SearchRequest slice) {
|
public DummyAbstractBulkByScrollRequest forSlice(TaskId slicingTask, SearchRequest slice, int totalSlices) {
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.admin.indices.alias.Alias;
|
import org.elasticsearch.action.admin.indices.alias.Alias;
|
||||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
|
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
import org.elasticsearch.index.query.QueryBuilders;
|
import org.elasticsearch.index.query.QueryBuilders;
|
||||||
|
@ -31,7 +30,10 @@ import org.elasticsearch.test.InternalSettingsPlugin;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY;
|
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY;
|
||||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE;
|
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE;
|
||||||
|
@ -224,7 +226,7 @@ public class DeleteByQueryBasicTests extends ReindexTestCase {
|
||||||
assertHitCount(client().prepareSearch("test").setSize(0).get(), docs);
|
assertHitCount(client().prepareSearch("test").setSize(0).get(), docs);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWorkers() throws Exception {
|
public void testSlices() throws Exception {
|
||||||
indexRandom(true,
|
indexRandom(true,
|
||||||
client().prepareIndex("test", "test", "1").setSource("foo", "a"),
|
client().prepareIndex("test", "test", "1").setSource("foo", "a"),
|
||||||
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
|
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
|
||||||
|
@ -236,18 +238,74 @@ public class DeleteByQueryBasicTests extends ReindexTestCase {
|
||||||
);
|
);
|
||||||
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 7);
|
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 7);
|
||||||
|
|
||||||
|
int slices = randomSlices();
|
||||||
|
int expectedSlices = expectedSliceStatuses(slices, "test");
|
||||||
|
|
||||||
// Deletes the two docs that matches "foo:a"
|
// Deletes the two docs that matches "foo:a"
|
||||||
assertThat(deleteByQuery().source("test").filter(termQuery("foo", "a")).refresh(true).setSlices(5).get(),
|
assertThat(
|
||||||
matcher().deleted(2).slices(hasSize(5)));
|
deleteByQuery()
|
||||||
|
.source("test")
|
||||||
|
.filter(termQuery("foo", "a"))
|
||||||
|
.refresh(true)
|
||||||
|
.setSlices(slices).get(),
|
||||||
|
matcher()
|
||||||
|
.deleted(2)
|
||||||
|
.slices(hasSize(expectedSlices)));
|
||||||
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 5);
|
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 5);
|
||||||
|
|
||||||
// Delete remaining docs
|
// Delete remaining docs
|
||||||
DeleteByQueryRequestBuilder request = deleteByQuery().source("test").filter(QueryBuilders.matchAllQuery()).refresh(true)
|
assertThat(
|
||||||
.setSlices(5);
|
deleteByQuery()
|
||||||
assertThat(request.get(), matcher().deleted(5).slices(hasSize(5)));
|
.source("test")
|
||||||
|
.filter(QueryBuilders.matchAllQuery())
|
||||||
|
.refresh(true)
|
||||||
|
.setSlices(slices).get(),
|
||||||
|
matcher()
|
||||||
|
.deleted(5)
|
||||||
|
.slices(hasSize(expectedSlices)));
|
||||||
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 0);
|
assertHitCount(client().prepareSearch("test").setTypes("test").setSize(0).get(), 0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMultipleSources() throws Exception {
|
||||||
|
int sourceIndices = between(2, 5);
|
||||||
|
|
||||||
|
Map<String, List<IndexRequestBuilder>> docs = new HashMap<>();
|
||||||
|
for (int sourceIndex = 0; sourceIndex < sourceIndices; sourceIndex++) {
|
||||||
|
String indexName = "test" + sourceIndex;
|
||||||
|
docs.put(indexName, new ArrayList<>());
|
||||||
|
int numDocs = between(5, 15);
|
||||||
|
for (int i = 0; i < numDocs; i++) {
|
||||||
|
docs.get(indexName).add(client().prepareIndex(indexName, "test", Integer.toString(i)).setSource("foo", "a"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
List<IndexRequestBuilder> allDocs = docs.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
|
||||||
|
indexRandom(true, allDocs);
|
||||||
|
for (Map.Entry<String, List<IndexRequestBuilder>> entry : docs.entrySet()) {
|
||||||
|
assertHitCount(client().prepareSearch(entry.getKey()).setSize(0).get(), entry.getValue().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
int slices = randomSlices(1, 10);
|
||||||
|
int expectedSlices = expectedSliceStatuses(slices, docs.keySet());
|
||||||
|
|
||||||
|
String[] sourceIndexNames = docs.keySet().toArray(new String[docs.size()]);
|
||||||
|
|
||||||
|
assertThat(
|
||||||
|
deleteByQuery()
|
||||||
|
.source(sourceIndexNames)
|
||||||
|
.filter(QueryBuilders.matchAllQuery())
|
||||||
|
.refresh(true)
|
||||||
|
.setSlices(slices).get(),
|
||||||
|
matcher()
|
||||||
|
.deleted(allDocs.size())
|
||||||
|
.slices(hasSize(expectedSlices)));
|
||||||
|
|
||||||
|
for (String index : docs.keySet()) {
|
||||||
|
assertHitCount(client().prepareSearch(index).setTypes("test").setSize(0).get(), 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test delete by query support for filtering by type. This entire feature
|
* Test delete by query support for filtering by type. This entire feature
|
||||||
* can and should be removed when we drop support for types index with
|
* can and should be removed when we drop support for types index with
|
||||||
|
|
|
@ -22,7 +22,11 @@ package org.elasticsearch.index.reindex;
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
@ -88,8 +92,6 @@ public class ReindexBasicTests extends ReindexTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testCopyManyWithSlices() throws Exception {
|
public void testCopyManyWithSlices() throws Exception {
|
||||||
int workers = between(2, 10);
|
|
||||||
|
|
||||||
List<IndexRequestBuilder> docs = new ArrayList<>();
|
List<IndexRequestBuilder> docs = new ArrayList<>();
|
||||||
int max = between(150, 500);
|
int max = between(150, 500);
|
||||||
for (int i = 0; i < max; i++) {
|
for (int i = 0; i < max; i++) {
|
||||||
|
@ -99,21 +101,61 @@ public class ReindexBasicTests extends ReindexTestCase {
|
||||||
indexRandom(true, docs);
|
indexRandom(true, docs);
|
||||||
assertHitCount(client().prepareSearch("source").setSize(0).get(), max);
|
assertHitCount(client().prepareSearch("source").setSize(0).get(), max);
|
||||||
|
|
||||||
|
int slices = randomSlices();
|
||||||
|
int expectedSlices = expectedSliceStatuses(slices, "source");
|
||||||
|
|
||||||
// Copy all the docs
|
// Copy all the docs
|
||||||
ReindexRequestBuilder copy = reindex().source("source").destination("dest", "type").refresh(true).setSlices(workers);
|
ReindexRequestBuilder copy = reindex().source("source").destination("dest", "type").refresh(true).setSlices(slices);
|
||||||
// Use a small batch size so we have to use more than one batch
|
// Use a small batch size so we have to use more than one batch
|
||||||
copy.source().setSize(5);
|
copy.source().setSize(5);
|
||||||
assertThat(copy.get(), matcher().created(max).batches(greaterThanOrEqualTo(max / 5)).slices(hasSize(workers)));
|
assertThat(copy.get(), matcher().created(max).batches(greaterThanOrEqualTo(max / 5)).slices(hasSize(expectedSlices)));
|
||||||
assertHitCount(client().prepareSearch("dest").setTypes("type").setSize(0).get(), max);
|
assertHitCount(client().prepareSearch("dest").setTypes("type").setSize(0).get(), max);
|
||||||
|
|
||||||
// Copy some of the docs
|
// Copy some of the docs
|
||||||
int half = max / 2;
|
int half = max / 2;
|
||||||
copy = reindex().source("source").destination("dest_half", "type").refresh(true).setSlices(workers);
|
copy = reindex().source("source").destination("dest_half", "type").refresh(true).setSlices(slices);
|
||||||
// Use a small batch size so we have to use more than one batch
|
// Use a small batch size so we have to use more than one batch
|
||||||
copy.source().setSize(5);
|
copy.source().setSize(5);
|
||||||
copy.size(half); // The real "size" of the request.
|
copy.size(half); // The real "size" of the request.
|
||||||
BulkByScrollResponse response = copy.get();
|
BulkByScrollResponse response = copy.get();
|
||||||
assertThat(response, matcher().created(lessThanOrEqualTo((long) half)).slices(hasSize(workers)));
|
assertThat(response, matcher().created(lessThanOrEqualTo((long) half)).slices(hasSize(expectedSlices)));
|
||||||
assertHitCount(client().prepareSearch("dest_half").setSize(0).get(), response.getCreated());
|
assertHitCount(client().prepareSearch("dest_half").setSize(0).get(), response.getCreated());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMultipleSources() throws Exception {
|
||||||
|
int sourceIndices = between(2, 5);
|
||||||
|
|
||||||
|
Map<String, List<IndexRequestBuilder>> docs = new HashMap<>();
|
||||||
|
for (int sourceIndex = 0; sourceIndex < sourceIndices; sourceIndex++) {
|
||||||
|
String indexName = "source" + sourceIndex;
|
||||||
|
String typeName = "test" + sourceIndex;
|
||||||
|
docs.put(indexName, new ArrayList<>());
|
||||||
|
int numDocs = between(50, 200);
|
||||||
|
for (int i = 0; i < numDocs; i++) {
|
||||||
|
docs.get(indexName).add(client().prepareIndex(indexName, typeName, "id_" + sourceIndex + "_" + i).setSource("foo", "a"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
List<IndexRequestBuilder> allDocs = docs.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
|
||||||
|
indexRandom(true, allDocs);
|
||||||
|
for (Map.Entry<String, List<IndexRequestBuilder>> entry : docs.entrySet()) {
|
||||||
|
assertHitCount(client().prepareSearch(entry.getKey()).setSize(0).get(), entry.getValue().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
int slices = randomSlices(1, 10);
|
||||||
|
int expectedSlices = expectedSliceStatuses(slices, docs.keySet());
|
||||||
|
|
||||||
|
String[] sourceIndexNames = docs.keySet().toArray(new String[docs.size()]);
|
||||||
|
ReindexRequestBuilder request = reindex()
|
||||||
|
.source(sourceIndexNames)
|
||||||
|
.destination("dest", "type")
|
||||||
|
.refresh(true)
|
||||||
|
.setSlices(slices);
|
||||||
|
|
||||||
|
BulkByScrollResponse response = request.get();
|
||||||
|
assertThat(response, matcher().created(allDocs.size()).slices(hasSize(expectedSlices)));
|
||||||
|
assertHitCount(client().prepareSearch("dest").setSize(0).get(), allDocs.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,10 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static java.util.Collections.singleton;
|
||||||
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
|
import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -62,4 +65,51 @@ public abstract class ReindexTestCase extends ESIntegTestCase {
|
||||||
public static BulkIndexByScrollResponseMatcher matcher() {
|
public static BulkIndexByScrollResponseMatcher matcher() {
|
||||||
return new BulkIndexByScrollResponseMatcher();
|
return new BulkIndexByScrollResponseMatcher();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static int randomSlices(int min, int max) {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
return AbstractBulkByScrollRequest.AUTO_SLICES;
|
||||||
|
} else {
|
||||||
|
return between(min, max);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static int randomSlices() {
|
||||||
|
return randomSlices(2, 10);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Figures out how many slices the request handling will use
|
||||||
|
*/
|
||||||
|
protected int expectedSlices(int requestSlices, Collection<String> indices) {
|
||||||
|
if (requestSlices == AbstractBulkByScrollRequest.AUTO_SLICES) {
|
||||||
|
int leastNumShards = Collections.min(indices.stream()
|
||||||
|
.map(sourceIndex -> getNumShards(sourceIndex).numPrimaries)
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
return Math.min(leastNumShards, BulkByScrollParallelizationHelper.AUTO_SLICE_CEILING);
|
||||||
|
} else {
|
||||||
|
return requestSlices;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int expectedSlices(int requestSlices, String index) {
|
||||||
|
return expectedSlices(requestSlices, singleton(index));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Figures out how many slice statuses to expect in the response
|
||||||
|
*/
|
||||||
|
protected int expectedSliceStatuses(int requestSlices, Collection<String> indices) {
|
||||||
|
int slicesConfigured = expectedSlices(requestSlices, indices);
|
||||||
|
|
||||||
|
if (slicesConfigured > 1) {
|
||||||
|
return slicesConfigured;
|
||||||
|
} else {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
protected int expectedSliceStatuses(int slicesConfigured, String index) {
|
||||||
|
return expectedSliceStatuses(slicesConfigured, singleton(index));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -60,15 +60,15 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testReindexWithWorkers() throws Exception {
|
public void testReindexWithWorkers() throws Exception {
|
||||||
testCase(reindex().source("test").destination("dest").setSlices(between(2, 10)), ReindexAction.NAME);
|
testCase(reindex().source("test").destination("dest").setSlices(randomSlices()), ReindexAction.NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testUpdateByQueryWithWorkers() throws Exception {
|
public void testUpdateByQueryWithWorkers() throws Exception {
|
||||||
testCase(updateByQuery().source("test").setSlices(between(2, 10)), UpdateByQueryAction.NAME);
|
testCase(updateByQuery().source("test").setSlices(randomSlices()), UpdateByQueryAction.NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testDeleteByQueryWithWorkers() throws Exception {
|
public void testDeleteByQueryWithWorkers() throws Exception {
|
||||||
testCase(deleteByQuery().source("test").filter(QueryBuilders.matchAllQuery()).setSlices(between(2, 10)), DeleteByQueryAction.NAME);
|
testCase(deleteByQuery().source("test").filter(QueryBuilders.matchAllQuery()).setSlices(randomSlices()), DeleteByQueryAction.NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void testCase(AbstractBulkByScrollRequestBuilder<?, ?> request, String actionName) throws Exception {
|
private void testCase(AbstractBulkByScrollRequestBuilder<?, ?> request, String actionName) throws Exception {
|
||||||
|
@ -76,8 +76,12 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
/* Add ten documents per slice so most slices will have many documents to process, having to go to multiple batches.
|
/* 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
|
* we can't rely on all of them doing so, but
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
createIndex("test");
|
||||||
|
int numSlices = expectedSlices(request.request().getSlices(), "test");
|
||||||
|
|
||||||
List<IndexRequestBuilder> docs = new ArrayList<>();
|
List<IndexRequestBuilder> docs = new ArrayList<>();
|
||||||
for (int i = 0; i < request.request().getSlices() * 10; i++) {
|
for (int i = 0; i < numSlices * 10; i++) {
|
||||||
docs.add(client().prepareIndex("test", "test", Integer.toString(i)).setSource("foo", "bar"));
|
docs.add(client().prepareIndex("test", "test", Integer.toString(i)).setSource("foo", "bar"));
|
||||||
}
|
}
|
||||||
indexRandom(true, docs);
|
indexRandom(true, docs);
|
||||||
|
@ -87,15 +91,15 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
request.source().setSize(1); // Make sure we use multiple batches
|
request.source().setSize(1); // Make sure we use multiple batches
|
||||||
ActionFuture<? extends BulkByScrollResponse> responseListener = request.execute();
|
ActionFuture<? extends BulkByScrollResponse> responseListener = request.execute();
|
||||||
|
|
||||||
TaskGroup taskGroupToRethrottle = findTaskToRethrottle(actionName, request.request().getSlices());
|
TaskGroup taskGroupToRethrottle = findTaskToRethrottle(actionName, numSlices);
|
||||||
TaskId taskToRethrottle = taskGroupToRethrottle.getTaskInfo().getTaskId();
|
TaskId taskToRethrottle = taskGroupToRethrottle.getTaskInfo().getTaskId();
|
||||||
|
|
||||||
if (request.request().getSlices() == 1) {
|
if (numSlices == 1) {
|
||||||
assertThat(taskGroupToRethrottle.getChildTasks(), empty());
|
assertThat(taskGroupToRethrottle.getChildTasks(), empty());
|
||||||
} else {
|
} else {
|
||||||
// There should be a sane number of child tasks running
|
// There should be a sane number of child tasks running
|
||||||
assertThat(taskGroupToRethrottle.getChildTasks(),
|
assertThat(taskGroupToRethrottle.getChildTasks(),
|
||||||
hasSize(allOf(greaterThanOrEqualTo(1), lessThanOrEqualTo(request.request().getSlices()))));
|
hasSize(allOf(greaterThanOrEqualTo(1), lessThanOrEqualTo(numSlices))));
|
||||||
// Wait for all of the sub tasks to start (or finish, some might finish early, all that matters is that not all do)
|
// Wait for all of the sub tasks to start (or finish, some might finish early, all that matters is that not all do)
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
BulkByScrollTask.Status parent = (BulkByScrollTask.Status) client().admin().cluster().prepareGetTask(taskToRethrottle).get()
|
BulkByScrollTask.Status parent = (BulkByScrollTask.Status) client().admin().cluster().prepareGetTask(taskToRethrottle).get()
|
||||||
|
@ -103,7 +107,7 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
long finishedSubTasks = parent.getSliceStatuses().stream().filter(Objects::nonNull).count();
|
long finishedSubTasks = parent.getSliceStatuses().stream().filter(Objects::nonNull).count();
|
||||||
ListTasksResponse list = client().admin().cluster().prepareListTasks().setParentTaskId(taskToRethrottle).get();
|
ListTasksResponse list = client().admin().cluster().prepareListTasks().setParentTaskId(taskToRethrottle).get();
|
||||||
list.rethrowFailures("subtasks");
|
list.rethrowFailures("subtasks");
|
||||||
assertThat(finishedSubTasks + list.getTasks().size(), greaterThanOrEqualTo((long) request.request().getSlices()));
|
assertThat(finishedSubTasks + list.getTasks().size(), greaterThanOrEqualTo((long) numSlices));
|
||||||
assertThat(list.getTasks().size(), greaterThan(0));
|
assertThat(list.getTasks().size(), greaterThan(0));
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -114,8 +118,9 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
rethrottleResponse.rethrowFailures("Rethrottle");
|
rethrottleResponse.rethrowFailures("Rethrottle");
|
||||||
assertThat(rethrottleResponse.getTasks(), hasSize(1));
|
assertThat(rethrottleResponse.getTasks(), hasSize(1));
|
||||||
BulkByScrollTask.Status status = (BulkByScrollTask.Status) rethrottleResponse.getTasks().get(0).getStatus();
|
BulkByScrollTask.Status status = (BulkByScrollTask.Status) rethrottleResponse.getTasks().get(0).getStatus();
|
||||||
|
|
||||||
// Now check the resulting requests per second.
|
// Now check the resulting requests per second.
|
||||||
if (request.request().getSlices() == 1) {
|
if (numSlices == 1) {
|
||||||
// If there is a single slice it should match perfectly
|
// If there is a single slice it should match perfectly
|
||||||
assertEquals(newRequestsPerSecond, status.getRequestsPerSecond(), Float.MIN_NORMAL);
|
assertEquals(newRequestsPerSecond, status.getRequestsPerSecond(), Float.MIN_NORMAL);
|
||||||
} else {
|
} else {
|
||||||
|
@ -128,7 +133,7 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
float maxExpectedSliceRequestsPerSecond = newRequestsPerSecond == Float.POSITIVE_INFINITY ?
|
float maxExpectedSliceRequestsPerSecond = newRequestsPerSecond == Float.POSITIVE_INFINITY ?
|
||||||
Float.POSITIVE_INFINITY : (newRequestsPerSecond / unfinished) * 1.01F;
|
Float.POSITIVE_INFINITY : (newRequestsPerSecond / unfinished) * 1.01F;
|
||||||
float minExpectedSliceRequestsPerSecond = newRequestsPerSecond == Float.POSITIVE_INFINITY ?
|
float minExpectedSliceRequestsPerSecond = newRequestsPerSecond == Float.POSITIVE_INFINITY ?
|
||||||
Float.POSITIVE_INFINITY : (newRequestsPerSecond / request.request().getSlices()) * 0.99F;
|
Float.POSITIVE_INFINITY : (newRequestsPerSecond / numSlices) * 0.99F;
|
||||||
boolean oneSliceRethrottled = false;
|
boolean oneSliceRethrottled = false;
|
||||||
float totalRequestsPerSecond = 0;
|
float totalRequestsPerSecond = 0;
|
||||||
for (BulkByScrollTask.StatusOrException statusOrException : status.getSliceStatuses()) {
|
for (BulkByScrollTask.StatusOrException statusOrException : status.getSliceStatuses()) {
|
||||||
|
@ -164,7 +169,7 @@ public class RethrottleTests extends ReindexTestCase {
|
||||||
// Now the response should come back quickly because we've rethrottled the request
|
// Now the response should come back quickly because we've rethrottled the request
|
||||||
BulkByScrollResponse response = responseListener.get();
|
BulkByScrollResponse response = responseListener.get();
|
||||||
assertThat("Entire request completed in a single batch. This may invalidate the test as throttling is done between batches.",
|
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()));
|
response.getBatches(), greaterThanOrEqualTo(numSlices));
|
||||||
}
|
}
|
||||||
|
|
||||||
private TaskGroup findTaskToRethrottle(String actionName, int sliceCount) {
|
private TaskGroup findTaskToRethrottle(String actionName, int sliceCount) {
|
||||||
|
|
|
@ -70,17 +70,16 @@ public class RoundTripTests extends ESTestCase {
|
||||||
roundTrip(reindex, tripped);
|
roundTrip(reindex, tripped);
|
||||||
assertRequestEquals(reindex, tripped);
|
assertRequestEquals(reindex, tripped);
|
||||||
|
|
||||||
// Try slices with a version that doesn't support slices. That should fail.
|
// Try slices=auto with a version that doesn't support it, which should fail
|
||||||
reindex.setSlices(between(2, 1000));
|
reindex.setSlices(AbstractBulkByScrollRequest.AUTO_SLICES);
|
||||||
Exception e = expectThrows(IllegalArgumentException.class, () -> roundTrip(Version.V_5_0_0_rc1, reindex, null));
|
Exception e = expectThrows(IllegalArgumentException.class, () -> roundTrip(Version.V_6_0_0_alpha1, reindex, null));
|
||||||
assertEquals("Attempting to send sliced reindex-style request to a node that doesn't support it. "
|
assertEquals("Slices set as \"auto\" are not supported before version [6.1.0]. Found version [6.0.0-alpha1]", e.getMessage());
|
||||||
+ "Version is [5.0.0-rc1] but must be [5.1.1]", e.getMessage());
|
|
||||||
|
|
||||||
// Try without slices with a version that doesn't support slices. That should work.
|
// Try regular slices with a version that doesn't support slices=auto, which should succeed
|
||||||
tripped = new ReindexRequest();
|
tripped = new ReindexRequest();
|
||||||
reindex.setSlices(1);
|
reindex.setSlices(between(1, Integer.MAX_VALUE));
|
||||||
roundTrip(Version.V_5_0_0_rc1, reindex, tripped);
|
roundTrip(Version.V_6_0_0_alpha1, reindex, tripped);
|
||||||
assertRequestEquals(Version.V_5_0_0_rc1, reindex, tripped);
|
assertRequestEquals(Version.V_6_0_0_alpha1, reindex, tripped);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testUpdateByQueryRequest() throws IOException {
|
public void testUpdateByQueryRequest() throws IOException {
|
||||||
|
@ -94,16 +93,15 @@ public class RoundTripTests extends ESTestCase {
|
||||||
assertRequestEquals(update, tripped);
|
assertRequestEquals(update, tripped);
|
||||||
assertEquals(update.getPipeline(), tripped.getPipeline());
|
assertEquals(update.getPipeline(), tripped.getPipeline());
|
||||||
|
|
||||||
// Try slices with a version that doesn't support slices. That should fail.
|
// Try slices=auto with a version that doesn't support it, which should fail
|
||||||
update.setSlices(between(2, 1000));
|
update.setSlices(AbstractBulkByScrollRequest.AUTO_SLICES);
|
||||||
Exception e = expectThrows(IllegalArgumentException.class, () -> roundTrip(Version.V_5_0_0_rc1, update, null));
|
Exception e = expectThrows(IllegalArgumentException.class, () -> roundTrip(Version.V_6_0_0_alpha1, update, null));
|
||||||
assertEquals("Attempting to send sliced reindex-style request to a node that doesn't support it. "
|
assertEquals("Slices set as \"auto\" are not supported before version [6.1.0]. Found version [6.0.0-alpha1]", e.getMessage());
|
||||||
+ "Version is [5.0.0-rc1] but must be [5.1.1]", e.getMessage());
|
|
||||||
|
|
||||||
// Try without slices with a version that doesn't support slices. That should work.
|
// Try regular slices with a version that doesn't support slices=auto, which should succeed
|
||||||
tripped = new UpdateByQueryRequest();
|
tripped = new UpdateByQueryRequest();
|
||||||
update.setSlices(1);
|
update.setSlices(between(1, Integer.MAX_VALUE));
|
||||||
roundTrip(Version.V_5_0_0_rc1, update, tripped);
|
roundTrip(Version.V_6_0_0_alpha1, update, tripped);
|
||||||
assertRequestEquals(update, tripped);
|
assertRequestEquals(update, tripped);
|
||||||
assertEquals(update.getPipeline(), tripped.getPipeline());
|
assertEquals(update.getPipeline(), tripped.getPipeline());
|
||||||
}
|
}
|
||||||
|
@ -115,16 +113,15 @@ public class RoundTripTests extends ESTestCase {
|
||||||
roundTrip(delete, tripped);
|
roundTrip(delete, tripped);
|
||||||
assertRequestEquals(delete, tripped);
|
assertRequestEquals(delete, tripped);
|
||||||
|
|
||||||
// Try slices with a version that doesn't support slices. That should fail.
|
// Try slices=auto with a version that doesn't support it, which should fail
|
||||||
delete.setSlices(between(2, 1000));
|
delete.setSlices(AbstractBulkByScrollRequest.AUTO_SLICES);
|
||||||
Exception e = expectThrows(IllegalArgumentException.class, () -> roundTrip(Version.V_5_0_0_rc1, delete, null));
|
Exception e = expectThrows(IllegalArgumentException.class, () -> roundTrip(Version.V_6_0_0_alpha1, delete, null));
|
||||||
assertEquals("Attempting to send sliced reindex-style request to a node that doesn't support it. "
|
assertEquals("Slices set as \"auto\" are not supported before version [6.1.0]. Found version [6.0.0-alpha1]", e.getMessage());
|
||||||
+ "Version is [5.0.0-rc1] but must be [5.1.1]", e.getMessage());
|
|
||||||
|
|
||||||
// Try without slices with a version that doesn't support slices. That should work.
|
// Try regular slices with a version that doesn't support slices=auto, which should succeed
|
||||||
tripped = new DeleteByQueryRequest();
|
tripped = new DeleteByQueryRequest();
|
||||||
delete.setSlices(1);
|
delete.setSlices(between(1, Integer.MAX_VALUE));
|
||||||
roundTrip(Version.V_5_0_0_rc1, delete, tripped);
|
roundTrip(Version.V_6_0_0_alpha1, delete, tripped);
|
||||||
assertRequestEquals(delete, tripped);
|
assertRequestEquals(delete, tripped);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -139,7 +136,9 @@ public class RoundTripTests extends ESTestCase {
|
||||||
request.setTimeout(TimeValue.parseTimeValue(randomTimeValue(), null, "test"));
|
request.setTimeout(TimeValue.parseTimeValue(randomTimeValue(), null, "test"));
|
||||||
request.setWaitForActiveShards(randomIntBetween(0, 10));
|
request.setWaitForActiveShards(randomIntBetween(0, 10));
|
||||||
request.setRequestsPerSecond(between(0, Integer.MAX_VALUE));
|
request.setRequestsPerSecond(between(0, Integer.MAX_VALUE));
|
||||||
request.setSlices(between(1, Integer.MAX_VALUE));
|
|
||||||
|
int slices = ReindexTestCase.randomSlices(1, Integer.MAX_VALUE);
|
||||||
|
request.setSlices(slices);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void randomRequest(AbstractBulkIndexByScrollRequest<?> request) {
|
private void randomRequest(AbstractBulkIndexByScrollRequest<?> request) {
|
||||||
|
|
|
@ -48,12 +48,13 @@ import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
public class TransportRethrottleActionTests extends ESTestCase {
|
public class TransportRethrottleActionTests extends ESTestCase {
|
||||||
private int slices;
|
private int slices;
|
||||||
private ParentBulkByScrollTask task;
|
private BulkByScrollTask task;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void createTask() {
|
public void createTask() {
|
||||||
slices = between(2, 50);
|
slices = between(2, 50);
|
||||||
task = new ParentBulkByScrollTask(1, "test_type", "test_action", "test", null, slices);
|
task = new BulkByScrollTask(1, "test_type", "test_action", "test", TaskId.EMPTY_TASK_ID);
|
||||||
|
task.setWorkerCount(slices);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -113,7 +114,7 @@ public class TransportRethrottleActionTests extends ESTestCase {
|
||||||
List<BulkByScrollTask.StatusOrException> sliceStatuses = new ArrayList<>(slices);
|
List<BulkByScrollTask.StatusOrException> sliceStatuses = new ArrayList<>(slices);
|
||||||
for (int i = 0; i < succeeded; i++) {
|
for (int i = 0; i < succeeded; i++) {
|
||||||
BulkByScrollTask.Status status = believeableCompletedStatus(i);
|
BulkByScrollTask.Status status = believeableCompletedStatus(i);
|
||||||
task.onSliceResponse(neverCalled(), i,
|
task.getLeaderState().onSliceResponse(neverCalled(), i,
|
||||||
new BulkByScrollResponse(timeValueMillis(10), status, emptyList(), emptyList(), false));
|
new BulkByScrollResponse(timeValueMillis(10), status, emptyList(), emptyList(), false));
|
||||||
sliceStatuses.add(new BulkByScrollTask.StatusOrException(status));
|
sliceStatuses.add(new BulkByScrollTask.StatusOrException(status));
|
||||||
}
|
}
|
||||||
|
@ -134,7 +135,8 @@ public class TransportRethrottleActionTests extends ESTestCase {
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
ActionListener<BulkByScrollResponse> listener = i < slices - 1 ? neverCalled() : mock(ActionListener.class);
|
ActionListener<BulkByScrollResponse> listener = i < slices - 1 ? neverCalled() : mock(ActionListener.class);
|
||||||
BulkByScrollTask.Status status = believeableCompletedStatus(i);
|
BulkByScrollTask.Status status = believeableCompletedStatus(i);
|
||||||
task.onSliceResponse(listener, i, new BulkByScrollResponse(timeValueMillis(10), status, emptyList(), emptyList(), false));
|
task.getLeaderState().onSliceResponse(listener, i, new BulkByScrollResponse(timeValueMillis(10), status, emptyList(),
|
||||||
|
emptyList(), false));
|
||||||
if (i == slices - 1) {
|
if (i == slices - 1) {
|
||||||
// The whole thing succeeded so we should have got the success
|
// The whole thing succeeded so we should have got the success
|
||||||
captureResponse(BulkByScrollResponse.class, listener).getStatus();
|
captureResponse(BulkByScrollResponse.class, listener).getStatus();
|
||||||
|
|
|
@ -19,8 +19,16 @@
|
||||||
|
|
||||||
package org.elasticsearch.index.reindex;
|
package org.elasticsearch.index.reindex;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||||
import org.elasticsearch.search.sort.SortOrder;
|
import org.elasticsearch.search.sort.SortOrder;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
import static org.hamcrest.Matchers.hasSize;
|
import static org.hamcrest.Matchers.hasSize;
|
||||||
|
@ -63,32 +71,91 @@ public class UpdateByQueryBasicTests extends ReindexTestCase {
|
||||||
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testWorkers() throws Exception {
|
public void testSlices() throws Exception {
|
||||||
indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "a"),
|
indexRandom(true,
|
||||||
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
|
client().prepareIndex("test", "test", "1").setSource("foo", "a"),
|
||||||
client().prepareIndex("test", "test", "3").setSource("foo", "b"),
|
client().prepareIndex("test", "test", "2").setSource("foo", "a"),
|
||||||
client().prepareIndex("test", "test", "4").setSource("foo", "c"));
|
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);
|
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", "1").get().getVersion());
|
||||||
assertEquals(1, client().prepareGet("test", "test", "4").get().getVersion());
|
assertEquals(1, client().prepareGet("test", "test", "4").get().getVersion());
|
||||||
|
|
||||||
|
int slices = randomSlices(2, 10);
|
||||||
|
int expectedSlices = expectedSliceStatuses(slices, "test");
|
||||||
|
|
||||||
// Reindex all the docs
|
// Reindex all the docs
|
||||||
assertThat(updateByQuery().source("test").refresh(true).setSlices(5).get(), matcher().updated(4).slices(hasSize(5)));
|
assertThat(
|
||||||
|
updateByQuery()
|
||||||
|
.source("test")
|
||||||
|
.refresh(true)
|
||||||
|
.setSlices(slices).get(),
|
||||||
|
matcher()
|
||||||
|
.updated(4)
|
||||||
|
.slices(hasSize(expectedSlices)));
|
||||||
assertEquals(2, client().prepareGet("test", "test", "1").get().getVersion());
|
assertEquals(2, client().prepareGet("test", "test", "1").get().getVersion());
|
||||||
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
||||||
|
|
||||||
// Now none of them
|
// Now none of them
|
||||||
assertThat(updateByQuery().source("test").filter(termQuery("foo", "no_match")).setSlices(5).refresh(true).get(),
|
assertThat(
|
||||||
matcher().updated(0).slices(hasSize(5)));
|
updateByQuery()
|
||||||
|
.source("test")
|
||||||
|
.filter(termQuery("foo", "no_match"))
|
||||||
|
.setSlices(slices)
|
||||||
|
.refresh(true).get(),
|
||||||
|
matcher()
|
||||||
|
.updated(0)
|
||||||
|
.slices(hasSize(expectedSlices)));
|
||||||
assertEquals(2, client().prepareGet("test", "test", "1").get().getVersion());
|
assertEquals(2, client().prepareGet("test", "test", "1").get().getVersion());
|
||||||
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
||||||
|
|
||||||
// Now half of them
|
// Now half of them
|
||||||
assertThat(updateByQuery().source("test").filter(termQuery("foo", "a")).refresh(true).setSlices(5).get(),
|
assertThat(
|
||||||
matcher().updated(2).slices(hasSize(5)));
|
updateByQuery()
|
||||||
|
.source("test")
|
||||||
|
.filter(termQuery("foo", "a"))
|
||||||
|
.refresh(true)
|
||||||
|
.setSlices(slices).get(),
|
||||||
|
matcher()
|
||||||
|
.updated(2)
|
||||||
|
.slices(hasSize(expectedSlices)));
|
||||||
assertEquals(3, client().prepareGet("test", "test", "1").get().getVersion());
|
assertEquals(3, client().prepareGet("test", "test", "1").get().getVersion());
|
||||||
assertEquals(3, client().prepareGet("test", "test", "2").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", "3").get().getVersion());
|
||||||
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
assertEquals(2, client().prepareGet("test", "test", "4").get().getVersion());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMultipleSources() throws Exception {
|
||||||
|
int sourceIndices = between(2, 5);
|
||||||
|
|
||||||
|
Map<String, List<IndexRequestBuilder>> docs = new HashMap<>();
|
||||||
|
for (int sourceIndex = 0; sourceIndex < sourceIndices; sourceIndex++) {
|
||||||
|
String indexName = "test" + sourceIndex;
|
||||||
|
docs.put(indexName, new ArrayList<>());
|
||||||
|
int numDocs = between(5, 15);
|
||||||
|
for (int i = 0; i < numDocs; i++) {
|
||||||
|
docs.get(indexName).add(client().prepareIndex(indexName, "test", Integer.toString(i)).setSource("foo", "a"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
List<IndexRequestBuilder> allDocs = docs.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
|
||||||
|
indexRandom(true, allDocs);
|
||||||
|
for (Map.Entry<String, List<IndexRequestBuilder>> entry : docs.entrySet()) {
|
||||||
|
assertHitCount(client().prepareSearch(entry.getKey()).setSize(0).get(), entry.getValue().size());
|
||||||
|
}
|
||||||
|
|
||||||
|
int slices = randomSlices(1, 10);
|
||||||
|
int expectedSlices = expectedSliceStatuses(slices, docs.keySet());
|
||||||
|
|
||||||
|
String[] sourceIndexNames = docs.keySet().toArray(new String[docs.size()]);
|
||||||
|
BulkByScrollResponse response = updateByQuery().source(sourceIndexNames).refresh(true).setSlices(slices).get();
|
||||||
|
assertThat(response, matcher().updated(allDocs.size()).slices(hasSize(expectedSlices)));
|
||||||
|
|
||||||
|
for (Map.Entry<String, List<IndexRequestBuilder>> entry : docs.entrySet()) {
|
||||||
|
String index = entry.getKey();
|
||||||
|
List<IndexRequestBuilder> indexDocs = entry.getValue();
|
||||||
|
int randomDoc = between(0, indexDocs.size() - 1);
|
||||||
|
assertEquals(2, client().prepareGet(index, "test", Integer.toString(randomDoc)).get().getVersion());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -126,7 +126,7 @@
|
||||||
---
|
---
|
||||||
"junk in slices fails":
|
"junk in slices fails":
|
||||||
- do:
|
- do:
|
||||||
catch: /Failed to parse int parameter \[slices\] with value \[junk\]/
|
catch: /\[slices\] must be a positive integer or the string "auto"/
|
||||||
delete_by_query:
|
delete_by_query:
|
||||||
slices: junk
|
slices: junk
|
||||||
index: test
|
index: test
|
||||||
|
@ -136,7 +136,7 @@
|
||||||
---
|
---
|
||||||
"zero slices fails":
|
"zero slices fails":
|
||||||
- do:
|
- do:
|
||||||
catch: /\[slices\] must be at least 1/
|
catch: /\[slices\] must be a positive integer or the string "auto"/
|
||||||
delete_by_query:
|
delete_by_query:
|
||||||
slices: 0
|
slices: 0
|
||||||
index: test
|
index: test
|
||||||
|
|
|
@ -45,6 +45,7 @@
|
||||||
- match: {throttled_millis: 0}
|
- match: {throttled_millis: 0}
|
||||||
- gte: { took: 0 }
|
- gte: { took: 0 }
|
||||||
- is_false: task
|
- is_false: task
|
||||||
|
- length: {slices: 5}
|
||||||
- match: {slices.0.version_conflicts: 0}
|
- match: {slices.0.version_conflicts: 0}
|
||||||
- match: {slices.0.throttled_millis: 0}
|
- match: {slices.0.throttled_millis: 0}
|
||||||
- match: {slices.1.version_conflicts: 0}
|
- match: {slices.1.version_conflicts: 0}
|
||||||
|
@ -128,6 +129,7 @@
|
||||||
- match: {response.throttled_millis: 0}
|
- match: {response.throttled_millis: 0}
|
||||||
- gte: { response.took: 0 }
|
- gte: { response.took: 0 }
|
||||||
- is_false: response.task
|
- is_false: response.task
|
||||||
|
- length: {response.slices: 5}
|
||||||
- match: {response.slices.0.version_conflicts: 0}
|
- match: {response.slices.0.version_conflicts: 0}
|
||||||
- match: {response.slices.0.throttled_millis: 0}
|
- match: {response.slices.0.throttled_millis: 0}
|
||||||
- match: {response.slices.1.version_conflicts: 0}
|
- match: {response.slices.1.version_conflicts: 0}
|
||||||
|
@ -142,6 +144,7 @@
|
||||||
- match: {task.status.deleted: 4}
|
- match: {task.status.deleted: 4}
|
||||||
- match: {task.status.version_conflicts: 0}
|
- match: {task.status.version_conflicts: 0}
|
||||||
- match: {task.status.throttled_millis: 0}
|
- match: {task.status.throttled_millis: 0}
|
||||||
|
- length: {task.status.slices: 5}
|
||||||
- match: {task.status.slices.0.version_conflicts: 0}
|
- match: {task.status.slices.0.version_conflicts: 0}
|
||||||
- match: {task.status.slices.0.throttled_millis: 0}
|
- match: {task.status.slices.0.throttled_millis: 0}
|
||||||
- match: {task.status.slices.1.version_conflicts: 0}
|
- match: {task.status.slices.1.version_conflicts: 0}
|
||||||
|
@ -252,6 +255,7 @@
|
||||||
- match: {response.throttled_millis: 0}
|
- match: {response.throttled_millis: 0}
|
||||||
- gte: { response.took: 0 }
|
- gte: { response.took: 0 }
|
||||||
- is_false: response.task
|
- is_false: response.task
|
||||||
|
- length: {response.slices: 2}
|
||||||
- match: {response.slices.0.version_conflicts: 0}
|
- match: {response.slices.0.version_conflicts: 0}
|
||||||
- match: {response.slices.0.throttled_millis: 0}
|
- match: {response.slices.0.throttled_millis: 0}
|
||||||
- match: {response.slices.1.version_conflicts: 0}
|
- match: {response.slices.1.version_conflicts: 0}
|
||||||
|
@ -260,6 +264,7 @@
|
||||||
- match: {task.status.deleted: 6}
|
- match: {task.status.deleted: 6}
|
||||||
- match: {task.status.version_conflicts: 0}
|
- match: {task.status.version_conflicts: 0}
|
||||||
- match: {task.status.throttled_millis: 0}
|
- match: {task.status.throttled_millis: 0}
|
||||||
|
- length: {task.status.slices: 2}
|
||||||
- match: {task.status.slices.0.version_conflicts: 0}
|
- match: {task.status.slices.0.version_conflicts: 0}
|
||||||
- match: {task.status.slices.0.throttled_millis: 0}
|
- match: {task.status.slices.0.throttled_millis: 0}
|
||||||
- match: {task.status.slices.1.version_conflicts: 0}
|
- match: {task.status.slices.1.version_conflicts: 0}
|
||||||
|
@ -277,3 +282,72 @@
|
||||||
count:
|
count:
|
||||||
index: test
|
index: test
|
||||||
- match: {count: 0}
|
- match: {count: 0}
|
||||||
|
|
||||||
|
---
|
||||||
|
"Multiple slices with auto slice":
|
||||||
|
- do:
|
||||||
|
indices.create:
|
||||||
|
index: test
|
||||||
|
body:
|
||||||
|
settings:
|
||||||
|
index:
|
||||||
|
number_of_shards: 3
|
||||||
|
- 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: auto
|
||||||
|
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
|
||||||
|
- length: {slices: 3}
|
||||||
|
- 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}
|
||||||
|
|
||||||
|
- do:
|
||||||
|
indices.refresh: {}
|
||||||
|
- do:
|
||||||
|
count:
|
||||||
|
index: test
|
||||||
|
- match: {count: 0}
|
||||||
|
|
|
@ -275,7 +275,7 @@
|
||||||
---
|
---
|
||||||
"junk in slices fails":
|
"junk in slices fails":
|
||||||
- do:
|
- do:
|
||||||
catch: /Failed to parse int parameter \[slices\] with value \[junk\]/
|
catch: /\[slices\] must be a positive integer or the string "auto"/
|
||||||
reindex:
|
reindex:
|
||||||
slices: junk
|
slices: junk
|
||||||
body:
|
body:
|
||||||
|
@ -287,7 +287,7 @@
|
||||||
---
|
---
|
||||||
"zero slices fails":
|
"zero slices fails":
|
||||||
- do:
|
- do:
|
||||||
catch: /\[slices\] must be at least 1/
|
catch: /\[slices\] must be a positive integer or the string "auto"/
|
||||||
reindex:
|
reindex:
|
||||||
slices: 0
|
slices: 0
|
||||||
body:
|
body:
|
||||||
|
|
|
@ -43,6 +43,7 @@
|
||||||
- gte: { took: 0 }
|
- gte: { took: 0 }
|
||||||
- is_false: task
|
- is_false: task
|
||||||
- is_false: deleted
|
- is_false: deleted
|
||||||
|
- length: {slices: 5}
|
||||||
- match: {slices.0.updated: 0}
|
- match: {slices.0.updated: 0}
|
||||||
- match: {slices.0.version_conflicts: 0}
|
- match: {slices.0.version_conflicts: 0}
|
||||||
- match: {slices.0.throttled_millis: 0}
|
- match: {slices.0.throttled_millis: 0}
|
||||||
|
@ -127,6 +128,7 @@
|
||||||
- gte: { response.took: 0 }
|
- gte: { response.took: 0 }
|
||||||
- is_false: response.task
|
- is_false: response.task
|
||||||
- is_false: response.deleted
|
- is_false: response.deleted
|
||||||
|
- length: {response.slices: 5}
|
||||||
- match: {response.slices.0.updated: 0}
|
- match: {response.slices.0.updated: 0}
|
||||||
- match: {response.slices.0.version_conflicts: 0}
|
- match: {response.slices.0.version_conflicts: 0}
|
||||||
- match: {response.slices.0.throttled_millis: 0}
|
- match: {response.slices.0.throttled_millis: 0}
|
||||||
|
@ -147,6 +149,7 @@
|
||||||
- match: {task.status.updated: 0}
|
- match: {task.status.updated: 0}
|
||||||
- match: {task.status.version_conflicts: 0}
|
- match: {task.status.version_conflicts: 0}
|
||||||
- match: {task.status.throttled_millis: 0}
|
- match: {task.status.throttled_millis: 0}
|
||||||
|
- length: {task.status.slices: 5}
|
||||||
- match: {task.status.slices.0.updated: 0}
|
- match: {task.status.slices.0.updated: 0}
|
||||||
- match: {task.status.slices.0.version_conflicts: 0}
|
- match: {task.status.slices.0.version_conflicts: 0}
|
||||||
- match: {task.status.slices.0.throttled_millis: 0}
|
- match: {task.status.slices.0.throttled_millis: 0}
|
||||||
|
@ -260,6 +263,7 @@
|
||||||
- gte: { response.took: 0 }
|
- gte: { response.took: 0 }
|
||||||
- is_false: response.task
|
- is_false: response.task
|
||||||
- is_false: response.deleted
|
- is_false: response.deleted
|
||||||
|
- length: {response.slices: 2}
|
||||||
- match: {response.slices.0.updated: 0}
|
- match: {response.slices.0.updated: 0}
|
||||||
- match: {response.slices.0.version_conflicts: 0}
|
- match: {response.slices.0.version_conflicts: 0}
|
||||||
- match: {response.slices.0.throttled_millis: 0}
|
- match: {response.slices.0.throttled_millis: 0}
|
||||||
|
@ -271,6 +275,7 @@
|
||||||
- match: {task.status.updated: 0}
|
- match: {task.status.updated: 0}
|
||||||
- match: {task.status.version_conflicts: 0}
|
- match: {task.status.version_conflicts: 0}
|
||||||
- match: {task.status.throttled_millis: 0}
|
- match: {task.status.throttled_millis: 0}
|
||||||
|
- length: {task.status.slices: 2}
|
||||||
- match: {task.status.slices.0.updated: 0}
|
- match: {task.status.slices.0.updated: 0}
|
||||||
- match: {task.status.slices.0.version_conflicts: 0}
|
- match: {task.status.slices.0.version_conflicts: 0}
|
||||||
- match: {task.status.slices.0.throttled_millis: 0}
|
- match: {task.status.slices.0.throttled_millis: 0}
|
||||||
|
@ -285,3 +290,67 @@
|
||||||
search:
|
search:
|
||||||
index: .tasks
|
index: .tasks
|
||||||
- match: { hits.total: 1 }
|
- match: { hits.total: 1 }
|
||||||
|
|
||||||
|
|
||||||
|
---
|
||||||
|
"Multiple slices with auto slice":
|
||||||
|
- do:
|
||||||
|
indices.create:
|
||||||
|
index: source
|
||||||
|
body:
|
||||||
|
settings:
|
||||||
|
index:
|
||||||
|
number_of_shards: 3
|
||||||
|
- 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: auto
|
||||||
|
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
|
||||||
|
- length: {slices: 3}
|
||||||
|
- 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}
|
||||||
|
|
|
@ -108,7 +108,7 @@
|
||||||
---
|
---
|
||||||
"junk in slices fails":
|
"junk in slices fails":
|
||||||
- do:
|
- do:
|
||||||
catch: /Failed to parse int parameter \[slices\] with value \[junk\]/
|
catch: /\[slices\] must be a positive integer or the string "auto"/
|
||||||
update_by_query:
|
update_by_query:
|
||||||
slices: junk
|
slices: junk
|
||||||
index: test
|
index: test
|
||||||
|
@ -116,7 +116,7 @@
|
||||||
---
|
---
|
||||||
"zero slices fails":
|
"zero slices fails":
|
||||||
- do:
|
- do:
|
||||||
catch: /\[slices\] must be at least 1/
|
catch: /\[slices\] must be a positive integer or the string "auto"/
|
||||||
update_by_query:
|
update_by_query:
|
||||||
slices: 0
|
slices: 0
|
||||||
index: test
|
index: test
|
||||||
|
|
|
@ -44,6 +44,7 @@
|
||||||
- match: {throttled_millis: 0}
|
- match: {throttled_millis: 0}
|
||||||
- gte: { took: 0 }
|
- gte: { took: 0 }
|
||||||
- is_false: task
|
- is_false: task
|
||||||
|
- length: {slices: 5}
|
||||||
- match: {slices.0.version_conflicts: 0}
|
- match: {slices.0.version_conflicts: 0}
|
||||||
- match: {slices.0.throttled_millis: 0}
|
- match: {slices.0.throttled_millis: 0}
|
||||||
- match: {slices.1.version_conflicts: 0}
|
- match: {slices.1.version_conflicts: 0}
|
||||||
|
@ -120,6 +121,7 @@
|
||||||
- match: {response.throttled_millis: 0}
|
- match: {response.throttled_millis: 0}
|
||||||
- gte: { response.took: 0 }
|
- gte: { response.took: 0 }
|
||||||
- is_false: response.task
|
- is_false: response.task
|
||||||
|
- length: {response.slices: 5}
|
||||||
- match: {response.slices.0.version_conflicts: 0}
|
- match: {response.slices.0.version_conflicts: 0}
|
||||||
- match: {response.slices.0.throttled_millis: 0}
|
- match: {response.slices.0.throttled_millis: 0}
|
||||||
- match: {response.slices.1.version_conflicts: 0}
|
- match: {response.slices.1.version_conflicts: 0}
|
||||||
|
@ -134,6 +136,7 @@
|
||||||
- match: {task.status.updated: 4}
|
- match: {task.status.updated: 4}
|
||||||
- match: {task.status.version_conflicts: 0}
|
- match: {task.status.version_conflicts: 0}
|
||||||
- match: {task.status.throttled_millis: 0}
|
- match: {task.status.throttled_millis: 0}
|
||||||
|
- length: {task.status.slices: 5}
|
||||||
- match: {task.status.slices.0.version_conflicts: 0}
|
- match: {task.status.slices.0.version_conflicts: 0}
|
||||||
- match: {task.status.slices.0.throttled_millis: 0}
|
- match: {task.status.slices.0.throttled_millis: 0}
|
||||||
- match: {task.status.slices.1.version_conflicts: 0}
|
- match: {task.status.slices.1.version_conflicts: 0}
|
||||||
|
@ -239,6 +242,7 @@
|
||||||
- match: {response.throttled_millis: 0}
|
- match: {response.throttled_millis: 0}
|
||||||
- gte: { response.took: 0 }
|
- gte: { response.took: 0 }
|
||||||
- is_false: response.task
|
- is_false: response.task
|
||||||
|
- length: {response.slices: 2}
|
||||||
- match: {response.slices.0.version_conflicts: 0}
|
- match: {response.slices.0.version_conflicts: 0}
|
||||||
- match: {response.slices.0.throttled_millis: 0}
|
- match: {response.slices.0.throttled_millis: 0}
|
||||||
- match: {response.slices.1.version_conflicts: 0}
|
- match: {response.slices.1.version_conflicts: 0}
|
||||||
|
@ -247,6 +251,7 @@
|
||||||
- match: {task.status.updated: 6}
|
- match: {task.status.updated: 6}
|
||||||
- match: {task.status.version_conflicts: 0}
|
- match: {task.status.version_conflicts: 0}
|
||||||
- match: {task.status.throttled_millis: 0}
|
- match: {task.status.throttled_millis: 0}
|
||||||
|
- length: {task.status.slices: 2}
|
||||||
- match: {task.status.slices.0.version_conflicts: 0}
|
- match: {task.status.slices.0.version_conflicts: 0}
|
||||||
- match: {task.status.slices.0.throttled_millis: 0}
|
- match: {task.status.slices.0.throttled_millis: 0}
|
||||||
- match: {task.status.slices.1.version_conflicts: 0}
|
- match: {task.status.slices.1.version_conflicts: 0}
|
||||||
|
@ -259,3 +264,65 @@
|
||||||
search:
|
search:
|
||||||
index: .tasks
|
index: .tasks
|
||||||
- match: { hits.total: 1 }
|
- match: { hits.total: 1 }
|
||||||
|
|
||||||
|
|
||||||
|
---
|
||||||
|
"Multiple slices with auto slice":
|
||||||
|
- do:
|
||||||
|
indices.create:
|
||||||
|
index: test
|
||||||
|
body:
|
||||||
|
settings:
|
||||||
|
index:
|
||||||
|
number_of_shards: 3
|
||||||
|
- 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: auto
|
||||||
|
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
|
||||||
|
- length: {slices: 3}
|
||||||
|
- 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}
|
||||||
|
|
|
@ -32,12 +32,14 @@ public abstract class AbstractAsyncBulkByScrollActionTestCase<
|
||||||
Response extends BulkByScrollResponse>
|
Response extends BulkByScrollResponse>
|
||||||
extends ESTestCase {
|
extends ESTestCase {
|
||||||
protected ThreadPool threadPool;
|
protected ThreadPool threadPool;
|
||||||
protected WorkingBulkByScrollTask task;
|
protected BulkByScrollTask task;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setupForTest() {
|
public void setupForTest() {
|
||||||
threadPool = new TestThreadPool(getTestName());
|
threadPool = new TestThreadPool(getTestName());
|
||||||
task = new WorkingBulkByScrollTask(1, "test", "test", "test", TaskId.EMPTY_TASK_ID, null, Float.MAX_VALUE);
|
task = new BulkByScrollTask(1, "test", "test", "test", TaskId.EMPTY_TASK_ID);
|
||||||
|
task.setWorker(Float.POSITIVE_INFINITY, null);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
Loading…
Reference in New Issue