Add Sequence Numbers and enforce Primary Terms

Adds a counter to each write operation on a shard. This sequence numbers is indexed into lucene using doc values, for now (we will probably require indexing to support range searchers in the future).

On top of this, primary term semantics are enforced and shards will refuse write operation coming from an older primary.

Other notes:
- The add SequenceServiceNumber is just a skeleton and will be replaced with much heavier one, once we have all the building blocks (i.e., checkpoints).
- I completely ignored recovery - for this we will need checkpoints as well.
- A new based class is introduced for all single doc write operations. This is handy to unify common logic (like toXContent).
- For now, we don't use seq# as versioning. We could in the future.

Relates to #10708
Closes #14651
This commit is contained in:
Boaz Leskes 2015-11-03 13:43:48 +00:00
parent 1e67ef85c6
commit 5fb0f9a88f
52 changed files with 1094 additions and 706 deletions

View File

@ -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;
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.action;
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.io.stream.Streamable;
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

View File

@ -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 @@ import java.util.List;
/**
* Flush Action.
*/
public class TransportFlushAction extends TransportBroadcastReplicationAction<FlushRequest, FlushResponse, ShardFlushRequest, ActionWriteResponse> {
public class TransportFlushAction extends TransportBroadcastReplicationAction<FlushRequest, FlushResponse, ShardFlushRequest, ReplicationResponse> {
@Inject
public TransportFlushAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
@ -47,8 +47,8 @@ public class TransportFlushAction extends TransportBroadcastReplicationAction<Fl
}
@Override
protected ActionWriteResponse newShardResponse() {
return new ActionWriteResponse();
protected ReplicationResponse newShardResponse() {
return new ReplicationResponse();
}
@Override

View File

@ -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 @@ import org.elasticsearch.transport.TransportService;
/**
*
*/
public class TransportShardFlushAction extends TransportReplicationAction<ShardFlushRequest, ShardFlushRequest, ActionWriteResponse> {
public class TransportShardFlushAction extends TransportReplicationAction<ShardFlushRequest, ShardFlushRequest, ReplicationResponse> {
public static final String NAME = FlushAction.NAME + "[s]";
@ -56,16 +56,16 @@ public class TransportShardFlushAction extends TransportReplicationAction<ShardF
}
@Override
protected ActionWriteResponse newResponseInstance() {
return new ActionWriteResponse();
protected ReplicationResponse newResponseInstance() {
return new ReplicationResponse();
}
@Override
protected Tuple<ActionWriteResponse, ShardFlushRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable {
protected Tuple<ReplicationResponse, ShardFlushRequest> 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

View File

@ -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 @@ import java.util.List;
/**
* Refresh action.
*/
public class TransportRefreshAction extends TransportBroadcastReplicationAction<RefreshRequest, RefreshResponse, ReplicationRequest, ActionWriteResponse> {
public class TransportRefreshAction extends TransportBroadcastReplicationAction<RefreshRequest, RefreshResponse, ReplicationRequest, ReplicationResponse> {
@Inject
public TransportRefreshAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
@ -48,8 +48,8 @@ public class TransportRefreshAction extends TransportBroadcastReplicationAction<
}
@Override
protected ActionWriteResponse newShardResponse() {
return new ActionWriteResponse();
protected ReplicationResponse newShardResponse() {
return new ReplicationResponse();
}
@Override

View File

@ -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 @@ import org.elasticsearch.transport.TransportService;
/**
*
*/
public class TransportShardRefreshAction extends TransportReplicationAction<ReplicationRequest, ReplicationRequest, ActionWriteResponse> {
public class TransportShardRefreshAction extends TransportReplicationAction<ReplicationRequest, ReplicationRequest, ReplicationResponse> {
public static final String NAME = RefreshAction.NAME + "[s]";
@ -57,16 +57,16 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Repl
}
@Override
protected ActionWriteResponse newResponseInstance() {
return new ActionWriteResponse();
protected ReplicationResponse newResponseInstance() {
return new ReplicationResponse();
}
@Override
protected Tuple<ActionWriteResponse, ReplicationRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable {
protected Tuple<ReplicationResponse, ReplicationRequest> 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

View File

@ -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 class BulkItemResponse implements Streamable {
private String opType;
private ActionWriteResponse response;
private DocWriteResponse response;
private Failure failure;
@ -107,7 +107,7 @@ public class BulkItemResponse implements Streamable {
}
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 class BulkItemResponse implements Streamable {
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 class BulkItemResponse implements Streamable {
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 class BulkItemResponse implements Streamable {
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 class BulkItemResponse implements Streamable {
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}). <tt>null</tt> in
* case of failure.
*/
public <T extends ActionWriteResponse> T getResponse() {
public <T extends DocWriteResponse> T getResponse() {
return (T) response;
}

View File

@ -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 @@ import java.io.IOException;
/**
*
*/
public class BulkShardResponse extends ActionWriteResponse {
public class BulkShardResponse extends ReplicationResponse {
private ShardId shardId;
private BulkItemResponse[] responses;

View File

@ -25,8 +25,10 @@ import org.elasticsearch.action.ActionRequest;
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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
try {
// add the response
final WriteResult<DeleteResponse> writeResult = shardDeleteOperation(request, deleteRequest, indexShard);
final WriteResult<DeleteResponse> 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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
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<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, shardRequest.request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes));
@ -228,7 +236,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
WriteResult<DeleteResponse> 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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
}
}
private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, ClusterState clusterState,
private WriteResult<IndexResponse> 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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
indexRequest.process(clusterState.metaData(), mappingMd, allowIdGeneration, request.index());
}
return executeIndexRequestOnPrimary(request, indexRequest, indexShard);
}
private WriteResult<DeleteResponse> 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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
} 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 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
}
}
processAfter(request.refresh(), indexShard, location);
processAfterWrite(request.refresh(), indexShard, location);
}
private void applyVersion(BulkItemRequest item, long version, VersionType versionType) {

View File

@ -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 @@ import java.io.IOException;
* @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 <tt>true</tt> if a doc was found to delete.
*/
@ -89,20 +57,23 @@ public class DeleteResponse extends ActionWriteResponse {
@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;
}
}

View File

@ -130,26 +130,34 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
protected Tuple<DeleteResponse, DeleteRequest> 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<DeleteResponse> result = executeDeleteRequestOnPrimary(request, indexShard);
processAfterWrite(request.refresh(), indexShard, result.location);
return new Tuple<>(result.response, shardRequest.request);
}
public static WriteResult<DeleteResponse> 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

View File

@ -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 @@ import java.io.IOException;
* @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 class IndexResponse extends ActionWriteResponse {
@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 class IndexResponse extends ActionWriteResponse {
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;
}
}

View File

@ -166,11 +166,11 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
IndexService indexService = indicesService.indexServiceSafe(shardRequest.shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardRequest.shardId.id());
final WriteResult<IndexResponse> result = executeIndexRequestOnPrimary(null, request, indexShard);
final WriteResult<IndexResponse> 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 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
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<IndexResponse> 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());
}
}

View File

@ -44,6 +44,9 @@ public class ReplicationRequest<T extends ReplicationRequest> extends ActionRequ
ShardId internalShardId;
long seqNo;
long primaryTerm;
protected TimeValue timeout = DEFAULT_TIMEOUT;
protected String index;
@ -76,6 +79,9 @@ public class ReplicationRequest<T extends ReplicationRequest> extends ActionRequ
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 class ReplicationRequest<T extends ReplicationRequest> extends ActionRequ
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 class ReplicationRequest<T extends ReplicationRequest> extends ActionRequ
consistencyLevel = WriteConsistencyLevel.fromId(in.readByte());
timeout = TimeValue.readTimeValue(in);
index = in.readString();
seqNo = in.readVLong();
primaryTerm = in.readVLong();
}
@Override
@ -175,6 +206,8 @@ public class ReplicationRequest<T extends ReplicationRequest> extends ActionRequ
out.writeByte(consistencyLevel.id());
timeout.writeTo(out);
out.writeString(index);
out.writeVLong(seqNo);
out.writeVLong(primaryTerm);
}
public T setShardId(ShardId shardId) {

View File

@ -22,9 +22,8 @@ package org.elasticsearch.action.support.replication;
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 @@ import java.util.function.Supplier;
* 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<Request extends BroadcastRequest, Response extends BroadcastResponse, ShardRequest extends ReplicationRequest, ShardResponse extends ActionWriteResponse> extends HandledTransportAction<Request, Response> {
public abstract class TransportBroadcastReplicationAction<Request extends BroadcastRequest, Response extends BroadcastResponse, ShardRequest extends ReplicationRequest, ShardResponse extends ReplicationResponse> extends HandledTransportAction<Request, Response> {
private final TransportReplicationAction replicatedBroadcastShardAction;
private final ClusterService clusterService;
@ -91,15 +90,15 @@ public abstract class TransportBroadcastReplicationAction<Request extends Broadc
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 @@ public abstract class TransportBroadcastReplicationAction<Request extends Broadc
int totalNumCopies = 0;
List<ShardOperationFailedException> 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 @@ public abstract class TransportBroadcastReplicationAction<Request extends Broadc
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())));
}
}

