Teach reindex to stop when cancelled
All we do is check the cancelled flag and stop the request at a few key points. Adds the cancellation cause to the status so any request that is cancelled but doesn't die can be seen in the task list.
This commit is contained in:
parent
a9fa0cbb51
commit
30107b4a74
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package org.elasticsearch.tasks;
|
package org.elasticsearch.tasks;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -56,4 +58,11 @@ public class CancellableTask extends Task {
|
||||||
return reason.get() != null;
|
return reason.get() != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The reason the task was cancelled or null if it hasn't been cancelled.
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public String getReasonCancelled() {
|
||||||
|
return reason.get();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -113,7 +113,11 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
return task;
|
return task;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initialSearch() {
|
void initialSearch() {
|
||||||
|
if (task.isCancelled()) {
|
||||||
|
finishHim(null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
try {
|
try {
|
||||||
// Default to sorting by _doc if it hasn't been changed.
|
// Default to sorting by _doc if it hasn't been changed.
|
||||||
if (firstSearchRequest.source().sorts() == null) {
|
if (firstSearchRequest.source().sorts() == null) {
|
||||||
|
@ -144,8 +148,19 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Set the last returned scrollId. Package private for testing.
|
||||||
|
*/
|
||||||
|
void setScroll(String scroll) {
|
||||||
|
this.scroll.set(scroll);
|
||||||
|
}
|
||||||
|
|
||||||
void onScrollResponse(SearchResponse searchResponse) {
|
void onScrollResponse(SearchResponse searchResponse) {
|
||||||
scroll.set(searchResponse.getScrollId());
|
if (task.isCancelled()) {
|
||||||
|
finishHim(null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
setScroll(searchResponse.getScrollId());
|
||||||
if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) {
|
if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) {
|
||||||
startNormalTermination(emptyList(), unmodifiableList(Arrays.asList(searchResponse.getShardFailures())));
|
startNormalTermination(emptyList(), unmodifiableList(Arrays.asList(searchResponse.getShardFailures())));
|
||||||
return;
|
return;
|
||||||
|
@ -178,7 +193,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
/*
|
/*
|
||||||
* If we noop-ed the entire batch then just skip to the next batch or the BulkRequest would fail validation.
|
* If we noop-ed the entire batch then just skip to the next batch or the BulkRequest would fail validation.
|
||||||
*/
|
*/
|
||||||
startNextScrollRequest();
|
startNextScroll();
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
request.timeout(mainRequest.getTimeout());
|
request.timeout(mainRequest.getTimeout());
|
||||||
|
@ -198,6 +213,10 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
}
|
}
|
||||||
|
|
||||||
void sendBulkRequest(BulkRequest request) {
|
void sendBulkRequest(BulkRequest request) {
|
||||||
|
if (task.isCancelled()) {
|
||||||
|
finishHim(null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
retry.withAsyncBackoff(client, request, new ActionListener<BulkResponse>() {
|
retry.withAsyncBackoff(client, request, new ActionListener<BulkResponse>() {
|
||||||
@Override
|
@Override
|
||||||
public void onResponse(BulkResponse response) {
|
public void onResponse(BulkResponse response) {
|
||||||
|
@ -212,6 +231,10 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
}
|
}
|
||||||
|
|
||||||
void onBulkResponse(BulkResponse response) {
|
void onBulkResponse(BulkResponse response) {
|
||||||
|
if (task.isCancelled()) {
|
||||||
|
finishHim(null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
try {
|
try {
|
||||||
List<Failure> failures = new ArrayList<Failure>();
|
List<Failure> failures = new ArrayList<Failure>();
|
||||||
Set<String> destinationIndicesThisBatch = new HashSet<>();
|
Set<String> destinationIndicesThisBatch = new HashSet<>();
|
||||||
|
@ -252,13 +275,17 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
|
||||||
startNormalTermination(emptyList(), emptyList());
|
startNormalTermination(emptyList(), emptyList());
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
startNextScrollRequest();
|
startNextScroll();
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
finishHim(t);
|
finishHim(t);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void startNextScrollRequest() {
|
void startNextScroll() {
|
||||||
|
if (task.isCancelled()) {
|
||||||
|
finishHim(null);
|
||||||
|
return;
|
||||||
|
}
|
||||||
SearchScrollRequest request = new SearchScrollRequest();
|
SearchScrollRequest request = new SearchScrollRequest();
|
||||||
request.scrollId(scroll.get()).scroll(firstSearchRequest.scroll());
|
request.scrollId(scroll.get()).scroll(firstSearchRequest.scroll());
|
||||||
client.searchScroll(request, new ActionListener<SearchResponse>() {
|
client.searchScroll(request, new ActionListener<SearchResponse>() {
|
||||||
|
|
|
@ -19,9 +19,11 @@
|
||||||
|
|
||||||
package org.elasticsearch.plugin.reindex;
|
package org.elasticsearch.plugin.reindex;
|
||||||
|
|
||||||
|
import org.elasticsearch.common.Nullable;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
|
import org.elasticsearch.tasks.CancellableTask;
|
||||||
import org.elasticsearch.tasks.Task;
|
import org.elasticsearch.tasks.Task;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -31,7 +33,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||||
/**
|
/**
|
||||||
* Task storing information about a currently running BulkByScroll request.
|
* Task storing information about a currently running BulkByScroll request.
|
||||||
*/
|
*/
|
||||||
public class BulkByScrollTask extends Task {
|
public class BulkByScrollTask extends CancellableTask {
|
||||||
/**
|
/**
|
||||||
* 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.
|
||||||
|
@ -52,7 +54,7 @@ public class BulkByScrollTask extends Task {
|
||||||
@Override
|
@Override
|
||||||
public Status getStatus() {
|
public Status getStatus() {
|
||||||
return new Status(total.get(), updated.get(), created.get(), deleted.get(), batch.get(), versionConflicts.get(), noops.get(),
|
return new Status(total.get(), updated.get(), created.get(), deleted.get(), batch.get(), versionConflicts.get(), noops.get(),
|
||||||
retries.get());
|
retries.get(), getReasonCancelled());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -63,7 +65,7 @@ public class BulkByScrollTask extends Task {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Status implements Task.Status {
|
public static class Status implements Task.Status {
|
||||||
public static final Status PROTOTYPE = new Status(0, 0, 0, 0, 0, 0, 0, 0);
|
public static final Status PROTOTYPE = new Status(0, 0, 0, 0, 0, 0, 0, 0, null);
|
||||||
|
|
||||||
private final long total;
|
private final long total;
|
||||||
private final long updated;
|
private final long updated;
|
||||||
|
@ -73,8 +75,10 @@ public class BulkByScrollTask extends Task {
|
||||||
private final long versionConflicts;
|
private final long versionConflicts;
|
||||||
private final long noops;
|
private final long noops;
|
||||||
private final long retries;
|
private final long retries;
|
||||||
|
private final String reasonCancelled;
|
||||||
|
|
||||||
public Status(long total, long updated, long created, long deleted, int batches, long versionConflicts, long noops, long retries) {
|
public Status(long total, long updated, long created, long deleted, int batches, long versionConflicts, long noops, long retries,
|
||||||
|
@Nullable String reasonCancelled) {
|
||||||
this.total = checkPositive(total, "total");
|
this.total = checkPositive(total, "total");
|
||||||
this.updated = checkPositive(updated, "updated");
|
this.updated = checkPositive(updated, "updated");
|
||||||
this.created = checkPositive(created, "created");
|
this.created = checkPositive(created, "created");
|
||||||
|
@ -83,6 +87,7 @@ public class BulkByScrollTask extends Task {
|
||||||
this.versionConflicts = checkPositive(versionConflicts, "versionConflicts");
|
this.versionConflicts = checkPositive(versionConflicts, "versionConflicts");
|
||||||
this.noops = checkPositive(noops, "noops");
|
this.noops = checkPositive(noops, "noops");
|
||||||
this.retries = checkPositive(retries, "retries");
|
this.retries = checkPositive(retries, "retries");
|
||||||
|
this.reasonCancelled = reasonCancelled;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Status(StreamInput in) throws IOException {
|
public Status(StreamInput in) throws IOException {
|
||||||
|
@ -94,6 +99,7 @@ public class BulkByScrollTask extends Task {
|
||||||
versionConflicts = in.readVLong();
|
versionConflicts = in.readVLong();
|
||||||
noops = in.readVLong();
|
noops = in.readVLong();
|
||||||
retries = in.readVLong();
|
retries = in.readVLong();
|
||||||
|
reasonCancelled = in.readOptionalString();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -106,6 +112,7 @@ public class BulkByScrollTask extends Task {
|
||||||
out.writeVLong(versionConflicts);
|
out.writeVLong(versionConflicts);
|
||||||
out.writeVLong(noops);
|
out.writeVLong(noops);
|
||||||
out.writeVLong(retries);
|
out.writeVLong(retries);
|
||||||
|
out.writeOptionalString(reasonCancelled);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -129,6 +136,9 @@ public class BulkByScrollTask extends Task {
|
||||||
builder.field("version_conflicts", versionConflicts);
|
builder.field("version_conflicts", versionConflicts);
|
||||||
builder.field("noops", noops);
|
builder.field("noops", noops);
|
||||||
builder.field("retries", retries);
|
builder.field("retries", retries);
|
||||||
|
if (reasonCancelled != null) {
|
||||||
|
builder.field("canceled", reasonCancelled);
|
||||||
|
}
|
||||||
return builder;
|
return builder;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -152,6 +162,9 @@ public class BulkByScrollTask extends Task {
|
||||||
builder.append(",versionConflicts=").append(versionConflicts);
|
builder.append(",versionConflicts=").append(versionConflicts);
|
||||||
builder.append(",noops=").append(noops);
|
builder.append(",noops=").append(noops);
|
||||||
builder.append(",retries=").append(retries);
|
builder.append(",retries=").append(retries);
|
||||||
|
if (reasonCancelled != null) {
|
||||||
|
builder.append(",canceled=").append(reasonCancelled);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -221,6 +234,13 @@ public class BulkByScrollTask extends Task {
|
||||||
return retries;
|
return retries;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The reason that the request was canceled or null if it hasn't been.
|
||||||
|
*/
|
||||||
|
public String getReasonCancelled() {
|
||||||
|
return reasonCancelled;
|
||||||
|
}
|
||||||
|
|
||||||
private int checkPositive(int value, String name) {
|
private int checkPositive(int value, String name) {
|
||||||
if (value < 0) {
|
if (value < 0) {
|
||||||
throw new IllegalArgumentException(name + " must be greater than 0 but was [" + value + "]");
|
throw new IllegalArgumentException(name + " must be greater than 0 but was [" + value + "]");
|
||||||
|
|
|
@ -81,6 +81,13 @@ public class BulkIndexByScrollResponse extends ActionResponse implements ToXCont
|
||||||
return status.getNoops();
|
return status.getNoops();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The reason that the request was canceled or null if it hasn't been.
|
||||||
|
*/
|
||||||
|
public String getReasonCancelled() {
|
||||||
|
return status.getReasonCancelled();
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* All of the indexing failures. Version conflicts are only included if the request sets abortOnVersionConflict to true (the
|
* All of the indexing failures. Version conflicts are only included if the request sets abortOnVersionConflict to true (the
|
||||||
* default).
|
* default).
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.hamcrest.Matcher;
|
||||||
import org.hamcrest.TypeSafeMatcher;
|
import org.hamcrest.TypeSafeMatcher;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
import static org.hamcrest.Matchers.nullValue;
|
||||||
|
|
||||||
public abstract class AbstractBulkIndexByScrollResponseMatcher<
|
public abstract class AbstractBulkIndexByScrollResponseMatcher<
|
||||||
Response extends BulkIndexByScrollResponse,
|
Response extends BulkIndexByScrollResponse,
|
||||||
|
@ -36,6 +37,7 @@ public abstract class AbstractBulkIndexByScrollResponseMatcher<
|
||||||
private Matcher<Integer> batchesMatcher;
|
private Matcher<Integer> batchesMatcher;
|
||||||
private Matcher<Long> versionConflictsMatcher = equalTo(0L);
|
private Matcher<Long> versionConflictsMatcher = equalTo(0L);
|
||||||
private Matcher<Integer> failuresMatcher = equalTo(0);
|
private Matcher<Integer> failuresMatcher = equalTo(0);
|
||||||
|
private Matcher<String> reasonCancelledMatcher = nullValue(String.class);
|
||||||
|
|
||||||
protected abstract Self self();
|
protected abstract Self self();
|
||||||
|
|
||||||
|
@ -93,13 +95,18 @@ public abstract class AbstractBulkIndexByScrollResponseMatcher<
|
||||||
return failures(equalTo(failures));
|
return failures(equalTo(failures));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Self reasonCancelled(Matcher<String> reasonCancelledMatcher) {
|
||||||
|
this.reasonCancelledMatcher = reasonCancelledMatcher;
|
||||||
|
return self();
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected boolean matchesSafely(Response item) {
|
protected boolean matchesSafely(Response item) {
|
||||||
return updatedMatcher.matches(item.getUpdated()) &&
|
return updatedMatcher.matches(item.getUpdated()) &&
|
||||||
(batchesMatcher == null || batchesMatcher.matches(item.getBatches())) &&
|
(batchesMatcher == null || batchesMatcher.matches(item.getBatches())) &&
|
||||||
versionConflictsMatcher.matches(item.getVersionConflicts()) &&
|
versionConflictsMatcher.matches(item.getVersionConflicts()) &&
|
||||||
failuresMatcher.matches(item.getIndexingFailures().size());
|
failuresMatcher.matches(item.getIndexingFailures().size()) &&
|
||||||
|
reasonCancelledMatcher.matches(item.getReasonCancelled());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -110,5 +117,6 @@ public abstract class AbstractBulkIndexByScrollResponseMatcher<
|
||||||
}
|
}
|
||||||
description.appendText(" and versionConflicts matches ").appendDescriptionOf(versionConflictsMatcher);
|
description.appendText(" and versionConflicts matches ").appendDescriptionOf(versionConflictsMatcher);
|
||||||
description.appendText(" and failures size matches ").appendDescriptionOf(failuresMatcher);
|
description.appendText(" and failures size matches ").appendDescriptionOf(failuresMatcher);
|
||||||
|
description.appendText(" and reason cancelled matches ").appendDescriptionOf(reasonCancelledMatcher);
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -45,6 +45,7 @@ import org.elasticsearch.action.update.UpdateRequest;
|
||||||
import org.elasticsearch.action.update.UpdateResponse;
|
import org.elasticsearch.action.update.UpdateResponse;
|
||||||
import org.elasticsearch.client.Client;
|
import org.elasticsearch.client.Client;
|
||||||
import org.elasticsearch.client.FilterClient;
|
import org.elasticsearch.client.FilterClient;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.text.Text;
|
import org.elasticsearch.common.text.Text;
|
||||||
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;
|
||||||
|
@ -57,6 +58,7 @@ import org.elasticsearch.search.SearchHit;
|
||||||
import org.elasticsearch.search.internal.InternalSearchHit;
|
import org.elasticsearch.search.internal.InternalSearchHit;
|
||||||
import org.elasticsearch.search.internal.InternalSearchHits;
|
import org.elasticsearch.search.internal.InternalSearchHits;
|
||||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||||
|
import org.elasticsearch.tasks.TaskManager;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.client.NoOpClient;
|
import org.elasticsearch.test.client.NoOpClient;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
@ -66,12 +68,15 @@ import org.junit.Before;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static org.apache.lucene.util.TestUtil.randomSimpleString;
|
import static org.apache.lucene.util.TestUtil.randomSimpleString;
|
||||||
import static org.elasticsearch.action.bulk.BackoffPolicy.constantBackoff;
|
import static org.elasticsearch.action.bulk.BackoffPolicy.constantBackoff;
|
||||||
|
@ -79,8 +84,8 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
||||||
import static org.hamcrest.Matchers.contains;
|
import static org.hamcrest.Matchers.contains;
|
||||||
import static org.hamcrest.Matchers.emptyCollectionOf;
|
import static org.hamcrest.Matchers.emptyCollectionOf;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
import static org.hamcrest.Matchers.hasSize;
|
||||||
import static org.hamcrest.Matchers.instanceOf;
|
import static org.hamcrest.Matchers.instanceOf;
|
||||||
import static org.hamcrest.Matchers.iterableWithSize;
|
|
||||||
|
|
||||||
public class AsyncBulkByScrollActionTests extends ESTestCase {
|
public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
private MyMockClient client;
|
private MyMockClient client;
|
||||||
|
@ -89,6 +94,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
private SearchRequest firstSearchRequest;
|
private SearchRequest firstSearchRequest;
|
||||||
private PlainActionFuture<BulkIndexByScrollResponse> listener;
|
private PlainActionFuture<BulkIndexByScrollResponse> listener;
|
||||||
private String scrollId;
|
private String scrollId;
|
||||||
|
private TaskManager taskManager;
|
||||||
private BulkByScrollTask task;
|
private BulkByScrollTask task;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
|
@ -99,7 +105,8 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
firstSearchRequest = null;
|
firstSearchRequest = null;
|
||||||
listener = new PlainActionFuture<>();
|
listener = new PlainActionFuture<>();
|
||||||
scrollId = null;
|
scrollId = null;
|
||||||
task = new BulkByScrollTask(0, "test", "test", "test");
|
taskManager = new TaskManager(Settings.EMPTY);
|
||||||
|
task = (BulkByScrollTask) taskManager.register("don'tcare", "hereeither", mainRequest);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
@ -246,6 +253,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
BulkIndexByScrollResponse response = listener.get();
|
BulkIndexByScrollResponse response = listener.get();
|
||||||
assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class));
|
assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class));
|
||||||
assertThat(response.getSearchFailures(), contains(shardFailure));
|
assertThat(response.getSearchFailures(), contains(shardFailure));
|
||||||
|
assertNull(response.getReasonCancelled());
|
||||||
assertThat(client.scrollsCleared, contains(scrollId));
|
assertThat(client.scrollsCleared, contains(scrollId));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -259,6 +267,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
BulkIndexByScrollResponse response = listener.get();
|
BulkIndexByScrollResponse response = listener.get();
|
||||||
assertThat(response.getIndexingFailures(), contains(failure));
|
assertThat(response.getIndexingFailures(), contains(failure));
|
||||||
assertThat(response.getSearchFailures(), emptyCollectionOf(ShardSearchFailure.class));
|
assertThat(response.getSearchFailures(), emptyCollectionOf(ShardSearchFailure.class));
|
||||||
|
assertNull(response.getReasonCancelled());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -323,7 +332,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void startNextScrollRequest() {
|
void startNextScroll() {
|
||||||
successLatch.countDown();
|
successLatch.countDown();
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
@ -334,9 +343,10 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
action.sendBulkRequest(request);
|
action.sendBulkRequest(request);
|
||||||
if (failWithRejection) {
|
if (failWithRejection) {
|
||||||
BulkIndexByScrollResponse response = listener.get();
|
BulkIndexByScrollResponse response = listener.get();
|
||||||
assertThat(response.getIndexingFailures(), iterableWithSize(equalTo(1)));
|
assertThat(response.getIndexingFailures(), hasSize(1));
|
||||||
assertEquals(response.getIndexingFailures().get(0).getStatus(), RestStatus.TOO_MANY_REQUESTS);
|
assertEquals(response.getIndexingFailures().get(0).getStatus(), RestStatus.TOO_MANY_REQUESTS);
|
||||||
assertThat(response.getSearchFailures(), emptyCollectionOf(ShardSearchFailure.class));
|
assertThat(response.getSearchFailures(), emptyCollectionOf(ShardSearchFailure.class));
|
||||||
|
assertNull(response.getReasonCancelled());
|
||||||
} else {
|
} else {
|
||||||
successLatch.await(10, TimeUnit.SECONDS);
|
successLatch.await(10, TimeUnit.SECONDS);
|
||||||
}
|
}
|
||||||
|
@ -360,6 +370,52 @@ public class AsyncBulkByScrollActionTests extends ESTestCase {
|
||||||
assertEquals(defaultBackoffBeforeFailing, millis);
|
assertEquals(defaultBackoffBeforeFailing, millis);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testCancelBeforeInitialSearch() throws Exception {
|
||||||
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.initialSearch());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCancelBeforeScrollResponse() throws Exception {
|
||||||
|
// We bail so early we don't need to pass in a half way valid response.
|
||||||
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.onScrollResponse(null));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCancelBeforeSendBulkRequest() throws Exception {
|
||||||
|
// We bail so early we don't need to pass in a half way valid request.
|
||||||
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.sendBulkRequest(null));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCancelBeforeOnBulkResponse() throws Exception {
|
||||||
|
// We bail so early we don't need to pass in a half way valid response.
|
||||||
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.onBulkResponse(null));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCancelBeforeStartNextScroll() throws Exception {
|
||||||
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNextScroll());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCancelBeforeStartNormalTermination() throws Exception {
|
||||||
|
// Refresh or not doesn't matter - we don't try to refresh.
|
||||||
|
mainRequest.setRefresh(usually());
|
||||||
|
cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNormalTermination(emptyList(), emptyList()));
|
||||||
|
// This wouldn't return if we called refresh - the action would hang waiting for the refresh that we haven't mocked.
|
||||||
|
}
|
||||||
|
|
||||||
|
private void cancelTaskCase(Consumer<DummyAbstractAsyncBulkByScrollAction> testMe) throws Exception {
|
||||||
|
DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction();
|
||||||
|
boolean previousScrollSet = usually();
|
||||||
|
if (previousScrollSet) {
|
||||||
|
action.setScroll(scrollId());
|
||||||
|
}
|
||||||
|
String reason = randomSimpleString(random());
|
||||||
|
taskManager.cancel(task, reason, (Set<String> s) -> {});
|
||||||
|
testMe.accept(action);
|
||||||
|
assertEquals(reason, listener.get().getReasonCancelled());
|
||||||
|
if (previousScrollSet) {
|
||||||
|
// Canceled tasks always start to clear the scroll before they die.
|
||||||
|
assertThat(client.scrollsCleared, contains(scrollId));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private class DummyAbstractAsyncBulkByScrollAction
|
private class DummyAbstractAsyncBulkByScrollAction
|
||||||
extends AbstractAsyncBulkByScrollAction<DummyAbstractBulkByScrollRequest, BulkIndexByScrollResponse> {
|
extends AbstractAsyncBulkByScrollAction<DummyAbstractBulkByScrollRequest, BulkIndexByScrollResponse> {
|
||||||
public DummyAbstractAsyncBulkByScrollAction() {
|
public DummyAbstractAsyncBulkByScrollAction() {
|
||||||
|
|
|
@ -101,13 +101,13 @@ public class BulkByScrollTaskTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testStatusHatesNegatives() {
|
public void testStatusHatesNegatives() {
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(-1, 0, 0, 0, 0, 0, 0, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(-1, 0, 0, 0, 0, 0, 0, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, -1, 0, 0, 0, 0, 0, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, -1, 0, 0, 0, 0, 0, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, -1, 0, 0, 0, 0, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, -1, 0, 0, 0, 0, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, -1, 0, 0, 0, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, -1, 0, 0, 0, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, -1, 0, 0, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, -1, 0, 0, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, 0, -1, 0, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, 0, -1, 0, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, 0, 0, -1, 0));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, 0, 0, -1, 0, null));
|
||||||
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, 0, 0, 0, -1));
|
expectThrows(IllegalArgumentException.class, () -> new BulkByScrollTask.Status(0, 0, 0, 0, 0, 0, 0, -1, null));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,146 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.plugin.reindex;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionResponse;
|
||||||
|
import org.elasticsearch.action.ListenableActionFuture;
|
||||||
|
import org.elasticsearch.action.admin.cluster.node.tasks.list.TaskInfo;
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
import org.elasticsearch.script.ExecutableScript;
|
||||||
|
import org.elasticsearch.script.NativeScriptFactory;
|
||||||
|
import org.elasticsearch.script.Script;
|
||||||
|
import org.elasticsearch.script.ScriptModule;
|
||||||
|
import org.elasticsearch.script.ScriptService.ScriptType;
|
||||||
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.concurrent.BrokenBarrierException;
|
||||||
|
import java.util.concurrent.CyclicBarrier;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.concurrent.TimeoutException;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyMap;
|
||||||
|
import static org.elasticsearch.test.ESIntegTestCase.client;
|
||||||
|
import static org.hamcrest.Matchers.hasSize;
|
||||||
|
import static org.junit.Assert.assertThat;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utilities for testing reindex and update-by-query cancelation. This whole class isn't thread safe. Luckily we run out tests in separate
|
||||||
|
* jvms.
|
||||||
|
*/
|
||||||
|
public class CancelTestUtils {
|
||||||
|
public static Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
|
return Arrays.asList(ReindexPlugin.class, StickyScriptPlugin.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||||
|
|
||||||
|
public static <Request extends AbstractBulkIndexByScrollRequest<Request>,
|
||||||
|
Response extends ActionResponse,
|
||||||
|
Builder extends AbstractBulkIndexByScrollRequestBuilder<Request, Response, Builder>>
|
||||||
|
Response testCancel(ESIntegTestCase test, Builder request, String actionToCancel) throws Exception {
|
||||||
|
|
||||||
|
test.indexRandom(true, client().prepareIndex("source", "test", "1").setSource("foo", "a"),
|
||||||
|
client().prepareIndex("source", "test", "2").setSource("foo", "a"));
|
||||||
|
|
||||||
|
request.source("source").script(new Script("sticky", ScriptType.INLINE, "native", emptyMap()));
|
||||||
|
request.source().setSize(1);
|
||||||
|
ListenableActionFuture<Response> response = request.execute();
|
||||||
|
|
||||||
|
// Wait until the script is on the first document.
|
||||||
|
barrier.await(30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
// Let just one document through.
|
||||||
|
barrier.await(30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
// Wait until the script is on the second document.
|
||||||
|
barrier.await(30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
// Cancel the request while the script is running. This will prevent the request from being sent at all.
|
||||||
|
List<TaskInfo> cancelledTasks = client().admin().cluster().prepareCancelTasks().setActions(actionToCancel).get().getTasks();
|
||||||
|
assertThat(cancelledTasks, hasSize(1));
|
||||||
|
|
||||||
|
// Now let the next document through. It won't be sent because the request is cancelled but we need to unblock the script.
|
||||||
|
barrier.await();
|
||||||
|
|
||||||
|
// Now we can just wait on the request and make sure it was actually cancelled half way through.
|
||||||
|
return response.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class StickyScriptPlugin extends Plugin {
|
||||||
|
@Override
|
||||||
|
public String name() {
|
||||||
|
return "sticky-script";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String description() {
|
||||||
|
return "installs a script that \"sticks\" when it runs for testing reindex";
|
||||||
|
}
|
||||||
|
|
||||||
|
public void onModule(ScriptModule module) {
|
||||||
|
module.registerScript("sticky", StickyScriptFactory.class);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class StickyScriptFactory implements NativeScriptFactory {
|
||||||
|
@Override
|
||||||
|
public ExecutableScript newScript(Map<String, Object> params) {
|
||||||
|
return new ExecutableScript() {
|
||||||
|
private Map<String, Object> source;
|
||||||
|
@Override
|
||||||
|
@SuppressWarnings("unchecked") // Safe because _ctx always has this shape
|
||||||
|
public void setNextVar(String name, Object value) {
|
||||||
|
if ("ctx".equals(name)) {
|
||||||
|
Map<String, Object> ctx = (Map<String, Object>) value;
|
||||||
|
source = (Map<String, Object>) ctx.get("_source");
|
||||||
|
} else {
|
||||||
|
throw new IllegalArgumentException("Unexpected var: " + name);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object run() {
|
||||||
|
try {
|
||||||
|
// Tell the test we've started a document.
|
||||||
|
barrier.await(30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
// Wait for the test to tell us to proceed.
|
||||||
|
barrier.await(30, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
// Make some change to the source so that update-by-query tests can make sure only one document was changed.
|
||||||
|
source.put("giraffes", "giraffes");
|
||||||
|
return null;
|
||||||
|
} catch (InterruptedException | BrokenBarrierException | TimeoutException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean needsScores() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,52 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.plugin.reindex;
|
||||||
|
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
|
||||||
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that you can actually cancel a reindex request and all the plumbing works. Doesn't test all of the different cancellation places -
|
||||||
|
* that is the responsibility of {@link AsyncBulkByScrollActionTests} which have more precise control to simulate failures but do not
|
||||||
|
* exercise important portion of the stack like transport and task management.
|
||||||
|
*/
|
||||||
|
public class ReindexCancelTests extends ReindexTestCase {
|
||||||
|
public void testCancel() throws Exception {
|
||||||
|
ReindexResponse response = CancelTestUtils.testCancel(this, reindex().destination("dest", "test"), ReindexAction.NAME);
|
||||||
|
|
||||||
|
assertThat(response, responseMatcher().created(1).reasonCancelled(equalTo("by user request")));
|
||||||
|
refresh("dest");
|
||||||
|
assertHitCount(client().prepareSearch("dest").setSize(0).get(), 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numberOfShards() {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
|
return CancelTestUtils.nodePlugins();
|
||||||
|
}
|
||||||
|
}
|
|
@ -118,7 +118,8 @@ public class RoundTripTests extends ESTestCase {
|
||||||
|
|
||||||
private BulkByScrollTask.Status randomStatus() {
|
private BulkByScrollTask.Status randomStatus() {
|
||||||
return new BulkByScrollTask.Status(randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(),
|
return new BulkByScrollTask.Status(randomPositiveLong(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(),
|
||||||
randomPositiveInt(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong());
|
randomPositiveInt(), randomPositiveLong(), randomPositiveLong(), randomPositiveLong(),
|
||||||
|
random().nextBoolean() ? null : randomSimpleString(random()));
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<Failure> randomIndexingFailures() {
|
private List<Failure> randomIndexingFailures() {
|
||||||
|
|
|
@ -0,0 +1,53 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.elasticsearch.plugin.reindex;
|
||||||
|
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
|
||||||
|
import java.util.Collection;
|
||||||
|
|
||||||
|
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
|
||||||
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests that you can actually cancel an update-by-query request and all the plumbing works. Doesn't test all of the different cancellation
|
||||||
|
* places - that is the responsibility of {@link AsyncBulkByScrollActionTests} which have more precise control to simulate failures but do
|
||||||
|
* not exercise important portion of the stack like transport and task management.
|
||||||
|
*/
|
||||||
|
public class UpdateByQueryCancelTests extends UpdateByQueryTestCase {
|
||||||
|
public void testCancel() throws Exception {
|
||||||
|
BulkIndexByScrollResponse response = CancelTestUtils.testCancel(this, request(), UpdateByQueryAction.NAME);
|
||||||
|
|
||||||
|
assertThat(response, responseMatcher().updated(1).reasonCancelled(equalTo("by user request")));
|
||||||
|
refresh("source");
|
||||||
|
assertHitCount(client().prepareSearch("source").setSize(0).setQuery(matchQuery("giraffes", "giraffes")).get(), 1);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected int numberOfShards() {
|
||||||
|
return 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
|
return CancelTestUtils.nodePlugins();
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,23 @@
|
||||||
|
---
|
||||||
|
"Totally broken scripts report the error properly":
|
||||||
|
- do:
|
||||||
|
index:
|
||||||
|
index: twitter
|
||||||
|
type: tweet
|
||||||
|
id: 1
|
||||||
|
body: { "user": "kimchy" }
|
||||||
|
- do:
|
||||||
|
indices.refresh: {}
|
||||||
|
|
||||||
|
- do:
|
||||||
|
catch: request
|
||||||
|
reindex:
|
||||||
|
refresh: true
|
||||||
|
body:
|
||||||
|
source:
|
||||||
|
index: twitter
|
||||||
|
dest:
|
||||||
|
index: new_twitter
|
||||||
|
script:
|
||||||
|
inline: syntax errors are fun!
|
||||||
|
- match: {error.reason: 'Failed to compile inline script [syntax errors are fun!] using lang [groovy]'}
|
|
@ -0,0 +1,20 @@
|
||||||
|
---
|
||||||
|
"Totally broken scripts report the error properly":
|
||||||
|
- do:
|
||||||
|
index:
|
||||||
|
index: twitter
|
||||||
|
type: tweet
|
||||||
|
id: 1
|
||||||
|
body: { "user": "kimchy" }
|
||||||
|
- do:
|
||||||
|
indices.refresh: {}
|
||||||
|
|
||||||
|
- do:
|
||||||
|
catch: request
|
||||||
|
update-by-query:
|
||||||
|
index: twitter
|
||||||
|
refresh: true
|
||||||
|
body:
|
||||||
|
script:
|
||||||
|
inline: syntax errors are fun!
|
||||||
|
- match: {error.reason: 'Failed to compile inline script [syntax errors are fun!] using lang [groovy]'}
|
Loading…
Reference in New Issue