diff --git a/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java b/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java new file mode 100644 index 0000000000000..bb63075a0b902 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java @@ -0,0 +1,143 @@ +/* + * 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.action; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; +import org.elasticsearch.index.seqno.SequenceNumbersService; +import org.elasticsearch.index.shard.ShardId; + +import java.io.IOException; + +/** + * A base class for the response of a write operation that involves a single doc + */ +public abstract class DocWriteResponse extends ReplicationResponse implements ToXContent { + + private ShardId shardId; + private String id; + private String type; + private long version; + private long seqNo; + + public DocWriteResponse(ShardId shardId, String type, String id, long seqNo, long version) { + this.shardId = shardId; + this.type = type; + this.id = id; + this.seqNo = seqNo; + this.version = version; + } + + // needed for deserialization + protected DocWriteResponse() { + } + + /** + * The index the document was changed in. + */ + public String getIndex() { + return this.shardId.getIndex(); + } + + + /** + * The exact shard the document was changed in. + */ + public ShardId getShardId() { + return this.shardId; + } + + /** + * The type of the document changed. + */ + public String getType() { + return this.type; + } + + /** + * The id of the document changed. + */ + public String getId() { + return this.id; + } + + /** + * Returns the current version of the doc. + */ + public long getVersion() { + return this.version; + } + + /** + * Returns the sequence number assigned for this change. Returns {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} if the operation wasn't + * performed (i.e., an update operation that resulted in a NOOP). + */ + public long getSeqNo() { + return seqNo; + } + + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + shardId = ShardId.readShardId(in); + type = in.readString(); + id = in.readString(); + version = in.readZLong(); + seqNo = in.readZLong(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + out.writeString(type); + out.writeString(id); + out.writeZLong(version); + out.writeZLong(seqNo); + } + + static final class Fields { + static final XContentBuilderString _INDEX = new XContentBuilderString("_index"); + static final XContentBuilderString _TYPE = new XContentBuilderString("_type"); + static final XContentBuilderString _ID = new XContentBuilderString("_id"); + static final XContentBuilderString _VERSION = new XContentBuilderString("_version"); + static final XContentBuilderString _SHARD_ID = new XContentBuilderString("_shard_id"); + static final XContentBuilderString _SEQ_NO = new XContentBuilderString("_seq_no"); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + ReplicationResponse.ShardInfo shardInfo = getShardInfo(); + builder.field(Fields._INDEX, getIndex()) + .field(Fields._TYPE, getType()) + .field(Fields._ID, getId()) + .field(Fields._VERSION, getVersion()); + //nocommit: i'm not sure we want to expose it in the api but it will be handy for debugging while we work... + builder.field(Fields._SHARD_ID, shardId.id()); + if (getSeqNo() >= 0) { + builder.field(Fields._SEQ_NO, getSeqNo()); + } + shardInfo.toXContent(builder, params); + return builder; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/ActionWriteResponse.java b/core/src/main/java/org/elasticsearch/action/ReplicationResponse.java similarity index 96% rename from core/src/main/java/org/elasticsearch/action/ActionWriteResponse.java rename to core/src/main/java/org/elasticsearch/action/ReplicationResponse.java index f4152ac85e4aa..4e358c8d42a73 100644 --- a/core/src/main/java/org/elasticsearch/action/ActionWriteResponse.java +++ b/core/src/main/java/org/elasticsearch/action/ReplicationResponse.java @@ -21,7 +21,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.bootstrap.Elasticsearch; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -30,25 +29,23 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; -import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.rest.RestStatus; import java.io.IOException; -import java.util.Collections; /** * Base class for write action responses. */ -public class ActionWriteResponse extends ActionResponse { +public class ReplicationResponse extends ActionResponse { - public final static ActionWriteResponse.ShardInfo.Failure[] EMPTY = new ActionWriteResponse.ShardInfo.Failure[0]; + public final static ReplicationResponse.ShardInfo.Failure[] EMPTY = new ReplicationResponse.ShardInfo.Failure[0]; private ShardInfo shardInfo; @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - shardInfo = ActionWriteResponse.ShardInfo.readShardInfo(in); + shardInfo = ReplicationResponse.ShardInfo.readShardInfo(in); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportFlushAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportFlushAction.java index ac159625420e7..bbcaf7ef5398f 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportFlushAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportFlushAction.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.indices.flush; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction; @@ -36,7 +36,7 @@ /** * Flush Action. */ -public class TransportFlushAction extends TransportBroadcastReplicationAction { +public class TransportFlushAction extends TransportBroadcastReplicationAction { @Inject public TransportFlushAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, @@ -47,8 +47,8 @@ public TransportFlushAction(Settings settings, ThreadPool threadPool, ClusterSer } @Override - protected ActionWriteResponse newShardResponse() { - return new ActionWriteResponse(); + protected ReplicationResponse newShardResponse() { + return new ReplicationResponse(); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java index f768cfedc9444..7e8bd9eb8cbd6 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/flush/TransportShardFlushAction.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.indices.flush; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.ClusterService; @@ -42,7 +42,7 @@ /** * */ -public class TransportShardFlushAction extends TransportReplicationAction { +public class TransportShardFlushAction extends TransportReplicationAction { public static final String NAME = FlushAction.NAME + "[s]"; @@ -56,16 +56,16 @@ public TransportShardFlushAction(Settings settings, TransportService transportSe } @Override - protected ActionWriteResponse newResponseInstance() { - return new ActionWriteResponse(); + protected ReplicationResponse newResponseInstance() { + return new ReplicationResponse(); } @Override - protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable { + protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).getShard(shardRequest.shardId.id()); indexShard.flush(shardRequest.request.getRequest()); logger.trace("{} flush request executed on primary", indexShard.shardId()); - return new Tuple<>(new ActionWriteResponse(), shardRequest.request); + return new Tuple<>(new ReplicationResponse(), shardRequest.request); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java index e2d978d306f26..55b3696c10bcf 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.indices.refresh; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.ReplicationRequest; @@ -37,7 +37,7 @@ /** * Refresh action. */ -public class TransportRefreshAction extends TransportBroadcastReplicationAction { +public class TransportRefreshAction extends TransportBroadcastReplicationAction { @Inject public TransportRefreshAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, @@ -48,8 +48,8 @@ public TransportRefreshAction(Settings settings, ThreadPool threadPool, ClusterS } @Override - protected ActionWriteResponse newShardResponse() { - return new ActionWriteResponse(); + protected ReplicationResponse newShardResponse() { + return new ReplicationResponse(); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java index a06483acb1d0d..229475b80f786 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.admin.indices.refresh; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.TransportReplicationAction; @@ -43,7 +43,7 @@ /** * */ -public class TransportShardRefreshAction extends TransportReplicationAction { +public class TransportShardRefreshAction extends TransportReplicationAction { public static final String NAME = RefreshAction.NAME + "[s]"; @@ -57,16 +57,16 @@ public TransportShardRefreshAction(Settings settings, TransportService transport } @Override - protected ActionWriteResponse newResponseInstance() { - return new ActionWriteResponse(); + protected ReplicationResponse newResponseInstance() { + return new ReplicationResponse(); } @Override - protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable { + protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).getShard(shardRequest.shardId.id()); indexShard.refresh("api"); logger.trace("{} refresh request executed on primary", indexShard.shardId()); - return new Tuple<>(new ActionWriteResponse(), shardRequest.request); + return new Tuple<>(new ReplicationResponse(), shardRequest.request); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java index 80e86eaaf1719..7b6be2d8dbfc7 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkItemResponse.java @@ -20,7 +20,7 @@ package org.elasticsearch.action.bulk; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.update.UpdateResponse; @@ -99,7 +99,7 @@ public Throwable getCause() { private String opType; - private ActionWriteResponse response; + private DocWriteResponse response; private Failure failure; @@ -107,7 +107,7 @@ public Throwable getCause() { } - public BulkItemResponse(int id, String opType, ActionWriteResponse response) { + public BulkItemResponse(int id, String opType, DocWriteResponse response) { this.id = id; this.opType = opType; this.response = response; @@ -140,14 +140,7 @@ public String getIndex() { if (failure != null) { return failure.getIndex(); } - if (response instanceof IndexResponse) { - return ((IndexResponse) response).getIndex(); - } else if (response instanceof DeleteResponse) { - return ((DeleteResponse) response).getIndex(); - } else if (response instanceof UpdateResponse) { - return ((UpdateResponse) response).getIndex(); - } - return null; + return response.getIndex(); } /** @@ -157,14 +150,7 @@ public String getType() { if (failure != null) { return failure.getType(); } - if (response instanceof IndexResponse) { - return ((IndexResponse) response).getType(); - } else if (response instanceof DeleteResponse) { - return ((DeleteResponse) response).getType(); - } else if (response instanceof UpdateResponse) { - return ((UpdateResponse) response).getType(); - } - return null; + return response.getType(); } /** @@ -174,14 +160,7 @@ public String getId() { if (failure != null) { return failure.getId(); } - if (response instanceof IndexResponse) { - return ((IndexResponse) response).getId(); - } else if (response instanceof DeleteResponse) { - return ((DeleteResponse) response).getId(); - } else if (response instanceof UpdateResponse) { - return ((UpdateResponse) response).getId(); - } - return null; + return response.getId(); } /** @@ -191,21 +170,14 @@ public long getVersion() { if (failure != null) { return -1; } - if (response instanceof IndexResponse) { - return ((IndexResponse) response).getVersion(); - } else if (response instanceof DeleteResponse) { - return ((DeleteResponse) response).getVersion(); - } else if (response instanceof UpdateResponse) { - return ((UpdateResponse) response).getVersion(); - } - return -1; + return response.getVersion(); } /** * The actual response ({@link IndexResponse} or {@link DeleteResponse}). null in * case of failure. */ - public T getResponse() { + public T getResponse() { return (T) response; } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java index 6b08627f5de60..76c80a9b0640a 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java @@ -19,7 +19,7 @@ package org.elasticsearch.action.bulk; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.shard.ShardId; @@ -29,7 +29,7 @@ /** * */ -public class BulkShardResponse extends ActionWriteResponse { +public class BulkShardResponse extends ReplicationResponse { private ShardId shardId; private BulkItemResponse[] responses; diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 0f00b87b12a3f..2283f779ef5eb 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -25,8 +25,10 @@ import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.delete.TransportDeleteAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.action.update.UpdateHelper; @@ -134,6 +136,9 @@ protected Tuple shardOperationOnPrimary(Clu IndexResponse indexResponse = result.response(); setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse)); } catch (Throwable e) { + // nocommit: since we now have RetryOnPrimaryException, retrying doesn't always mean the shard is closed. + // some operations were already perform and have a seqno assigned. we shouldn't just reindex them + // if we have a pending mapping update // rethrow the failure if we are going to retry on primary and let parent failure to handle it if (retryPrimaryException(e)) { // restore updated versions... @@ -164,11 +169,13 @@ protected Tuple shardOperationOnPrimary(Clu try { // add the response - final WriteResult writeResult = shardDeleteOperation(request, deleteRequest, indexShard); + final WriteResult writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard); DeleteResponse deleteResponse = writeResult.response(); location = locationToSync(location, writeResult.location); setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse)); } catch (Throwable e) { + // nocommit: since we now have RetryOnPrimaryException, retrying doesn't always mean the shard is closed. + // some operations were already perform and have a seqno assigned. we shouldn't just reindex them // rethrow the failure if we are going to retry on primary and let parent failure to handle it if (retryPrimaryException(e)) { // restore updated versions... @@ -216,7 +223,8 @@ protected Tuple shardOperationOnPrimary(Clu BytesReference indexSourceAsBytes = indexRequest.source(); // add the response IndexResponse indexResponse = result.response(); - UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getIndex(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); + UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), + indexResponse.getType(), indexResponse.getId(), indexResponse.getSeqNo(), indexResponse.getVersion(), indexResponse.isCreated()); if (updateRequest.fields() != null && updateRequest.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, shardRequest.request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); @@ -228,7 +236,8 @@ protected Tuple shardOperationOnPrimary(Clu WriteResult writeResult = updateResult.writeResult; DeleteResponse response = writeResult.response(); DeleteRequest deleteRequest = updateResult.request(); - updateResponse = new UpdateResponse(response.getShardInfo(), response.getIndex(), response.getType(), response.getId(), response.getVersion(), false); + updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), + response.getId(), response.getSeqNo(), response.getVersion(), false); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, shardRequest.request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null)); // Replace the update request to the translated delete request to execute on the replica. item = request.items()[requestIndex] = new BulkItemRequest(request.items()[requestIndex].id(), deleteRequest); @@ -250,6 +259,8 @@ protected Tuple shardOperationOnPrimary(Clu new BulkItemResponse.Failure(request.index(), updateRequest.type(), updateRequest.id(), t))); } } else { + // nocommit: since we now have RetryOnPrimaryException, retrying doesn't always mean the shard is closed. + // some operations were already perform and have a seqno assigned. we shouldn't just reindex them // rethrow the failure if we are going to retry on primary and let parent failure to handle it if (retryPrimaryException(t)) { // restore updated versions... @@ -304,7 +315,7 @@ protected Tuple shardOperationOnPrimary(Clu assert preVersionTypes[requestIndex] != null; } - processAfter(request.refresh(), indexShard, location); + processAfterWrite(request.refresh(), indexShard, location); BulkItemResponse[] responses = new BulkItemResponse[request.items().length]; BulkItemRequest[] items = request.items(); for (int i = 0; i < items.length; i++) { @@ -320,7 +331,7 @@ private void setResponse(BulkItemRequest request, BulkItemResponse response) { } } - private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, ClusterState clusterState, + private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, ClusterState clusterState, IndexShard indexShard, boolean processed) throws Throwable { // validate, if routing is required, that we got routing @@ -335,20 +346,7 @@ private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest i indexRequest.process(clusterState.metaData(), mappingMd, allowIdGeneration, request.index()); } - return executeIndexRequestOnPrimary(request, indexRequest, indexShard); - } - - private WriteResult shardDeleteOperation(BulkShardRequest request, DeleteRequest deleteRequest, IndexShard indexShard) { - Engine.Delete delete = indexShard.prepareDelete(deleteRequest.type(), deleteRequest.id(), deleteRequest.version(), deleteRequest.versionType(), Engine.Operation.Origin.PRIMARY); - indexShard.delete(delete); - // update the request with the version so it will go to the replicas - deleteRequest.versionType(delete.versionType().versionTypeForReplicationAndRecovery()); - deleteRequest.version(delete.version()); - - assert deleteRequest.versionType().validateVersionForWrites(deleteRequest.version()); - - DeleteResponse deleteResponse = new DeleteResponse(request.index(), deleteRequest.type(), deleteRequest.id(), delete.version(), delete.found()); - return new WriteResult(deleteResponse, delete.getTranslogLocation()); + return TransportIndexAction.executeIndexRequestOnPrimary(indexRequest, indexShard, mappingUpdatedAction); } static class UpdateResult { @@ -424,7 +422,7 @@ private UpdateResult shardUpdateOperation(ClusterState clusterState, BulkShardRe case DELETE: DeleteRequest deleteRequest = translate.action(); try { - WriteResult result = shardDeleteOperation(bulkShardRequest, deleteRequest, indexShard); + WriteResult result = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard); return new UpdateResult(translate, deleteRequest, result); } catch (Throwable t) { t = ExceptionsHelper.unwrapCause(t); @@ -460,7 +458,8 @@ protected void shardOperationOnReplica(ShardId shardId, BulkShardRequest request SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, indexRequest.source()).index(shardId.getIndex()).type(indexRequest.type()).id(indexRequest.id()) .routing(indexRequest.routing()).parent(indexRequest.parent()).timestamp(indexRequest.timestamp()).ttl(indexRequest.ttl()); - final Engine.Index operation = indexShard.prepareIndex(sourceToParse, indexRequest.version(), indexRequest.versionType(), Engine.Operation.Origin.REPLICA); + final Engine.Index operation = indexShard.prepareIndexOnReplica(sourceToParse, + indexRequest.seqNo(), indexRequest.version(), indexRequest.versionType()); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { throw new RetryOnReplicaException(shardId, "Mappings are not available on the replica yet, triggered update: " + update); @@ -477,7 +476,8 @@ protected void shardOperationOnReplica(ShardId shardId, BulkShardRequest request } else if (item.request() instanceof DeleteRequest) { DeleteRequest deleteRequest = (DeleteRequest) item.request(); try { - Engine.Delete delete = indexShard.prepareDelete(deleteRequest.type(), deleteRequest.id(), deleteRequest.version(), deleteRequest.versionType(), Engine.Operation.Origin.REPLICA); + Engine.Delete delete = indexShard.prepareDeleteOnReplica(deleteRequest.type(), deleteRequest.id(), + deleteRequest.seqNo(), deleteRequest.version(), deleteRequest.versionType()); indexShard.delete(delete); location = locationToSync(location, delete.getTranslogLocation()); } catch (Throwable e) { @@ -492,7 +492,7 @@ protected void shardOperationOnReplica(ShardId shardId, BulkShardRequest request } } - processAfter(request.refresh(), indexShard, location); + processAfterWrite(request.refresh(), indexShard, location); } private void applyVersion(BulkItemRequest item, long version, VersionType versionType) { diff --git a/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java b/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java index 26cfa57a13d39..1b79d96f114d3 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java +++ b/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java @@ -19,9 +19,12 @@ package org.elasticsearch.action.delete; -import org.elasticsearch.action.ActionWriteResponse; +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.common.xcontent.XContentBuilderString; +import org.elasticsearch.index.shard.ShardId; import java.io.IOException; @@ -31,54 +34,19 @@ * @see org.elasticsearch.action.delete.DeleteRequest * @see org.elasticsearch.client.Client#delete(DeleteRequest) */ -public class DeleteResponse extends ActionWriteResponse { +public class DeleteResponse extends DocWriteResponse { - private String index; - private String id; - private String type; - private long version; private boolean found; public DeleteResponse() { } - public DeleteResponse(String index, String type, String id, long version, boolean found) { - this.index = index; - this.id = id; - this.type = type; - this.version = version; + public DeleteResponse(ShardId shardId, String type, String id, long seqNo, long version, boolean found) { + super(shardId, type, id, seqNo, version); this.found = found; } - /** - * The index the document was deleted from. - */ - public String getIndex() { - return this.index; - } - - /** - * The type of the document deleted. - */ - public String getType() { - return this.type; - } - - /** - * The id of the document deleted. - */ - public String getId() { - return this.id; - } - - /** - * The version of the delete operation. - */ - public long getVersion() { - return this.version; - } - /** * Returns true if a doc was found to delete. */ @@ -89,20 +57,23 @@ public boolean isFound() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - index = in.readString(); - type = in.readString(); - id = in.readString(); - version = in.readLong(); found = in.readBoolean(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(index); - out.writeString(type); - out.writeString(id); - out.writeLong(version); out.writeBoolean(found); } + + static final class Fields { + static final XContentBuilderString FOUND = new XContentBuilderString("found"); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(Fields.FOUND, isFound()); + super.toXContent(builder, params); + return builder; + } } diff --git a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index 227d8728dbe97..29d23f7cb5a3a 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -130,26 +130,34 @@ protected DeleteResponse newResponseInstance() { protected Tuple shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) { DeleteRequest request = shardRequest.request; IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()).getShard(shardRequest.shardId.id()); - Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY); + final WriteResult result = executeDeleteRequestOnPrimary(request, indexShard); + + processAfterWrite(request.refresh(), indexShard, result.location); + + return new Tuple<>(result.response, shardRequest.request); + } + + public static WriteResult executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard indexShard) { + Engine.Delete delete = indexShard.prepareDeleteOnPrimary(request.type(), request.id(), request.version(), request.versionType()); indexShard.delete(delete); // update the request with the version so it will go to the replicas request.versionType(delete.versionType().versionTypeForReplicationAndRecovery()); request.version(delete.version()); + request.seqNo(delete.seqNo()); assert request.versionType().validateVersionForWrites(request.version()); - processAfter(request.refresh(), indexShard, delete.getTranslogLocation()); - - DeleteResponse response = new DeleteResponse(shardRequest.shardId.getIndex(), request.type(), request.id(), delete.version(), delete.found()); - return new Tuple<>(response, shardRequest.request); + return new WriteResult<>(new DeleteResponse(indexShard.shardId(), request.type(), request.id(), + delete.seqNo(), delete.version(), delete.found()), delete.getTranslogLocation()); } @Override protected void shardOperationOnReplica(ShardId shardId, DeleteRequest request) { IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id()); - Engine.Delete delete = indexShard.prepareDelete(request.type(), request.id(), request.version(), request.versionType(), Engine.Operation.Origin.REPLICA); + Engine.Delete delete = indexShard.prepareDeleteOnReplica(request.type(), request.id(), + request.seqNo(), request.version(), request.versionType()); indexShard.delete(delete); - processAfter(request.refresh(), indexShard, delete.getTranslogLocation()); + processAfterWrite(request.refresh(), indexShard, delete.getTranslogLocation()); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java b/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java index 5727b2b673b97..5765a7463731b 100644 --- a/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java +++ b/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java @@ -19,9 +19,12 @@ package org.elasticsearch.action.index; -import org.elasticsearch.action.ActionWriteResponse; +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.common.xcontent.XContentBuilderString; +import org.elasticsearch.index.shard.ShardId; import java.io.IOException; @@ -31,54 +34,18 @@ * @see org.elasticsearch.action.index.IndexRequest * @see org.elasticsearch.client.Client#index(IndexRequest) */ -public class IndexResponse extends ActionWriteResponse { +public class IndexResponse extends DocWriteResponse { - private String index; - private String id; - private String type; - private long version; private boolean created; public IndexResponse() { - } - public IndexResponse(String index, String type, String id, long version, boolean created) { - this.index = index; - this.id = id; - this.type = type; - this.version = version; + public IndexResponse(ShardId shardId, String type, String id, long seqNo, long version, boolean created) { + super(shardId, type, id, seqNo, version); this.created = created; } - /** - * The index the document was indexed into. - */ - public String getIndex() { - return this.index; - } - - /** - * The type of the document indexed. - */ - public String getType() { - return this.type; - } - - /** - * The id of the document indexed. - */ - public String getId() { - return this.id; - } - - /** - * Returns the current version of the doc indexed. - */ - public long getVersion() { - return this.version; - } - /** * Returns true if the document was created, false if updated. */ @@ -89,20 +56,12 @@ public boolean isCreated() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - index = in.readString(); - type = in.readString(); - id = in.readString(); - version = in.readLong(); created = in.readBoolean(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(index); - out.writeString(type); - out.writeString(id); - out.writeLong(version); out.writeBoolean(created); } @@ -110,12 +69,24 @@ public void writeTo(StreamOutput out) throws IOException { public String toString() { StringBuilder builder = new StringBuilder(); builder.append("IndexResponse["); - builder.append("index=").append(index); - builder.append(",type=").append(type); - builder.append(",id=").append(id); - builder.append(",version=").append(version); + builder.append("index=").append(getIndex()); + builder.append(",type=").append(getType()); + builder.append(",id=").append(getId()); + builder.append(",version=").append(getVersion()); builder.append(",created=").append(created); + builder.append(",seqNo=").append(getSeqNo()); builder.append(",shards=").append(getShardInfo()); return builder.append("]").toString(); } + + static final class Fields { + static final XContentBuilderString CREATED = new XContentBuilderString("created"); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + super.toXContent(builder, params); + builder.field(Fields.CREATED, isCreated()); + return builder; + } } diff --git a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 63b82377d8a47..015b6a702f778 100644 --- a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -166,11 +166,11 @@ protected Tuple shardOperationOnPrimary(ClusterStat IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex()); IndexShard indexShard = indexService.getShard(shardRequest.shardId.id()); - final WriteResult result = executeIndexRequestOnPrimary(null, request, indexShard); + final WriteResult result = executeIndexRequestOnPrimary(request, indexShard, mappingUpdatedAction); final IndexResponse response = result.response; final Translog.Location location = result.location; - processAfter(request.refresh(), indexShard, location); + processAfterWrite(request.refresh(), indexShard, location); return new Tuple<>(response, shardRequest.request); } @@ -181,13 +181,51 @@ protected void shardOperationOnReplica(ShardId shardId, IndexRequest request) { SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.REPLICA, request.source()).index(shardId.getIndex()).type(request.type()).id(request.id()) .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); - final Engine.Index operation = indexShard.prepareIndex(sourceToParse, request.version(), request.versionType(), Engine.Operation.Origin.REPLICA); + final Engine.Index operation = indexShard.prepareIndexOnReplica(sourceToParse, request.seqNo(), request.version(), request.versionType()); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); if (update != null) { throw new RetryOnReplicaException(shardId, "Mappings are not available on the replica yet, triggered update: " + update); } indexShard.index(operation); - processAfter(request.refresh(), indexShard, operation.getTranslogLocation()); + processAfterWrite(request.refresh(), indexShard, operation.getTranslogLocation()); } + /** utility method to prepare indexing operations on the primary */ + public static Engine.Index prepareIndexOperationOnPrimary(IndexRequest request, IndexShard indexShard) { + SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).index(request.index()).type(request.type()).id(request.id()) + .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); + return indexShard.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType()); + + } + + /** + * Execute the given {@link IndexRequest} on a primary shard, throwing a + * {@link RetryOnPrimaryException} if the operation needs to be re-tried. + */ + public static WriteResult executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, MappingUpdatedAction mappingUpdatedAction) throws Throwable { + Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard); + Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); + final ShardId shardId = indexShard.shardId(); + if (update != null) { + final String indexName = shardId.getIndex(); + mappingUpdatedAction.updateMappingOnMasterSynchronously(indexName, request.type(), update); + operation = prepareIndexOperationOnPrimary(request, indexShard); + update = operation.parsedDoc().dynamicMappingsUpdate(); + if (update != null) { + throw new RetryOnPrimaryException(shardId, + "Dynamics mappings are not available on the node that holds the primary yet"); + } + } + final boolean created = indexShard.index(operation); + + // update the version on request so it will happen on the replicas + final long version = operation.version(); + request.version(version); + request.versionType(request.versionType().versionTypeForReplicationAndRecovery()); + request.seqNo(operation.seqNo()); + + assert request.versionType().validateVersionForWrites(request.version()); + + return new WriteResult<>(new IndexResponse(shardId, request.type(), request.id(), request.seqNo(), request.version(), created), operation.getTranslogLocation()); + } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java index c629a70d6f973..e7c7664d10a38 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java @@ -44,6 +44,9 @@ public class ReplicationRequest extends ActionRequ ShardId internalShardId; + long seqNo; + long primaryTerm; + protected TimeValue timeout = DEFAULT_TIMEOUT; protected String index; @@ -76,6 +79,9 @@ protected ReplicationRequest(T request, ActionRequest originalRequest) { this.timeout = request.timeout(); this.index = request.index(); this.consistencyLevel = request.consistencyLevel(); + this.internalShardId = request.internalShardId; + this.seqNo = request.seqNo; + this.primaryTerm = request.primaryTerm; } /** @@ -141,6 +147,29 @@ public final T consistencyLevel(WriteConsistencyLevel consistencyLevel) { return (T) this; } + /** + * Returns the sequence number for this operation. The sequence number is assigned while the operation + * is performed on the primary shard. + */ + public long seqNo() { + return seqNo; + } + + /** sets the sequence number for this operation. should only be called on the primary shard */ + public void seqNo(long seqNo) { + this.seqNo = seqNo; + } + + /** returns the primary term active at the time the operation was performed on the primary shard */ + public long primaryTerm() { + return primaryTerm; + } + + /** marks the primary term in which the operation was performed */ + public void primaryTerm(long term) { + primaryTerm = term; + } + @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; @@ -161,6 +190,8 @@ public void readFrom(StreamInput in) throws IOException { consistencyLevel = WriteConsistencyLevel.fromId(in.readByte()); timeout = TimeValue.readTimeValue(in); index = in.readString(); + seqNo = in.readVLong(); + primaryTerm = in.readVLong(); } @Override @@ -175,6 +206,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeByte(consistencyLevel.id()); timeout.writeTo(out); out.writeString(index); + out.writeVLong(seqNo); + out.writeVLong(primaryTerm); } public T setShardId(ShardId shardId) { diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java index ddd4d42f7a68b..33a9d349e8064 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportBroadcastReplicationAction.java @@ -22,9 +22,8 @@ import com.carrotsearch.hppc.cursors.IntObjectCursor; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.ShardOperationFailedException; -import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.HandledTransportAction; @@ -53,7 +52,7 @@ * Base class for requests that should be executed on all shards of an index or several indices. * This action sends shard requests to all primary shards of the indices and they are then replicated like write requests */ -public abstract class TransportBroadcastReplicationAction extends HandledTransportAction { +public abstract class TransportBroadcastReplicationAction extends HandledTransportAction { private final TransportReplicationAction replicatedBroadcastShardAction; private final ClusterService clusterService; @@ -91,15 +90,15 @@ public void onFailure(Throwable e) { logger.trace("{}: got failure from {}", actionName, shardId); int totalNumCopies = clusterState.getMetaData().index(shardId.index().getName()).getNumberOfReplicas() + 1; ShardResponse shardResponse = newShardResponse(); - ActionWriteResponse.ShardInfo.Failure[] failures; + ReplicationResponse.ShardInfo.Failure[] failures; if (TransportActions.isShardNotAvailableException(e)) { - failures = new ActionWriteResponse.ShardInfo.Failure[0]; + failures = new ReplicationResponse.ShardInfo.Failure[0]; } else { - ActionWriteResponse.ShardInfo.Failure failure = new ActionWriteResponse.ShardInfo.Failure(shardId.index().name(), shardId.id(), null, e, ExceptionsHelper.status(e), true); - failures = new ActionWriteResponse.ShardInfo.Failure[totalNumCopies]; + ReplicationResponse.ShardInfo.Failure failure = new ReplicationResponse.ShardInfo.Failure(shardId.index().name(), shardId.id(), null, e, ExceptionsHelper.status(e), true); + failures = new ReplicationResponse.ShardInfo.Failure[totalNumCopies]; Arrays.fill(failures, failure); } - shardResponse.setShardInfo(new ActionWriteResponse.ShardInfo(totalNumCopies, 0, failures)); + shardResponse.setShardInfo(new ReplicationResponse.ShardInfo(totalNumCopies, 0, failures)); shardsResponses.add(shardResponse); if (responsesCountDown.countDown()) { finishAndNotifyListener(listener, shardsResponses); @@ -142,7 +141,7 @@ private void finishAndNotifyListener(ActionListener listener, CopyOnWriteArrayLi int totalNumCopies = 0; List shardFailures = null; for (int i = 0; i < shardsResponses.size(); i++) { - ActionWriteResponse shardResponse = shardsResponses.get(i); + ReplicationResponse shardResponse = shardsResponses.get(i); if (shardResponse == null) { // non active shard, ignore } else { @@ -152,7 +151,7 @@ private void finishAndNotifyListener(ActionListener listener, CopyOnWriteArrayLi if (shardFailures == null) { shardFailures = new ArrayList<>(); } - for (ActionWriteResponse.ShardInfo.Failure failure : shardResponse.getShardInfo().getFailures()) { + for (ReplicationResponse.ShardInfo.Failure failure : shardResponse.getShardInfo().getFailures()) { shardFailures.add(new DefaultShardOperationFailedException(new BroadcastShardOperationFailedException(new ShardId(failure.index(), failure.shardId()), failure.getCause()))); } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 23bf21bd83a11..a1f5b5edcff6b 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -22,13 +22,9 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.WriteConsistencyLevel; -import org.elasticsearch.action.bulk.BulkShardRequest; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexRequest.OpType; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; @@ -56,10 +52,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.VersionConflictEngineException; -import org.elasticsearch.index.mapper.Mapping; -import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; @@ -78,7 +71,7 @@ /** */ -public abstract class TransportReplicationAction extends TransportAction { +public abstract class TransportReplicationAction extends TransportAction { public static final String SHARD_FAILURE_TIMEOUT = "action.support.replication.shard.failure_timeout"; @@ -195,7 +188,7 @@ protected boolean isConflictException(Throwable e) { return false; } - protected static class WriteResult { + protected static class WriteResult { public final T response; public final Translog.Location location; @@ -206,10 +199,10 @@ public WriteResult(T response, Translog.Location location) { } @SuppressWarnings("unchecked") - public T response() { + public T response() { // this sets total, pending and failed to 0 and this is ok, because we will embed this into the replica // request and not use it - response.setShardInfo(new ActionWriteResponse.ShardInfo()); + response.setShardInfo(new ReplicationResponse.ShardInfo()); return (T) response; } @@ -314,7 +307,7 @@ protected void responseWithFailure(Throwable t) { @Override protected void doRun() throws Exception { - try (Releasable shardReference = getIndexShardOperationsCounter(request.internalShardId)) { + try (Releasable shardReference = getIndexShardOperationsCounter(request.internalShardId, request.primaryTerm)) { shardOperationOnReplica(request.internalShardId, request); } channel.sendResponse(TransportResponse.Empty.INSTANCE); @@ -577,9 +570,10 @@ void performOnPrimary(final ShardRouting primary, final ShardIterator shardsIt) } final ReplicationPhase replicationPhase; try { - indexShardReference = getIndexShardOperationsCounter(primary.shardId()); + indexShardReference = getIndexShardOperationsCounter(primary.shardId(), primary.primaryTerm()); PrimaryOperationRequest por = new PrimaryOperationRequest(primary.id(), internalRequest.concreteIndex(), internalRequest.request()); Tuple primaryResponse = shardOperationOnPrimary(observer.observedState(), por); + primaryResponse.v2().primaryTerm(primary.primaryTerm()); logger.trace("operation completed on primary [{}]", primary); replicationPhase = new ReplicationPhase(shardsIt, primaryResponse.v2(), primaryResponse.v1(), observer, primary, internalRequest, listener, indexShardReference, shardFailedTimeout); } catch (Throwable e) { @@ -664,10 +658,10 @@ void retryBecauseUnavailable(ShardId shardId, String message) { } - protected Releasable getIndexShardOperationsCounter(ShardId shardId) { + protected Releasable getIndexShardOperationsCounter(ShardId shardId, long opPrimaryTerm) { IndexService indexService = indicesService.indexServiceSafe(shardId.index().getName()); IndexShard indexShard = indexService.getShard(shardId.id()); - return new IndexShardReference(indexShard); + return new IndexShardReference(indexShard, opPrimaryTerm); } private void failReplicaIfNeeded(String index, int shardId, Throwable t) { @@ -961,20 +955,20 @@ private void doFinish() { if (finished.compareAndSet(false, true)) { Releasables.close(indexShardReference); final ShardId shardId = shardIt.shardId(); - final ActionWriteResponse.ShardInfo.Failure[] failuresArray; + final ReplicationResponse.ShardInfo.Failure[] failuresArray; if (!shardReplicaFailures.isEmpty()) { int slot = 0; - failuresArray = new ActionWriteResponse.ShardInfo.Failure[shardReplicaFailures.size()]; + failuresArray = new ReplicationResponse.ShardInfo.Failure[shardReplicaFailures.size()]; for (Map.Entry entry : shardReplicaFailures.entrySet()) { RestStatus restStatus = ExceptionsHelper.status(entry.getValue()); - failuresArray[slot++] = new ActionWriteResponse.ShardInfo.Failure( + failuresArray[slot++] = new ReplicationResponse.ShardInfo.Failure( shardId.getIndex(), shardId.getId(), entry.getKey(), entry.getValue(), restStatus, false ); } } else { - failuresArray = ActionWriteResponse.EMPTY; + failuresArray = ReplicationResponse.EMPTY; } - finalResponse.setShardInfo(new ActionWriteResponse.ShardInfo( + finalResponse.setShardInfo(new ReplicationResponse.ShardInfo( totalShards, success.get(), failuresArray @@ -1046,13 +1040,15 @@ public String concreteIndex() { } } + static class IndexShardReference implements Releasable { final private IndexShard counter; private final AtomicBoolean closed = new AtomicBoolean(false); - IndexShardReference(IndexShard counter) { - counter.incrementOperationCounter(); + IndexShardReference(IndexShard counter, long opPrimaryTerm) { + // this enforces primary terms, if we're lagging an exception will be thrown. + counter.incrementOperationCounter(opPrimaryTerm); this.counter = counter; } @@ -1064,44 +1060,8 @@ public void close() { } } - /** Utility method to create either an index or a create operation depending - * on the {@link OpType} of the request. */ - private final Engine.Index prepareIndexOperationOnPrimary(BulkShardRequest shardRequest, IndexRequest request, IndexShard indexShard) { - SourceToParse sourceToParse = SourceToParse.source(SourceToParse.Origin.PRIMARY, request.source()).index(request.index()).type(request.type()).id(request.id()) - .routing(request.routing()).parent(request.parent()).timestamp(request.timestamp()).ttl(request.ttl()); - return indexShard.prepareIndex(sourceToParse, request.version(), request.versionType(), Engine.Operation.Origin.PRIMARY); - - } - - /** Execute the given {@link IndexRequest} on a primary shard, throwing a - * {@link RetryOnPrimaryException} if the operation needs to be re-tried. */ - protected final WriteResult executeIndexRequestOnPrimary(BulkShardRequest shardRequest, IndexRequest request, IndexShard indexShard) throws Throwable { - Engine.Index operation = prepareIndexOperationOnPrimary(shardRequest, request, indexShard); - Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); - final ShardId shardId = indexShard.shardId(); - if (update != null) { - final String indexName = shardId.getIndex(); - mappingUpdatedAction.updateMappingOnMasterSynchronously(indexName, request.type(), update); - operation = prepareIndexOperationOnPrimary(shardRequest, request, indexShard); - update = operation.parsedDoc().dynamicMappingsUpdate(); - if (update != null) { - throw new RetryOnPrimaryException(shardId, - "Dynamics mappings are not available on the node that holds the primary yet"); - } - } - final boolean created = indexShard.index(operation); - - // update the version on request so it will happen on the replicas - final long version = operation.version(); - request.version(version); - request.versionType(request.versionType().versionTypeForReplicationAndRecovery()); - - assert request.versionType().validateVersionForWrites(request.version()); - - return new WriteResult(new IndexResponse(shardId.getIndex(), request.type(), request.id(), request.version(), created), operation.getTranslogLocation()); - } - - protected final void processAfter(boolean refresh, IndexShard indexShard, Translog.Location location) { + /** utility method for common tasks that should be done after a write operation */ + public static void processAfterWrite(boolean refresh, IndexShard indexShard, Translog.Location location) { if (refresh) { try { indexShard.refresh("refresh_flag_index"); diff --git a/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index 2a639c83ad1b4..b3c211f292d56 100644 --- a/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -175,7 +175,8 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< indexAction.execute(upsertRequest, new ActionListener() { @Override public void onResponse(IndexResponse response) { - UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); + UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), + response.getId(), response.getSeqNo(), response.getVersion(), response.isCreated()); if (request.fields() != null && request.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); @@ -210,7 +211,8 @@ protected void doRun() { indexAction.execute(indexRequest, new ActionListener() { @Override public void onResponse(IndexResponse response) { - UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); + UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), + response.getSeqNo(), response.getVersion(), response.isCreated()); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); listener.onResponse(update); } @@ -238,7 +240,8 @@ protected void doRun() { deleteAction.execute(deleteRequest, new ActionListener() { @Override public void onResponse(DeleteResponse response) { - UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getIndex(), response.getType(), response.getId(), response.getVersion(), false); + UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), + response.getId(), response.getSeqNo(), response.getVersion(), false); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); listener.onResponse(update); } @@ -264,7 +267,7 @@ protected void doRun() { case NONE: UpdateResponse update = result.action(); IndexService indexServiceOrNull = indicesService.indexService(request.concreteIndex()); - if (indexServiceOrNull != null) { + if (indexServiceOrNull != null) { IndexShard shard = indexService.getShardOrNull(request.shardId()); if (shard != null) { shard.indexingService().noopUpdate(request.type()); diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 010142b0b4c15..247c672b8edbd 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -83,9 +83,10 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { @SuppressWarnings("unchecked") protected Result prepare(UpdateRequest request, final GetResult getResult) { long getDateNS = System.nanoTime(); + final ShardId shardId = new ShardId(request.index(), request.shardId()); if (!getResult.isExists()) { if (request.upsertRequest() == null && !request.docAsUpsert()) { - throw new DocumentMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id()); + throw new DocumentMissingException(shardId, request.type(), request.id()); } IndexRequest indexRequest = request.docAsUpsert() ? request.doc() : request.upsertRequest(); Long ttl = indexRequest.ttl(); @@ -113,7 +114,7 @@ protected Result prepare(UpdateRequest request, final GetResult getResult) { logger.warn("Used upsert operation [{}] for script [{}], doing nothing...", scriptOpChoice, request.script.getScript()); } - UpdateResponse update = new UpdateResponse(getResult.getIndex(), getResult.getType(), getResult.getId(), + UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), false); update.setGetResult(getResult); return new Result(update, Operation.NONE, upsertDoc, XContentType.JSON); @@ -145,7 +146,7 @@ protected Result prepare(UpdateRequest request, final GetResult getResult) { if (getResult.internalSourceRef() == null) { // no source, we can't do nothing, through a failure... - throw new DocumentSourceMissingException(new ShardId(request.index(), request.shardId()), request.type(), request.id()); + throw new DocumentSourceMissingException(shardId, request.type(), request.id()); } Tuple> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); @@ -231,12 +232,12 @@ protected Result prepare(UpdateRequest request, final GetResult getResult) { .consistencyLevel(request.consistencyLevel()); return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType); } else if ("none".equals(operation)) { - UpdateResponse update = new UpdateResponse(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), false); + UpdateResponse update = new UpdateResponse(new ShardId(getResult.getIndex(), request.shardId()), getResult.getType(), getResult.getId(), getResult.getVersion(), false); update.setGetResult(extractGetResult(request, request.index(), getResult.getVersion(), updatedSourceAsMap, updateSourceContentType, getResult.internalSourceRef())); return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType); } else { logger.warn("Used update operation [{}] for script [{}], doing nothing...", operation, request.script.getScript()); - UpdateResponse update = new UpdateResponse(getResult.getIndex(), getResult.getType(), getResult.getId(), getResult.getVersion(), false); + UpdateResponse update = new UpdateResponse(new ShardId(getResult.getIndex(), request.shardId()), getResult.getType(), getResult.getId(), getResult.getVersion(), false); return new Result(update, Operation.NONE, updatedSourceAsMap, updateSourceContentType); } } diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java b/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java index af6438097c800..34e808e71c4a1 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java @@ -19,21 +19,21 @@ package org.elasticsearch.action.update; -import org.elasticsearch.action.ActionWriteResponse; +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.common.xcontent.XContentBuilderString; import org.elasticsearch.index.get.GetResult; +import org.elasticsearch.index.seqno.SequenceNumbersService; +import org.elasticsearch.index.shard.ShardId; import java.io.IOException; /** */ -public class UpdateResponse extends ActionWriteResponse { +public class UpdateResponse extends DocWriteResponse { - private String index; - private String id; - private String type; - private long version; private boolean created; private GetResult getResult; @@ -44,47 +44,16 @@ public UpdateResponse() { * Constructor to be used when a update didn't translate in a write. * For example: update script with operation set to none */ - public UpdateResponse(String index, String type, String id, long version, boolean created) { - this(new ShardInfo(0, 0), index, type, id, version, created); + public UpdateResponse(ShardId shardId, String type, String id, long version, boolean created) { + this(new ShardInfo(0, 0), shardId, type, id, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, created); } - public UpdateResponse(ShardInfo shardInfo, String index, String type, String id, long version, boolean created) { + public UpdateResponse(ShardInfo shardInfo, ShardId shardId, String type, String id, long seqNo, long version, boolean created) { + super(shardId, type, id, seqNo, version); setShardInfo(shardInfo); - this.index = index; - this.id = id; - this.type = type; - this.version = version; this.created = created; } - /** - * The index the document was indexed into. - */ - public String getIndex() { - return this.index; - } - - /** - * The type of the document indexed. - */ - public String getType() { - return this.type; - } - - /** - * The id of the document indexed. - */ - public String getId() { - return this.id; - } - - /** - * Returns the current version of the doc indexed. - */ - public long getVersion() { - return this.version; - } - public void setGetResult(GetResult getResult) { this.getResult = getResult; } @@ -104,10 +73,6 @@ public boolean isCreated() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - index = in.readString(); - type = in.readString(); - id = in.readString(); - version = in.readLong(); created = in.readBoolean(); if (in.readBoolean()) { getResult = GetResult.readGetResult(in); @@ -117,10 +82,6 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(index); - out.writeString(type); - out.writeString(id); - out.writeLong(version); out.writeBoolean(created); if (getResult == null) { out.writeBoolean(false); @@ -129,4 +90,19 @@ public void writeTo(StreamOutput out) throws IOException { getResult.writeTo(out); } } + + static final class Fields { + static final XContentBuilderString GET = new XContentBuilderString("get"); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + super.toXContent(builder, params); + if (getGetResult() != null) { + builder.startObject(Fields.GET); + getGetResult().toXContentEmbedded(builder, params); + builder.endObject(); + } + return builder; + } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java b/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java index d5f9bd602e70e..13a4cc66b8419 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java @@ -191,6 +191,15 @@ public boolean relocating() { return state == ShardRoutingState.RELOCATING; } + /** + * Returns true if this shard is a relocation target for another shard (i.e., was created with {@link #buildTargetRelocatingShard()} + * + */ + public boolean isRelocationTarget() { + return state == ShardRoutingState.INITIALIZING && relocatingNodeId != null; + } + + /** * Returns true iff this shard is assigned to a node ie. not * {@link ShardRoutingState#UNASSIGNED unassigned}. Otherwise false diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 0504fdfa99694..bccae2e46642a 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -45,6 +45,7 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -577,7 +578,8 @@ public interface EventListener { /** * Called when a fatal exception occurred */ - default void onFailedEngine(String reason, @Nullable Throwable t) {} + default void onFailedEngine(String reason, @Nullable Throwable t) { + } } public static class Searcher implements Releasable { @@ -602,7 +604,7 @@ public IndexReader reader() { } public DirectoryReader getDirectoryReader() { - if (reader() instanceof DirectoryReader) { + if (reader() instanceof DirectoryReader) { return (DirectoryReader) reader(); } throw new IllegalStateException("Can't use " + reader().getClass() + " as a directory reader"); @@ -621,14 +623,18 @@ public void close() { public static abstract class Operation { private final Term uid; private long version; + private long seqNo; private final VersionType versionType; private final Origin origin; private Translog.Location location; private final long startTime; private long endTime; - public Operation(Term uid, long version, VersionType versionType, Origin origin, long startTime) { + public Operation(Term uid, long seqNo, long version, VersionType versionType, Origin origin, long startTime) { this.uid = uid; + assert origin != Origin.PRIMARY || seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO : "seqNo should not be set when origin is PRIMARY"; + assert origin == Origin.PRIMARY || seqNo >= 0 : "seqNo should be set when origin is not PRIMARY"; + this.seqNo = seqNo; this.version = version; this.versionType = versionType; this.origin = origin; @@ -657,6 +663,14 @@ public void updateVersion(long version) { this.version = version; } + public long seqNo() { + return seqNo; + } + + public void updateSeqNo(long seqNo) { + this.seqNo = seqNo; + } + public void setTranslogLocation(Translog.Location location) { this.location = location; } @@ -692,8 +706,8 @@ public static class Index extends Operation { private final ParsedDocument doc; - public Index(Term uid, ParsedDocument doc, long version, VersionType versionType, Origin origin, long startTime) { - super(uid, version, versionType, origin, startTime); + public Index(Term uid, ParsedDocument doc, long seqNo, long version, VersionType versionType, Origin origin, long startTime) { + super(uid, seqNo, version, versionType, origin, startTime); this.doc = doc; } @@ -702,7 +716,7 @@ public Index(Term uid, ParsedDocument doc) { } public Index(Term uid, ParsedDocument doc, long version) { - this(uid, doc, version, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime()); + this(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime()); } public ParsedDocument parsedDoc() { @@ -735,6 +749,12 @@ public void updateVersion(long version) { this.doc.version().setLongValue(version); } + @Override + public void updateSeqNo(long seqNo) { + super.updateSeqNo(seqNo); + this.doc.seqNo().setLongValue(seqNo); + } + public String parent() { return this.doc.parent(); } @@ -753,19 +773,15 @@ public static class Delete extends Operation { private final String id; private boolean found; - public Delete(String type, String id, Term uid, long version, VersionType versionType, Origin origin, long startTime, boolean found) { - super(uid, version, versionType, origin, startTime); + public Delete(String type, String id, Term uid, long seqNo, long version, VersionType versionType, Origin origin, long startTime, boolean found) { + super(uid, seqNo, version, versionType, origin, startTime); this.type = type; this.id = id; this.found = found; } public Delete(String type, String id, Term uid) { - this(type, id, uid, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), false); - } - - public Delete(Delete template, VersionType versionType) { - this(template.type(), template.id(), template.uid(), template.version(), versionType, template.origin(), template.startTime(), template.found()); + this(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), false); } public String type() { @@ -1060,6 +1076,7 @@ public void onSettingsChanged() { * Returns the timestamp of the last write in nanoseconds. * Note: this time might not be absolutely accurate since the {@link Operation#startTime()} is used which might be * slightly inaccurate. + * * @see System#nanoTime() * @see Operation#startTime() */ @@ -1069,12 +1086,14 @@ public long getLastWriteNanos() { /** * Called for each new opened engine searcher to warm new segments + * * @see EngineConfig#getWarmer() */ public interface Warmer { /** * Called once a new Searcher is opened. - * @param searcher the searcer to warm + * + * @param searcher the searcer to warm * @param isTopLevelReader true iff the searcher is build from a top-level reader. * Otherwise the searcher might be build from a leaf reader to warm in isolation */ diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index c5da8e83b3db8..a7e2e9b8854d8 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -73,6 +73,7 @@ public final class EngineConfig { private final QueryCache queryCache; private final QueryCachingPolicy queryCachingPolicy; + /** * Index setting for compound file on flush. This setting is realtime updateable. */ @@ -123,7 +124,8 @@ public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService this.indexSettings = indexSettings; this.threadPool = threadPool; this.indexingService = indexingService; - this.warmer = warmer == null ? (a,b) -> {} : warmer; + this.warmer = warmer == null ? (a, b) -> { + } : warmer; this.store = store; this.deletionPolicy = deletionPolicy; this.mergePolicy = mergePolicy; @@ -226,9 +228,9 @@ public long getGcDeletesInMillis() { * Returns true iff delete garbage collection in the engine should be enabled. This setting is updateable * in realtime and forces a volatile read. Consumers can safely read this value directly go fetch it's latest value. The default is true *

- * Engine GC deletion if enabled collects deleted documents from in-memory realtime data structures after a certain amount of - * time ({@link #getGcDeletesInMillis()} if enabled. Before deletes are GCed they will cause re-adding the document that was deleted - * to fail. + * Engine GC deletion if enabled collects deleted documents from in-memory realtime data structures after a certain amount of + * time ({@link #getGcDeletesInMillis()} if enabled. Before deletes are GCed they will cause re-adding the document that was deleted + * to fail. *

*/ public boolean isEnableGcDeletes() { @@ -238,7 +240,7 @@ public boolean isEnableGcDeletes() { /** * Returns the {@link Codec} used in the engines {@link org.apache.lucene.index.IndexWriter} *

- * Note: this settings is only read on startup. + * Note: this settings is only read on startup. *

*/ public Codec getCodec() { @@ -259,7 +261,6 @@ public ThreadPool getThreadPool() { * * @see org.elasticsearch.index.indexing.ShardIndexingService#postIndex(Engine.Index) * @see org.elasticsearch.index.indexing.ShardIndexingService#preIndex(Engine.Index) - * */ public ShardIndexingService getIndexingService() { return indexingService; @@ -323,7 +324,9 @@ public IndexSettings getIndexSettings() { /** * Returns the engines shard ID */ - public ShardId getShardId() { return shardId; } + public ShardId getShardId() { + return shardId; + } /** * Returns the analyzer as the default analyzer in the engines {@link org.apache.lucene.index.IndexWriter} @@ -404,6 +407,7 @@ public boolean isCreate() { * should be automatically flushed. This is used to free up transient disk usage of potentially large segments that * are written after the engine became inactive from an indexing perspective. */ - public TimeValue getFlushMergesAfter() { return flushMergesAfter; } - + public TimeValue getFlushMergesAfter() { + return flushMergesAfter; + } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1404b61b8ec8e..dbb62a735a064 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -50,6 +50,7 @@ import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.MergeSchedulerConfig; import org.elasticsearch.index.shard.ShardId; @@ -102,6 +103,8 @@ public class InternalEngine extends Engine { private final IndexThrottle throttle; + private final SequenceNumbersService seqNoService; + public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogRecovery) throws EngineException { super(engineConfig); this.versionMap = new LiveVersionMap(); @@ -115,6 +118,7 @@ public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogReco this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis(); this.indexingService = engineConfig.getIndexingService(); this.warmer = engineConfig.getWarmer(); + seqNoService = new SequenceNumbersService(shardId, engineConfig.getIndexSettings()); mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings(), engineConfig.getMergeSchedulerConfig()); this.dirtyLocks = new Object[Runtime.getRuntime().availableProcessors() * 10]; // we multiply it to have enough... for (int i = 0; i < dirtyLocks.length; i++) { @@ -344,6 +348,10 @@ public boolean index(Index index) { } catch (OutOfMemoryError | IllegalStateException | IOException t) { maybeFailEngine("index", t); throw new IndexFailedEngineException(shardId, index.type(), index.id(), t); + } finally { + if (index.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoService.markSeqNoAsCompleted(index.seqNo()); + } } checkVersionMapRefresh(); return created; @@ -380,6 +388,9 @@ private boolean innerIndex(Index index) throws IOException { final boolean created; index.updateVersion(updatedVersion); + if (index.origin() == Operation.Origin.PRIMARY) { + index.updateSeqNo(seqNoService.generateSeqNo()); + } if (currentVersion == Versions.NOT_FOUND) { // document does not exists, we can optimize for create @@ -447,6 +458,10 @@ public void delete(Delete delete) throws EngineException { } catch (OutOfMemoryError | IllegalStateException | IOException t) { maybeFailEngine("delete", t); throw new DeleteFailedEngineException(shardId, delete, t); + } finally { + if (delete.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoService.markSeqNoAsCompleted(delete.seqNo()); + } } maybePruneDeletedTombstones(); @@ -490,6 +505,11 @@ private void innerDelete(Delete delete) throws IOException { } } updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); + + if (delete.origin() == Operation.Origin.PRIMARY) { + delete.updateSeqNo(seqNoService.generateSeqNo()); + } + final boolean found; if (currentVersion == Versions.NOT_FOUND) { // doc does not exist and no prior deletes diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index 1be873d3e9748..0a888731cf103 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -35,18 +35,7 @@ import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.mapper.internal.AllFieldMapper; -import org.elasticsearch.index.mapper.internal.FieldNamesFieldMapper; -import org.elasticsearch.index.mapper.internal.IdFieldMapper; -import org.elasticsearch.index.mapper.internal.IndexFieldMapper; -import org.elasticsearch.index.mapper.internal.ParentFieldMapper; -import org.elasticsearch.index.mapper.internal.RoutingFieldMapper; -import org.elasticsearch.index.mapper.internal.SourceFieldMapper; -import org.elasticsearch.index.mapper.internal.TTLFieldMapper; -import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; -import org.elasticsearch.index.mapper.internal.TypeFieldMapper; -import org.elasticsearch.index.mapper.internal.UidFieldMapper; -import org.elasticsearch.index.mapper.internal.VersionFieldMapper; +import org.elasticsearch.index.mapper.internal.*; import org.elasticsearch.index.mapper.object.ObjectMapper; import org.elasticsearch.index.mapper.object.RootObjectMapper; import org.elasticsearch.script.ExecutableScript; @@ -107,6 +96,7 @@ public Builder(Settings indexSettings, RootObjectMapper.Builder builder, MapperS this.rootMappers.put(TimestampFieldMapper.class, new TimestampFieldMapper(indexSettings, mapperService.fullName(TimestampFieldMapper.NAME))); this.rootMappers.put(TTLFieldMapper.class, new TTLFieldMapper(indexSettings)); this.rootMappers.put(VersionFieldMapper.class, new VersionFieldMapper(indexSettings)); + this.rootMappers.put(SeqNoFieldMapper.class, new SeqNoFieldMapper(indexSettings)); this.rootMappers.put(ParentFieldMapper.class, new ParentFieldMapper(indexSettings, mapperService.fullName(ParentFieldMapper.NAME), /* parent type */builder.name())); // _field_names last so that it can see all other fields this.rootMappers.put(FieldNamesFieldMapper.class, new FieldNamesFieldMapper(indexSettings, mapperService.fullName(FieldNamesFieldMapper.NAME))); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index de4dc387c8891..87dfce192b6e9 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -28,8 +28,6 @@ import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ReleasableLock; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.core.DateFieldMapper.DateFieldType; @@ -47,7 +45,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; /** A parser for documents, given mappings from a DocumentMapper */ @@ -184,7 +181,7 @@ private ParsedDocument innerParseDocument(SourceToParse source) throws MapperPar update = mapping.mappingUpdate(rootDynamicUpdate); } - ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(), + ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.seqNo(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(), context.source(), update).parent(source.parent()); // reset the context to free up memory context.reset(null, null, null); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index edf75621c1e3e..b68f7d85dc471 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -323,6 +323,16 @@ public void version(Field version) { in.version(version); } + @Override + public Field seqNo() { + return in.seqNo(); + } + + @Override + public void seqNo(Field seqNo) { + in.seqNo(seqNo); + } + @Override public AllEntries allEntries() { return in.allEntries(); @@ -386,7 +396,7 @@ public static class InternalParseContext extends ParseContext { private String id; - private Field uid, version; + private Field uid, version, seqNo; private StringBuilder stringBuilder = new StringBuilder(); @@ -564,6 +574,17 @@ public void version(Field version) { this.version = version; } + @Override + public Field seqNo() { + return this.seqNo; + } + + @Override + public void seqNo(Field seqNo) { + this.seqNo = seqNo; + } + + @Override public AllEntries allEntries() { return this.allEntries; @@ -730,6 +751,10 @@ public boolean isWithinMultiFields() { public abstract void version(Field version); + public abstract Field seqNo(); + + public abstract void seqNo(Field seqNo); + public final boolean includeInAll(Boolean includeInAll, FieldMapper mapper) { return includeInAll(includeInAll, mapper.fieldType().indexOptions() != IndexOptions.NONE); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index ed8314c6f7de8..ea0b786354d46 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -30,7 +30,7 @@ */ public class ParsedDocument { - private final Field uid, version; + private final Field uid, version, seqNo; private final String id; @@ -50,9 +50,10 @@ public class ParsedDocument { private String parent; - public ParsedDocument(Field uid, Field version, String id, String type, String routing, long timestamp, long ttl, List documents, BytesReference source, Mapping dynamicMappingsUpdate) { + public ParsedDocument(Field uid, Field version, Field seqNo, String id, String type, String routing, long timestamp, long ttl, List documents, BytesReference source, Mapping dynamicMappingsUpdate) { this.uid = uid; this.version = version; + this.seqNo = seqNo; this.id = id; this.type = type; this.routing = routing; @@ -71,6 +72,10 @@ public Field version() { return version; } + public Field seqNo() { + return seqNo; + } + public String id() { return this.id; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/SeqNoFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/SeqNoFieldMapper.java new file mode 100644 index 0000000000000..8f0709e986eef --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/SeqNoFieldMapper.java @@ -0,0 +1,151 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.mapper.internal; + +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.index.DocValuesType; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.mapper.*; +import org.elasticsearch.index.mapper.ParseContext.Document; +import org.elasticsearch.index.seqno.SequenceNumbersService; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +/** Mapper for the _version field. */ +public class SeqNoFieldMapper extends MetadataFieldMapper { + + public static final String NAME = "_seq_no"; + public static final String CONTENT_TYPE = "_seq_no"; + + public static class Defaults { + + public static final String NAME = SeqNoFieldMapper.NAME; + public static final MappedFieldType FIELD_TYPE = new SeqNoFieldType(); + + static { + FIELD_TYPE.setNames(new MappedFieldType.Names(NAME)); + FIELD_TYPE.setDocValuesType(DocValuesType.NUMERIC); + FIELD_TYPE.setHasDocValues(true); + FIELD_TYPE.freeze(); + } + } + + public static class Builder extends MetadataFieldMapper.Builder { + + public Builder() { + super(Defaults.NAME, Defaults.FIELD_TYPE); + } + + @Override + public SeqNoFieldMapper build(BuilderContext context) { + return new SeqNoFieldMapper(context.indexSettings()); + } + } + + public static class TypeParser implements Mapper.TypeParser { + @Override + public Mapper.Builder parse(String name, Map node, ParserContext parserContext) throws MapperParsingException { + throw new MapperParsingException(NAME + " is not configurable"); + } + } + + static final class SeqNoFieldType extends MappedFieldType { + + public SeqNoFieldType() { + setFieldDataType(new FieldDataType("long")); + } + + protected SeqNoFieldType(SeqNoFieldType ref) { + super(ref); + } + + @Override + public MappedFieldType clone() { + return new SeqNoFieldType(this); + } + + @Override + public String typeName() { + return CONTENT_TYPE; + } + + @Override + public Long value(Object value) { + if (value == null || (value instanceof Long)) { + return (Long) value; + } else { + return Long.parseLong(value.toString()); + } + } + } + + public SeqNoFieldMapper(Settings indexSettings) { + super(NAME, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE, indexSettings); + } + + @Override + public void preParse(ParseContext context) throws IOException { + super.parse(context); + } + + @Override + protected void parseCreateField(ParseContext context, List fields) throws IOException { + // see InternalEngine.updateVersion to see where the real version value is set + final Field seqNo = new NumericDocValuesField(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO); + context.seqNo(seqNo); + fields.add(seqNo); + } + + @Override + public Mapper parse(ParseContext context) throws IOException { + // _seqno added in preparse + return null; + } + + @Override + public void postParse(ParseContext context) throws IOException { + // In the case of nested docs, let's fill nested docs with seqNo=1 so that Lucene doesn't write a Bitset for documents + // that don't have the field. This is consistent with the default value for efficiency. + for (int i = 1; i < context.docs().size(); i++) { + final Document doc = context.docs().get(i); + doc.add(new NumericDocValuesField(NAME, 1L)); + } + } + + @Override + protected String contentType() { + return CONTENT_TYPE; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder; + } + + @Override + public void merge(Mapper mergeWith, MergeResult mergeResult) throws MergeMappingException { + // nothing to do + } +} diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java new file mode 100644 index 0000000000000..46b033622432b --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -0,0 +1,60 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.index.seqno; + +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.ShardId; + +import java.util.concurrent.atomic.AtomicLong; + +/** a very light weight implementation. will be replaced with proper machinery later */ +public class SequenceNumbersService extends AbstractIndexShardComponent { + + public final static long UNASSIGNED_SEQ_NO = -1L; + + AtomicLong seqNoGenerator = new AtomicLong(); + + public SequenceNumbersService(ShardId shardId, IndexSettings indexSettings) { + super(shardId, indexSettings); + } + + /** + * generates a new sequence number. + * Note: you must call {@link #markSeqNoAsCompleted(long)} after the operation for which this seq# was generated + * was completed (whether successfully or with a failure + */ + public long generateSeqNo() { + return seqNoGenerator.getAndIncrement(); + } + + public void markSeqNoAsCompleted(long seqNo) { + // this is temporary to make things semi sane on primary promotion and recovery. will be replaced with better machinery + boolean success; + do { + long maxSeqNo = seqNoGenerator.get(); + if (seqNo > maxSeqNo) { + success = seqNoGenerator.compareAndSet(maxSeqNo, seqNo); + } else { + success = true; + } + } while (success == false); + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java b/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java index 31c235e09ecd1..e632c0669f6cb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.shard; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.rest.RestStatus; @@ -34,12 +33,12 @@ public class IllegalIndexShardStateException extends ElasticsearchException { private final IndexShardState currentState; - public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg) { - this(shardId, currentState, msg, null); + public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Object... args) { + this(shardId, currentState, msg, null, args); } - public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Throwable ex) { - super("CurrentState[" + currentState + "] " + msg, ex); + public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Throwable ex, Object... args) { + super("CurrentState[" + currentState + "] " + msg, ex, args); setShard(shardId); this.currentState = currentState; } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 2370ace0464e9..8af89deac9fdb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -19,10 +19,7 @@ package org.elasticsearch.index.shard; -import org.apache.lucene.index.CheckIndex; -import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; -import org.apache.lucene.index.SnapshotDeletionPolicy; +import org.apache.lucene.index.*; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.UsageTrackingQueryCachingPolicy; import org.apache.lucene.store.AlreadyClosedException; @@ -82,6 +79,7 @@ import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.snapshots.IndexShardRepository; import org.elasticsearch.index.store.Store; @@ -194,8 +192,10 @@ public class IndexShard extends AbstractIndexShardComponent { private final IndexSearcherWrapper searcherWrapper; private final TimeValue inactiveTime; - /** True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link - * IndexingMemoryController}). */ + /** + * True if this shard is still indexing (recently) and false if we've been idle for long enough (as periodically checked by {@link + * IndexingMemoryController}). + */ private final AtomicBoolean active = new AtomicBoolean(); public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache, @@ -223,7 +223,7 @@ public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, this.termVectorsService = provider.getTermVectorsService(); this.searchService = new ShardSearchStats(settings); this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings); - this.indicesQueryCache = provider.getIndicesQueryCache(); + this.indicesQueryCache = provider.getIndicesQueryCache(); this.shardQueryCache = new ShardRequestCache(shardId, indexSettings); this.shardFieldData = new ShardFieldData(); this.indexFieldDataService = indexFieldDataService; @@ -233,6 +233,7 @@ public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, this.flushOnClose = settings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true); this.path = path; this.mergePolicyConfig = new MergePolicyConfig(logger, settings); + /* create engine config */ logger.debug("state: [CREATED]"); @@ -445,22 +446,38 @@ private IndexShardState changeState(IndexShardState newState, String reason) { return previousState; } - public Engine.Index prepareIndex(SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin) { + public Engine.Index prepareIndexOnPrimary(SourceToParse source, long version, VersionType versionType) { try { - return prepareIndex(docMapper(source.type()), source, version, versionType, origin); + if (shardRouting.primary() == false) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary"); + } + return prepareIndex(docMapper(source.type()), source, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, versionType, Engine.Operation.Origin.PRIMARY); } catch (Throwable t) { verifyNotClosed(t); throw t; } } - static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long version, VersionType versionType, Engine.Operation.Origin origin) { + public Engine.Index prepareIndexOnReplica(SourceToParse source, long seqNo, long version, VersionType versionType) { + try { + if (shardRouting.primary() && shardRouting.isRelocationTarget() == false) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a replica"); + } + return prepareIndex(docMapper(source.type()), source, seqNo, version, versionType, Engine.Operation.Origin.REPLICA); + } catch (Throwable t) { + verifyNotClosed(t); + throw t; + } + } + + static Engine.Index prepareIndex(DocumentMapperForType docMapper, SourceToParse source, long seqNo, long version, VersionType versionType, Engine.Operation.Origin origin) { long startTime = System.nanoTime(); ParsedDocument doc = docMapper.getDocumentMapper().parse(source); if (docMapper.getMapping() != null) { doc.addDynamicMappingsUpdate(docMapper.getMapping()); } - return new Engine.Index(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime); + doc.seqNo().setLongValue(seqNo); + return new Engine.Index(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, seqNo, version, versionType, origin, startTime); } /** @@ -486,10 +503,27 @@ public boolean index(Engine.Index index) { return created; } - public Engine.Delete prepareDelete(String type, String id, long version, VersionType versionType, Engine.Operation.Origin origin) { - long startTime = System.nanoTime(); + public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) { + if (shardRouting.primary() == false) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary"); + } + final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); + final Term uid = documentMapper.uidMapper().term(Uid.createUid(type, id)); + return prepareDelete(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, versionType, Engine.Operation.Origin.PRIMARY); + } + + public Engine.Delete prepareDeleteOnReplica(String type, String id, long seqNo, long version, VersionType versionType) { + if (shardRouting.primary()) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a replica"); + } final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); - return new Engine.Delete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, origin, startTime, false); + final Term uid = documentMapper.uidMapper().term(Uid.createUid(type, id)); + return prepareDelete(type, id, uid, seqNo, version, versionType, Engine.Operation.Origin.REPLICA); + } + + static Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo, long version, VersionType versionType, Engine.Operation.Origin origin) { + long startTime = System.nanoTime(); + return new Engine.Delete(type, id, uid, seqNo, version, versionType, origin, startTime, false); } public void delete(Engine.Delete delete) { @@ -533,11 +567,8 @@ public FlushStats flushStats() { } public DocsStats docStats() { - final Engine.Searcher searcher = acquireSearcher("doc_stats"); - try { + try (Engine.Searcher searcher = acquireSearcher("doc_stats")) { return new DocsStats(searcher.reader().numDocs(), searcher.reader().numDeletedDocs()); - } finally { - searcher.close(); } } @@ -977,8 +1008,10 @@ public void addShardFailureCallback(Callback onShardFailure) { this.shardEventListener.delegates.add(onShardFailure); } - /** Change the indexing and translog buffer sizes. If {@code IndexWriter} is currently using more than - * the new buffering indexing size then we do a refresh to free up the heap. */ + /** + * Change the indexing and translog buffer sizes. If {@code IndexWriter} is currently using more than + * the new buffering indexing size then we do a refresh to free up the heap. + */ public void updateBufferSize(ByteSizeValue shardIndexingBufferSize, ByteSizeValue shardTranslogBufferSize) { final EngineConfig config = engineConfig; @@ -1021,9 +1054,11 @@ public void updateBufferSize(ByteSizeValue shardIndexingBufferSize, ByteSizeValu engine.getTranslog().updateBuffer(shardTranslogBufferSize); } - /** Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last - * indexing operation, and become inactive (reducing indexing and translog buffers to tiny values) if so. This returns true - * if the shard is inactive. */ + /** + * Called by {@link IndexingMemoryController} to check whether more than {@code inactiveTimeNS} has passed since the last + * indexing operation, and become inactive (reducing indexing and translog buffers to tiny values) if so. This returns true + * if the shard is inactive. + */ public boolean checkIdle() { return checkIdle(inactiveTime.nanos()); } @@ -1042,8 +1077,10 @@ final boolean checkIdle(long inactiveTimeNS) { // pkg private for testing return active.get() == false; } - /** Returns {@code true} if this shard is active (has seen indexing ops in the last {@link - * IndexShard#INDEX_SHARD_INACTIVE_TIME_SETTING} (default 5 minutes), else {@code false}. */ + /** + * Returns {@code true} if this shard is active (has seen indexing ops in the last {@link + * IndexShard#INDEX_SHARD_INACTIVE_TIME_SETTING} (default 5 minutes), else {@code false}. + */ public boolean getActive() { return active.get(); } @@ -1077,10 +1114,10 @@ public boolean recoverFromStore(DiscoveryNode localNode) { return storeRecovery.recoverFromStore(this, shouldExist, localNode); } - public boolean restoreFromRepository(IndexShardRepository repository, DiscoveryNode locaNode) { + public boolean restoreFromRepository(IndexShardRepository repository, DiscoveryNode localNode) { assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard"; StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); - return storeRecovery.recoverFromRepository(this, repository, locaNode); + return storeRecovery.recoverFromRepository(this, repository, localNode); } /** @@ -1244,35 +1281,32 @@ public void run() { reschedule(); return; } - threadPool.executor(ThreadPool.Names.REFRESH).execute(new Runnable() { - @Override - public void run() { - try { - if (getEngine().refreshNeeded()) { - refresh("schedule"); - } - } catch (EngineClosedException e) { - // we are being closed, ignore - } catch (RefreshFailedEngineException e) { - if (e.getCause() instanceof InterruptedException) { - // ignore, we are being shutdown - } else if (e.getCause() instanceof ClosedByInterruptException) { - // ignore, we are being shutdown - } else if (e.getCause() instanceof ThreadInterruptedException) { - // ignore, we are being shutdown - } else { - if (state != IndexShardState.CLOSED) { - logger.warn("Failed to perform scheduled engine refresh", e); - } - } - } catch (Exception e) { + threadPool.executor(ThreadPool.Names.REFRESH).execute(() -> { + try { + if (getEngine().refreshNeeded()) { + refresh("schedule"); + } + } catch (EngineClosedException e) { + // we are being closed, ignore + } catch (RefreshFailedEngineException e) { + if (e.getCause() instanceof InterruptedException) { + // ignore, we are being shutdown + } else if (e.getCause() instanceof ClosedByInterruptException) { + // ignore, we are being shutdown + } else if (e.getCause() instanceof ThreadInterruptedException) { + // ignore, we are being shutdown + } else { if (state != IndexShardState.CLOSED) { logger.warn("Failed to perform scheduled engine refresh", e); } } - - reschedule(); + } catch (Exception e) { + if (state != IndexShardState.CLOSED) { + logger.warn("Failed to perform scheduled engine refresh", e); + } } + + reschedule(); }); } @@ -1369,8 +1403,10 @@ Engine getEngine() { return engine; } - /** NOTE: returns null if engine is not yet started (e.g. recovery phase 1, copying over index files, is still running), or if engine is - * closed. */ + /** + * NOTE: returns null if engine is not yet started (e.g. recovery phase 1, copying over index files, is still running), or if engine is + * closed. + */ protected Engine getEngineOrNull() { return this.currentEngineReference.get(); } @@ -1462,7 +1498,8 @@ protected void operationProcessed() { final Engine.Warmer engineWarmer = (searcher, toLevel) -> warmer.warm(searcher, this, idxSettings, toLevel); return new EngineConfig(shardId, threadPool, indexingService, indexSettings, engineWarmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig, - mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, inactiveTime); + mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, + indexCache.query(), cachingPolicy, translogConfig, inactiveTime); } private static class IndexShardOperationCounter extends AbstractRefCounted { @@ -1486,7 +1523,14 @@ protected void alreadyClosed() { } } - public void incrementOperationCounter() { + /** + * increments the ongoing operations counter. If the given primary term is lower then the one in {@link #shardRouting} + * an {@link IllegalIndexShardStateException} is thrown. + */ + public void incrementOperationCounter(long opPrimaryTerm) { + if (shardRouting.primaryTerm() > opPrimaryTerm) { + throw new IllegalIndexShardStateException(shardId, state, "operation term [{}] is too old (current [{}])", opPrimaryTerm, shardRouting.primaryTerm()); + } indexShardOperationCounter.incRef(); } @@ -1578,6 +1622,7 @@ public void onAfter() { /** * Simple struct encapsulating a shard failure + * * @see IndexShard#addShardFailureCallback(Callback) */ public static final class ShardFailure { @@ -1604,7 +1649,7 @@ protected QueryShardContext initialValue() { }; private QueryShardContext newQueryShardContext() { - return new QueryShardContext(idxSettings, provider.getClient(), indexCache.bitsetFilterCache(), indexFieldDataService, mapperService, similarityService, provider.getScriptService(), provider.getIndicesQueriesRegistry()); + return new QueryShardContext(idxSettings, provider.getClient(), indexCache.bitsetFilterCache(), indexFieldDataService, mapperService, similarityService, provider.getScriptService(), provider.getIndicesQueriesRegistry()); } /** diff --git a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java index 68c552d4419b6..20a681678cd3a 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java +++ b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java @@ -128,7 +128,7 @@ public void performRecoveryOperation(Engine engine, Translog.Operation operation Translog.Index index = (Translog.Index) operation; Engine.Index engineIndex = IndexShard.prepareIndex(docMapper(index.type()), source(index.source()).type(index.type()).id(index.id()) .routing(index.routing()).parent(index.parent()).timestamp(index.timestamp()).ttl(index.ttl()), - index.version(), index.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY); + index.seqNo(), index.version(), index.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY); maybeAddMappingUpdate(engineIndex.type(), engineIndex.parsedDoc().dynamicMappingsUpdate(), engineIndex.id(), allowMappingUpdates); if (logger.isTraceEnabled()) { logger.trace("[translog] recover [index] op of [{}][{}]", index.type(), index.id()); @@ -141,8 +141,9 @@ public void performRecoveryOperation(Engine engine, Translog.Operation operation if (logger.isTraceEnabled()) { logger.trace("[translog] recover [delete] op of [{}][{}]", uid.type(), uid.id()); } - engine.delete(new Engine.Delete(uid.type(), uid.id(), delete.uid(), delete.version(), - delete.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY, System.nanoTime(), false)); + Engine.Delete engineDelete = IndexShard.prepareDelete(uid.type(), uid.id(), delete.uid(), delete.seqNo(), + delete.version(), delete.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY); + engine.delete(engineDelete); break; default: throw new IllegalStateException("No operation defined for [" + operation + "]"); diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index f69cac45f7b33..f46d32b3f4d39 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -129,11 +129,11 @@ public void handle(View view) { }; - /** * Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogConfig} has * a non-null {@link org.elasticsearch.index.translog.Translog.TranslogGeneration}. If the generation is null this method * us destructive and will delete all files in the translog path given. + * * @see TranslogConfig#getTranslogPath() */ public Translog(TranslogConfig config) throws IOException { @@ -141,7 +141,7 @@ public Translog(TranslogConfig config) throws IOException { this.config = config; TranslogGeneration translogGeneration = config.getTranslogGeneration(); - if (translogGeneration == null || translogGeneration.translogUUID == null) { // legacy case + if (translogGeneration == null || translogGeneration.translogUUID == null) { // legacy case translogUUID = Strings.randomBase64UUID(); } else { translogUUID = translogGeneration.translogUUID; @@ -347,7 +347,6 @@ public long sizeInBytes() { } - TranslogWriter createWriter(long fileGeneration) throws IOException { TranslogWriter newFile; try { @@ -508,6 +507,7 @@ static String getCommitCheckpointFileName(long generation) { /** * Ensures that the given location has be synced / written to the underlying storage. + * * @return Returns true iff this call caused an actual sync operation otherwise false */ public boolean ensureSynced(Location location) throws IOException { @@ -749,13 +749,21 @@ public int compareTo(Location o) { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } Location location = (Location) o; - if (generation != location.generation) return false; - if (translogLocation != location.translogLocation) return false; + if (generation != location.generation) { + return false; + } + if (translogLocation != location.translogLocation) { + return false; + } return size == location.size; } @@ -846,10 +854,11 @@ public Source(BytesReference source, String routing, String parent, long timesta } public static class Index implements Operation { - public static final int SERIALIZATION_FORMAT = 6; + public static final int SERIALIZATION_FORMAT = 7; private String id; private String type; + private long seqNo = -1; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; private BytesReference source; @@ -867,6 +876,7 @@ public Index(Engine.Index index) { this.source = index.source(); this.routing = index.routing(); this.parent = index.parent(); + this.seqNo = index.seqNo(); this.version = index.version(); this.timestamp = index.timestamp(); this.ttl = index.ttl(); @@ -877,6 +887,8 @@ public Index(String type, String id, byte[] source) { this.type = type; this.id = id; this.source = new BytesArray(source); + this.seqNo = 0; + this.version = 0; } @Override @@ -917,6 +929,10 @@ public BytesReference source() { return this.source; } + public long seqNo() { + return seqNo; + } + public long version() { return this.version; } @@ -959,6 +975,9 @@ public void readFrom(StreamInput in) throws IOException { if (version >= 6) { this.versionType = VersionType.fromValue(in.readByte()); } + if (version >= 7) { + this.seqNo = in.readVLong(); + } } catch (Exception e) { throw new ElasticsearchException("failed to read [" + type + "][" + id + "]", e); } @@ -988,6 +1007,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(timestamp); out.writeLong(ttl); out.writeByte(versionType.getValue()); + out.writeVLong(seqNo); } @Override @@ -1002,6 +1022,7 @@ public boolean equals(Object o) { Index index = (Index) o; if (version != index.version || + seqNo != index.seqNo || timestamp != index.timestamp || ttl != index.ttl || id.equals(index.id) == false || @@ -1021,6 +1042,7 @@ public boolean equals(Object o) { public int hashCode() { int result = id.hashCode(); result = 31 * result + type.hashCode(); + result = 31 * result + Long.hashCode(seqNo); result = 31 * result + Long.hashCode(version); result = 31 * result + versionType.hashCode(); result = 31 * result + source.hashCode(); @@ -1041,9 +1063,10 @@ public String toString() { } public static class Delete implements Operation { - public static final int SERIALIZATION_FORMAT = 2; + public static final int SERIALIZATION_FORMAT = 3; private Term uid; + private long seqNo = -1L; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; @@ -1051,19 +1074,19 @@ public Delete() { } public Delete(Engine.Delete delete) { - this(delete.uid()); - this.version = delete.version(); - this.versionType = delete.versionType(); + this(delete.uid(), delete.seqNo(), delete.version(), delete.versionType()); } + /** utility for testing */ public Delete(Term uid) { - this.uid = uid; + this(uid, 0, 0, VersionType.EXTERNAL); } - public Delete(Term uid, long version, VersionType versionType) { + public Delete(Term uid, long seqNo, long version, VersionType versionType) { this.uid = uid; this.version = version; this.versionType = versionType; + this.seqNo = seqNo; } @Override @@ -1080,6 +1103,10 @@ public Term uid() { return this.uid; } + public long seqNo() { + return seqNo; + } + public long version() { return this.version; } @@ -1089,7 +1116,7 @@ public VersionType versionType() { } @Override - public Source getSource(){ + public Source getSource() { throw new IllegalStateException("trying to read doc source from delete operation"); } @@ -1103,6 +1130,9 @@ public void readFrom(StreamInput in) throws IOException { if (version >= 2) { this.versionType = VersionType.fromValue(in.readByte()); } + if (version >= 3) { + this.seqNo = in.readVLong(); + } assert versionType.validateVersionForWrites(version); } @@ -1114,6 +1144,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(uid.text()); out.writeLong(version); out.writeByte(versionType.getValue()); + out.writeVLong(seqNo); } @Override @@ -1127,7 +1158,7 @@ public boolean equals(Object o) { Delete delete = (Delete) o; - return version == delete.version && + return version == delete.version && seqNo == delete.seqNo && uid.equals(delete.uid) && versionType == delete.versionType; } @@ -1135,6 +1166,7 @@ public boolean equals(Object o) { @Override public int hashCode() { int result = uid.hashCode(); + result = 31 * result + Long.hashCode(seqNo); result = 31 * result + Long.hashCode(version); result = 31 * result + versionType.hashCode(); return result; @@ -1198,7 +1230,7 @@ static Translog.Operation readOperation(BufferedChecksumStreamInput in) throws I // to prevent this unfortunately. in.mark(opSize); - in.skip(opSize-4); + in.skip(opSize - 4); verifyChecksum(in); in.reset(); } @@ -1250,7 +1282,7 @@ public static void writeOperationNoSize(BufferedChecksumStreamOutput out, Transl out.writeByte(op.opType().id()); op.writeTo(out); long checksum = out.getChecksum(); - out.writeInt((int)checksum); + out.writeInt((int) checksum); } /** diff --git a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java index 9018435271413..cb4d09742840d 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java @@ -20,7 +20,7 @@ package org.elasticsearch.rest.action.bulk; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; @@ -97,49 +97,35 @@ public RestResponse buildResponse(BulkResponse response, XContentBuilder builder for (BulkItemResponse itemResponse : response) { builder.startObject(); builder.startObject(itemResponse.getOpType()); - builder.field(Fields._INDEX, itemResponse.getIndex()); - builder.field(Fields._TYPE, itemResponse.getType()); - builder.field(Fields._ID, itemResponse.getId()); - long version = itemResponse.getVersion(); - if (version != -1) { - builder.field(Fields._VERSION, itemResponse.getVersion()); - } if (itemResponse.isFailed()) { + builder.field(Fields._INDEX, itemResponse.getIndex()); + builder.field(Fields._TYPE, itemResponse.getType()); + builder.field(Fields._ID, itemResponse.getId()); builder.field(Fields.STATUS, itemResponse.getFailure().getStatus().getStatus()); builder.startObject(Fields.ERROR); ElasticsearchException.toXContent(builder, request, itemResponse.getFailure().getCause()); builder.endObject(); } else { - ActionWriteResponse.ShardInfo shardInfo = itemResponse.getResponse().getShardInfo(); - shardInfo.toXContent(builder, request); - if (itemResponse.getResponse() instanceof DeleteResponse) { - DeleteResponse deleteResponse = itemResponse.getResponse(); - if (deleteResponse.isFound()) { - builder.field(Fields.STATUS, shardInfo.status().getStatus()); - } else { - builder.field(Fields.STATUS, RestStatus.NOT_FOUND.getStatus()); + final DocWriteResponse docResponse = itemResponse.getResponse(); + docResponse.toXContent(builder, request); + RestStatus status = docResponse.getShardInfo().status(); + if (docResponse instanceof DeleteResponse) { + DeleteResponse deleteResponse = (DeleteResponse) docResponse; + if (deleteResponse.isFound() == false) { + status = RestStatus.NOT_FOUND; } - builder.field(Fields.FOUND, deleteResponse.isFound()); - } else if (itemResponse.getResponse() instanceof IndexResponse) { - IndexResponse indexResponse = itemResponse.getResponse(); + } else if (docResponse instanceof IndexResponse) { + IndexResponse indexResponse = (IndexResponse) docResponse; if (indexResponse.isCreated()) { - builder.field(Fields.STATUS, RestStatus.CREATED.getStatus()); - } else { - builder.field(Fields.STATUS, shardInfo.status().getStatus()); + status = RestStatus.CREATED; } - } else if (itemResponse.getResponse() instanceof UpdateResponse) { - UpdateResponse updateResponse = itemResponse.getResponse(); + } else if (docResponse instanceof UpdateResponse) { + UpdateResponse updateResponse = (UpdateResponse) docResponse; if (updateResponse.isCreated()) { - builder.field(Fields.STATUS, RestStatus.CREATED.getStatus()); - } else { - builder.field(Fields.STATUS, shardInfo.status().getStatus()); - } - if (updateResponse.getGetResult() != null) { - builder.startObject(Fields.GET); - updateResponse.getGetResult().toXContentEmbedded(builder, request); - builder.endObject(); + status = RestStatus.CREATED; } } + builder.field(Fields.STATUS, status.getStatus()); } builder.endObject(); builder.endObject(); @@ -161,9 +147,5 @@ static final class Fields { static final XContentBuilderString STATUS = new XContentBuilderString("status"); static final XContentBuilderString ERROR = new XContentBuilderString("error"); static final XContentBuilderString TOOK = new XContentBuilderString("took"); - static final XContentBuilderString _VERSION = new XContentBuilderString("_version"); - static final XContentBuilderString FOUND = new XContentBuilderString("found"); - static final XContentBuilderString GET = new XContentBuilderString("get"); } - } diff --git a/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java b/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java index 209ab686ce5b3..9b740954aaf21 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.rest.action.delete; -import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; @@ -27,7 +26,6 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.index.VersionType; import org.elasticsearch.rest.*; import org.elasticsearch.rest.action.support.RestActions; @@ -65,15 +63,10 @@ public void handleRequest(final RestRequest request, final RestChannel channel, client.delete(deleteRequest, new RestBuilderListener(channel) { @Override public RestResponse buildResponse(DeleteResponse result, XContentBuilder builder) throws Exception { - ActionWriteResponse.ShardInfo shardInfo = result.getShardInfo(); - builder.startObject().field(Fields.FOUND, result.isFound()) - .field(Fields._INDEX, result.getIndex()) - .field(Fields._TYPE, result.getType()) - .field(Fields._ID, result.getId()) - .field(Fields._VERSION, result.getVersion()) - .value(shardInfo) - .endObject(); - RestStatus status = shardInfo.status(); + builder.startObject(); + result.toXContent(builder, request); + builder.endObject(); + RestStatus status = result.getShardInfo().status(); if (!result.isFound()) { status = NOT_FOUND; } @@ -81,12 +74,4 @@ public RestResponse buildResponse(DeleteResponse result, XContentBuilder builder } }); } - - static final class Fields { - static final XContentBuilderString FOUND = new XContentBuilderString("found"); - static final XContentBuilderString _INDEX = new XContentBuilderString("_index"); - static final XContentBuilderString _TYPE = new XContentBuilderString("_type"); - static final XContentBuilderString _ID = new XContentBuilderString("_id"); - static final XContentBuilderString _VERSION = new XContentBuilderString("_version"); - } } diff --git a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java index d0d0fe68a13c1..ec673952b7c63 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.rest.action.index; -import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; @@ -27,7 +26,6 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.index.VersionType; import org.elasticsearch.rest.*; import org.elasticsearch.rest.action.support.RestActions; @@ -37,7 +35,8 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; -import static org.elasticsearch.rest.RestStatus.*; +import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; +import static org.elasticsearch.rest.RestStatus.CREATED; /** * @@ -85,7 +84,7 @@ public void handleRequest(final RestRequest request, final RestChannel channel, if (sOpType != null) { try { indexRequest.opType(IndexRequest.OpType.fromString(sOpType)); - } catch (IllegalArgumentException eia){ + } catch (IllegalArgumentException eia) { try { XContentBuilder builder = channel.newErrorBuilder(); channel.sendResponse(new BytesRestResponse(BAD_REQUEST, builder.startObject().field("error", eia.getMessage()).endObject())); @@ -103,15 +102,8 @@ public void handleRequest(final RestRequest request, final RestChannel channel, @Override public RestResponse buildResponse(IndexResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - ActionWriteResponse.ShardInfo shardInfo = response.getShardInfo(); - builder.field(Fields._INDEX, response.getIndex()) - .field(Fields._TYPE, response.getType()) - .field(Fields._ID, response.getId()) - .field(Fields._VERSION, response.getVersion()); - shardInfo.toXContent(builder, request); - builder.field(Fields.CREATED, response.isCreated()); - builder.endObject(); - RestStatus status = shardInfo.status(); + response.toXContent(builder, request); + RestStatus status = response.getShardInfo().status(); if (response.isCreated()) { status = CREATED; } @@ -119,13 +111,4 @@ public RestResponse buildResponse(IndexResponse response, XContentBuilder builde } }); } - - static final class Fields { - static final XContentBuilderString _INDEX = new XContentBuilderString("_index"); - static final XContentBuilderString _TYPE = new XContentBuilderString("_type"); - static final XContentBuilderString _ID = new XContentBuilderString("_id"); - static final XContentBuilderString _VERSION = new XContentBuilderString("_version"); - static final XContentBuilderString CREATED = new XContentBuilderString("created"); - } - } diff --git a/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java b/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java index a23780db62ee5..7a46aadde8113 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.rest.action.update; -import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.update.UpdateRequest; @@ -29,15 +28,8 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.index.VersionType; -import org.elasticsearch.rest.BaseRestHandler; -import org.elasticsearch.rest.BytesRestResponse; -import org.elasticsearch.rest.RestChannel; -import org.elasticsearch.rest.RestController; -import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.rest.RestResponse; -import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.*; import org.elasticsearch.rest.action.support.RestActions; import org.elasticsearch.rest.action.support.RestBuilderListener; import org.elasticsearch.script.Script; @@ -127,21 +119,9 @@ public void handleRequest(final RestRequest request, final RestChannel channel, @Override public RestResponse buildResponse(UpdateResponse response, XContentBuilder builder) throws Exception { builder.startObject(); - ActionWriteResponse.ShardInfo shardInfo = response.getShardInfo(); - builder.field(Fields._INDEX, response.getIndex()) - .field(Fields._TYPE, response.getType()) - .field(Fields._ID, response.getId()) - .field(Fields._VERSION, response.getVersion()); - - shardInfo.toXContent(builder, request); - if (response.getGetResult() != null) { - builder.startObject(Fields.GET); - response.getGetResult().toXContentEmbedded(builder, request); - builder.endObject(); - } - + response.toXContent(builder, request); builder.endObject(); - RestStatus status = shardInfo.status(); + RestStatus status = response.getShardInfo().status(); if (response.isCreated()) { status = CREATED; } @@ -149,12 +129,4 @@ public RestResponse buildResponse(UpdateResponse response, XContentBuilder build } }); } - - static final class Fields { - static final XContentBuilderString _INDEX = new XContentBuilderString("_index"); - static final XContentBuilderString _TYPE = new XContentBuilderString("_type"); - static final XContentBuilderString _ID = new XContentBuilderString("_id"); - static final XContentBuilderString _VERSION = new XContentBuilderString("_version"); - static final XContentBuilderString GET = new XContentBuilderString("get"); - } } diff --git a/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java b/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java index df2362dd47bad..e842b571fa856 100644 --- a/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java +++ b/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java @@ -20,7 +20,7 @@ package org.elasticsearch.transport; /** - * A simple based class that always spawns. + * A simple base class that always spawns. */ public abstract class BaseTransportResponseHandler implements TransportResponseHandler { diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index a7fc01e9677a6..68f27a16c4fa6 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -20,8 +20,8 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionWriteResponse; import org.elasticsearch.action.NoShardAvailableActionException; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushResponse; @@ -58,12 +58,8 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithAssignedPrimariesAndOneReplica; -import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithNoShard; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.*; +import static org.hamcrest.Matchers.*; public class BroadcastReplicationTests extends ESTestCase { @@ -101,7 +97,7 @@ public void testNotStartedPrimary() throws InterruptedException, ExecutionExcept randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED)); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); - for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { + for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { if (randomBoolean()) { shardRequests.v2().onFailure(new NoShardAvailableActionException(shardRequests.v1())); } else { @@ -120,10 +116,10 @@ public void testStartedPrimary() throws InterruptedException, ExecutionException ShardRoutingState.STARTED)); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); - for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { - ActionWriteResponse actionWriteResponse = new ActionWriteResponse(); - actionWriteResponse.setShardInfo(new ActionWriteResponse.ShardInfo(1, 1, new ActionWriteResponse.ShardInfo.Failure[0])); - shardRequests.v2().onResponse(actionWriteResponse); + for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { + ReplicationResponse replicationResponse = new ReplicationResponse(); + replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(1, 1, new ReplicationResponse.ShardInfo.Failure[0])); + shardRequests.v2().onResponse(replicationResponse); } logger.info("total shards: {}, ", response.get().getTotalShards()); assertBroadcastResponse(1, 1, 0, response.get(), null); @@ -137,20 +133,20 @@ public void testResultCombine() throws InterruptedException, ExecutionException, Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); int succeeded = 0; int failed = 0; - for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { + for (Tuple> shardRequests : broadcastReplicationAction.capturedShardRequests) { if (randomBoolean()) { - ActionWriteResponse.ShardInfo.Failure[] failures = new ActionWriteResponse.ShardInfo.Failure[0]; + ReplicationResponse.ShardInfo.Failure[] failures = new ReplicationResponse.ShardInfo.Failure[0]; int shardsSucceeded = randomInt(1) + 1; succeeded += shardsSucceeded; - ActionWriteResponse actionWriteResponse = new ActionWriteResponse(); + ReplicationResponse replicationResponse = new ReplicationResponse(); if (shardsSucceeded == 1 && randomBoolean()) { //sometimes add failure (no failure means shard unavailable) - failures = new ActionWriteResponse.ShardInfo.Failure[1]; - failures[0] = new ActionWriteResponse.ShardInfo.Failure(index, shardRequests.v1().id(), null, new Exception("pretend shard failed"), RestStatus.GATEWAY_TIMEOUT, false); + failures = new ReplicationResponse.ShardInfo.Failure[1]; + failures[0] = new ReplicationResponse.ShardInfo.Failure(index, shardRequests.v1().id(), null, new Exception("pretend shard failed"), RestStatus.GATEWAY_TIMEOUT, false); failed++; } - actionWriteResponse.setShardInfo(new ActionWriteResponse.ShardInfo(2, shardsSucceeded, failures)); - shardRequests.v2().onResponse(actionWriteResponse); + replicationResponse.setShardInfo(new ReplicationResponse.ShardInfo(2, shardsSucceeded, failures)); + shardRequests.v2().onResponse(replicationResponse); } else { // sometimes fail failed += 2; @@ -179,16 +175,16 @@ public void testShardsList() throws InterruptedException, ExecutionException { assertThat(shards.get(0), equalTo(shardId)); } - private class TestBroadcastReplicationAction extends TransportBroadcastReplicationAction { - protected final Set>> capturedShardRequests = ConcurrentCollections.newConcurrentSet(); + private class TestBroadcastReplicationAction extends TransportBroadcastReplicationAction { + protected final Set>> capturedShardRequests = ConcurrentCollections.newConcurrentSet(); public TestBroadcastReplicationAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, TransportReplicationAction replicatedBroadcastShardAction) { super("test-broadcast-replication-action", BroadcastRequest::new, settings, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, replicatedBroadcastShardAction); } @Override - protected ActionWriteResponse newShardResponse() { - return new ActionWriteResponse(); + protected ReplicationResponse newShardResponse() { + return new ReplicationResponse(); } @Override @@ -202,7 +198,7 @@ protected BroadcastResponse newResponse(int successfulShards, int failedShards, } @Override - protected void shardExecute(BroadcastRequest request, ShardId shardId, ActionListener shardActionListener) { + protected void shardExecute(BroadcastRequest request, ShardId shardId, ActionListener shardActionListener) { capturedShardRequests.add(new Tuple<>(shardId, shardActionListener)); } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java index fb7c59f353c66..7a63e9cb8af23 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java @@ -27,12 +27,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.TestShardRouting; -import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.cluster.routing.*; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.index.shard.ShardId; @@ -80,10 +75,11 @@ public static ClusterState state(String index, boolean primaryLocal, ShardRoutin } discoBuilder.localNodeId(newNode(0).id()); discoBuilder.masterNodeId(newNode(1).id()); // we need a non-local master to test shard failures + final int primaryTerm = randomInt(200); IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder() .put(SETTING_VERSION_CREATED, Version.CURRENT) .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicas) - .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build(); + .put(SETTING_CREATION_DATE, System.currentTimeMillis())).primaryTerm(0, primaryTerm).build(); RoutingTable.Builder routing = new RoutingTable.Builder(); routing.addAsNew(indexMetaData); @@ -105,7 +101,6 @@ public static ClusterState state(String index, boolean primaryLocal, ShardRoutin } else { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null); } - final int primaryTerm = randomInt(200); indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, 0, primaryNode, relocatingNode, null, primaryTerm, true, primaryState, 0, unassignedInfo)); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 79f3853b0a8d0..47b9beac5e6a9 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -20,7 +20,7 @@ import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.support.ActionFilter; @@ -74,11 +74,7 @@ import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithStartedPrimary; -import static org.hamcrest.Matchers.arrayWithSize; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; public class TransportReplicationActionTests extends ESTestCase { @@ -344,6 +340,8 @@ protected void runReplicateTest(IndexShardRoutingTable shardRoutingTable, int as final ShardIterator shardIt = shardRoutingTable.shardsIt(); final ShardId shardId = shardIt.shardId(); final Request request = new Request(); + final long primaryTerm = randomInt(200); + request.primaryTerm(primaryTerm); PlainActionFuture listener = new PlainActionFuture<>(); logger.debug("expecting [{}] assigned replicas, [{}] total shards. using state: \n{}", assignedReplicas, totalShards, clusterService.state().prettyPrint()); @@ -401,7 +399,7 @@ action.new ReplicationPhase(shardIt, request, } assertThat(listener.isDone(), equalTo(true)); Response response = listener.get(); - final ActionWriteResponse.ShardInfo shardInfo = response.getShardInfo(); + final ReplicationResponse.ShardInfo shardInfo = response.getShardInfo(); assertThat(shardInfo.getFailed(), equalTo(criticalFailures)); assertThat(shardInfo.getFailures(), arrayWithSize(criticalFailures)); assertThat(shardInfo.getSuccessful(), equalTo(successful)); @@ -413,6 +411,25 @@ action.new ReplicationPhase(shardIt, request, } // all replicas have responded so the counter should be decreased again assertIndexShardCounter(1); + + // assert that nothing in the replica logic changes the primary term of the operation + assertThat(request.primaryTerm(), equalTo(primaryTerm)); + } + + public void testSeqNoIsSetOnPrimary() { + final String index = "test"; + final ShardId shardId = new ShardId(index, 0); + // we use one replica to check the primary term was set on the operation and sent to the replica + clusterService.setState(state(index, true, + ShardRoutingState.STARTED, randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.STARTED))); + logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + Request request = new Request(shardId); + PlainActionFuture listener = new PlainActionFuture<>(); + TransportReplicationAction.PrimaryPhase primaryPhase = action.new PrimaryPhase(request, listener); + primaryPhase.doRun(); + CapturingTransport.CapturedRequest[] requestsToReplicas = transport.capturedRequests(); + assertThat(requestsToReplicas, arrayWithSize(1)); + assertThat(((Request) requestsToReplicas[0].request).primaryTerm(), equalTo(clusterService.state().getMetaData().index(index).primaryTerm(0))); } public void testCounterOnPrimary() throws InterruptedException, ExecutionException, IOException { @@ -586,7 +603,7 @@ public void readFrom(StreamInput in) throws IOException { } } - static class Response extends ActionWriteResponse { + static class Response extends ReplicationResponse { } class Action extends TransportReplicationAction { @@ -632,7 +649,7 @@ protected boolean resolveIndex() { } @Override - protected Releasable getIndexShardOperationsCounter(ShardId shardId) { + protected Releasable getIndexShardOperationsCounter(ShardId shardId, long opPrimaryTerm) { return getOrCreateIndexShardOperationsCounter(); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java index c547994b3fdc8..01a35d277fa66 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java @@ -115,12 +115,18 @@ public void testIsSourceTargetRelocation() { ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "node1", randomInt(200), randomBoolean(), ShardRoutingState.INITIALIZING, 1); ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "node1", randomInt(200), randomBoolean(), ShardRoutingState.INITIALIZING, 1); ShardRouting startedShard0 = new ShardRouting(initializingShard0); + assertFalse(startedShard0.isRelocationTarget()); startedShard0.moveToStarted(); + assertFalse(startedShard0.isRelocationTarget()); ShardRouting startedShard1 = new ShardRouting(initializingShard1); + assertFalse(startedShard1.isRelocationTarget()); startedShard1.moveToStarted(); + assertFalse(startedShard1.isRelocationTarget()); ShardRouting sourceShard0a = new ShardRouting(startedShard0); sourceShard0a.relocate("node2", -1); + assertFalse(sourceShard0a.isRelocationTarget()); ShardRouting targetShard0a = sourceShard0a.buildTargetRelocatingShard(); + assertTrue(targetShard0a.isRelocationTarget()); ShardRouting sourceShard0b = new ShardRouting(startedShard0); sourceShard0b.relocate("node2", -1); ShardRouting sourceShard1 = new ShardRouting(startedShard1); diff --git a/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java b/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java index d4907d821285a..64190091a02fa 100644 --- a/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java +++ b/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java @@ -19,7 +19,7 @@ package org.elasticsearch.document; -import org.elasticsearch.action.ActionWriteResponse; +import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -34,10 +34,7 @@ import org.elasticsearch.test.ESIntegTestCase; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; /** */ @@ -117,11 +114,11 @@ private void prepareIndex(int numberOfPrimaryShards, boolean routingRequired) th } } - private void assertShardInfo(ActionWriteResponse response) { + private void assertShardInfo(ReplicationResponse response) { assertShardInfo(response, numCopies, numNodes); } - private void assertShardInfo(ActionWriteResponse response, int expectedTotal, int expectedSuccessful) { + private void assertShardInfo(ReplicationResponse response, int expectedTotal, int expectedSuccessful) { assertThat(response.getShardInfo().getTotal(), greaterThanOrEqualTo(expectedTotal)); assertThat(response.getShardInfo().getSuccessful(), greaterThanOrEqualTo(expectedSuccessful)); } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 2937461cb8c40..b30a38e228846 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -68,6 +68,7 @@ import org.elasticsearch.index.mapper.internal.SourceFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.object.RootObjectMapper; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.*; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.DirectoryService; @@ -190,9 +191,10 @@ private Document testDocument() { private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, Document document, BytesReference source, Mapping mappingUpdate) { Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); + Field seqNoField = new NumericDocValuesField("_seq_no", 0); document.add(uidField); document.add(versionField); - return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); + return new ParsedDocument(uidField, versionField, seqNoField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); } protected Store createStore() throws IOException { @@ -246,7 +248,8 @@ public EngineConfig config(IndexSettings indexSettings, Store store, Path transl public void onFailedEngine(String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test } - }, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + }, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), + translogConfig, TimeValue.timeValueMinutes(5)); try { config.setCreate(Lucene.indexExists(store.directory()) == false); } catch (IOException e) { @@ -784,7 +787,9 @@ public void testRenewSyncFlush() throws Exception { engine.flush(); final boolean forceMergeFlushes = randomBoolean(); if (forceMergeFlushes) { - engine.index(new Engine.Index(newUid("3"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos())); + engine.index(new Engine.Index(newUid("3"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, + VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, + System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos())); } else { engine.index(new Engine.Index(newUid("3"), doc)); } @@ -824,7 +829,7 @@ public void testRenewSyncFlush() throws Exception { } } - public void testSycnedFlushSurvivesEngineRestart() throws IOException { + public void testSyncedFlushSurvivesEngineRestart() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); engine.index(new Engine.Index(newUid("1"), doc)); @@ -843,7 +848,7 @@ public void testSycnedFlushSurvivesEngineRestart() throws IOException { assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); } - public void testSycnedFlushVanishesOnReplay() throws IOException { + public void testSyncedFlushVanishesOnReplay() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); engine.index(new Engine.Index(newUid("1"), doc)); @@ -872,8 +877,7 @@ public void testVersioningNewCreate() { Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED); engine.index(create); assertThat(create.version(), equalTo(1l)); - - create = new Engine.Index(newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + create = new Engine.Index(newUid("1"), doc, create.seqNo(), create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(create); assertThat(create.version(), equalTo(1l)); } @@ -884,18 +888,18 @@ public void testVersioningNewIndex() { engine.index(index); assertThat(index.version(), equalTo(1l)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(1l)); } public void testExternalVersioningNewIndex() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(12l)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(12l)); } @@ -910,7 +914,7 @@ public void testVersioningIndexConflict() { engine.index(index); assertThat(index.version(), equalTo(2l)); - index = new Engine.Index(newUid("1"), doc, 1l, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0); try { engine.index(index); fail(); @@ -919,7 +923,7 @@ public void testVersioningIndexConflict() { } // future versions should not work as well - index = new Engine.Index(newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -930,15 +934,15 @@ public void testVersioningIndexConflict() { public void testExternalVersioningIndexConflict() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(12l)); - index = new Engine.Index(newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 14, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(14l)); - index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 13, VersionType.EXTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -959,7 +963,7 @@ public void testVersioningIndexConflictWithFlush() { engine.flush(); - index = new Engine.Index(newUid("1"), doc, 1l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -968,7 +972,7 @@ public void testVersioningIndexConflictWithFlush() { } // future versions should not work as well - index = new Engine.Index(newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -979,17 +983,17 @@ public void testVersioningIndexConflictWithFlush() { public void testExternalVersioningIndexConflictWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(12l)); - index = new Engine.Index(newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 14, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(14l)); engine.flush(); - index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 13, VersionType.EXTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -1103,7 +1107,7 @@ public void testVersioningDeleteConflict() { engine.index(index); assertThat(index.version(), equalTo(2l)); - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1112,7 +1116,7 @@ public void testVersioningDeleteConflict() { } // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1121,12 +1125,12 @@ public void testVersioningDeleteConflict() { } // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0, false); engine.delete(delete); assertThat(delete.version(), equalTo(3l)); // now check if we can index to a delete doc with version - index = new Engine.Index(newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -1134,13 +1138,9 @@ public void testVersioningDeleteConflict() { // all is well } - // we shouldn't be able to create as well - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(create); - } catch (VersionConflictEngineException e) { - // all is well - } + // we should be able to create as well + Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + engine.index(create); } public void testVersioningDeleteConflictWithFlush() { @@ -1155,7 +1155,7 @@ public void testVersioningDeleteConflictWithFlush() { engine.flush(); - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1164,7 +1164,7 @@ public void testVersioningDeleteConflictWithFlush() { } // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1175,14 +1175,14 @@ public void testVersioningDeleteConflictWithFlush() { engine.flush(); // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0, false); engine.delete(delete); assertThat(delete.version(), equalTo(3l)); engine.flush(); // now check if we can index to a delete doc with version - index = new Engine.Index(newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -1190,22 +1190,18 @@ public void testVersioningDeleteConflictWithFlush() { // all is well } - // we shouldn't be able to create as well - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(create); - } catch (VersionConflictEngineException e) { - // all is well - } + // we should be able to create + Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + engine.index(create); } public void testVersioningCreateExistsException() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); engine.index(create); assertThat(create.version(), equalTo(1l)); - create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(create); fail(); @@ -1216,13 +1212,13 @@ public void testVersioningCreateExistsException() { public void testVersioningCreateExistsExceptionWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + Engine.Index create = new Engine.Index(newUid("1"), doc, -1, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); engine.index(create); assertThat(create.version(), equalTo(1l)); engine.flush(); - create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + create = new Engine.Index(newUid("1"), doc, -1, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(create); fail(); @@ -1242,12 +1238,12 @@ public void testVersioningReplicaConflict1() { assertThat(index.version(), equalTo(2l)); // apply the second index to the replica, should work fine - index = new Engine.Index(newUid("1"), doc, index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(2l)); // now, the old one should not work - index = new Engine.Index(newUid("1"), doc, 1l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 1l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); try { replicaEngine.index(index); fail(); @@ -1257,7 +1253,7 @@ public void testVersioningReplicaConflict1() { // second version on replica should fail as well try { - index = new Engine.Index(newUid("1"), doc, 2l + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 2l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(2l)); @@ -1273,7 +1269,7 @@ public void testVersioningReplicaConflict2() { assertThat(index.version(), equalTo(1l)); // apply the first index to the replica, should work fine - index = new Engine.Index(newUid("1"), doc, 1l + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 1l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(1l)); @@ -1289,14 +1285,14 @@ public void testVersioningReplicaConflict2() { assertThat(delete.version(), equalTo(3l)); // apply the delete on the replica (skipping the second index) - delete = new Engine.Delete("test", "1", newUid("1"), 3l + delete = new Engine.Delete("test", "1", newUid("1"), delete.seqNo(), 3l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); replicaEngine.delete(delete); assertThat(delete.version(), equalTo(3l)); // second time delete with same version should fail try { - delete = new Engine.Delete("test", "1", newUid("1"), 3l + delete = new Engine.Delete("test", "1", newUid("1"), delete.seqNo(), 3l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); replicaEngine.delete(delete); fail("excepted VersionConflictEngineException to be thrown"); @@ -1306,7 +1302,7 @@ public void testVersioningReplicaConflict2() { // now do the second index on the replica, it should fail try { - index = new Engine.Index(newUid("1"), doc, 2l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 2l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); fail("excepted VersionConflictEngineException to be thrown"); } catch (VersionConflictEngineException e) { @@ -1447,10 +1443,10 @@ public void testEnableGcDeletes() throws Exception { document.add(new TextField("value", "test1", Field.Store.YES)); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_2, null); - engine.index(new Engine.Index(newUid("1"), doc, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); // Delete document we just added: - engine.delete(new Engine.Delete("test", "1", newUid("1"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); // Get should not find the document Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid("1"))); @@ -1464,7 +1460,7 @@ public void testEnableGcDeletes() throws Exception { } // Delete non-existent document - engine.delete(new Engine.Delete("test", "2", newUid("2"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); // Get should not find the document (we never indexed uid=2): getResult = engine.get(new Engine.Get(true, newUid("2"))); @@ -1472,7 +1468,7 @@ public void testEnableGcDeletes() throws Exception { // Try to index uid=1 with a too-old version, should fail: try { - engine.index(new Engine.Index(newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); fail("did not hit expected exception"); } catch (VersionConflictEngineException vcee) { // expected @@ -1484,7 +1480,7 @@ public void testEnableGcDeletes() throws Exception { // Try to index uid=2 with a too-old version, should fail: try { - engine.index(new Engine.Index(newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid("2"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); fail("did not hit expected exception"); } catch (VersionConflictEngineException vcee) { // expected @@ -1569,7 +1565,7 @@ public void testDeletesAloneCanTriggerRefresh() throws Exception { for (int i = 0; i < 100; i++) { String id = Integer.toString(i); ParsedDocument doc = testParsedDocument(id, id, "test", null, -1, -1, testDocument(), B_1, null); - engine.index(new Engine.Index(newUid(id), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid(id), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); } // Force merge so we know all merges are done before we start deleting: @@ -1580,20 +1576,17 @@ public void testDeletesAloneCanTriggerRefresh() throws Exception { s.close(); for (int i = 0; i < 100; i++) { String id = Integer.toString(i); - engine.delete(new Engine.Delete("test", id, newUid(id), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", id, newUid(id), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); } // We must assertBusy because refresh due to version map being full is done in background (REFRESH) thread pool: - assertBusy(new Runnable() { - @Override - public void run() { - Searcher s2 = engine.acquireSearcher("test"); - long version2 = ((DirectoryReader) s2.reader()).getVersion(); - s2.close(); - - // 100 buffered deletes will easily exceed 25% of our 1 KB indexing buffer so it should have forced a refresh: - assertThat(version2, greaterThan(version1)); - } + assertBusy(() -> { + Searcher s2 = engine.acquireSearcher("test"); + long version2 = ((DirectoryReader) s2.reader()).getVersion(); + s2.close(); + + // 100 buffered deletes will easily exceed 25% of our 1 KB indexing buffer so it should have forced a refresh: + assertThat(version2, greaterThan(version1)); }); } } @@ -1622,8 +1615,8 @@ public void testMissingTranslog() throws IOException { public void testTranslogReplayWithFailure() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, SequenceNumbersService.UNASSIGNED_SEQ_NO, -1, testDocument(), new BytesArray("{}"), null); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1675,7 +1668,8 @@ public void testSkipTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1770,7 +1764,7 @@ public void testUpgradeOldIndex() throws IOException { final int numExtraDocs = randomIntBetween(1, 10); for (int i = 0; i < numExtraDocs; i++) { ParsedDocument doc = testParsedDocument("extra" + Integer.toString(i), "extra" + Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1799,7 +1793,8 @@ public void testTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1848,7 +1843,7 @@ public void testTranslogReplay() throws IOException { int randomId = randomIntBetween(numDocs + 1, numDocs + 10); String uuidValue = "test#" + Integer.toString(randomId); ParsedDocument doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); if (flush) { @@ -1856,7 +1851,7 @@ public void testTranslogReplay() throws IOException { } doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); + Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); engine.index(idxRequest); engine.refresh("test"); assertThat(idxRequest.version(), equalTo(2l)); @@ -1922,7 +1917,8 @@ public void testRecoverFromForeignTranslog() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } diff --git a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 3fe7a540bf8af..78fe08cd79b64 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -161,9 +161,10 @@ private ParseContext.Document testDocument() { private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, ParseContext.Document document, BytesReference source, Mapping mappingsUpdate) { Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); + Field seqNoField = new NumericDocValuesField("_seq_no", 0); document.add(uidField); document.add(versionField); - return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingsUpdate); + return new ParsedDocument(uidField, versionField, seqNoField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingsUpdate); } protected Store createStore(Path p) throws IOException { @@ -220,7 +221,9 @@ public EngineConfig config(IndexSettings indexSettings, Store store, Path transl @Override public void onFailedEngine(String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test - }}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + } + }, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5) + ); try { config.setCreate(Lucene.indexExists(store.directory()) == false); } catch (IOException e) { diff --git a/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java b/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java index ccbef6837c98a..10dd82a7ae84a 100644 --- a/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java +++ b/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java @@ -36,7 +36,8 @@ public class IndexingSlowLogTests extends ESTestCase { public void testSlowLogParsedDocumentPrinterSourceToLog() throws IOException { BytesReference source = JsonXContent.contentBuilder().startObject().field("foo", "bar").endObject().bytes(); - ParsedDocument pd = new ParsedDocument(new StringField("uid", "test:id", Store.YES), new IntField("version", 1, Store.YES), "id", + ParsedDocument pd = new ParsedDocument(new StringField("uid", "test:id", Store.YES), new IntField("version", 1, Store.YES), + new IntField("seqNo", 1, Store.YES), "id", "test", null, 0, -1, null, source, null); // Turning off document logging doesn't log source[] diff --git a/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java index 4753f903cbf0c..e5e42f4d0a3c3 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java @@ -88,8 +88,8 @@ public void testInjectIntoDocDuringParsing() throws Exception { .endObject() .endObject() .bytes()); - - assertFieldNames(set("a", "b", "b.c", "_uid", "_type", "_version", "_source", "_all"), doc); + + assertFieldNames(set("a", "b", "b.c", "_uid", "_type", "_version", "_seq_no", "_source", "_all"), doc); } public void testExplicitEnabled() throws Exception { @@ -106,7 +106,7 @@ public void testExplicitEnabled() throws Exception { .endObject() .bytes()); - assertFieldNames(set("field", "_uid", "_type", "_version", "_source", "_all"), doc); + assertFieldNames(set("field", "_uid", "_type", "_version", "_seq_no", "_source", "_all"), doc); } public void testDisabled() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 6c4f0d59cae87..73b758b90a05d 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -34,6 +34,8 @@ import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterInfoService; @@ -41,6 +43,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.SnapshotId; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.*; @@ -101,6 +104,7 @@ import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; /** @@ -312,7 +316,7 @@ public void testDeleteIndexDecreasesCounter() throws InterruptedException, Execu client().admin().indices().prepareDelete("test").get(); assertThat(indexShard.getOperationsCount(), equalTo(0)); try { - indexShard.incrementOperationCounter(); + indexShard.incrementOperationCounter(indexShard.routingEntry().primaryTerm()); fail("we should not be able to increment anymore"); } catch (IndexShardClosedException e) { // expected @@ -325,11 +329,26 @@ public void testIndexShardCounter() throws InterruptedException, ExecutionExcept IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService indexService = indicesService.indexServiceSafe("test"); IndexShard indexShard = indexService.getShardOrNull(0); + final long primaryTerm = indexShard.shardRouting.primaryTerm(); assertEquals(0, indexShard.getOperationsCount()); - indexShard.incrementOperationCounter(); + indexShard.incrementOperationCounter(primaryTerm); assertEquals(1, indexShard.getOperationsCount()); - indexShard.incrementOperationCounter(); + indexShard.incrementOperationCounter(primaryTerm); assertEquals(2, indexShard.getOperationsCount()); + + try { + indexShard.incrementOperationCounter(primaryTerm - 1); + fail("you can not increment the operation counter with an older primary term"); + } catch (IllegalIndexShardStateException e) { + assertThat(e.getMessage(), containsString("operation term")); + assertThat(e.getMessage(), containsString("too old")); + } + + // but you can increment with a newer one.. + indexShard.incrementOperationCounter(primaryTerm + 1 + randomInt(20)); + + + indexShard.decrementOperationCounter(); indexShard.decrementOperationCounter(); indexShard.decrementOperationCounter(); assertEquals(0, indexShard.getOperationsCount()); @@ -580,9 +599,10 @@ public void testShardStats() throws IOException { private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); + Field seqNoField = new NumericDocValuesField("_seq_no", 0); document.add(uidField); document.add(versionField); - return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); + return new ParsedDocument(uidField, versionField, seqNoField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); } public void testPreIndex() throws IOException { @@ -771,7 +791,7 @@ public void testRecoverFromStore() throws IOException { assertHitCount(response, 1); } - public void testFailIfIndexNotPresentInRecoverFromStore() throws IOException { + public void testFailIfIndexNotPresentInRecoverFromStore() throws Throwable { createIndex("test"); ensureGreen(); IndicesService indicesService = getInstanceFromNode(IndicesService.class); @@ -821,7 +841,12 @@ public void testFailIfIndexNotPresentInRecoverFromStore() throws IOException { newShard.updateRoutingEntry(routing, true); SearchResponse response = client().prepareSearch().get(); assertHitCount(response, 0); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(true).get(); + // we can't issue this request through a client because of the inconsistencies we created with the cluster state + // doing it directly instead + IndexRequest request = client().prepareIndex("test", "test", "0").setSource("{}").request(); + request.process(MetaData.builder().put(test.getMetaData(), false).build(), null, false, "test"); + TransportIndexAction.executeIndexRequestOnPrimary(request, newShard, null); + newShard.refresh("test"); assertHitCount(client().prepareSearch().get(), 1); } diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index e76e09a6cfb88..db21fbc384d6b 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -302,7 +302,7 @@ public void testStats() throws IOException { assertThat(stats.estimatedNumberOfOperations(), equalTo(0l)); assertThat(stats.getTranslogSizeInBytes(), equalTo(firstOperationPosition)); assertEquals(6, total.estimatedNumberOfOperations()); - assertEquals(431, total.getTranslogSizeInBytes()); + assertEquals(437, total.getTranslogSizeInBytes()); BytesStreamOutput out = new BytesStreamOutput(); total.writeTo(out); @@ -310,10 +310,10 @@ public void testStats() throws IOException { copy.readFrom(StreamInput.wrap(out.bytes())); assertEquals(6, copy.estimatedNumberOfOperations()); - assertEquals(431, copy.getTranslogSizeInBytes()); + assertEquals(437, copy.getTranslogSizeInBytes()); assertEquals("\"translog\"{\n" + " \"operations\" : 6,\n" + - " \"size_in_bytes\" : 431\n" + + " \"size_in_bytes\" : 437\n" + "}", copy.toString().trim()); try { @@ -475,7 +475,7 @@ public void run() { break; case DELETE: op = new Translog.Delete(new Term("_uid", threadId + "_" + opCount), - 1 + randomInt(100000), + opCount, 1 + randomInt(100000), randomFrom(VersionType.values())); break; default: @@ -1127,7 +1127,7 @@ public void testRecoveryUncommittedCorryptedCheckpoint() throws IOException { try (Translog translog = new Translog(config)) { fail("corrupted"); } catch (IllegalStateException ex) { - assertEquals(ex.getMessage(), "Checkpoint file translog-2.ckp already exists but has corrupted content expected: Checkpoint{offset=2683, numOps=55, translogFileGeneration= 2} but got: Checkpoint{offset=0, numOps=0, translogFileGeneration= 0}"); + assertEquals(ex.getMessage(), "Checkpoint file translog-2.ckp already exists but has corrupted content expected: Checkpoint{offset=2738, numOps=55, translogFileGeneration= 2} but got: Checkpoint{offset=0, numOps=0, translogFileGeneration= 0}"); } Checkpoint.write(config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation)), read, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING); try (Translog translog = new Translog(config)) { diff --git a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java index 1a4bf8fd3f764..f1322d16a782a 100644 --- a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java @@ -110,7 +110,7 @@ public void testSyncFailsIfOperationIsInFlight() throws InterruptedException { SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); final ShardId shardId = shard.shardId(); - shard.incrementOperationCounter(); + shard.incrementOperationCounter(shard.routingEntry().primaryTerm()); try { SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener<>(); flushService.attemptSyncedFlush(shardId, listener); diff --git a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java index a5b7da7796f86..fa41c9fef2d46 100644 --- a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java @@ -253,7 +253,7 @@ public void testRequiredRoutingWithPathMapping() throws Exception { assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); } } - + public void testRequiredRoutingWithPathMappingBulk() throws Exception { client().admin().indices().prepareCreate("test") .addAlias(new Alias("alias")) diff --git a/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java b/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java index 2b708341c7fd9..2a57af862b088 100644 --- a/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java +++ b/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.internal.InternalSearchHit; @@ -225,7 +226,7 @@ public void testOnBulkResponse() { } else { deleted++; } - items[i] = new BulkItemResponse(i, "delete", new DeleteResponse("test", "type", String.valueOf(i), 1, delete)); + items[i] = new BulkItemResponse(i, "delete", new DeleteResponse(new ShardId("test", 0), "type", String.valueOf(i), i, 1, delete)); } else { items[i] = new BulkItemResponse(i, "delete", new BulkItemResponse.Failure("test", "type", String.valueOf(i), new Throwable("item failed"))); failed++; @@ -281,7 +282,7 @@ public void testOnBulkResponseMultipleIndices() { deleted[0] = deleted[0] + 1; deleted[index] = deleted[index] + 1; } - items[i] = new BulkItemResponse(i, "delete", new DeleteResponse("test-" + index, "type", String.valueOf(i), 1, delete)); + items[i] = new BulkItemResponse(i, "delete", new DeleteResponse(new ShardId("test-" + index, 0), "type", String.valueOf(i), i, 1, delete)); } else { items[i] = new BulkItemResponse(i, "delete", new BulkItemResponse.Failure("test-" + index, "type", String.valueOf(i), new Throwable("item failed"))); failed[0] = failed[0] + 1; @@ -408,7 +409,7 @@ private void assertNoFailures(TestActionListener listener) { private void assertSearchContextsClosed() { NodesStatsResponse nodesStats = client().admin().cluster().prepareNodesStats().setIndices(true).get(); - for (NodeStats nodeStat : nodesStats.getNodes()){ + for (NodeStats nodeStat : nodesStats.getNodes()) { assertThat(nodeStat.getIndices().getSearch().getOpenContexts(), equalTo(0L)); } }