View File

@ -22,13 +22,9 @@ package org.elasticsearch.action.support.replication;
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.unit.TimeValue;
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 @@ import java.util.function.Supplier;
/**
*/
public abstract class TransportReplicationAction<Request extends ReplicationRequest, ReplicaRequest extends ReplicationRequest, Response extends ActionWriteResponse> extends TransportAction<Request, Response> {
public abstract class TransportReplicationAction<Request extends ReplicationRequest, ReplicaRequest extends ReplicationRequest, Response extends ReplicationResponse> extends TransportAction<Request, Response> {
public static final String SHARD_FAILURE_TIMEOUT = "action.support.replication.shard.failure_timeout";
@ -195,7 +188,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
return false;
}
protected static class WriteResult<T extends ActionWriteResponse> {
protected static class WriteResult<T extends ReplicationResponse> {
public final T response;
public final Translog.Location location;
@ -206,10 +199,10 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
@SuppressWarnings("unchecked")
public <T extends ActionWriteResponse> T response() {
public <T extends ReplicationResponse> 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 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
@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 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
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<Response, ReplicaRequest> 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 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
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 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
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<String, Throwable> 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 abstract class TransportReplicationAction<Request extends ReplicationRequ
}
}
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 abstract class TransportReplicationAction<Request extends ReplicationRequ
}
}
/** 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<IndexResponse> 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");

View File

@ -175,7 +175,8 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
indexAction.execute(upsertRequest, new ActionListener<IndexResponse>() {
@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<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true);
update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes));
@ -210,7 +211,8 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
indexAction.execute(indexRequest, new ActionListener<IndexResponse>() {
@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 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
deleteAction.execute(deleteRequest, new ActionListener<DeleteResponse>() {
@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 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
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());

View File

@ -83,9 +83,10 @@ public class UpdateHelper extends AbstractComponent {
@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 @@ public class UpdateHelper extends AbstractComponent {
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 @@ public class UpdateHelper extends AbstractComponent {
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<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true);
@ -231,12 +232,12 @@ public class UpdateHelper extends AbstractComponent {
.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);
}
}

View File

@ -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 class UpdateResponse extends ActionWriteResponse {
* 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 class UpdateResponse extends ActionWriteResponse {
@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 class UpdateResponse extends ActionWriteResponse {
@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 class UpdateResponse extends ActionWriteResponse {
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;
}
}

View File

@ -191,6 +191,15 @@ public final class ShardRouting implements Streamable, ToXContent {
return state == ShardRoutingState.RELOCATING;
}
/**
* Returns <code>true</code> 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 <code>true</code> iff this shard is assigned to a node ie. not
* {@link ShardRoutingState#UNASSIGNED unassigned}. Otherwise <code>false</code>

View File

@ -45,6 +45,7 @@ import org.elasticsearch.index.mapper.ParseContext.Document;
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 abstract class Engine implements Closeable {
/**
* 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 abstract class Engine implements Closeable {
}
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 abstract class Engine implements Closeable {
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 abstract class Engine implements Closeable {
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 abstract class Engine implements Closeable {
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 abstract class Engine implements Closeable {
}
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 abstract class Engine implements Closeable {
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 abstract class Engine implements Closeable {
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 abstract class Engine implements Closeable {
* 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 abstract class Engine implements Closeable {
/**
* 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 <code>true</code> iff the searcher is build from a top-level reader.
* Otherwise the searcher might be build from a leaf reader to warm in isolation
*/

