Remove isCreated and isFound from the Java API

This is cleanup work from #19566, where @nik9000 suggested trying to nuke the isCreated and isFound methods. I've combined nuking the two methods with removing UpdateHelper.Operation in favor of DocWriteResponse.Operation here.

Closes #19631.
This commit is contained in:
Alexander Lin 2016-07-29 11:21:43 -07:00 committed by Nik Everett
parent c9790a1257
commit 119026b4fb
31 changed files with 131 additions and 145 deletions

View File

@ -240,7 +240,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
location = locationToSync(location, updateResult.writeResult.getLocation());
}
switch (updateResult.result.operation()) {
case UPSERT:
case CREATE:
case INDEX:
@SuppressWarnings("unchecked")
WriteResult<IndexResponse> result = updateResult.writeResult;
@ -267,7 +267,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
item = request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest);
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break;
case NONE:
case NOOP:
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResult.noopResult));
item.setIgnoreOnReplica(); // no need to go to the replica
break;
@ -300,7 +300,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), e)));
} else {
switch (updateResult.result.operation()) {
case UPSERT:
case CREATE:
case INDEX:
IndexRequest indexRequest = updateResult.request();
logFailure(e, "index", request.shardId(), indexRequest);
@ -400,7 +400,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
private UpdateResult shardUpdateOperation(IndexMetaData metaData, BulkShardRequest bulkShardRequest, UpdateRequest updateRequest, IndexShard indexShard) {
UpdateHelper.Result translate = updateHelper.prepare(updateRequest, indexShard);
switch (translate.operation()) {
case UPSERT:
case CREATE:
case INDEX:
IndexRequest indexRequest = translate.action();
try {
@ -427,7 +427,7 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
}
return new UpdateResult(translate, deleteRequest, retry, cause, null);
}
case NONE:
case NOOP:
UpdateResponse updateResponse = translate.action();
indexShard.noopUpdate(updateRequest.type());
return new UpdateResult(translate, updateResponse);

View File

@ -42,21 +42,14 @@ public class DeleteResponse extends DocWriteResponse {
super(shardId, type, id, version, found ? Operation.DELETE : Operation.NOOP);
}
/**
* Returns <tt>true</tt> if a doc was found to delete.
*/
public boolean isFound() {
return operation == Operation.DELETE;
}
@Override
public RestStatus status() {
return isFound() ? super.status() : RestStatus.NOT_FOUND;
return operation == Operation.DELETE ? super.status() : RestStatus.NOT_FOUND;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.field("found", isFound());
builder.field("found", operation == Operation.DELETE);
super.toXContent(builder, params);
return builder;
}

View File

@ -20,8 +20,6 @@
package org.elasticsearch.action.index;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
@ -44,16 +42,9 @@ public class IndexResponse extends DocWriteResponse {
super(shardId, type, id, version, created ? Operation.CREATE : Operation.INDEX);
}
/**
* Returns true if the document was created, false if updated.
*/
public boolean isCreated() {
return this.operation == Operation.CREATE;
}
@Override
public RestStatus status() {
return isCreated() ? RestStatus.CREATED : super.status();
return operation == Operation.CREATE ? RestStatus.CREATED : super.status();
}
@Override
@ -72,7 +63,7 @@ public class IndexResponse extends DocWriteResponse {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
super.toXContent(builder, params);
builder.field("created", isCreated());
builder.field("created", operation == Operation.CREATE);
return builder;
}
}

View File

@ -179,7 +179,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
final IndexShard indexShard = indexService.getShard(shardId.getId());
final UpdateHelper.Result result = updateHelper.prepare(request, indexShard);
switch (result.operation()) {
case UPSERT:
case CREATE:
IndexRequest upsertRequest = result.action();
// we fetch it from the index request so we don't generate the bytes twice, its already done in the index request
final BytesReference upsertSourceBytes = upsertRequest.source();
@ -277,7 +277,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
}
});
break;
case NONE:
case NOOP:
UpdateResponse update = result.action();
IndexService indexServiceOrNull = indicesService.indexService(shardId.getIndex());
if (indexServiceOrNull != null) {

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.update;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.client.Requests;
@ -116,9 +117,9 @@ public class UpdateHelper extends AbstractComponent {
request.script.getScript());
}
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(),
getResult.getVersion(), UpdateResponse.convert(Operation.NONE));
getResult.getVersion(), DocWriteResponse.Operation.NOOP);
update.setGetResult(getResult);
return new Result(update, Operation.NONE, upsertDoc, XContentType.JSON);
return new Result(update, DocWriteResponse.Operation.NOOP, upsertDoc, XContentType.JSON);
}
indexRequest.source((Map) ctx.get("_source"));
}
@ -135,7 +136,7 @@ public class UpdateHelper extends AbstractComponent {
// in all but the internal versioning mode, we want to create the new document using the given version.
indexRequest.version(request.version()).versionType(request.versionType());
}
return new Result(indexRequest, Operation.UPSERT, null, null);
return new Result(indexRequest, DocWriteResponse.Operation.CREATE, null, null);
}
long updateVersion = getResult.getVersion();
@ -226,21 +227,21 @@ public class UpdateHelper extends AbstractComponent {
.consistencyLevel(request.consistencyLevel())
.timestamp(timestamp).ttl(ttl)
.setRefreshPolicy(request.getRefreshPolicy());
return new Result(indexRequest, Operation.INDEX, updatedSourceAsMap, updateSourceContentType);
return new Result(indexRequest, DocWriteResponse.Operation.INDEX, updatedSourceAsMap, updateSourceContentType);
} else if ("delete".equals(operation)) {
DeleteRequest deleteRequest = Requests.deleteRequest(request.index()).type(request.type()).id(request.id()).routing(routing).parent(parent)
.version(updateVersion).versionType(request.versionType())
.consistencyLevel(request.consistencyLevel())
.setRefreshPolicy(request.getRefreshPolicy());
return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType);
return new Result(deleteRequest, DocWriteResponse.Operation.DELETE, updatedSourceAsMap, updateSourceContentType);
} else if ("none".equals(operation)) {
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), UpdateResponse.convert(Operation.NONE));
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), DocWriteResponse.Operation.NOOP);
update.setGetResult(extractGetResult(request, request.index(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef()));
return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType);
return new Result(update, DocWriteResponse.Operation.NOOP, updatedSourceAsMap, updateSourceContentType);
} else {
logger.warn("Used update operation [{}] for script [{}], doing nothing...", operation, request.script.getScript());
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), UpdateResponse.convert(Operation.NONE));
return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType);
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), DocWriteResponse.Operation.NOOP);
return new Result(update, DocWriteResponse.Operation.NOOP, updatedSourceAsMap, updateSourceContentType);
}
}
@ -309,11 +310,11 @@ public class UpdateHelper extends AbstractComponent {
public static class Result {
private final Streamable action;
private final Operation operation;
private final DocWriteResponse.Operation operation;
private final Map<String, Object> updatedSourceAsMap;
private final XContentType updateSourceContentType;
public Result(Streamable action, Operation operation, Map<String, Object> updatedSourceAsMap, XContentType updateSourceContentType) {
public Result(Streamable action, DocWriteResponse.Operation operation, Map<String, Object> updatedSourceAsMap, XContentType updateSourceContentType) {
this.action = action;
this.operation = operation;
this.updatedSourceAsMap = updatedSourceAsMap;
@ -325,7 +326,7 @@ public class UpdateHelper extends AbstractComponent {
return (T) action;
}
public Operation operation() {
public DocWriteResponse.Operation operation() {
return operation;
}
@ -338,10 +339,4 @@ public class UpdateHelper extends AbstractComponent {
}
}
public enum Operation {
UPSERT,
INDEX,
DELETE,
NONE
}
}

View File

@ -50,20 +50,6 @@ public class UpdateResponse extends DocWriteResponse {
setShardInfo(shardInfo);
}
public static Operation convert(UpdateHelper.Operation op) {
switch(op) {
case UPSERT:
return Operation.CREATE;
case INDEX:
return Operation.INDEX;
case DELETE:
return Operation.DELETE;
case NONE:
return Operation.NOOP;
}
throw new IllegalArgumentException();
}
public void setGetResult(GetResult getResult) {
this.getResult = getResult;
}
@ -72,16 +58,9 @@ public class UpdateResponse extends DocWriteResponse {
return this.getResult;
}
/**
* Returns true if document was created due to an UPSERT operation
*/
public boolean isCreated() {
return this.operation == Operation.CREATE;
}
@Override
public RestStatus status() {
return isCreated() ? RestStatus.CREATED : super.status();
return this.operation == Operation.CREATE ? RestStatus.CREATED : super.status();
}
@Override

View File

@ -234,7 +234,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
client().prepareIndex(indexOrAlias, "type", "id").setSource("field", "value").get();
UpdateRequest updateRequest = new UpdateRequest(indexOrAlias, "type", "id").doc("field1", "value1");
UpdateResponse updateResponse = internalCluster().coordOnlyNodeClient().update(updateRequest).actionGet();
assertThat(updateResponse.isCreated(), equalTo(false));
assertEquals(DocWriteResponse.Operation.INDEX, updateResponse.getOperation());
clearInterceptedActions();
assertSameIndices(updateRequest, updateShardActions);
@ -248,7 +248,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
String indexOrAlias = randomIndexOrAlias();
UpdateRequest updateRequest = new UpdateRequest(indexOrAlias, "type", "id").upsert("field", "value").doc("field1", "value1");
UpdateResponse updateResponse = internalCluster().coordOnlyNodeClient().update(updateRequest).actionGet();
assertThat(updateResponse.isCreated(), equalTo(true));
assertEquals(DocWriteResponse.Operation.CREATE, updateResponse.getOperation());
clearInterceptedActions();
assertSameIndices(updateRequest, updateShardActions);
@ -264,7 +264,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
UpdateRequest updateRequest = new UpdateRequest(indexOrAlias, "type", "id")
.script(new Script("ctx.op='delete'", ScriptService.ScriptType.INLINE, CustomScriptPlugin.NAME, Collections.emptyMap()));
UpdateResponse updateResponse = internalCluster().coordOnlyNodeClient().update(updateRequest).actionGet();
assertThat(updateResponse.isCreated(), equalTo(false));
assertEquals(DocWriteResponse.Operation.DELETE, updateResponse.getOperation());
clearInterceptedActions();
assertSameIndices(updateRequest, updateShardActions);

View File

@ -19,11 +19,11 @@
package org.elasticsearch.action.bulk;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateRequest;
@ -207,11 +207,11 @@ public class BulkWithUpdatesIT extends ESIntegTestCase {
.add(client().prepareIndex("test", "type", "2").setCreate(true).setSource("field", "1"))
.add(client().prepareIndex("test", "type", "1").setSource("field", "2")).get();
assertTrue(((IndexResponse) bulkResponse.getItems()[0].getResponse()).isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, bulkResponse.getItems()[0].getResponse().getOperation());
assertThat(bulkResponse.getItems()[0].getResponse().getVersion(), equalTo(1L));
assertTrue(((IndexResponse) bulkResponse.getItems()[1].getResponse()).isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, bulkResponse.getItems()[1].getResponse().getOperation());
assertThat(bulkResponse.getItems()[1].getResponse().getVersion(), equalTo(1L));
assertFalse(((IndexResponse) bulkResponse.getItems()[2].getResponse()).isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, bulkResponse.getItems()[2].getResponse().getOperation());
assertThat(bulkResponse.getItems()[2].getResponse().getVersion(), equalTo(2L));
bulkResponse = client().prepareBulk()
@ -232,11 +232,11 @@ public class BulkWithUpdatesIT extends ESIntegTestCase {
.setSource("field", "2").setVersion(12).setVersionType(VersionType.EXTERNAL))
.get();
assertTrue(((IndexResponse) bulkResponse.getItems()[0].getResponse()).isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, bulkResponse.getItems()[0].getResponse().getOperation());
assertThat(bulkResponse.getItems()[0].getResponse().getVersion(), equalTo(10L));
assertTrue(((IndexResponse) bulkResponse.getItems()[1].getResponse()).isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, bulkResponse.getItems()[1].getResponse().getOperation());
assertThat(bulkResponse.getItems()[1].getResponse().getVersion(), equalTo(10L));
assertFalse(((IndexResponse) bulkResponse.getItems()[2].getResponse()).isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, bulkResponse.getItems()[2].getResponse().getOperation());
assertThat(bulkResponse.getItems()[2].getResponse().getVersion(), equalTo(12L));
bulkResponse = client().prepareBulk()

View File

@ -19,6 +19,7 @@ package org.elasticsearch.action.support.master;
* under the License.
*/
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.DiscoverySettings;
@ -97,7 +98,7 @@ public class IndexingMasterFailoverIT extends ESIntegTestCase {
for (int i = 0; i < 10; i++) {
// index data with mapping changes
IndexResponse response = client(dataNode).prepareIndex("myindex", "mytype").setSource("field_" + i, "val").get();
assertThat(response.isCreated(), equalTo(true));
assertEquals(DocWriteResponse.Operation.CREATE, response.getOperation());
}
}
});

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.util.English;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse;
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
@ -118,7 +119,8 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase {
for (int i = 0; i < numDocs; i++) {
String routingKey = routing ? randomRealisticUnicodeOfLength(10) : null;
String id = Integer.toString(i);
assertThat(id, client().prepareIndex("test", "type1", id).setRouting(routingKey).setSource("field1", English.intToEnglish(i)).get().isCreated(), is(true));
assertEquals(id, DocWriteResponse.Operation.CREATE, client().prepareIndex("test", "type1", id)
.setRouting(routingKey).setSource("field1", English.intToEnglish(i)).get().getOperation());
GetResponse get = client().prepareGet("test", "type1", id).setRouting(routingKey).setVersion(1).get();
assertThat("Document with ID " + id + " should exist but doesn't", get.isExists(), is(true));
assertThat(get.getVersion(), equalTo(1L));
@ -476,7 +478,7 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase {
assertThat(searchResponse.getHits().totalHits(), equalTo((long) numDocs));
DeleteResponse deleteResponse = client().prepareDelete("test", "test", firstDocId).setRouting("routing").get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
GetResponse getResponse = client().prepareGet("test", "test", firstDocId).setRouting("routing").get();
assertThat(getResponse.isExists(), equalTo(false));
refresh();
@ -491,7 +493,7 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase {
int numDocs = iterations(10, 50);
for (int i = 0; i < numDocs; i++) {
IndexResponse indexResponse = client().prepareIndex(indexOrAlias(), "type", Integer.toString(i)).setSource("field", "value-" + i).get();
assertThat(indexResponse.isCreated(), equalTo(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
assertThat(indexResponse.getIndex(), equalTo("test"));
assertThat(indexResponse.getType(), equalTo("type"));
assertThat(indexResponse.getId(), equalTo(Integer.toString(i)));
@ -506,7 +508,7 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase {
assertThat(getResponse.getId(), equalTo(docId));
DeleteResponse deleteResponse = client().prepareDelete(indexOrAlias(), "type", docId).get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getIndex(), equalTo("test"));
assertThat(deleteResponse.getType(), equalTo("type"));
assertThat(deleteResponse.getId(), equalTo(docId));
@ -530,7 +532,7 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase {
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getType(), equalTo("type1"));
assertThat(updateResponse.getId(), equalTo("1"));
assertThat(updateResponse.isCreated(), equalTo(true));
assertEquals(DocWriteResponse.Operation.CREATE, updateResponse.getOperation());
GetResponse getResponse = client().prepareGet("test", "type1", "1").get();
assertThat(getResponse.isExists(), equalTo(true));
@ -541,7 +543,7 @@ public class BasicBackwardsCompatibilityIT extends ESBackcompatTestCase {
assertThat(updateResponse.getIndex(), equalTo("test"));
assertThat(updateResponse.getType(), equalTo("type1"));
assertThat(updateResponse.getId(), equalTo("1"));
assertThat(updateResponse.isCreated(), equalTo(false));
assertEquals(DocWriteResponse.Operation.INDEX, updateResponse.getOperation());
getResponse = client().prepareGet("test", "type1", "1").get();
assertThat(getResponse.isExists(), equalTo(true));

View File

@ -21,6 +21,7 @@ package org.elasticsearch.discovery;
import org.apache.lucene.index.CorruptIndexException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.index.IndexResponse;
@ -490,7 +491,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
logger.trace("[{}] indexing id [{}] through node [{}] targeting shard [{}]", name, id, node, shard);
IndexResponse response =
client.prepareIndex("test", "type", id).setSource("{}").setTimeout(timeout).get(timeout);
assertTrue("doc [" + id + "] should have been created", response.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, response.getOperation());
ackedDocs.put(id, node);
logger.trace("[{}] indexed id [{}] through node [{}]", name, id, node);
} catch (ElasticsearchException e) {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.get;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
@ -176,7 +177,7 @@ public class GetActionIT extends ESIntegTestCase {
assertThat(response.getSourceAsMap().get("field2").toString(), equalTo("value2_2"));
DeleteResponse deleteResponse = client().prepareDelete("test", "type1", "1").get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
response = client().prepareGet(indexOrAlias(), "type1", "1").get();
assertThat(response.isExists(), equalTo(false));

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
@ -414,7 +415,7 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
try {
final IndexResponse indexResponse = client().prepareIndex(IDX, "doc",
Integer.toString(counter.incrementAndGet())).setSource("foo", "bar").get();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
} catch (Exception e) {
exceptions.add(e);
}
@ -507,7 +508,7 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
while (counter.get() < (numPhase1Docs + numPhase2Docs + numPhase3Docs)) {
final IndexResponse indexResponse = client().prepareIndex(IDX, "doc",
Integer.toString(counter.incrementAndGet())).setSource("foo", "bar").get();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
final int docCount = counter.get();
if (docCount == numPhase1Docs) {
phase1finished.countDown();

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.ListenableActionFuture;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder;
@ -27,7 +28,6 @@ import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.ScriptPlugin;
@ -35,7 +35,6 @@ import org.elasticsearch.rest.RestStatus;
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 org.junit.Before;
@ -85,7 +84,7 @@ public class WaitUntilRefreshIT extends ESIntegTestCase {
// Now delete with blockUntilRefresh
DeleteResponse delete = client().prepareDelete("test", "test", "1").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get();
assertTrue("document was deleted", delete.isFound());
assertEquals(DocWriteResponse.Operation.DELETE, delete.getOperation());
assertFalse("request shouldn't have forced a refresh", delete.forcedRefresh());
assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get());
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.index.mapper;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.get.GetIndexResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.bulk.BulkResponse;
@ -99,7 +100,8 @@ public class DynamicMappingIT extends ESIntegTestCase {
public void run() {
try {
startLatch.await();
assertTrue(client().prepareIndex("index", "type", id).setSource("field" + id, "bar").get().isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, client().prepareIndex("index", "type", id)
.setSource("field" + id, "bar").get().getOperation());
} catch (Exception e) {
error.compareAndSet(null, e);
}
@ -140,7 +142,7 @@ public class DynamicMappingIT extends ESIntegTestCase {
() -> client().prepareIndex("index_2", "bar", "1").setSource("field", "abc").get());
assertEquals("type[bar] missing", e1.getMessage());
assertEquals("trying to auto create mapping, but dynamic mapping is disabled", e1.getCause().getMessage());
// make sure no mappings were created for bar
GetIndexResponse getIndexResponse = client().admin().indices().prepareGetIndex().addIndices("index_2").get();
assertFalse(getIndexResponse.mappings().containsKey("bar"));

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index.mapper.core;
import com.carrotsearch.randomizedtesting.annotations.Name;
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchRequestBuilder;
@ -136,12 +137,13 @@ public class TokenCountFieldMapperIntegrationIT extends ESIntegTestCase {
.endObject().endObject()).get();
ensureGreen();
assertTrue(prepareIndex("single", "I have four terms").get().isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, prepareIndex("single", "I have four terms").get().getOperation());
BulkResponse bulk = client().prepareBulk()
.add(prepareIndex("bulk1", "bulk three terms"))
.add(prepareIndex("bulk2", "this has five bulk terms")).get();
assertFalse(bulk.buildFailureMessage(), bulk.hasFailures());
assertTrue(prepareIndex("multi", "two terms", "wow now I have seven lucky terms").get().isCreated());
assertEquals(DocWriteResponse.Operation.CREATE,
prepareIndex("multi", "two terms", "wow now I have seven lucky terms").get().getOperation());
bulk = client().prepareBulk()
.add(prepareIndex("multibulk1", "one", "oh wow now I have eight unlucky terms"))
.add(prepareIndex("multibulk2", "six is a bunch of terms", "ten! ten terms is just crazy! too many too count!")).get();

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.search.LegacyNumericRangeQuery;
import org.apache.lucene.util.Constants;
import org.elasticsearch.Version;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -448,7 +449,7 @@ public class LegacyDateMappingTests extends ESSingleNodeTestCase {
ParsedDocument doc = defaultMapper.parse("test", "type", "1", document.bytes());
assertThat(getDateAsMillis(doc.rootDoc(), "date_field"), equalTo(1433239200000L));
IndexResponse indexResponse = client().prepareIndex("test2", "test").setSource(document).get();
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
// integers should always be parsed as well... cannot be sure it is a unix timestamp only
doc = defaultMapper.parse("test", "type", "1", XContentFactory.jsonBuilder()
@ -458,7 +459,7 @@ public class LegacyDateMappingTests extends ESSingleNodeTestCase {
.bytes());
assertThat(getDateAsMillis(doc.rootDoc(), "date_field"), equalTo(1433239200000L));
indexResponse = client().prepareIndex("test", "test").setSource(document).get();
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
}
public void testThatNewIndicesOnlyAllowStrictDates() throws Exception {

View File

@ -28,6 +28,7 @@ import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.recovery.RecoveryRequest;
import org.elasticsearch.action.admin.indices.stats.IndexShardStats;
@ -256,7 +257,7 @@ public abstract class ESIndexLevelReplicationTestCase extends ESTestCase {
final IndexRequest indexRequest = new IndexRequest(index.getName(), "type", Integer.toString(docId.incrementAndGet()))
.source("{}");
final IndexResponse response = index(indexRequest);
assertThat(response.isCreated(), equalTo(true));
assertEquals(DocWriteResponse.Operation.CREATE, response.getOperation());
}
return numOfDoc;
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.indexing;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.index.IndexResponse;
@ -93,15 +94,15 @@ public class IndexActionIT extends ESIntegTestCase {
ensureGreen();
IndexResponse indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").execute().actionGet();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_2").execute().actionGet();
assertFalse(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, indexResponse.getOperation());
client().prepareDelete("test", "type", "1").execute().actionGet();
indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_2").execute().actionGet();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
}
@ -110,14 +111,14 @@ public class IndexActionIT extends ESIntegTestCase {
ensureGreen();
IndexResponse indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").execute().actionGet();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
client().prepareDelete("test", "type", "1").execute().actionGet();
flush();
indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_2").execute().actionGet();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
}
public void testCreatedFlagParallelExecution() throws Exception {
@ -138,7 +139,9 @@ public class IndexActionIT extends ESIntegTestCase {
public Void call() throws Exception {
int docId = random.nextInt(docCount);
IndexResponse indexResponse = index("test", "type", Integer.toString(docId), "field1", "value");
if (indexResponse.isCreated()) createdCounts.incrementAndGet(docId);
if (indexResponse.getOperation() == DocWriteResponse.Operation.CREATE) {
createdCounts.incrementAndGet(docId);
}
return null;
}
});
@ -158,7 +161,7 @@ public class IndexActionIT extends ESIntegTestCase {
IndexResponse indexResponse = client().prepareIndex("test", "type", "1").setSource("field1", "value1_1").setVersion(123)
.setVersionType(VersionType.EXTERNAL).execute().actionGet();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
}
public void testCreateFlagWithBulk() {
@ -169,7 +172,7 @@ public class IndexActionIT extends ESIntegTestCase {
assertThat(bulkResponse.hasFailures(), equalTo(false));
assertThat(bulkResponse.getItems().length, equalTo(1));
IndexResponse indexResponse = bulkResponse.getItems()[0].getResponse();
assertTrue(indexResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
}
public void testCreateIndexWithLongName() {

View File

@ -19,10 +19,12 @@
package org.elasticsearch.indices;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.update.UpdateHelper;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.test.ESIntegTestCase;
import org.joda.time.DateTime;
@ -74,15 +76,15 @@ public class DateMathIndexExpressionsIntegrationIT extends ESIntegTestCase {
assertThat(indicesStatsResponse.getIndex(index3), notNullValue());
DeleteResponse deleteResponse = client().prepareDelete(dateMathExp1, "type", "1").get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getId(), equalTo("1"));
deleteResponse = client().prepareDelete(dateMathExp2, "type", "2").get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getId(), equalTo("2"));
deleteResponse = client().prepareDelete(dateMathExp3, "type", "3").get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getId(), equalTo("3"));
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.indices.recovery;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.index.IndexResponse;
@ -55,9 +56,9 @@ public class IndexPrimaryRelocationIT extends ESIntegTestCase {
public void run() {
while (finished.get() == false) {
IndexResponse indexResponse = client().prepareIndex("test", "type", "id").setSource("field", "value").get();
assertThat("deleted document was found", indexResponse.isCreated(), equalTo(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
DeleteResponse deleteResponse = client().prepareDelete("test", "type", "id").get();
assertThat("indexed document was not found", deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
}
}
};

View File

@ -20,6 +20,7 @@
package org.elasticsearch.indices.stats;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
@ -1036,8 +1037,8 @@ public class IndexStatsIT extends ESIntegTestCase {
assertThat(stats.getTotal().queryCache.getCacheSize(), greaterThan(0L));
});
assertTrue(client().prepareDelete("index", "type", "1").get().isFound());
assertTrue(client().prepareDelete("index", "type", "2").get().isFound());
assertEquals(DocWriteResponse.Operation.DELETE, client().prepareDelete("index", "type", "1").get().getOperation());
assertEquals(DocWriteResponse.Operation.DELETE, client().prepareDelete("index", "type", "2").get().getOperation());
refresh();
response = client().admin().indices().prepareStats("index").setQueryCache(true).get();
assertCumulativeQueryCacheStats(response);

View File

@ -22,6 +22,7 @@ package org.elasticsearch.ingest;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse;
@ -161,7 +162,7 @@ public class IngestClientIT extends ESIntegTestCase {
itemResponse.isFailed(), is(false));
assertThat(indexResponse, notNullValue());
assertThat(indexResponse.getId(), equalTo(Integer.toString(i)));
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
}
}
}

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.util.English;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
@ -107,7 +108,7 @@ public class SearchWithRandomExceptionsIT extends ESIntegTestCase {
for (int i = 0; i < numDocs; i++) {
try {
IndexResponse indexResponse = client().prepareIndex("test", "type", "" + i).setTimeout(TimeValue.timeValueSeconds(1)).setSource("test", English.intToEnglish(i)).get();
if (indexResponse.isCreated()) {
if (indexResponse.getOperation() == DocWriteResponse.Operation.CREATE) {
numCreated++;
added[i] = true;
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.basic;
import org.apache.lucene.util.English;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.index.IndexResponse;
@ -136,7 +137,7 @@ public class SearchWithRandomIOExceptionsIT extends ESIntegTestCase {
added[i] = false;
try {
IndexResponse indexResponse = client().prepareIndex("test", "type", Integer.toString(i)).setTimeout(TimeValue.timeValueSeconds(1)).setSource("test", English.intToEnglish(i)).get();
if (indexResponse.isCreated()) {
if (indexResponse.getOperation() == DocWriteResponse.Operation.CREATE) {
numCreated++;
added[i] = true;
}

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.nested;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.cluster.stats.ClusterStatsResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.delete.DeleteResponse;
@ -148,7 +149,7 @@ public class SimpleNestedIT extends ESIntegTestCase {
// check delete, so all is gone...
DeleteResponse deleteResponse = client().prepareDelete("test", "type1", "2").execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
// flush, so we fetch it from the index (as see that we filter nested docs)
flush();

View File

@ -20,6 +20,7 @@
package org.elasticsearch.ttl;
import org.elasticsearch.Version;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
@ -105,14 +106,14 @@ public class SimpleTTLIT extends ESIntegTestCase {
long now = System.currentTimeMillis();
IndexResponse indexResponse = client().prepareIndex("test", "type1", "1").setSource("field1", "value1")
.setTimestamp(String.valueOf(now)).setTTL(providedTTLValue).setRefreshPolicy(IMMEDIATE).get();
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
indexResponse = client().prepareIndex("test", "type1", "with_routing").setSource("field1", "value1")
.setTimestamp(String.valueOf(now)).setTTL(providedTTLValue).setRouting("routing").setRefreshPolicy(IMMEDIATE).get();
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
indexResponse = client().prepareIndex("test", "type1", "no_ttl").setSource("field1", "value1").get();
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
indexResponse = client().prepareIndex("test", "type2", "default_ttl").setSource("field1", "value1").get();
assertThat(indexResponse.isCreated(), is(true));
assertEquals(DocWriteResponse.Operation.CREATE, indexResponse.getOperation());
// realtime get check
long currentTime = System.currentTimeMillis();
@ -258,7 +259,7 @@ public class SimpleTTLIT extends ESIntegTestCase {
long thirdTtl = aLongTime * 1;
IndexResponse indexResponse = client().prepareIndex("test", "type1", "1").setSource("field1", "value1")
.setTTL(firstTtl).setRefreshPolicy(IMMEDIATE).get();
assertTrue(indexResponse.isCreated());
assertTrue(indexResponse.getOperation() == DocWriteResponse.Operation.CREATE);
assertThat(getTtl("type1", 1), both(lessThanOrEqualTo(firstTtl)).and(greaterThan(secondTtl)));
// Updating with the default detect_noop without a change to the document doesn't change the ttl.

View File

@ -22,6 +22,7 @@ package org.elasticsearch.update;
import org.elasticsearch.ElasticsearchTimeoutException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.alias.Alias;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteResponse;
@ -370,7 +371,7 @@ public class UpdateIT extends ESIntegTestCase {
.setUpsert(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject())
.setScript(new Script("field", ScriptService.ScriptType.INLINE, "field_inc", null))
.execute().actionGet();
assertTrue(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -382,7 +383,7 @@ public class UpdateIT extends ESIntegTestCase {
.setUpsert(XContentFactory.jsonBuilder().startObject().field("field", 1).endObject())
.setScript(new Script("field", ScriptService.ScriptType.INLINE, "field_inc", null))
.execute().actionGet();
assertFalse(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -411,7 +412,7 @@ public class UpdateIT extends ESIntegTestCase {
.setScriptedUpsert(true)
.setScript(new Script("", ScriptService.ScriptType.INLINE, "scripted_upsert", params))
.execute().actionGet();
assertTrue(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.CREATE, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -425,7 +426,7 @@ public class UpdateIT extends ESIntegTestCase {
.setScriptedUpsert(true)
.setScript(new Script("", ScriptService.ScriptType.INLINE, "scripted_upsert", params))
.execute().actionGet();
assertFalse(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -581,7 +582,7 @@ public class UpdateIT extends ESIntegTestCase {
UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript(new Script("field", ScriptService.ScriptType.INLINE, "field_inc", null)).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(2L));
assertFalse(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -594,7 +595,7 @@ public class UpdateIT extends ESIntegTestCase {
updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript(new Script("field", ScriptService.ScriptType.INLINE, "field_inc", params)).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(3L));
assertFalse(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.INDEX, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -606,7 +607,7 @@ public class UpdateIT extends ESIntegTestCase {
updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript(new Script("", ScriptService.ScriptType.INLINE, "put_values", Collections.singletonMap("_ctx", Collections.singletonMap("op", "none")))).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(3L));
assertFalse(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.NOOP, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {
@ -618,7 +619,7 @@ public class UpdateIT extends ESIntegTestCase {
updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1")
.setScript(new Script("", ScriptService.ScriptType.INLINE, "put_values", Collections.singletonMap("_ctx", Collections.singletonMap("op", "delete")))).execute().actionGet();
assertThat(updateResponse.getVersion(), equalTo(4L));
assertFalse(updateResponse.isCreated());
assertEquals(DocWriteResponse.Operation.DELETE, updateResponse.getOperation());
assertThat(updateResponse.getIndex(), equalTo("test"));
for (int i = 0; i < 5; i++) {

View File

@ -20,6 +20,7 @@ package org.elasticsearch.versioning;
import org.apache.lucene.util.TestUtil;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.index.IndexRequest;
@ -58,7 +59,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// Note - external version doesn't throw version conflicts on deletes of non existent records. This is different from internal versioning
DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setVersion(17).setVersionType(VersionType.EXTERNAL).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(false));
assertEquals(DocWriteResponse.Operation.NOOP, deleteResponse.getOperation());
// this should conflict with the delete command transaction which told us that the object was deleted at version 17.
assertThrows(
@ -97,7 +98,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// deleting with a lower version works.
long v = randomIntBetween(12, 14);
DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setVersion(v).setVersionType(VersionType.FORCE).get();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(v));
}
@ -132,7 +133,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// Delete with a higher or equal version deletes all versions up to the given one.
long v = randomIntBetween(14, 17);
DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setVersion(v).setVersionType(VersionType.EXTERNAL_GTE).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(v));
// Deleting with a lower version keeps on failing after a delete.
@ -143,7 +144,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// But delete with a higher version is OK.
deleteResponse = client().prepareDelete("test", "type", "1").setVersion(18).setVersionType(VersionType.EXTERNAL_GTE).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(false));
assertEquals(DocWriteResponse.Operation.NOOP, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(18L));
}
@ -174,7 +175,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// Delete with a higher version deletes all versions up to the given one.
DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setVersion(17).setVersionType(VersionType.EXTERNAL).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(17L));
// Deleting with a lower version keeps on failing after a delete.
@ -185,7 +186,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// But delete with a higher version is OK.
deleteResponse = client().prepareDelete("test", "type", "1").setVersion(18).setVersionType(VersionType.EXTERNAL).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(false));
assertEquals(DocWriteResponse.Operation.NOOP, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(18L));
@ -195,7 +196,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
deleteResponse = client().prepareDelete("test", "type", "1").setVersion(20).setVersionType(VersionType.EXTERNAL).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(20L));
// Make sure that the next delete will be GC. Note we do it on the index settings so it will be cleaned up
@ -280,7 +281,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
}
DeleteResponse deleteResponse = client().prepareDelete("test", "type", "1").setVersion(2).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(true));
assertEquals(DocWriteResponse.Operation.DELETE, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(3L));
assertThrows(client().prepareDelete("test", "type", "1").setVersion(2).execute(), VersionConflictEngineException.class);
@ -289,7 +290,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
// This is intricate - the object was deleted but a delete transaction was with the right version. We add another one
// and thus the transaction is increased.
deleteResponse = client().prepareDelete("test", "type", "1").setVersion(3).execute().actionGet();
assertThat(deleteResponse.isFound(), equalTo(false));
assertEquals(DocWriteResponse.Operation.NOOP, deleteResponse.getOperation());
assertThat(deleteResponse.getVersion(), equalTo(4L));
}
@ -478,7 +479,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
sb.append(" version=");
sb.append(deleteResponse.getVersion());
sb.append(" found=");
sb.append(deleteResponse.isFound());
sb.append(deleteResponse.getOperation() == DocWriteResponse.Operation.DELETE);
} else if (response instanceof IndexResponse) {
IndexResponse indexResponse = (IndexResponse) response;
sb.append(" index=");
@ -490,7 +491,7 @@ public class SimpleVersioningIT extends ESIntegTestCase {
sb.append(" version=");
sb.append(indexResponse.getVersion());
sb.append(" created=");
sb.append(indexResponse.isCreated());
sb.append(indexResponse.getOperation() == DocWriteResponse.Operation.CREATE);
} else {
sb.append(" response: " + response);
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BackoffPolicy;
@ -78,7 +79,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
private final ParentTaskAssigningClient client;
private final ActionListener<BulkIndexByScrollResponse> listener;
private final Retry bulkRetry;
private final ScrollableHitSource scrollSource;
private final ScrollableHitSource scrollSource;
public AbstractAsyncBulkByScrollAction(BulkByScrollTask task, ESLogger logger, ParentTaskAssigningClient client,
ThreadPool threadPool, Request mainRequest, ActionListener<BulkIndexByScrollResponse> listener) {
@ -254,7 +255,7 @@ public abstract class AbstractAsyncBulkByScrollAction<Request extends AbstractBu
case "index":
case "create":
IndexResponse ir = item.getResponse();
if (ir.isCreated()) {
if (ir.getOperation() == DocWriteResponse.Operation.CREATE) {
task.countCreated();
} else {
task.countUpdated();

View File

@ -28,6 +28,7 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.discovery.DiscoveryModule;
import org.elasticsearch.client.RestClientBuilder;
import org.elasticsearch.transport.MockTcpTransportPlugin;
@ -1380,8 +1381,9 @@ public abstract class ESIntegTestCase extends ESTestCase {
if (!bogusIds.isEmpty()) {
// delete the bogus types again - it might trigger merges or at least holes in the segments and enforces deleted docs!
for (Tuple<String, String> doc : bogusIds) {
assertTrue("failed to delete a dummy doc [" + doc.v1() + "][" + doc.v2() + "]",
client().prepareDelete(doc.v1(), RANDOM_BOGUS_TYPE, doc.v2()).get().isFound());
assertEquals("failed to delete a dummy doc [" + doc.v1() + "][" + doc.v2() + "]",
DocWriteResponse.Operation.DELETE,
client().prepareDelete(doc.v1(), RANDOM_BOGUS_TYPE, doc.v2()).get().getOperation());
}
}
if (forceRefresh) {