View File

@ -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 final class EngineConfig {
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 final class EngineConfig {
* Returns <code>true</code> 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 <code>true</code>
* <p>
* 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.
* </p>
*/
public boolean isEnableGcDeletes() {
@ -238,7 +240,7 @@ public final class EngineConfig {
/**
* Returns the {@link Codec} used in the engines {@link org.apache.lucene.index.IndexWriter}
* <p>
* Note: this settings is only read on startup.
* Note: this settings is only read on startup.
* </p>
*/
public Codec getCodec() {
@ -259,7 +261,6 @@ public final class EngineConfig {
*
* @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 final class EngineConfig {
/**
* 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 final class EngineConfig {
* 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;
}
}

View File

@ -50,6 +50,7 @@ import org.elasticsearch.index.indexing.ShardIndexingService;
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 class InternalEngine extends Engine {
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 class InternalEngine extends Engine {
} 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 @@ public class InternalEngine extends Engine {
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 class InternalEngine extends Engine {
} 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 @@ public class InternalEngine extends Engine {
}
}
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

View File

@ -35,18 +35,7 @@ import org.elasticsearch.common.util.concurrent.ReleasableLock;
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 class DocumentMapper implements ToXContent {
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)));

View File

@ -28,8 +28,6 @@ import org.elasticsearch.common.Strings;
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.io.IOException;
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 @@ class DocumentParser implements Closeable {
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);

View File

@ -323,6 +323,16 @@ public abstract class ParseContext {
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 abstract class ParseContext {
private String id;
private Field uid, version;
private Field uid, version, seqNo;
private StringBuilder stringBuilder = new StringBuilder();
@ -564,6 +574,17 @@ public abstract class ParseContext {
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 abstract class ParseContext {
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);
}

View File

@ -30,7 +30,7 @@ import java.util.List;
*/
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<Document> documents, BytesReference source, Mapping dynamicMappingsUpdate) {
public ParsedDocument(Field uid, Field version, Field seqNo, String id, String type, String routing, long timestamp, long ttl, List<Document> 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 class ParsedDocument {
return version;
}
public Field seqNo() {
return seqNo;
}
public String id() {
return this.id;
}

View File

@ -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<Builder, SeqNoFieldMapper> {
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<String, Object> 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<Field> 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
}
}

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -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.recovery.RecoveryStats;
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 class IndexShard extends AbstractIndexShardComponent {
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 class IndexShard extends AbstractIndexShardComponent {
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 @@ public class IndexShard extends AbstractIndexShardComponent {
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 class IndexShard extends AbstractIndexShardComponent {
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();
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, 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();
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 class IndexShard extends AbstractIndexShardComponent {
}
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 class IndexShard extends AbstractIndexShardComponent {
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 class IndexShard extends AbstractIndexShardComponent {
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 @@ public class IndexShard extends AbstractIndexShardComponent {
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 class IndexShard extends AbstractIndexShardComponent {
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 class IndexShard extends AbstractIndexShardComponent {
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 @@ public class IndexShard extends AbstractIndexShardComponent {
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 @@ public class IndexShard extends AbstractIndexShardComponent {
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 @@ public class IndexShard extends AbstractIndexShardComponent {
}
}
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 class IndexShard extends AbstractIndexShardComponent {
/**
* Simple struct encapsulating a shard failure
*
* @see IndexShard#addShardFailureCallback(Callback)
*/
public static final class ShardFailure {
@ -1604,7 +1649,7 @@ public class IndexShard extends AbstractIndexShardComponent {
};
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());
}
/**

View File

@ -128,7 +128,7 @@ public class TranslogRecoveryPerformer {
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 class TranslogRecoveryPerformer {
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 + "]");

View File

@ -129,11 +129,11 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
};
/**
* 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 class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
}
TranslogWriter createWriter(long fileGeneration) throws IOException {
TranslogWriter newFile;
try {
@ -508,6 +507,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
/**
* Ensures that the given location has be synced / written to the underlying storage.
*
* @return Returns <code>true</code> iff this call caused an actual sync operation otherwise <code>false</code>
*/
public boolean ensureSynced(Location location) throws IOException {
@ -749,13 +749,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
@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 class Translog extends AbstractIndexShardComponent implements IndexShardC
}
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
this.type = type;
this.id = id;
this.source = new BytesArray(source);
this.seqNo = 0;
this.version = 0;
}
@Override
@ -917,6 +929,10 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
return this.source;
}
public long seqNo() {
return seqNo;
}
public long version() {
return this.version;
}
@ -959,6 +975,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
out.writeLong(timestamp);
out.writeLong(ttl);
out.writeByte(versionType.getValue());
out.writeVLong(seqNo);
}
@Override
@ -1002,6 +1022,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
}
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
}
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
return this.uid;
}
public long seqNo() {
return seqNo;
}
public long version() {
return this.version;
}
@ -1089,7 +1116,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
}
@Override
public Source getSource(){
public Source getSource() {
throw new IllegalStateException("trying to read doc source from delete operation");
}
@ -1103,6 +1130,9 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
out.writeString(uid.text());
out.writeLong(version);
out.writeByte(versionType.getValue());
out.writeVLong(seqNo);
}
@Override
@ -1127,7 +1158,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
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 class Translog extends AbstractIndexShardComponent implements IndexShardC
@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 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
// to prevent this unfortunately.
in.mark(opSize);
in.skip(opSize-4);
in.skip(opSize - 4);
verifyChecksum(in);
in.reset();
}
@ -1250,7 +1282,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
out.writeByte(op.opType().id());
op.writeTo(out);
long checksum = out.getChecksum();
out.writeInt((int)checksum);
out.writeInt((int) checksum);
}
/**

View File

@ -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 class RestBulkAction extends BaseRestHandler {
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 @@ public class RestBulkAction extends BaseRestHandler {
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");
}
}

View File

@ -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.client.Client;
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 class RestDeleteAction extends BaseRestHandler {
client.delete(deleteRequest, new RestBuilderListener<DeleteResponse>(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 class RestDeleteAction extends BaseRestHandler {
}
});
}
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");
}
}

View File

@ -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.client.Client;
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 java.io.IOException;
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 class RestIndexAction extends BaseRestHandler {
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 class RestIndexAction extends BaseRestHandler {
@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 class RestIndexAction extends BaseRestHandler {
}
});
}
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");
}
}

View File

@ -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.Strings;
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 class RestUpdateAction extends BaseRestHandler {
@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 class RestUpdateAction extends BaseRestHandler {
}
});
}
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");
}
}

View File

@ -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<T extends TransportResponse> implements TransportResponseHandler<T> {

View File

@ -20,8 +20,8 @@ package org.elasticsearch.action.support.replication;
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.ExecutionException;
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 class BroadcastReplicationTests extends ESTestCase {
randomBoolean() ? ShardRoutingState.INITIALIZING : ShardRoutingState.UNASSIGNED, ShardRoutingState.UNASSIGNED));
logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
Future<BroadcastResponse> response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index)));
for (Tuple<ShardId, ActionListener<ActionWriteResponse>> shardRequests : broadcastReplicationAction.capturedShardRequests) {
for (Tuple<ShardId, ActionListener<ReplicationResponse>> shardRequests : broadcastReplicationAction.capturedShardRequests) {
if (randomBoolean()) {
shardRequests.v2().onFailure(new NoShardAvailableActionException(shardRequests.v1()));
} else {
@ -120,10 +116,10 @@ public class BroadcastReplicationTests extends ESTestCase {
ShardRoutingState.STARTED));
logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint());
Future<BroadcastResponse> response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index)));
for (Tuple<ShardId, ActionListener<ActionWriteResponse>> 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<ShardId, ActionListener<ReplicationResponse>> 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 class BroadcastReplicationTests extends ESTestCase {
Future<BroadcastResponse> response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index)));
int succeeded = 0;
int failed = 0;
for (Tuple<ShardId, ActionListener<ActionWriteResponse>> shardRequests : broadcastReplicationAction.capturedShardRequests) {
for (Tuple<ShardId, ActionListener<ReplicationResponse>> 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 class BroadcastReplicationTests extends ESTestCase {
assertThat(shards.get(0), equalTo(shardId));
}
private class TestBroadcastReplicationAction extends TransportBroadcastReplicationAction<BroadcastRequest, BroadcastResponse, ReplicationRequest, ActionWriteResponse> {
protected final Set<Tuple<ShardId, ActionListener<ActionWriteResponse>>> capturedShardRequests = ConcurrentCollections.newConcurrentSet();
private class TestBroadcastReplicationAction extends TransportBroadcastReplicationAction<BroadcastRequest, BroadcastResponse, ReplicationRequest, ReplicationResponse> {
protected final Set<Tuple<ShardId, ActionListener<ReplicationResponse>>> 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 @@ public class BroadcastReplicationTests extends ESTestCase {
}
@Override
protected void shardExecute(BroadcastRequest request, ShardId shardId, ActionListener<ActionWriteResponse> shardActionListener) {
protected void shardExecute(BroadcastRequest request, ShardId shardId, ActionListener<ReplicationResponse> shardActionListener) {
capturedShardRequests.add(new Tuple<>(shardId, shardActionListener));
}
}

View File

@ -27,12 +27,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
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 class ClusterStateCreationUtils {
}
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 class ClusterStateCreationUtils {
} 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));

View File

@ -20,7 +20,7 @@ package org.elasticsearch.action.support.replication;
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 java.util.concurrent.atomic.AtomicInteger;
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 @@ public class TransportReplicationActionTests extends ESTestCase {
final ShardIterator shardIt = shardRoutingTable.shardsIt();
final ShardId shardId = shardIt.shardId();
final Request request = new Request();
final long primaryTerm = randomInt(200);
request.primaryTerm(primaryTerm);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
logger.debug("expecting [{}] assigned replicas, [{}] total shards. using state: \n{}", assignedReplicas, totalShards, clusterService.state().prettyPrint());
@ -401,7 +399,7 @@ public class TransportReplicationActionTests extends ESTestCase {
}
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 @@ public class TransportReplicationActionTests extends ESTestCase {
}
// 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<Response> listener = new PlainActionFuture<>();
TransportReplicationAction<Request, Request, Response>.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 class TransportReplicationActionTests extends ESTestCase {
}
}
static class Response extends ActionWriteResponse {
static class Response extends ReplicationResponse {
}
class Action extends TransportReplicationAction<Request, Request, Response> {
@ -632,7 +649,7 @@ public class TransportReplicationActionTests extends ESTestCase {
}
@Override
protected Releasable getIndexShardOperationsCounter(ShardId shardId) {
protected Releasable getIndexShardOperationsCounter(ShardId shardId, long opPrimaryTerm) {
return getOrCreateIndexShardOperationsCounter();
}
}

View File

@ -115,12 +115,18 @@ public class ShardRoutingTests extends ESTestCase {
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);

View File

@ -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.common.settings.Settings;
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 @@ public class ShardInfoIT extends ESIntegTestCase {
}
}
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));
}

View File

@ -68,6 +68,7 @@ import org.elasticsearch.index.mapper.ParseContext.Document;
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 @@ public class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
}
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
// 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 class InternalEngineTests extends ESTestCase {
}
// 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 class InternalEngineTests extends ESTestCase {
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
// 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 class InternalEngineTests extends ESTestCase {
// 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 class InternalEngineTests extends ESTestCase {
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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();
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));
}
// 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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
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 class InternalEngineTests extends ESTestCase {
}
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 class InternalEngineTests extends ESTestCase {
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));
}

View File

@ -161,9 +161,10 @@ public class ShadowEngineTests extends ESTestCase {
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 class ShadowEngineTests extends ESTestCase {
@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) {

View File

@ -36,7 +36,8 @@ import static org.hamcrest.Matchers.not;
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[]

View File

@ -88,8 +88,8 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase {
.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 class FieldNamesFieldMapperTests extends ESSingleNodeTestCase {
.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 {

View File

@ -34,6 +34,8 @@ import org.elasticsearch.action.admin.indices.stats.CommonStats;
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.ClusterService;
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.settings.Settings.settingsBuilder;
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 class IndexShardTests extends ESSingleNodeTestCase {
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 class IndexShardTests extends ESSingleNodeTestCase {
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 class IndexShardTests extends ESSingleNodeTestCase {
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 class IndexShardTests extends ESSingleNodeTestCase {
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 class IndexShardTests extends ESSingleNodeTestCase {
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);
}

View File

@ -302,7 +302,7 @@ public class TranslogTests extends ESTestCase {
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 class TranslogTests extends ESTestCase {
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 class TranslogTests extends ESTestCase {
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 class TranslogTests extends ESTestCase {
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)) {

View File

@ -110,7 +110,7 @@ public class SyncedFlushSingleNodeTests extends ESSingleNodeTestCase {
SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class);
final ShardId shardId = shard.shardId();
shard.incrementOperationCounter();
shard.incrementOperationCounter(shard.routingEntry().primaryTerm());
try {
SyncedFlushUtil.LatchedListener<ShardsSyncedFlushResult> listener = new SyncedFlushUtil.LatchedListener<>();
flushService.attemptSyncedFlush(shardId, listener);

View File

@ -253,7 +253,7 @@ public class SimpleRoutingIT extends ESIntegTestCase {
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"))

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.text.StringText;
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 class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase {
} 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 class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase {
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 @@ public class TransportDeleteByQueryActionTests extends ESSingleNodeTestCase {
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));
}
}