diff --git a/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java b/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java index 009d3fc47a96c..6b12d23435bc1 100644 --- a/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java +++ b/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java @@ -21,9 +21,9 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.StatusToXContent; -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 org.elasticsearch.rest.RestStatus; @@ -38,11 +38,13 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St private String id; private String type; private long version; + private long seqNo; - public DocWriteResponse(ShardId shardId, String type, String id, long version) { + 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; } @@ -86,12 +88,19 @@ 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; + } + /** returns the rest status for this response (based on {@link ShardInfo#status()} */ public RestStatus status() { return getShardInfo().status(); } - @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -99,6 +108,7 @@ public void readFrom(StreamInput in) throws IOException { type = in.readString(); id = in.readString(); version = in.readZLong(); + seqNo = in.readZLong(); } @Override @@ -108,6 +118,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(type); out.writeString(id); out.writeZLong(version); + out.writeZLong(seqNo); } static final class Fields { @@ -115,6 +126,8 @@ static final class Fields { 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 @@ -125,6 +138,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws .field(Fields._ID, id) .field(Fields._VERSION, version); shardInfo.toXContent(builder, params); + //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()); + } return builder; } } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java index 3fba14e72bc79..91e1f82658739 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -19,7 +19,6 @@ package org.elasticsearch.action.admin.cluster.stats; -import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.indices.stats.CommonStats; @@ -30,6 +29,7 @@ import org.elasticsearch.action.support.nodes.TransportNodesAction; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterStateHealth; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.inject.Inject; @@ -105,7 +105,8 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq for (IndexShard indexShard : indexService) { if (indexShard.routingEntry() != null && indexShard.routingEntry().active()) { // only report on fully started shards - shardsStats.add(new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), new CommonStats(indexShard, SHARD_STATS_FLAGS), indexShard.commitStats())); + shardsStats.add(new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), + new CommonStats(indexShard, SHARD_STATS_FLAGS), indexShard.commitStats(), indexShard.seqNoStats())); } } } diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java b/core/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java index fb306337886af..037bf8575eeea 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStatsFlags.java @@ -227,7 +227,6 @@ public static enum Flag { RequestCache("request_cache"), Recovery("recovery"); - private final String restName; Flag(String restName) { diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/stats/ShardStats.java b/core/src/main/java/org/elasticsearch/action/admin/indices/stats/ShardStats.java index 8fea8c795ebd5..81586f0fa7cae 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/stats/ShardStats.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/stats/ShardStats.java @@ -28,7 +28,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.index.engine.CommitStats; -import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.shard.ShardPath; import java.io.IOException; @@ -42,6 +42,8 @@ public class ShardStats implements Streamable, ToXContent { private CommonStats commonStats; @Nullable private CommitStats commitStats; + @Nullable + private SeqNoStats seqNoStats; private String dataPath; private String statePath; private boolean isCustomDataPath; @@ -49,13 +51,14 @@ public class ShardStats implements Streamable, ToXContent { ShardStats() { } - public ShardStats(ShardRouting routing, ShardPath shardPath, CommonStats commonStats, CommitStats commitStats) { + public ShardStats(ShardRouting routing, ShardPath shardPath, CommonStats commonStats, CommitStats commitStats, SeqNoStats seqNoStats) { this.shardRouting = routing; this.dataPath = shardPath.getRootDataPath().toString(); this.statePath = shardPath.getRootStatePath().toString(); this.isCustomDataPath = shardPath.isCustomDataPath(); this.commitStats = commitStats; this.commonStats = commonStats; + this.seqNoStats = seqNoStats; } /** @@ -73,6 +76,11 @@ public CommitStats getCommitStats() { return this.commitStats; } + @Nullable + public SeqNoStats getSeqNoStats() { + return this.seqNoStats; + } + public String getDataPath() { return dataPath; } @@ -99,6 +107,7 @@ public void readFrom(StreamInput in) throws IOException { statePath = in.readString(); dataPath = in.readString(); isCustomDataPath = in.readBoolean(); + seqNoStats = in.readOptionalStreamableReader(SeqNoStats::new); } @Override @@ -109,6 +118,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(statePath); out.writeString(dataPath); out.writeBoolean(isCustomDataPath); + out.writeOptionalWritable(seqNoStats); } @Override @@ -124,6 +134,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (commitStats != null) { commitStats.toXContent(builder, params); } + if (seqNoStats != null) { + seqNoStats.toXContent(builder, params); + } builder.startObject(Fields.SHARD_PATH); builder.field(Fields.STATE_PATH, statePath); builder.field(Fields.DATA_PATH, dataPath); diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java index d5de67da478bb..e5fd4d4120879 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/stats/TransportIndicesStatsAction.java @@ -162,6 +162,7 @@ protected ShardStats shardOperation(IndicesStatsRequest request, ShardRouting sh flags.set(CommonStatsFlags.Flag.Recovery); } - return new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), new CommonStats(indexShard, flags), indexShard.commitStats()); + return new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), + new CommonStats(indexShard, flags), indexShard.commitStats(), indexShard.seqNoStats()); } } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 2597695a1e260..fad3953bc09ac 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -122,6 +122,9 @@ protected Tuple shardOperationOnPrimary(Met 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... @@ -157,6 +160,8 @@ protected Tuple shardOperationOnPrimary(Met 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... @@ -204,7 +209,8 @@ protected Tuple shardOperationOnPrimary(Met BytesReference indexSourceAsBytes = indexRequest.source(); // add the response IndexResponse indexResponse = result.response(); - UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); + UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), + indexResponse.getType(), indexResponse.getId(), indexResponse.getSeqNo(),indexResponse.getVersion(), indexResponse.isCreated()); if (updateRequest.fields() != null && updateRequest.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); @@ -216,7 +222,8 @@ protected Tuple shardOperationOnPrimary(Met WriteResult writeResult = updateResult.writeResult; DeleteResponse response = writeResult.response(); DeleteRequest deleteRequest = updateResult.request(); - updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), 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, 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); @@ -238,6 +245,8 @@ protected Tuple shardOperationOnPrimary(Met 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... @@ -308,7 +317,7 @@ private void setResponse(BulkItemRequest request, BulkItemResponse response) { } } - private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, MetaData metaData, + private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, MetaData metaData, IndexShard indexShard, boolean processed) throws Throwable { // validate, if routing is required, that we got routing diff --git a/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java b/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java index 57781547266c3..4852af7d01dd6 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java +++ b/core/src/main/java/org/elasticsearch/action/delete/DeleteResponse.java @@ -43,12 +43,11 @@ public DeleteResponse() { } - public DeleteResponse(ShardId shardId, String type, String id, long version, boolean found) { - super(shardId, type, id, version); + public DeleteResponse(ShardId shardId, String type, String id, long seqNo, long version, boolean found) { + super(shardId, type, id, seqNo, version); this.found = found; } - /** * Returns true if a doc was found to delete. */ diff --git a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index ca66b28575347..cca14ca041862 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -137,15 +137,16 @@ public static WriteResult executeDeleteRequestOnPrimary(DeleteRe // 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()); return new WriteResult<>( - new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found()), + new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.seqNo(), delete.version(), delete.found()), delete.getTranslogLocation()); } public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) { - Engine.Delete delete = indexShard.prepareDeleteOnReplica(request.type(), request.id(), request.version(), request.versionType()); + Engine.Delete delete = indexShard.prepareDeleteOnReplica(request.type(), request.id(), request.seqNo(), request.version(), request.versionType()); indexShard.delete(delete); return delete; } diff --git a/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java b/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java index 665327a749f78..e75a06b38dac3 100644 --- a/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java +++ b/core/src/main/java/org/elasticsearch/action/index/IndexResponse.java @@ -40,11 +40,10 @@ public class IndexResponse extends DocWriteResponse { private boolean created; public IndexResponse() { - } - public IndexResponse(ShardId shardId, String type, String id, long version, boolean created) { - super(shardId, type, id, version); + public IndexResponse(ShardId shardId, String type, String id, long seqNo, long version, boolean created) { + super(shardId, type, id, seqNo, version); this.created = created; } @@ -84,6 +83,7 @@ public String toString() { 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(); } diff --git a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 620056ded4e4c..8b38c4092619d 100644 --- a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -179,7 +179,7 @@ public static Engine.Index executeIndexRequestOnReplica(IndexRequest request, In 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.prepareIndexOnReplica(sourceToParse, request.version(), request.versionType()); + 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); @@ -219,11 +219,11 @@ public static WriteResult executeIndexRequestOnPrimary(IndexReque 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.version(), created), operation.getTranslogLocation()); + return new WriteResult<>(new IndexResponse(shardId, request.type(), request.id(), request.seqNo(), request.version(), created), operation.getTranslogLocation()); } - } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java index adbe199824ed8..b7f7f3b571779 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java @@ -49,6 +49,9 @@ public class ReplicationRequest extends ActionRequ */ protected ShardId shardId; + long seqNo; + long primaryTerm; + protected TimeValue timeout = DEFAULT_TIMEOUT; protected String index; @@ -90,6 +93,9 @@ protected ReplicationRequest(T request, ActionRequest originalRequest) { this.timeout = request.timeout(); this.index = request.index(); this.consistencyLevel = request.consistencyLevel(); + this.shardId = request.shardId(); + this.seqNo = request.seqNo(); + this.primaryTerm = request.primaryTerm(); } /** @@ -155,6 +161,29 @@ public final T consistencyLevel(WriteConsistencyLevel consistencyLevel) { return (T) this; } + /** + * Returns the sequence number for this operation. The sequence number is assigned while the operation + * is performed on the primary shard. + */ + public long seqNo() { + return seqNo; + } + + /** sets the sequence number for this operation. should only be called on the primary shard */ + public void seqNo(long seqNo) { + this.seqNo = seqNo; + } + + /** returns the primary term active at the time the operation was performed on the primary shard */ + public long primaryTerm() { + return primaryTerm; + } + + /** marks the primary term in which the operation was performed */ + public void primaryTerm(long term) { + primaryTerm = term; + } + @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; @@ -175,6 +204,8 @@ public void readFrom(StreamInput in) throws IOException { consistencyLevel = WriteConsistencyLevel.fromId(in.readByte()); timeout = TimeValue.readTimeValue(in); index = in.readString(); + seqNo = in.readVLong(); + primaryTerm = in.readVLong(); } @Override @@ -189,6 +220,8 @@ public void writeTo(StreamOutput out) throws IOException { out.writeByte(consistencyLevel.id()); timeout.writeTo(out); out.writeString(index); + out.writeVLong(seqNo); + out.writeVLong(primaryTerm); } /** diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 26c439c0a3d42..f081d4d2392c9 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -186,7 +186,7 @@ protected TransportRequestOptions transportOptions() { protected boolean retryPrimaryException(Throwable e) { return e.getClass() == RetryOnPrimaryException.class - || TransportActions.isShardNotAvailableException(e); + || TransportActions.isShardNotAvailableException(e); } /** @@ -280,7 +280,7 @@ public RetryOnReplicaException(ShardId shardId, String msg) { setShard(shardId); } - public RetryOnReplicaException(StreamInput in) throws IOException{ + public RetryOnReplicaException(StreamInput in) throws IOException { super(in); } } @@ -358,7 +358,7 @@ protected void responseWithFailure(Throwable t) { @Override protected void doRun() throws Exception { assert request.shardId() != null : "request shardId must be set"; - try (Releasable ignored = getIndexShardOperationsCounter(request.shardId())) { + try (Releasable ignored = getIndexShardOperationsCounterOnReplica(request.shardId(), request.primaryTerm)) { shardOperationOnReplica(request); if (logger.isTraceEnabled()) { logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, request.shardId(), request); @@ -374,7 +374,7 @@ public RetryOnPrimaryException(ShardId shardId, String msg) { setShard(shardId); } - public RetryOnPrimaryException(StreamInput in) throws IOException{ + public RetryOnPrimaryException(StreamInput in) throws IOException { super(in); } } @@ -562,7 +562,7 @@ final class PrimaryPhase extends AbstractRunnable { private final TransportChannel channel; private final ClusterState state; private final AtomicBoolean finished = new AtomicBoolean(); - private Releasable indexShardReference; + private IndexShardReference indexShardReference; PrimaryPhase(Request request, TransportChannel channel) { this.state = clusterService.state(); @@ -587,8 +587,9 @@ protected void doRun() throws Exception { } final ReplicationPhase replicationPhase; try { - indexShardReference = getIndexShardOperationsCounter(shardId); + indexShardReference = getIndexShardOperationsCounterOnPrimary(shardId); Tuple primaryResponse = shardOperationOnPrimary(state.metaData(), request); + primaryResponse.v2().primaryTerm(indexShardReference.opPrimaryTerm()); if (logger.isTraceEnabled()) { logger.trace("action [{}] completed on shard [{}] for request [{}] with cluster state version [{}]", transportPrimaryAction, shardId, request, state.version()); } @@ -690,10 +691,16 @@ void finishBecauseUnavailable(ShardId shardId, String message) { } } - protected Releasable getIndexShardOperationsCounter(ShardId shardId) { + protected IndexShardReference getIndexShardOperationsCounterOnReplica(ShardId shardId, long opPrimaryTerm) { IndexService indexService = indicesService.indexServiceSafe(shardId.index().getName()); IndexShard indexShard = indexService.getShard(shardId.id()); - return new IndexShardReference(indexShard); + return new IndexShardReferenceImpl(indexShard, opPrimaryTerm); + } + + protected IndexShardReference getIndexShardOperationsCounterOnPrimary(ShardId shardId) { + IndexService indexService = indicesService.indexServiceSafe(shardId.index().getName()); + IndexShard indexShard = indexService.getShard(shardId.id()); + return new IndexShardReferenceImpl(indexShard); } /** @@ -907,6 +914,9 @@ private void forceFinishAsFailed(Throwable t) { private void doFinish() { if (finished.compareAndSet(false, true)) { + if (logger.isTraceEnabled()) { + logger.trace("finished replicating action [{}], request [{}]", actionName, replicaRequest); + } Releasables.close(indexShardReference); final ReplicationResponse.ShardInfo.Failure[] failuresArray; if (!shardReplicaFailures.isEmpty()) { @@ -925,8 +935,7 @@ private void doFinish() { totalShards, success.get(), failuresArray - - ) + ) ); try { channel.sendResponse(finalResponse); @@ -976,13 +985,28 @@ protected boolean shouldExecuteReplication(Settings settings) { return IndexMetaData.isIndexUsingShadowReplicas(settings) == false; } - static class IndexShardReference implements Releasable { + interface IndexShardReference extends Releasable { + + /** returns the primary term of the current opration */ + long opPrimaryTerm(); + } + + static class IndexShardReferenceImpl implements IndexShardReference { final private IndexShard counter; private final AtomicBoolean closed = new AtomicBoolean(); + private final long opPrimaryTerm; + + IndexShardReferenceImpl(IndexShard counter) { + // this enforces primary terms, if we're lagging an exception will be thrown. + this. opPrimaryTerm = counter.incrementOperationCounterOnPrimary(); + this.counter = counter; + } - IndexShardReference(IndexShard counter) { - counter.incrementOperationCounter(); + IndexShardReferenceImpl(IndexShard counter, long opPrimaryTerm) { + // this enforces primary terms, if we're lagging an exception will be thrown. + counter.incrementOperationCounterOnReplica(opPrimaryTerm); + this.opPrimaryTerm = opPrimaryTerm; this.counter = counter; } @@ -992,6 +1016,11 @@ public void close() { counter.decrementOperationCounter(); } } + + @Override + public long opPrimaryTerm() { + return opPrimaryTerm; + } } protected final void processAfterWrite(boolean refresh, IndexShard indexShard, Translog.Location location) { diff --git a/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index e5edc1af96bac..622133026f685 100644 --- a/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -175,7 +175,8 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< indexAction.execute(upsertRequest, new ActionListener() { @Override public void onResponse(IndexResponse response) { - UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); + UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), + response.getId(), response.getSeqNo(), response.getVersion(), response.isCreated()); if (request.fields() != null && request.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); @@ -212,7 +213,8 @@ protected void doRun() { indexAction.execute(indexRequest, new ActionListener() { @Override public void onResponse(IndexResponse response) { - UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), 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); } @@ -240,7 +242,8 @@ protected void doRun() { deleteAction.execute(deleteRequest, new ActionListener() { @Override public void onResponse(DeleteResponse response) { - UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), 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); } @@ -266,7 +269,7 @@ protected void doRun() { case NONE: UpdateResponse update = result.action(); IndexService indexServiceOrNull = indicesService.indexService(request.concreteIndex()); - if (indexServiceOrNull != null) { + if (indexServiceOrNull != null) { IndexShard shard = indexService.getShardOrNull(request.shardId()); if (shard != null) { shard.indexingService().noopUpdate(request.type()); diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java b/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java index 2f3146b064418..915311e10b7a5 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateResponse.java @@ -25,6 +25,7 @@ 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 org.elasticsearch.rest.RestStatus; @@ -45,11 +46,11 @@ public UpdateResponse() { * For example: update script with operation set to none */ public UpdateResponse(ShardId shardId, String type, String id, long version, boolean created) { - this(new ShardInfo(0, 0), shardId, type, id, version, created); + this(new ShardInfo(0, 0), shardId, type, id, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, created); } - public UpdateResponse(ShardInfo shardInfo, ShardId shardId, String type, String id, long version, boolean created) { - super(shardId, type, id, version); + 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.created = created; } @@ -99,7 +100,6 @@ public void writeTo(StreamOutput out) throws IOException { } } - static final class Fields { static final XContentBuilderString GET = new XContentBuilderString("get"); } @@ -127,5 +127,4 @@ public String toString() { builder.append(",shards=").append(getShardInfo()); return builder.append("]").toString(); } - } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java index e20f21b4cecfc..e02775cc82a3a 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -127,7 +127,7 @@ public static T lookupPrototype(String type) { public static T lookupPrototypeSafe(String type) { @SuppressWarnings("unchecked") - T proto = (T)customPrototypes.get(type); + T proto = (T) customPrototypes.get(type); if (proto == null) { throw new IllegalArgumentException("No custom state prototype registered for type [" + type + "]"); } @@ -469,6 +469,12 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.endArray(); + builder.startObject("primary_terms"); + for (int shard = 0; shard < indexMetaData.getNumberOfShards(); shard++) { + builder.field(Integer.toString(shard), indexMetaData.primaryTerm(shard)); + } + builder.endObject(); + builder.startObject(IndexMetaData.KEY_ACTIVE_ALLOCATIONS); for (IntObjectCursor> cursor : indexMetaData.getActiveAllocationIds()) { builder.startArray(String.valueOf(cursor.key)); @@ -479,6 +485,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.endObject(); + // index metdata data builder.endObject(); } builder.endObject(); @@ -675,16 +682,16 @@ public static byte[] toBytes(ClusterState state) throws IOException { } /** - * @param data input bytes - * @param localNode used to set the local node in the cluster state. + * @param data input bytes + * @param localNode used to set the local node in the cluster state. */ public static ClusterState fromBytes(byte[] data, DiscoveryNode localNode) throws IOException { return readFrom(StreamInput.wrap(data), localNode); } /** - * @param in input stream - * @param localNode used to set the local node in the cluster state. can be null. + * @param in input stream + * @param localNode used to set the local node in the cluster state. can be null. */ public static ClusterState readFrom(StreamInput in, @Nullable DiscoveryNode localNode) throws IOException { return PROTO.readFrom(in, localNode); @@ -783,17 +790,17 @@ public ClusterStateDiff(StreamInput in, ClusterState proto) throws IOException { metaData = proto.metaData.readDiffFrom(in); blocks = proto.blocks.readDiffFrom(in); customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), - new DiffableUtils.DiffableValueSerializer() { - @Override - public Custom read(StreamInput in, String key) throws IOException { - return lookupPrototypeSafe(key).readFrom(in); - } + new DiffableUtils.DiffableValueSerializer() { + @Override + public Custom read(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } - @Override - public Diff readDiff(StreamInput in, String key) throws IOException { - return lookupPrototypeSafe(key).readDiffFrom(in); - } - }); + @Override + public Diff readDiff(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); } @Override diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 669d71477caaf..a1cdf26faab51 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -19,6 +19,7 @@ package org.elasticsearch.cluster.metadata; +import com.carrotsearch.hppc.LongArrayList; import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; @@ -29,6 +30,7 @@ import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.node.DiscoveryNodeFilters; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -48,13 +50,7 @@ import java.io.IOException; import java.text.ParseException; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Locale; -import java.util.Map; -import java.util.Set; +import java.util.*; import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.AND; import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.OR; @@ -63,11 +59,11 @@ /** * */ -public class IndexMetaData implements Diffable, FromXContentBuilder, ToXContent { +public class IndexMetaData implements Diffable, FromXContentBuilder, ToXContent { public static final IndexMetaData PROTO = IndexMetaData.builder("") - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .numberOfShards(1).numberOfReplicas(0).build(); + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0).build(); public interface Custom extends Diffable, ToXContent { @@ -150,6 +146,7 @@ public static State fromString(String state) { throw new IllegalStateException("No state match for [" + state + "]"); } } + public static final String INDEX_SETTING_PREFIX = "index."; public static final String SETTING_NUMBER_OF_SHARDS = "index.number_of_shards"; public static final String SETTING_NUMBER_OF_REPLICAS = "index.number_of_replicas"; @@ -174,12 +171,19 @@ public static State fromString(String state) { public static final String INDEX_UUID_NA_VALUE = "_na_"; public static final String KEY_ACTIVE_ALLOCATIONS = "active_allocations"; + static final String KEY_VERSION = "version"; + static final String KEY_SETTINGS = "settings"; + static final String KEY_STATE = "state"; + static final String KEY_MAPPINGS = "mappings"; + static final String KEY_ALIASES = "aliases"; + static final String KEY_PRIMARY_TERMS = "primary_terms"; private final int numberOfShards; private final int numberOfReplicas; private final String index; private final long version; + private final long[] primaryTerms; private final State state; @@ -203,7 +207,7 @@ public static State fromString(String state) { private final Version indexUpgradedVersion; private final org.apache.lucene.util.Version minimumCompatibleLuceneVersion; - private IndexMetaData(String index, long version, State state, int numberOfShards, int numberOfReplicas, Settings settings, + private IndexMetaData(String index, long version, long[] primaryTerms, State state, int numberOfShards, int numberOfReplicas, Settings settings, ImmutableOpenMap mappings, ImmutableOpenMap aliases, ImmutableOpenMap customs, ImmutableOpenIntMap> activeAllocationIds, DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters, @@ -211,6 +215,8 @@ private IndexMetaData(String index, long version, State state, int numberOfShard this.index = index; this.version = version; + this.primaryTerms = primaryTerms; + assert primaryTerms.length == numberOfShards; this.state = state; this.numberOfShards = numberOfShards; this.numberOfReplicas = numberOfReplicas; @@ -252,6 +258,16 @@ public long getVersion() { return this.version; } + + /** + * The term of the current selected primary. This is a non-negative number incremented when + * a primary shard is assigned after a full cluster restart (see {@link ShardRouting#initialize(java.lang.String, long)} + * or a replica shard is promoted to a primary (see {@link ShardRouting#moveToPrimary()}). + **/ + public long primaryTerm(int shardId) { + return this.primaryTerms[shardId]; + } + /** * Return the {@link Version} on which this index has been created. This * information is typically useful for backward compatibility. @@ -372,6 +388,10 @@ public boolean equals(Object o) { IndexMetaData that = (IndexMetaData) o; + if (version != that.version) { + return false; + } + if (!aliases.equals(that.aliases)) { return false; } @@ -390,6 +410,10 @@ public boolean equals(Object o) { if (!customs.equals(that.customs)) { return false; } + + if (Arrays.equals(primaryTerms, that.primaryTerms) == false) { + return false; + } if (!activeAllocationIds.equals(that.activeAllocationIds)) { return false; } @@ -399,14 +423,18 @@ public boolean equals(Object o) { @Override public int hashCode() { int result = index.hashCode(); + result = 31 * result + Long.hashCode(version); result = 31 * result + state.hashCode(); result = 31 * result + aliases.hashCode(); result = 31 * result + settings.hashCode(); result = 31 * result + mappings.hashCode(); + result = 31 * result + customs.hashCode(); + result = 31 * result + Arrays.hashCode(primaryTerms); result = 31 * result + activeAllocationIds.hashCode(); return result; } + @Override public Diff diff(IndexMetaData previousState) { return new IndexMetaDataDiff(previousState, this); @@ -432,6 +460,7 @@ private static class IndexMetaDataDiff implements Diff { private final String index; private final long version; + private final long[] primaryTerms; private final State state; private final Settings settings; private final Diff> mappings; @@ -444,11 +473,12 @@ public IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) { version = after.version; state = after.state; settings = after.settings; + primaryTerms = after.primaryTerms; mappings = DiffableUtils.diff(before.mappings, after.mappings, DiffableUtils.getStringKeySerializer()); aliases = DiffableUtils.diff(before.aliases, after.aliases, DiffableUtils.getStringKeySerializer()); customs = DiffableUtils.diff(before.customs, after.customs, DiffableUtils.getStringKeySerializer()); activeAllocationIds = DiffableUtils.diff(before.activeAllocationIds, after.activeAllocationIds, - DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance()); + DiffableUtils.getVIntKeySerializer(), DiffableUtils.StringSetValueSerializer.getInstance()); } public IndexMetaDataDiff(StreamInput in) throws IOException { @@ -456,22 +486,23 @@ public IndexMetaDataDiff(StreamInput in) throws IOException { version = in.readLong(); state = State.fromId(in.readByte()); settings = Settings.readSettingsFromStream(in); + primaryTerms = in.readVLongArray(); mappings = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), MappingMetaData.PROTO); aliases = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), AliasMetaData.PROTO); customs = DiffableUtils.readImmutableOpenMapDiff(in, DiffableUtils.getStringKeySerializer(), - new DiffableUtils.DiffableValueSerializer() { - @Override - public Custom read(StreamInput in, String key) throws IOException { - return lookupPrototypeSafe(key).readFrom(in); - } + new DiffableUtils.DiffableValueSerializer() { + @Override + public Custom read(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readFrom(in); + } - @Override - public Diff readDiff(StreamInput in, String key) throws IOException { - return lookupPrototypeSafe(key).readDiffFrom(in); - } - }); + @Override + public Diff readDiff(StreamInput in, String key) throws IOException { + return lookupPrototypeSafe(key).readDiffFrom(in); + } + }); activeAllocationIds = DiffableUtils.readImmutableOpenIntMapDiff(in, DiffableUtils.getVIntKeySerializer(), - DiffableUtils.StringSetValueSerializer.getInstance()); + DiffableUtils.StringSetValueSerializer.getInstance()); } @Override @@ -480,6 +511,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeByte(state.id); Settings.writeSettingsToStream(settings, out); + out.writeVLongArray(primaryTerms); mappings.writeTo(out); aliases.writeTo(out); customs.writeTo(out); @@ -492,6 +524,7 @@ public IndexMetaData apply(IndexMetaData part) { builder.version(version); builder.state(state); builder.settings(settings); + builder.primaryTerms(primaryTerms); builder.mappings.putAll(mappings.apply(part.mappings)); builder.aliases.putAll(aliases.apply(part.aliases)); builder.customs.putAll(customs.apply(part.customs)); @@ -506,6 +539,7 @@ public IndexMetaData readFrom(StreamInput in) throws IOException { builder.version(in.readLong()); builder.state(State.fromId(in.readByte())); builder.settings(readSettingsFromStream(in)); + builder.primaryTerms(in.readVLongArray()); int mappingsSize = in.readVInt(); for (int i = 0; i < mappingsSize; i++) { MappingMetaData mappingMd = MappingMetaData.PROTO.readFrom(in); @@ -537,6 +571,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(version); out.writeByte(state.id()); writeSettingsToStream(settings, out); + out.writeVLongArray(primaryTerms); out.writeVInt(mappings.size()); for (ObjectCursor cursor : mappings.values()) { cursor.value.writeTo(out); @@ -570,6 +605,7 @@ public static class Builder { private String index; private State state = State.OPEN; private long version = 1; + private long[] primaryTerms = null; private Settings settings = Settings.Builder.EMPTY_SETTINGS; private final ImmutableOpenMap.Builder mappings; private final ImmutableOpenMap.Builder aliases; @@ -589,6 +625,7 @@ public Builder(IndexMetaData indexMetaData) { this.state = indexMetaData.state; this.version = indexMetaData.version; this.settings = indexMetaData.getSettings(); + this.primaryTerms = indexMetaData.primaryTerms.clone(); this.mappings = ImmutableOpenMap.builder(indexMetaData.mappings); this.aliases = ImmutableOpenMap.builder(indexMetaData.aliases); this.customs = ImmutableOpenMap.builder(indexMetaData.customs); @@ -621,7 +658,7 @@ public Builder numberOfReplicas(int numberOfReplicas) { public int numberOfReplicas() { return settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, -1); } - + public Builder creationDate(long creationDate) { settings = settingsBuilder().put(settings).put(SETTING_CREATION_DATE, creationDate).build(); return this; @@ -632,8 +669,7 @@ public long creationDate() { } public Builder settings(Settings.Builder settings) { - this.settings = settings.build(); - return this; + return settings(settings.build()); } public Builder settings(Settings settings) { @@ -719,6 +755,42 @@ public Builder version(long version) { return this; } + /** + * returns the primary term for the given shard. + * See {@link IndexMetaData#primaryTerm(int)} for more information. + */ + public long primaryTerm(int shardId) { + if (primaryTerms == null) { + initializePrimaryTerms(); + } + return this.primaryTerms[shardId]; + } + + /** + * sets the primary term for the given shard. + * See {@link IndexMetaData#primaryTerm(int)} for more information. + */ + public Builder primaryTerm(int shardId, long primaryTerm) { + if (primaryTerms == null) { + initializePrimaryTerms(); + } + this.primaryTerms[shardId] = primaryTerm; + return this; + } + + private void primaryTerms(long[] primaryTerms) { + this.primaryTerms = primaryTerms.clone(); + } + + private void initializePrimaryTerms() { + assert primaryTerms == null; + if (numberOfShards() < 0) { + throw new IllegalStateException("you must set the number of shards before setting/reading primary terms"); + } + primaryTerms = new long[numberOfShards()]; + } + + public IndexMetaData build() { ImmutableOpenMap.Builder tmpAliases = aliases; Settings tmpSettings = settings; @@ -788,32 +860,39 @@ public IndexMetaData build() { try { minimumCompatibleLuceneVersion = org.apache.lucene.util.Version.parse(stringLuceneVersion); } catch (ParseException ex) { - throw new IllegalStateException("Cannot parse lucene version [" + stringLuceneVersion + "] in the [" + SETTING_VERSION_MINIMUM_COMPATIBLE +"] setting", ex); + throw new IllegalStateException("Cannot parse lucene version [" + stringLuceneVersion + "] in the [" + SETTING_VERSION_MINIMUM_COMPATIBLE + "] setting", ex); } } else { minimumCompatibleLuceneVersion = null; } - return new IndexMetaData(index, version, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(), - tmpAliases.build(), customs.build(), filledActiveAllocationIds.build(), requireFilters, includeFilters, excludeFilters, - indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion); + if (primaryTerms == null) { + initializePrimaryTerms(); + } else if (primaryTerms.length != numberOfShards) { + throw new IllegalStateException("primaryTerms length is [" + primaryTerms.length + + "] but should be equal to number of shards [" + numberOfShards() + "]"); + } + + return new IndexMetaData(index, version, primaryTerms, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(), + tmpAliases.build(), customs.build(), filledActiveAllocationIds.build(), requireFilters, includeFilters, excludeFilters, + indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion); } public static void toXContent(IndexMetaData indexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { builder.startObject(indexMetaData.getIndex(), XContentBuilder.FieldCaseConversion.NONE); - builder.field("version", indexMetaData.getVersion()); - builder.field("state", indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH)); + builder.field(KEY_VERSION, indexMetaData.getVersion()); + builder.field(KEY_STATE, indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH)); boolean binary = params.paramAsBoolean("binary", false); - builder.startObject("settings"); + builder.startObject(KEY_SETTINGS); for (Map.Entry entry : indexMetaData.getSettings().getAsMap().entrySet()) { builder.field(entry.getKey(), entry.getValue()); } builder.endObject(); - builder.startArray("mappings"); + builder.startArray(KEY_MAPPINGS); for (ObjectObjectCursor cursor : indexMetaData.getMappings()) { if (binary) { builder.value(cursor.value.source().compressed()); @@ -833,12 +912,18 @@ public static void toXContent(IndexMetaData indexMetaData, XContentBuilder build builder.endObject(); } - builder.startObject("aliases"); + builder.startObject(KEY_ALIASES); for (ObjectCursor cursor : indexMetaData.getAliases().values()) { AliasMetaData.Builder.toXContent(cursor.value, builder, params); } builder.endObject(); + builder.startArray(KEY_PRIMARY_TERMS); + for (int i = 0; i < indexMetaData.getNumberOfShards(); i++) { + builder.value(indexMetaData.primaryTerm(i)); + } + builder.endArray(); + builder.startObject(KEY_ACTIVE_ALLOCATIONS); for (IntObjectCursor> cursor : indexMetaData.activeAllocationIds) { builder.startArray(String.valueOf(cursor.key)); @@ -867,9 +952,9 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); } else if (token == XContentParser.Token.START_OBJECT) { - if ("settings".equals(currentFieldName)) { + if (KEY_SETTINGS.equals(currentFieldName)) { builder.settings(Settings.settingsBuilder().put(SettingsLoader.Helper.loadNestedFromMap(parser.mapOrdered()))); - } else if ("mappings".equals(currentFieldName)) { + } else if (KEY_MAPPINGS.equals(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); @@ -879,7 +964,7 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti builder.putMapping(new MappingMetaData(mappingType, mappingSource)); } } - } else if ("aliases".equals(currentFieldName)) { + } else if (KEY_ALIASES.equals(currentFieldName)) { while (parser.nextToken() != XContentParser.Token.END_OBJECT) { builder.putAlias(AliasMetaData.Builder.fromXContent(parser)); } @@ -910,7 +995,7 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti } } } else if (token == XContentParser.Token.START_ARRAY) { - if ("mappings".equals(currentFieldName)) { + if (KEY_MAPPINGS.equals(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) { builder.putMapping(new MappingMetaData(new CompressedXContent(parser.binaryValue()))); @@ -922,11 +1007,21 @@ public static IndexMetaData fromXContent(XContentParser parser) throws IOExcepti } } } + } else if (KEY_PRIMARY_TERMS.equals(currentFieldName)) { + LongArrayList list = new LongArrayList(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + if (token == XContentParser.Token.VALUE_NUMBER) { + list.add(parser.longValue()); + } else { + throw new IllegalStateException("found a non-numeric value under [" + KEY_PRIMARY_TERMS + "]"); + } + } + builder.primaryTerms(list.toArray()); } } else if (token.isValue()) { - if ("state".equals(currentFieldName)) { + if (KEY_STATE.equals(currentFieldName)) { builder.state(State.fromString(parser.text())); - } else if ("version".equals(currentFieldName)) { + } else if (KEY_VERSION.equals(currentFieldName)) { builder.version(parser.longValue()); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java index 00904af89155b..6c1d41da98301 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexUpgradeService.java @@ -19,7 +19,6 @@ package org.elasticsearch.cluster.metadata; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.lucene.analysis.Analyzer; import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.UnassignedInfo; @@ -30,6 +29,7 @@ import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.seqno.LocalCheckpointService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.indices.mapper.MapperRegistry; @@ -116,42 +116,43 @@ private static boolean isSupportedVersion(IndexMetaData indexMetaData) { /** All known byte-sized settings for an index. */ public static final Set INDEX_BYTES_SIZE_SETTINGS = unmodifiableSet(newHashSet( - "index.merge.policy.floor_segment", - "index.merge.policy.max_merged_segment", - "index.merge.policy.max_merge_size", - "index.merge.policy.min_merge_size", - "index.shard.recovery.file_chunk_size", - "index.shard.recovery.translog_size", - "index.store.throttle.max_bytes_per_sec", - "index.translog.flush_threshold_size", - "index.translog.fs.buffer_size", - "index.version_map_size")); + "index.merge.policy.floor_segment", + "index.merge.policy.max_merged_segment", + "index.merge.policy.max_merge_size", + "index.merge.policy.min_merge_size", + "index.shard.recovery.file_chunk_size", + "index.shard.recovery.translog_size", + "index.store.throttle.max_bytes_per_sec", + "index.translog.flush_threshold_size", + "index.translog.fs.buffer_size", + "index.version_map_size")); /** All known time settings for an index. */ public static final Set INDEX_TIME_SETTINGS = unmodifiableSet(newHashSet( - "index.gateway.wait_for_mapping_update_post_recovery", - "index.shard.wait_for_mapping_update_post_recovery", - "index.gc_deletes", - "index.indexing.slowlog.threshold.index.debug", - "index.indexing.slowlog.threshold.index.info", - "index.indexing.slowlog.threshold.index.trace", - "index.indexing.slowlog.threshold.index.warn", - "index.refresh_interval", - "index.search.slowlog.threshold.fetch.debug", - "index.search.slowlog.threshold.fetch.info", - "index.search.slowlog.threshold.fetch.trace", - "index.search.slowlog.threshold.fetch.warn", - "index.search.slowlog.threshold.query.debug", - "index.search.slowlog.threshold.query.info", - "index.search.slowlog.threshold.query.trace", - "index.search.slowlog.threshold.query.warn", - "index.shadow.wait_for_initial_commit", - "index.store.stats_refresh_interval", - "index.translog.flush_threshold_period", - "index.translog.interval", - "index.translog.sync_interval", - "index.shard.inactive_time", - UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING)); + "index.gateway.wait_for_mapping_update_post_recovery", + "index.shard.wait_for_mapping_update_post_recovery", + "index.gc_deletes", + "index.indexing.slowlog.threshold.index.debug", + "index.indexing.slowlog.threshold.index.info", + "index.indexing.slowlog.threshold.index.trace", + "index.indexing.slowlog.threshold.index.warn", + "index.refresh_interval", + "index.search.slowlog.threshold.fetch.debug", + "index.search.slowlog.threshold.fetch.info", + "index.search.slowlog.threshold.fetch.trace", + "index.search.slowlog.threshold.fetch.warn", + "index.search.slowlog.threshold.query.debug", + "index.search.slowlog.threshold.query.info", + "index.search.slowlog.threshold.query.trace", + "index.search.slowlog.threshold.query.warn", + "index.shadow.wait_for_initial_commit", + "index.store.stats_refresh_interval", + "index.translog.flush_threshold_period", + "index.translog.interval", + "index.translog.sync_interval", + "index.shard.inactive_time", + LocalCheckpointService.SETTINGS_BIT_ARRAYS_SIZE, + UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING)); /** * Elasticsearch 2.0 requires units on byte/memory and time settings; this method adds the default unit to any such settings that are @@ -163,7 +164,7 @@ private IndexMetaData addDefaultUnitsIfNeeded(IndexMetaData indexMetaData) { // Created lazily if we find any settings that are missing units: Settings settings = indexMetaData.getSettings(); Settings.Builder newSettings = null; - for(String byteSizeSetting : INDEX_BYTES_SIZE_SETTINGS) { + for (String byteSizeSetting : INDEX_BYTES_SIZE_SETTINGS) { String value = settings.get(byteSizeSetting); if (value != null) { try { @@ -180,7 +181,7 @@ private IndexMetaData addDefaultUnitsIfNeeded(IndexMetaData indexMetaData) { newSettings.put(byteSizeSetting, value + "b"); } } - for(String timeSetting : INDEX_TIME_SETTINGS) { + for (String timeSetting : INDEX_TIME_SETTINGS) { String value = settings.get(timeSetting); if (value != null) { try { @@ -200,9 +201,9 @@ private IndexMetaData addDefaultUnitsIfNeeded(IndexMetaData indexMetaData) { if (newSettings != null) { // At least one setting was changed: return IndexMetaData.builder(indexMetaData) - .version(indexMetaData.getVersion()) - .settings(newSettings.build()) - .build(); + .version(indexMetaData.getVersion()) + .settings(newSettings.build()) + .build(); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java b/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java index ca071c811e3e1..62054670e214f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/IndexRoutingTable.java @@ -99,14 +99,17 @@ public String getIndex() { } /** - * creates a new {@link IndexRoutingTable} with all shard versions normalized + * creates a new {@link IndexRoutingTable} with all shard versions & primary terms set to the highest found. + * This allows incrementing {@link ShardRouting#version()} and {@link ShardRouting#primaryTerm()} where we work on + * the individual shards without worrying about synchronization between {@link ShardRouting} instances. This method + * takes care of it. * * @return new {@link IndexRoutingTable} */ - public IndexRoutingTable normalizeVersions() { + public IndexRoutingTable normalizeVersionsAndPrimaryTerms() { IndexRoutingTable.Builder builder = new Builder(this.index); for (IntObjectCursor cursor : shards) { - builder.addIndexShard(cursor.value.normalizeVersions()); + builder.addIndexShard(cursor.value.normalizeVersionsAndPrimaryTerms()); } return builder.build(); } @@ -422,11 +425,12 @@ private Builder initializeAsRestore(IndexMetaData indexMetaData, RestoreSource r for (int shardId = 0; shardId < indexMetaData.getNumberOfShards(); shardId++) { IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(new ShardId(indexMetaData.getIndex(), shardId)); for (int i = 0; i <= indexMetaData.getNumberOfReplicas(); i++) { + final long primaryTerm = indexMetaData.primaryTerm(shardId); if (asNew && ignoreShards.contains(shardId)) { // This shards wasn't completely snapshotted - restore it as new shard - indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(index, shardId, null, i == 0, unassignedInfo)); + indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(index, shardId, null, primaryTerm, i == 0, unassignedInfo)); } else { - indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(index, shardId, i == 0 ? restoreSource : null, i == 0, unassignedInfo)); + indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(index, shardId, i == 0 ? restoreSource : null, primaryTerm, i == 0, unassignedInfo)); } } shards.put(shardId, indexShardRoutingBuilder.build()); @@ -442,9 +446,10 @@ private Builder initializeEmpty(IndexMetaData indexMetaData, UnassignedInfo unas throw new IllegalStateException("trying to initialize an index with fresh shards, but already has shards created"); } for (int shardId = 0; shardId < indexMetaData.getNumberOfShards(); shardId++) { + final long primaryTerm = indexMetaData.primaryTerm(shardId); IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(new ShardId(indexMetaData.getIndex(), shardId)); for (int i = 0; i <= indexMetaData.getNumberOfReplicas(); i++) { - indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(index, shardId, null, i == 0, unassignedInfo)); + indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(index, shardId, null,primaryTerm, i == 0, unassignedInfo)); } shards.put(shardId, indexShardRoutingBuilder.build()); } @@ -455,9 +460,11 @@ public Builder addReplica() { for (IntCursor cursor : shards.keys()) { int shardId = cursor.value; // version 0, will get updated when reroute will happen - ShardRouting shard = ShardRouting.newUnassigned(index, shardId, null, false, new UnassignedInfo(UnassignedInfo.Reason.REPLICA_ADDED, null)); + final IndexShardRoutingTable shardRoutingTable = shards.get(shardId); + ShardRouting shard = ShardRouting.newUnassigned(index, shardId, null, shardRoutingTable.primary.primaryTerm(), false, + new UnassignedInfo(UnassignedInfo.Reason.REPLICA_ADDED, null)); shards.put(shardId, - new IndexShardRoutingTable.Builder(shards.get(shard.id())).addShard(shard).build() + new IndexShardRoutingTable.Builder(shardRoutingTable).addShard(shard).build() ); } return this; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/core/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index 6512ee5cef799..fa3da3779ac46 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -28,14 +28,7 @@ import org.elasticsearch.index.shard.ShardId; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.ThreadLocalRandom; import static java.util.Collections.emptyMap; @@ -120,34 +113,37 @@ public class IndexShardRoutingTable implements Iterable { } /** - * Normalizes all shard routings to the same version. + * Normalizes all shard routings to the same (highest found) version & primary terms. */ - public IndexShardRoutingTable normalizeVersions() { + public IndexShardRoutingTable normalizeVersionsAndPrimaryTerms() { if (shards.isEmpty()) { return this; } + if (shards.size() == 1) { return this; } long highestVersion = shards.get(0).version(); + long highestPrimaryTerm = shards.get(0).primaryTerm(); boolean requiresNormalization = false; for (int i = 1; i < shards.size(); i++) { - if (shards.get(i).version() != highestVersion) { + final long version = shards.get(i).version(); + final long primaryTerm = shards.get(i).primaryTerm(); + if (highestVersion != version || highestPrimaryTerm != primaryTerm) { requiresNormalization = true; } - if (shards.get(i).version() > highestVersion) { - highestVersion = shards.get(i).version(); - } + highestVersion = Math.max(highestVersion, version); + highestPrimaryTerm = Math.max(highestPrimaryTerm, primaryTerm); } if (!requiresNormalization) { return this; } List shardRoutings = new ArrayList<>(shards.size()); for (int i = 0; i < shards.size(); i++) { - if (shards.get(i).version() == highestVersion) { + if (shards.get(i).version() == highestVersion && shards.get(i).primaryTerm() == highestPrimaryTerm) { shardRoutings.add(shards.get(i)); } else { - shardRoutings.add(new ShardRouting(shards.get(i), highestVersion)); + shardRoutings.add(new ShardRouting(shards.get(i), highestVersion, highestPrimaryTerm)); } } return new IndexShardRoutingTable(shardId, Collections.unmodifiableList(shardRoutings)); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index badf70a191e1c..f8861f3a0404b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -619,6 +619,8 @@ public ShardRouting next() { /** * Initializes the current unassigned shard and moves it from the unassigned list. + * + * If a primary is initalized, it's term is incremented. */ public void initialize(String nodeId, long version, long expectedShardSize) { innerRemove(); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java index fbabacd79fde3..ee48b5859b4ee 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java @@ -22,7 +22,6 @@ import com.carrotsearch.hppc.IntSet; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.Diffable; import org.elasticsearch.cluster.DiffableUtils; @@ -37,12 +36,7 @@ import org.elasticsearch.index.shard.ShardNotFoundException; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.function.Predicate; /** @@ -558,7 +552,7 @@ public RoutingTable build() { } // normalize the versions right before we build it... for (ObjectCursor indexRoutingTable : indicesRouting.values()) { - indicesRouting.put(indexRoutingTable.value.index(), indexRoutingTable.value.normalizeVersions()); + indicesRouting.put(indexRoutingTable.value.index(), indexRoutingTable.value.normalizeVersionsAndPrimaryTerms()); } RoutingTable table = new RoutingTable(version, indicesRouting.build()); indicesRouting = null; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java b/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java index 8dd71e3fba518..fbeb8a548ba5b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java @@ -47,6 +47,7 @@ public final class ShardRouting implements Streamable, ToXContent { private String currentNodeId; private String relocatingNodeId; private boolean primary; + private long primaryTerm; private ShardRoutingState state; private long version; private RestoreSource restoreSource; @@ -62,11 +63,15 @@ private ShardRouting() { } public ShardRouting(ShardRouting copy) { - this(copy, copy.version()); + this(copy, copy.version(), copy.primaryTerm()); } public ShardRouting(ShardRouting copy, long version) { - this(copy.index(), copy.id(), copy.currentNodeId(), copy.relocatingNodeId(), copy.restoreSource(), copy.primary(), copy.state(), version, copy.unassignedInfo(), copy.allocationId(), true, copy.getExpectedShardSize()); + this(copy, version, copy.primaryTerm()); + } + + public ShardRouting(ShardRouting copy, long version, long primaryTerm) { + this(copy.index(), copy.id(), copy.currentNodeId(), copy.relocatingNodeId(), copy.restoreSource(), primaryTerm, copy.primary(), copy.state(), version, copy.unassignedInfo(), copy.allocationId(), true, copy.getExpectedShardSize()); } /** @@ -74,12 +79,13 @@ public ShardRouting(ShardRouting copy, long version) { * by either this class or tests. Visible for testing. */ ShardRouting(String index, int shardId, String currentNodeId, - String relocatingNodeId, RestoreSource restoreSource, boolean primary, ShardRoutingState state, long version, + String relocatingNodeId, RestoreSource restoreSource, long primaryTerm, boolean primary, ShardRoutingState state, long version, UnassignedInfo unassignedInfo, AllocationId allocationId, boolean internal, long expectedShardSize) { this.index = index; this.shardId = shardId; this.currentNodeId = currentNodeId; this.relocatingNodeId = relocatingNodeId; + this.primaryTerm = primaryTerm; this.primary = primary; this.state = state; this.asList = Collections.singletonList(this); @@ -103,8 +109,8 @@ public ShardRouting(ShardRouting copy, long version) { /** * Creates a new unassigned shard. */ - public static ShardRouting newUnassigned(String index, int shardId, RestoreSource restoreSource, boolean primary, UnassignedInfo unassignedInfo) { - return new ShardRouting(index, shardId, null, null, restoreSource, primary, ShardRoutingState.UNASSIGNED, 0, unassignedInfo, null, true, UNAVAILABLE_EXPECTED_SHARD_SIZE); + public static ShardRouting newUnassigned(String index, int shardId, RestoreSource restoreSource, long primaryTerm, boolean primary, UnassignedInfo unassignedInfo) { + return new ShardRouting(index, shardId, null, null, restoreSource, primaryTerm, primary, ShardRoutingState.UNASSIGNED, 0, unassignedInfo, null, true, UNAVAILABLE_EXPECTED_SHARD_SIZE); } /** @@ -214,7 +220,7 @@ public String relocatingNodeId() { */ public ShardRouting buildTargetRelocatingShard() { assert relocating(); - return new ShardRouting(index, shardId, relocatingNodeId, currentNodeId, restoreSource, primary, ShardRoutingState.INITIALIZING, version, unassignedInfo, + return new ShardRouting(index, shardId, relocatingNodeId, currentNodeId, restoreSource, primaryTerm, primary, ShardRoutingState.INITIALIZING, version, unassignedInfo, AllocationId.newTargetRelocation(allocationId), true, expectedShardSize); } @@ -249,6 +255,16 @@ public boolean primary() { return this.primary; } + /** + * Returns the term of the current primary shard for this shard. + * The term is incremented with every primary promotion/initial assignment. + * + * See {@link org.elasticsearch.cluster.metadata.IndexMetaData#primaryTerm(int)} for more info. + */ + public long primaryTerm() { + return this.primaryTerm; + } + /** * The shard state. */ @@ -318,6 +334,7 @@ public void readFromThin(StreamInput in) throws IOException { } primary = in.readBoolean(); + primaryTerm = in.readVLong(); state = ShardRoutingState.fromValue(in.readByte()); restoreSource = RestoreSource.readOptionalRestoreSource(in); @@ -363,6 +380,7 @@ public void writeToThin(StreamOutput out) throws IOException { } out.writeBoolean(primary); + out.writeVLong(primaryTerm); out.writeByte(state.value()); if (restoreSource != null) { @@ -420,7 +438,7 @@ void moveToUnassigned(UnassignedInfo unassignedInfo) { } /** - * Initializes an unassigned shard on a node. + * Initializes an unassigned shard on a node. If the shard is primary, it's term is incremented. */ void initialize(String nodeId, long expectedShardSize) { ensureNotFrozen(); @@ -430,6 +448,9 @@ void initialize(String nodeId, long expectedShardSize) { state = ShardRoutingState.INITIALIZING; currentNodeId = nodeId; allocationId = AllocationId.newInitializing(); + if (primary) { + primaryTerm++; + } this.expectedShardSize = expectedShardSize; } @@ -507,6 +528,7 @@ void moveToPrimary() { throw new IllegalShardRoutingStateException(this, "Already primary, can't move to primary"); } primary = true; + primaryTerm++; } /** @@ -569,6 +591,9 @@ public boolean isRelocationTargetOf(ShardRouting other) { assert b == false || this.primary == other.primary : "ShardRouting is a relocation target but primary flag is different. This [" + this + "], target [" + other + "]"; + assert b == false || this.primaryTerm == other.primaryTerm : + "ShardRouting is a relocation target but primary term is different. This [" + this + "], target [" + other + "]"; + return b; } @@ -596,10 +621,13 @@ public boolean isRelocationSourceOf(ShardRouting other) { assert b == false || this.primary == other.primary : "ShardRouting is a relocation source but primary flag is different. This [" + this + "], target [" + other + "]"; + assert b == false || this.primaryTerm == other.primaryTerm : + "ShardRouting is a relocation source but primary term is different. This [" + this + "], target [" + other + "]"; + return b; } - /** returns true if the current routing is identical to the other routing in all but meta fields, i.e., version and unassigned info */ + /** returns true if the current routing is identical to the other routing in all but meta fields, i.e., version, primary term and unassigned info */ public boolean equalsIgnoringMetaData(ShardRouting other) { if (primary != other.primary) { return false; @@ -644,6 +672,9 @@ public boolean equals(Object o) { if (unassignedInfo != null ? !unassignedInfo.equals(that.unassignedInfo) : that.unassignedInfo != null) { return false; } + if (primaryTerm != that.primaryTerm) { + return false; + } return equalsIgnoringMetaData(that); } @@ -660,6 +691,7 @@ public int hashCode() { result = 31 * result + (currentNodeId != null ? currentNodeId.hashCode() : 0); result = 31 * result + (relocatingNodeId != null ? relocatingNodeId.hashCode() : 0); result = 31 * result + (primary ? 1 : 0); + result = 31 * result + Long.hashCode(primaryTerm); result = 31 * result + (state != null ? state.hashCode() : 0); result = 31 * result + Long.hashCode(version); result = 31 * result + (restoreSource != null ? restoreSource.hashCode() : 0); @@ -689,6 +721,7 @@ public String shortSummary() { sb.append("[R]"); } sb.append(", v[").append(version).append("]"); + sb.append(", t[").append(primaryTerm).append("]"); if (this.restoreSource != null) { sb.append(", restoring[" + restoreSource + "]"); } @@ -710,6 +743,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject() .field("state", state()) .field("primary", primary()) + .field("primary_term", primaryTerm()) .field("node", currentNodeId()) .field("relocating_node", relocatingNodeId()) .field("shard", shardId().id()) diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index feafb76a5f297..f7843aeef2e85 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -27,14 +27,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.AllocationId; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.RoutingNode; -import org.elasticsearch.cluster.routing.RoutingNodes; -import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.cluster.routing.*; import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -43,11 +36,7 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.Set; +import java.util.*; import java.util.function.Function; import java.util.stream.Collectors; @@ -98,21 +87,22 @@ public RoutingAllocation.Result applyStartedShards(ClusterState clusterState, Li String startedShardsAsString = firstListElementsToCommaDelimitedString(startedShards, s -> s.shardId().toString()); logClusterHealthStateChange( - new ClusterStateHealth(clusterState), - new ClusterStateHealth(clusterState.metaData(), result.routingTable()), - "shards started [" + startedShardsAsString + "] ..." + new ClusterStateHealth(clusterState), + new ClusterStateHealth(clusterState.metaData(), result.routingTable()), + "shards started [" + startedShardsAsString + "] ..." ); return result; - } + } protected RoutingAllocation.Result buildChangedResult(MetaData metaData, RoutingNodes routingNodes) { return buildChangedResult(metaData, routingNodes, new RoutingExplanations()); } + protected RoutingAllocation.Result buildChangedResult(MetaData metaData, RoutingNodes routingNodes, RoutingExplanations explanations) { final RoutingTable routingTable = new RoutingTable.Builder().updateNodes(routingNodes).build(); - MetaData newMetaData = updateMetaDataWithRoutingTable(metaData,routingTable); + MetaData newMetaData = updateMetaDataWithRoutingTable(metaData, routingTable); return new RoutingAllocation.Result(true, routingTable.validateRaiseException(newMetaData), newMetaData, explanations); } @@ -133,11 +123,13 @@ static MetaData updateMetaDataWithRoutingTable(MetaData currentMetaData, Routing } IndexMetaData.Builder indexMetaDataBuilder = null; for (IndexShardRoutingTable shardRoutings : indexRoutingTable) { + + // update activeAllocationIds Set activeAllocationIds = shardRoutings.activeShards().stream() - .map(ShardRouting::allocationId) - .filter(Objects::nonNull) - .map(AllocationId::getId) - .collect(Collectors.toSet()); + .map(ShardRouting::allocationId) + .filter(Objects::nonNull) + .map(AllocationId::getId) + .collect(Collectors.toSet()); // only update active allocation ids if there is an active shard if (activeAllocationIds.isEmpty() == false) { // get currently stored allocation ids @@ -146,10 +138,25 @@ static MetaData updateMetaDataWithRoutingTable(MetaData currentMetaData, Routing if (indexMetaDataBuilder == null) { indexMetaDataBuilder = IndexMetaData.builder(indexMetaData); } - indexMetaDataBuilder.putActiveAllocationIds(shardRoutings.shardId().id(), activeAllocationIds); } } + + // update primary terms + final ShardRouting primary = shardRoutings.primaryShard(); + if (primary == null) { + throw new IllegalStateException("missing primary shard for " + shardRoutings.shardId()); + } + final int shardId = primary.shardId().id(); + if (primary.primaryTerm() != indexMetaData.primaryTerm(shardId)) { + assert primary.primaryTerm() > indexMetaData.primaryTerm(shardId) : + "primary term should only increase. Index primary term [" + + indexMetaData.primaryTerm(shardId) + "] but primary routing is " + primary; + if (indexMetaDataBuilder == null) { + indexMetaDataBuilder = IndexMetaData.builder(indexMetaData); + } + indexMetaDataBuilder.primaryTerm(shardId, primary.primaryTerm()); + } } if (indexMetaDataBuilder != null) { if (metaDataBuilder == null) { @@ -182,7 +189,7 @@ public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, Lis boolean changed = false; for (FailedRerouteAllocation.FailedShard failedShard : failedShards) { changed |= applyFailedShard(allocation, failedShard.shard, true, new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, failedShard.message, failedShard.failure, - System.nanoTime(), System.currentTimeMillis())); + System.nanoTime(), System.currentTimeMillis())); } if (!changed) { return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData()); @@ -192,9 +199,9 @@ public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, Lis final RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes); String failedShardsAsString = firstListElementsToCommaDelimitedString(failedShards, s -> s.shard.shardId().toString()); logClusterHealthStateChange( - new ClusterStateHealth(clusterState), - new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()), - "shards failed [" + failedShardsAsString + "] ..." + new ClusterStateHealth(clusterState), + new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()), + "shards failed [" + failedShardsAsString + "] ..." ); return result; } @@ -210,10 +217,10 @@ public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, Lis private String firstListElementsToCommaDelimitedString(List elements, Function formatter) { final int maxNumberOfElements = 10; return elements - .stream() - .limit(maxNumberOfElements) - .map(formatter) - .collect(Collectors.joining(", ")); + .stream() + .limit(maxNumberOfElements) + .map(formatter) + .collect(Collectors.joining(", ")); } public RoutingAllocation.Result reroute(ClusterState clusterState, AllocationCommands commands) { @@ -238,13 +245,14 @@ public RoutingAllocation.Result reroute(ClusterState clusterState, AllocationCom reroute(allocation); RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes, explanations); logClusterHealthStateChange( - new ClusterStateHealth(clusterState), - new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()), - "reroute commands" + new ClusterStateHealth(clusterState), + new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()), + "reroute commands" ); return result; } + /** * Reroutes the routing table based on the live nodes. *

@@ -270,9 +278,9 @@ protected RoutingAllocation.Result reroute(ClusterState clusterState, String rea } RoutingAllocation.Result result = buildChangedResult(clusterState.metaData(), routingNodes); logClusterHealthStateChange( - new ClusterStateHealth(clusterState), - new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()), - reason + new ClusterStateHealth(clusterState), + new ClusterStateHealth(clusterState.getMetaData(), result.routingTable()), + reason ); return result; } @@ -384,8 +392,8 @@ private boolean electPrimariesAndUnassignedDanglingReplicas(RoutingAllocation al } for (ShardRouting shardToFail : shardsToFail) { changed |= applyFailedShard(allocation, shardToFail, false, - new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, "primary failed while replica initializing", - null, allocation.getCurrentNanoTime(), System.currentTimeMillis())); + new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, "primary failed while replica initializing", + null, allocation.getCurrentNanoTime(), System.currentTimeMillis())); } // now, go over and elect a new primary if possible, not, from this code block on, if one is elected, @@ -447,7 +455,7 @@ private boolean deassociateDeadNodes(RoutingAllocation allocation) { // now, go over all the shards routing on the node, and fail them for (ShardRouting shardRouting : node.copyShards()) { UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left[" + node.nodeId() + "]", null, - allocation.getCurrentNanoTime(), System.currentTimeMillis()); + allocation.getCurrentNanoTime(), System.currentTimeMillis()); applyFailedShard(allocation, shardRouting, false, unassignedInfo); } // its a dead node, remove it, note, its important to remove it *after* we apply failed shard diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java index 4e6ba0fb5ada0..fa9841173d270 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java @@ -39,12 +39,12 @@ /** * The {@link RoutingAllocation} keep the state of the current allocation * of shards and holds the {@link AllocationDeciders} which are responsible - * for the current routing state. + * for the current routing state. */ public class RoutingAllocation { /** - * this class is used to describe results of a {@link RoutingAllocation} + * this class is used to describe results of a {@link RoutingAllocation} */ public static class Result { @@ -58,9 +58,10 @@ public static class Result { /** * Creates a new {@link RoutingAllocation.Result} - * @param changed a flag to determine whether the actual {@link RoutingTable} has been changed + * + * @param changed a flag to determine whether the actual {@link RoutingTable} has been changed * @param routingTable the {@link RoutingTable} this Result references - * @param metaData the {@link MetaData} this Result references + * @param metaData the {@link MetaData} this Result references */ public Result(boolean changed, RoutingTable routingTable, MetaData metaData) { this.changed = changed; @@ -70,9 +71,10 @@ public Result(boolean changed, RoutingTable routingTable, MetaData metaData) { /** * Creates a new {@link RoutingAllocation.Result} - * @param changed a flag to determine whether the actual {@link RoutingTable} has been changed + * + * @param changed a flag to determine whether the actual {@link RoutingTable} has been changed * @param routingTable the {@link RoutingTable} this Result references - * @param metaData the {@link MetaData} this Result references + * @param metaData the {@link MetaData} this Result references * @param explanations Explanation for the reroute actions */ public Result(boolean changed, RoutingTable routingTable, MetaData metaData, RoutingExplanations explanations) { @@ -82,7 +84,9 @@ public Result(boolean changed, RoutingTable routingTable, MetaData metaData, Rou this.explanations = explanations; } - /** determine whether the actual {@link RoutingTable} has been changed + /** + * determine whether the actual {@link RoutingTable} has been changed + * * @return true if the {@link RoutingTable} has been changed by allocation. Otherwise false */ public boolean changed() { @@ -91,6 +95,7 @@ public boolean changed() { /** * Get the {@link MetaData} referenced by this result + * * @return referenced {@link MetaData} */ public MetaData metaData() { @@ -99,6 +104,7 @@ public MetaData metaData() { /** * Get the {@link RoutingTable} referenced by this result + * * @return referenced {@link RoutingTable} */ public RoutingTable routingTable() { @@ -107,6 +113,7 @@ public RoutingTable routingTable() { /** * Get the explanation of this result + * * @return explanation */ public RoutingExplanations explanations() { @@ -137,9 +144,10 @@ public RoutingExplanations explanations() { /** * Creates a new {@link RoutingAllocation} - * @param deciders {@link AllocationDeciders} to used to make decisions for routing allocations - * @param routingNodes Routing nodes in the current cluster - * @param nodes TODO: Documentation + * + * @param deciders {@link AllocationDeciders} to used to make decisions for routing allocations + * @param routingNodes Routing nodes in the current cluster + * @param nodes TODO: Documentation * @param currentNanoTime the nano time to use for all delay allocation calculation (typically {@link System#nanoTime()}) */ public RoutingAllocation(AllocationDeciders deciders, RoutingNodes routingNodes, DiscoveryNodes nodes, ClusterInfo clusterInfo, long currentNanoTime) { @@ -157,6 +165,7 @@ public long getCurrentNanoTime() { /** * Get {@link AllocationDeciders} used for allocation + * * @return {@link AllocationDeciders} used for allocation */ public AllocationDeciders deciders() { @@ -165,6 +174,7 @@ public AllocationDeciders deciders() { /** * Get routing table of current nodes + * * @return current routing table */ public RoutingTable routingTable() { @@ -173,6 +183,7 @@ public RoutingTable routingTable() { /** * Get current routing nodes + * * @return routing nodes */ public RoutingNodes routingNodes() { @@ -181,6 +192,7 @@ public RoutingNodes routingNodes() { /** * Get metadata of routing nodes + * * @return Metadata of routing nodes */ public MetaData metaData() { @@ -189,6 +201,7 @@ public MetaData metaData() { /** * Get discovery nodes in current routing + * * @return discovery nodes */ public DiscoveryNodes nodes() { @@ -201,6 +214,7 @@ public ClusterInfo clusterInfo() { /** * Get explanations of current routing + * * @return explanation of routing */ public AllocationExplanation explanation() { @@ -257,10 +271,11 @@ public Set getIgnoreNodes(ShardId shardId) { /** * Create a routing decision, including the reason if the debug flag is * turned on - * @param decision decision whether to allow/deny allocation + * + * @param decision decision whether to allow/deny allocation * @param deciderLabel a human readable label for the AllocationDecider - * @param reason a format string explanation of the decision - * @param params format string parameters + * @param reason a format string explanation of the decision + * @param params format string parameters */ public Decision decision(Decision decision, String deciderLabel, String reason, Object... params) { if (debugDecision()) { diff --git a/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java b/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java index d4b158618463b..d8393d07f5485 100644 --- a/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java +++ b/core/src/main/java/org/elasticsearch/cluster/service/InternalClusterService.java @@ -293,7 +293,8 @@ public void submitStateUpdateTask(final String source, final T task, updateTasksExecutor.execute(updateTask, threadPool.scheduler(), config.timeout(), () -> threadPool.generic().execute(() -> { if (updateTask.processed.getAndSet(true) == false) { listener.onFailure(source, new ProcessClusterEventTimeoutException(config.timeout(), source)); - }})); + } + })); } else { updateTasksExecutor.execute(updateTask); } @@ -420,7 +421,7 @@ void runTasksForExecutor(ClusterStateTaskExecutor executor) { for (UpdateTask updateTask : toExecute) { assert batchResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask.task.toString(); final ClusterStateTaskExecutor.TaskResult executionResult = - batchResult.executionResults.get(updateTask.task); + batchResult.executionResults.get(updateTask.task); executionResult.handle(() -> proccessedListeners.add(updateTask), ex -> updateTask.listener.onFailure(updateTask.source, ex)); } @@ -523,6 +524,7 @@ void runTasksForExecutor(ClusterStateTaskExecutor executor) { logger.debug("set local cluster state to version {}", newClusterState.version()); for (ClusterStateListener listener : preAppliedListeners) { try { + logger.trace("calling [{}] with change to version [{}]", listener, newClusterState.version()); listener.clusterChanged(clusterChangedEvent); } catch (Exception ex) { logger.warn("failed to notify ClusterStateListener", ex); @@ -541,6 +543,7 @@ void runTasksForExecutor(ClusterStateTaskExecutor executor) { for (ClusterStateListener listener : postAppliedListeners) { try { + logger.trace("calling [{}] with change to version [{}]", listener, newClusterState.version()); listener.clusterChanged(clusterChangedEvent); } catch (Exception ex) { logger.warn("failed to notify ClusterStateListener", ex); diff --git a/core/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/core/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java index 20859e2716a8d..4a47aa65f6754 100644 --- a/core/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java +++ b/core/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java @@ -40,19 +40,9 @@ import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import java.io.ByteArrayInputStream; -import java.io.EOFException; -import java.io.FileNotFoundException; -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; +import java.io.*; import java.nio.file.NoSuchFileException; -import java.util.ArrayList; -import java.util.Date; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.function.Supplier; import static org.elasticsearch.ElasticsearchException.readException; @@ -543,6 +533,17 @@ public T readOptionalStreamable(Supplier supplier) thr } } + /** + * Serializes a potential null value. + */ + public T readOptionalStreamableReader(StreamableReader streamableReader) throws IOException { + if (readBoolean()) { + return streamableReader.readFrom(this); + } else { + return null; + } + } + public T readThrowable() throws IOException { if (readBoolean()) { int key = readVInt(); diff --git a/core/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/core/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java index 5f1e7623d2822..d5e96d9b0dbdb 100644 --- a/core/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java +++ b/core/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java @@ -503,6 +503,18 @@ public void writeOptionalStreamable(@Nullable Streamable streamable) throws IOEx } } + /** + * Serializes a potential null value. + */ + public void writeOptionalWritable(@Nullable Writeable writeable) throws IOException { + if (writeable != null) { + writeBoolean(true); + writeable.writeTo(this); + } else { + writeBoolean(false); + } + } + public void writeThrowable(Throwable throwable) throws IOException { if (throwable == null) { writeBoolean(false); diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/EsRejectedExecutionException.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/EsRejectedExecutionException.java index d75b3ffa8c264..8033750d1d24e 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/EsRejectedExecutionException.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/EsRejectedExecutionException.java @@ -31,13 +31,17 @@ public class EsRejectedExecutionException extends ElasticsearchException { private final boolean isExecutorShutdown; - public EsRejectedExecutionException(String message, boolean isExecutorShutdown) { - super(message); + public EsRejectedExecutionException(String message, boolean isExecutorShutdown, Object... args) { + super(message, args); this.isExecutorShutdown = isExecutorShutdown; } - public EsRejectedExecutionException(String message) { - this(message, false); + public EsRejectedExecutionException(String message, Object... args) { + this(message, false, args); + } + + public EsRejectedExecutionException(String message, boolean isExecutorShutdown) { + this(message, isExecutorShutdown, new Object[0]); } public EsRejectedExecutionException() { diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 0504fdfa99694..2bd714c2d26b3 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -45,6 +45,8 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -324,6 +326,9 @@ public CommitStats commitStats() { return new CommitStats(getLastCommittedSegmentInfos()); } + /** get sequence number related stats */ + public abstract SeqNoStats seqNoStats(); + /** * Read the last segments info from the commit pointed to by the searcher manager */ @@ -577,7 +582,8 @@ public interface EventListener { /** * Called when a fatal exception occurred */ - default void onFailedEngine(String reason, @Nullable Throwable t) {} + default void onFailedEngine(String reason, @Nullable Throwable t) { + } } public static class Searcher implements Releasable { @@ -602,7 +608,7 @@ public IndexReader reader() { } public DirectoryReader getDirectoryReader() { - if (reader() instanceof DirectoryReader) { + if (reader() instanceof DirectoryReader) { return (DirectoryReader) reader(); } throw new IllegalStateException("Can't use " + reader().getClass() + " as a directory reader"); @@ -621,14 +627,18 @@ public void close() { public static abstract class Operation { private final Term uid; private long version; + private long seqNo; private final VersionType versionType; private final Origin origin; private Translog.Location location; private final long startTime; private long endTime; - public Operation(Term uid, long version, VersionType versionType, Origin origin, long startTime) { + public Operation(Term uid, long seqNo, long version, VersionType versionType, Origin origin, long startTime) { this.uid = uid; + assert origin != Origin.PRIMARY || seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO : "seqNo should not be set when origin is PRIMARY"; + assert origin == Origin.PRIMARY || seqNo >= 0 : "seqNo should be set when origin is not PRIMARY"; + this.seqNo = seqNo; this.version = version; this.versionType = versionType; this.origin = origin; @@ -657,6 +667,14 @@ public void updateVersion(long version) { this.version = version; } + public long seqNo() { + return seqNo; + } + + public void updateSeqNo(long seqNo) { + this.seqNo = seqNo; + } + public void setTranslogLocation(Translog.Location location) { this.location = location; } @@ -692,8 +710,8 @@ public static class Index extends Operation { private final ParsedDocument doc; - public Index(Term uid, ParsedDocument doc, long version, VersionType versionType, Origin origin, long startTime) { - super(uid, version, versionType, origin, startTime); + public Index(Term uid, ParsedDocument doc, long seqNo, long version, VersionType versionType, Origin origin, long startTime) { + super(uid, seqNo, version, versionType, origin, startTime); this.doc = doc; } @@ -702,7 +720,7 @@ public Index(Term uid, ParsedDocument doc) { } public Index(Term uid, ParsedDocument doc, long version) { - this(uid, doc, version, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime()); + this(uid, doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime()); } public ParsedDocument parsedDoc() { @@ -735,6 +753,12 @@ public void updateVersion(long version) { this.doc.version().setLongValue(version); } + @Override + public void updateSeqNo(long seqNo) { + super.updateSeqNo(seqNo); + this.doc.seqNo().setLongValue(seqNo); + } + public String parent() { return this.doc.parent(); } @@ -753,19 +777,15 @@ public static class Delete extends Operation { private final String id; private boolean found; - public Delete(String type, String id, Term uid, long version, VersionType versionType, Origin origin, long startTime, boolean found) { - super(uid, version, versionType, origin, startTime); + public Delete(String type, String id, Term uid, long seqNo, long version, VersionType versionType, Origin origin, long startTime, boolean found) { + super(uid, seqNo, version, versionType, origin, startTime); this.type = type; this.id = id; this.found = found; } public Delete(String type, String id, Term uid) { - this(type, id, uid, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), false); - } - - public Delete(Delete template, VersionType versionType) { - this(template.type(), template.id(), template.uid(), template.version(), versionType, template.origin(), template.startTime(), template.found()); + this(type, id, uid, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, VersionType.INTERNAL, Origin.PRIMARY, System.nanoTime(), false); } public String type() { @@ -1060,6 +1080,7 @@ public void onSettingsChanged() { * Returns the timestamp of the last write in nanoseconds. * Note: this time might not be absolutely accurate since the {@link Operation#startTime()} is used which might be * slightly inaccurate. + * * @see System#nanoTime() * @see Operation#startTime() */ @@ -1069,12 +1090,14 @@ public long getLastWriteNanos() { /** * Called for each new opened engine searcher to warm new segments + * * @see EngineConfig#getWarmer() */ public interface Warmer { /** * Called once a new Searcher is opened. - * @param searcher the searcer to warm + * + * @param searcher the searcer to warm * @param isTopLevelReader true iff the searcher is build from a top-level reader. * Otherwise the searcher might be build from a leaf reader to warm in isolation */ diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index c5da8e83b3db8..a7e2e9b8854d8 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -73,6 +73,7 @@ public final class EngineConfig { private final QueryCache queryCache; private final QueryCachingPolicy queryCachingPolicy; + /** * Index setting for compound file on flush. This setting is realtime updateable. */ @@ -123,7 +124,8 @@ public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService this.indexSettings = indexSettings; this.threadPool = threadPool; this.indexingService = indexingService; - this.warmer = warmer == null ? (a,b) -> {} : warmer; + this.warmer = warmer == null ? (a, b) -> { + } : warmer; this.store = store; this.deletionPolicy = deletionPolicy; this.mergePolicy = mergePolicy; @@ -226,9 +228,9 @@ public long getGcDeletesInMillis() { * Returns true iff delete garbage collection in the engine should be enabled. This setting is updateable * in realtime and forces a volatile read. Consumers can safely read this value directly go fetch it's latest value. The default is true *

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

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

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

*/ public Codec getCodec() { @@ -259,7 +261,6 @@ public ThreadPool getThreadPool() { * * @see org.elasticsearch.index.indexing.ShardIndexingService#postIndex(Engine.Index) * @see org.elasticsearch.index.indexing.ShardIndexingService#preIndex(Engine.Index) - * */ public ShardIndexingService getIndexingService() { return indexingService; @@ -323,7 +324,9 @@ public IndexSettings getIndexSettings() { /** * Returns the engines shard ID */ - public ShardId getShardId() { return shardId; } + public ShardId getShardId() { + return shardId; + } /** * Returns the analyzer as the default analyzer in the engines {@link org.apache.lucene.index.IndexWriter} @@ -404,6 +407,7 @@ public boolean isCreate() { * should be automatically flushed. This is used to free up transient disk usage of potentially large segments that * are written after the engine became inactive from an indexing perspective. */ - public TimeValue getFlushMergesAfter() { return flushMergesAfter; } - + public TimeValue getFlushMergesAfter() { + return flushMergesAfter; + } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1404b61b8ec8e..488222487630f 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -50,6 +50,8 @@ import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; +import org.elasticsearch.index.seqno.SeqNoStats; +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 +104,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 +119,7 @@ public InternalEngine(EngineConfig engineConfig, boolean skipInitialTranslogReco this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis(); this.indexingService = engineConfig.getIndexingService(); this.warmer = engineConfig.getWarmer(); + seqNoService = new SequenceNumbersService(shardId, engineConfig.getIndexSettings()); mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings(), engineConfig.getMergeSchedulerConfig()); this.dirtyLocks = new Object[Runtime.getRuntime().availableProcessors() * 10]; // we multiply it to have enough... for (int i = 0; i < dirtyLocks.length; i++) { @@ -351,63 +356,71 @@ public boolean index(Index index) { private boolean innerIndex(Index index) throws IOException { synchronized (dirtyLock(index.uid())) { - lastWriteNanos = index.startTime(); - final long currentVersion; - final boolean deleted; - VersionValue versionValue = versionMap.getUnderLock(index.uid().bytes()); - if (versionValue == null) { - currentVersion = loadCurrentVersionFromIndex(index.uid()); - deleted = currentVersion == Versions.NOT_FOUND; - } else { - deleted = versionValue.delete(); - if (engineConfig.isEnableGcDeletes() && versionValue.delete() && (engineConfig.getThreadPool().estimatedTimeInMillis() - versionValue.time()) > engineConfig.getGcDeletesInMillis()) { - currentVersion = Versions.NOT_FOUND; // deleted, and GC + try { + lastWriteNanos = index.startTime(); + final long currentVersion; + final boolean deleted; + VersionValue versionValue = versionMap.getUnderLock(index.uid().bytes()); + if (versionValue == null) { + currentVersion = loadCurrentVersionFromIndex(index.uid()); + deleted = currentVersion == Versions.NOT_FOUND; } else { - currentVersion = versionValue.version(); + deleted = versionValue.delete(); + if (engineConfig.isEnableGcDeletes() && versionValue.delete() && (engineConfig.getThreadPool().estimatedTimeInMillis() - versionValue.time()) > engineConfig.getGcDeletesInMillis()) { + currentVersion = Versions.NOT_FOUND; // deleted, and GC + } else { + currentVersion = versionValue.version(); + } } - } - long expectedVersion = index.version(); - if (index.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) { - if (index.origin() == Operation.Origin.RECOVERY) { - return false; - } else { - throw new VersionConflictEngineException(shardId, index.type(), index.id(), - index.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted)); + long expectedVersion = index.version(); + if (index.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) { + if (index.origin() == Operation.Origin.RECOVERY) { + return false; + } else { + throw new VersionConflictEngineException(shardId, index.type(), index.id(), + index.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted)); + } } - } - long updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); - - final boolean created; - index.updateVersion(updatedVersion); + long updatedVersion = index.versionType().updateVersion(currentVersion, expectedVersion); - if (currentVersion == Versions.NOT_FOUND) { - // document does not exists, we can optimize for create - created = true; - if (index.docs().size() > 1) { - indexWriter.addDocuments(index.docs()); - } else { - indexWriter.addDocument(index.docs().get(0)); + final boolean created; + index.updateVersion(updatedVersion); + if (index.origin() == Operation.Origin.PRIMARY) { + index.updateSeqNo(seqNoService.generateSeqNo()); } - } else { - if (versionValue != null) { - created = versionValue.delete(); // we have a delete which is not GC'ed... - } else { - created = false; - } - if (index.docs().size() > 1) { - indexWriter.updateDocuments(index.uid(), index.docs()); + if (currentVersion == Versions.NOT_FOUND) { + // document does not exists, we can optimize for create + created = true; + if (index.docs().size() > 1) { + indexWriter.addDocuments(index.docs()); + } else { + indexWriter.addDocument(index.docs().get(0)); + } } else { - indexWriter.updateDocument(index.uid(), index.docs().get(0)); + if (versionValue != null) { + created = versionValue.delete(); // we have a delete which is not GC'ed... + } else { + created = false; + } + if (index.docs().size() > 1) { + indexWriter.updateDocuments(index.uid(), index.docs()); + } else { + indexWriter.updateDocument(index.uid(), index.docs().get(0)); + } } - } - Translog.Location translogLocation = translog.add(new Translog.Index(index)); + Translog.Location translogLocation = translog.add(new Translog.Index(index)); - versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion, translogLocation)); - index.setTranslogLocation(translogLocation); + versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion, translogLocation)); + index.setTranslogLocation(translogLocation); - indexingService.postIndexUnderLock(index); - return created; + indexingService.postIndexUnderLock(index); + return created; + } finally { + if (index.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoService.markSeqNoAsCompleted(index.seqNo()); + } + } } } @@ -463,51 +476,62 @@ private void maybePruneDeletedTombstones() { private void innerDelete(Delete delete) throws IOException { synchronized (dirtyLock(delete.uid())) { - lastWriteNanos = delete.startTime(); - final long currentVersion; - final boolean deleted; - VersionValue versionValue = versionMap.getUnderLock(delete.uid().bytes()); - if (versionValue == null) { - currentVersion = loadCurrentVersionFromIndex(delete.uid()); - deleted = currentVersion == Versions.NOT_FOUND; - } else { - deleted = versionValue.delete(); - if (engineConfig.isEnableGcDeletes() && versionValue.delete() && (engineConfig.getThreadPool().estimatedTimeInMillis() - versionValue.time()) > engineConfig.getGcDeletesInMillis()) { - currentVersion = Versions.NOT_FOUND; // deleted, and GC + try { + lastWriteNanos = delete.startTime(); + final long currentVersion; + final boolean deleted; + VersionValue versionValue = versionMap.getUnderLock(delete.uid().bytes()); + if (versionValue == null) { + currentVersion = loadCurrentVersionFromIndex(delete.uid()); + deleted = currentVersion == Versions.NOT_FOUND; } else { - currentVersion = versionValue.version(); + deleted = versionValue.delete(); + if (engineConfig.isEnableGcDeletes() && versionValue.delete() && (engineConfig.getThreadPool().estimatedTimeInMillis() - versionValue.time()) > engineConfig.getGcDeletesInMillis()) { + currentVersion = Versions.NOT_FOUND; // deleted, and GC + } else { + currentVersion = versionValue.version(); + } } - } - long updatedVersion; - long expectedVersion = delete.version(); - if (delete.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) { - if (delete.origin() == Operation.Origin.RECOVERY) { - return; + long updatedVersion; + long expectedVersion = delete.version(); + if (delete.versionType().isVersionConflictForWrites(currentVersion, expectedVersion, deleted)) { + if (delete.origin() == Operation.Origin.RECOVERY) { + return; + } else { + throw new VersionConflictEngineException(shardId, delete.type(), delete.id(), + delete.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted)); + } + } + 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 + found = false; + } else if (versionValue != null && versionValue.delete()) { + // a "delete on delete", in this case, we still increment the version, log it, and return that version + found = false; } else { - throw new VersionConflictEngineException(shardId, delete.type(), delete.id(), - delete.versionType().explainConflictForWrites(currentVersion, expectedVersion, deleted)); + // we deleted a currently existing document + indexWriter.deleteDocuments(delete.uid()); + found = true; } - } - updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); - final boolean found; - if (currentVersion == Versions.NOT_FOUND) { - // doc does not exist and no prior deletes - found = false; - } else if (versionValue != null && versionValue.delete()) { - // a "delete on delete", in this case, we still increment the version, log it, and return that version - found = false; - } else { - // we deleted a currently existing document - indexWriter.deleteDocuments(delete.uid()); - found = true; - } - delete.updateVersion(updatedVersion, found); - Translog.Location translogLocation = translog.add(new Translog.Delete(delete)); - versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis(), translogLocation)); - delete.setTranslogLocation(translogLocation); - indexingService.postDeleteUnderLock(delete); + delete.updateVersion(updatedVersion, found); + Translog.Location translogLocation = translog.add(new Translog.Delete(delete)); + versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis(), translogLocation)); + delete.setTranslogLocation(translogLocation); + indexingService.postDeleteUnderLock(delete); + } finally { + if (delete.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoService.markSeqNoAsCompleted(delete.seqNo()); + } + } } } @@ -968,7 +992,7 @@ final static class SearchFactory extends EngineSearcherFactory { @Override public IndexSearcher newSearcher(IndexReader reader, IndexReader previousReader) throws IOException { IndexSearcher searcher = super.newSearcher(reader, previousReader); - if (reader instanceof LeafReader && isMergedSegment((LeafReader)reader)) { + if (reader instanceof LeafReader && isMergedSegment((LeafReader) reader)) { // we call newSearcher from the IndexReaderWarmer which warms segments during merging // in that case the reader is a LeafReader and all we need to do is to build a new Searcher // and return it since it does it's own warming for that particular reader. @@ -1158,4 +1182,9 @@ public void onSettingsChanged() { public MergeStats getMergeStats() { return mergeScheduler.stats(); } + + @Override + public SeqNoStats seqNoStats() { + return seqNoService.stats(); + } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index af3e0ae82a8ec..dd9ff4375e854 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; +import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.translog.Translog; import java.io.IOException; @@ -231,4 +232,9 @@ public long indexWriterRAMBytesUsed() { // No IndexWriter throw new UnsupportedOperationException("ShadowEngine has no IndexWriter"); } + + @Override + public SeqNoStats seqNoStats() { + throw new UnsupportedOperationException("ShadowEngine doesn't track sequence numbers"); + } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index b0ad972d57583..e0feed76092eb 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -185,7 +185,7 @@ private ParsedDocument innerParseDocument(SourceToParse source) throws MapperPar update = mapping.mappingUpdate(rootDynamicUpdate); } - ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(), + ParsedDocument doc = new ParsedDocument(context.uid(), context.version(), context.seqNo(), context.id(), context.type(), source.routing(), source.timestamp(), source.ttl(), context.docs(), context.source(), update).parent(source.parent()); // reset the context to free up memory context.reset(null, null, null); diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java index edf75621c1e3e..b68f7d85dc471 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParseContext.java @@ -323,6 +323,16 @@ public void version(Field version) { in.version(version); } + @Override + public Field seqNo() { + return in.seqNo(); + } + + @Override + public void seqNo(Field seqNo) { + in.seqNo(seqNo); + } + @Override public AllEntries allEntries() { return in.allEntries(); @@ -386,7 +396,7 @@ public static class InternalParseContext extends ParseContext { private String id; - private Field uid, version; + private Field uid, version, seqNo; private StringBuilder stringBuilder = new StringBuilder(); @@ -564,6 +574,17 @@ public void version(Field version) { this.version = version; } + @Override + public Field seqNo() { + return this.seqNo; + } + + @Override + public void seqNo(Field seqNo) { + this.seqNo = seqNo; + } + + @Override public AllEntries allEntries() { return this.allEntries; @@ -730,6 +751,10 @@ public boolean isWithinMultiFields() { public abstract void version(Field version); + public abstract Field seqNo(); + + public abstract void seqNo(Field seqNo); + public final boolean includeInAll(Boolean includeInAll, FieldMapper mapper) { return includeInAll(includeInAll, mapper.fieldType().indexOptions() != IndexOptions.NONE); } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java index ed8314c6f7de8..ea0b786354d46 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/ParsedDocument.java @@ -30,7 +30,7 @@ */ public class ParsedDocument { - private final Field uid, version; + private final Field uid, version, seqNo; private final String id; @@ -50,9 +50,10 @@ public class ParsedDocument { private String parent; - public ParsedDocument(Field uid, Field version, String id, String type, String routing, long timestamp, long ttl, List documents, BytesReference source, Mapping dynamicMappingsUpdate) { + public ParsedDocument(Field uid, Field version, Field seqNo, String id, String type, String routing, long timestamp, long ttl, List documents, BytesReference source, Mapping dynamicMappingsUpdate) { this.uid = uid; this.version = version; + this.seqNo = seqNo; this.id = id; this.type = type; this.routing = routing; @@ -71,6 +72,10 @@ public Field version() { return version; } + public Field seqNo() { + return seqNo; + } + public String id() { return this.id; } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/internal/SeqNoFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/internal/SeqNoFieldMapper.java new file mode 100644 index 0000000000000..64e6e8a1584d3 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/mapper/internal/SeqNoFieldMapper.java @@ -0,0 +1,156 @@ +/* + * 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 _seq_no field. */ +public class SeqNoFieldMapper extends MetadataFieldMapper { + + public static final String NAME = "_seq_no"; + public static final String CONTENT_TYPE = "_seq_no"; + + public static class Defaults { + + public static final String NAME = SeqNoFieldMapper.NAME; + public static final MappedFieldType FIELD_TYPE = new SeqNoFieldType(); + + static { + FIELD_TYPE.setNames(new MappedFieldType.Names(NAME)); + FIELD_TYPE.setDocValuesType(DocValuesType.NUMERIC); + FIELD_TYPE.setHasDocValues(true); + FIELD_TYPE.freeze(); + } + } + + public static class Builder extends MetadataFieldMapper.Builder { + + public Builder() { + super(Defaults.NAME, Defaults.FIELD_TYPE); + } + + @Override + public SeqNoFieldMapper build(BuilderContext context) { + return new SeqNoFieldMapper(context.indexSettings()); + } + } + + public static class TypeParser implements MetadataFieldMapper.TypeParser { + @Override + public MetadataFieldMapper.Builder parse(String name, Map node, ParserContext parserContext) throws MapperParsingException { + throw new MapperParsingException(NAME + " is not configurable"); + } + + @Override + public MetadataFieldMapper getDefault(Settings indexSettings, MappedFieldType fieldType, String typeName) { + return new SeqNoFieldMapper(indexSettings); + } + } + + static final class SeqNoFieldType extends MappedFieldType { + + public SeqNoFieldType() { + setFieldDataType(new FieldDataType("long")); + } + + protected SeqNoFieldType(SeqNoFieldType ref) { + super(ref); + } + + @Override + public MappedFieldType clone() { + return new SeqNoFieldType(this); + } + + @Override + public String typeName() { + return CONTENT_TYPE; + } + + @Override + public Long value(Object value) { + if (value == null || (value instanceof Long)) { + return (Long) value; + } else { + return Long.parseLong(value.toString()); + } + } + } + + public SeqNoFieldMapper(Settings indexSettings) { + super(NAME, Defaults.FIELD_TYPE, Defaults.FIELD_TYPE, indexSettings); + } + + @Override + public void preParse(ParseContext context) throws IOException { + super.parse(context); + } + + @Override + protected void parseCreateField(ParseContext context, List fields) throws IOException { + // see InternalEngine.updateVersion to see where the real version value is set + final Field seqNo = new NumericDocValuesField(NAME, SequenceNumbersService.UNASSIGNED_SEQ_NO); + context.seqNo(seqNo); + fields.add(seqNo); + } + + @Override + public Mapper parse(ParseContext context) throws IOException { + // _seqno added in preparse + return null; + } + + @Override + public void postParse(ParseContext context) throws IOException { + // In the case of nested docs, let's fill nested docs with seqNo=1 so that Lucene doesn't write a Bitset for documents + // that don't have the field. This is consistent with the default value for efficiency. + for (int i = 1; i < context.docs().size(); i++) { + final Document doc = context.docs().get(i); + doc.add(new NumericDocValuesField(NAME, 1L)); + } + } + + @Override + protected String contentType() { + return CONTENT_TYPE; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder; + } + + @Override + public void merge(Mapper mergeWith, MergeResult mergeResult) { + // nothing to do + } +} diff --git a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointService.java b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointService.java new file mode 100644 index 0000000000000..a1835171802c2 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointService.java @@ -0,0 +1,149 @@ +/* + * 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.apache.lucene.util.FixedBitSet; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.ShardId; + +import java.util.LinkedList; + +/** + * This class generates sequences numbers and keeps track of the so called local checkpoint - the highest number for which + * all previous seqNo have been processed (including) + */ +public class LocalCheckpointService extends AbstractIndexShardComponent { + + /** + * we keep a bit for each seq No that is still pending. to optimize allocation, we do so in multiple arrays + * allocating them on demand and cleaning up while completed. This setting controls the size of the arrays + */ + public static String SETTINGS_BIT_ARRAYS_SIZE = "index.seq_no.checkpoint.bit_arrays_size"; + + /** default value for {@link #SETTINGS_BIT_ARRAYS_SIZE} */ + final static int DEFAULT_BIT_ARRAYS_SIZE = 1024; + + + /** + * an order list of bit arrays representing pending seq nos. The list is "anchored" in {@link #firstSeqNoInProcessSeqNo} + * which marks the seqNo the fist bit in the first array corresponds to. + */ + final LinkedList processedSeqNo; + final int bitArraysSize; + long firstSeqNoInProcessSeqNo = 0; + + /** the current local checkpoint, i.e., all seqNo lower<= this number have been completed */ + volatile long checkpoint = -1; + + /** the next available seqNo - used for seqNo generation */ + volatile long nextSeqNo = 0; + + + public LocalCheckpointService(ShardId shardId, IndexSettings indexSettings) { + super(shardId, indexSettings); + bitArraysSize = indexSettings.getSettings().getAsInt(SETTINGS_BIT_ARRAYS_SIZE, DEFAULT_BIT_ARRAYS_SIZE); + if (bitArraysSize <= 0) { + throw new IllegalArgumentException("[" + SETTINGS_BIT_ARRAYS_SIZE + "] must be positive. got [" + bitArraysSize + "]"); + } + processedSeqNo = new LinkedList<>(); + } + + /** + * issue the next sequence number + **/ + public synchronized long generateSeqNo() { + return nextSeqNo++; + } + + /** + * marks the processing of the given seqNo have been completed + **/ + public synchronized void markSeqNoAsCompleted(long seqNo) { + // make sure we track highest seen seqNo + if (seqNo >= nextSeqNo) { + nextSeqNo = seqNo + 1; + } + if (seqNo <= checkpoint) { + // this is possible during recovery where we might replay an op that was also replicated + return; + } + FixedBitSet bitSet = getBitSetForSeqNo(seqNo); + int offset = seqNoToBitSetOffset(seqNo); + bitSet.set(offset); + if (seqNo == checkpoint + 1) { + updateCheckpoint(); + } + } + + /** get's the current check point */ + public long getCheckpoint() { + return checkpoint; + } + + /** get's the maximum seqno seen so far */ + public long getMaxSeqNo() { + return nextSeqNo - 1; + } + + /** + * moves the checkpoint to the last consecutively processed seqNo + * Note: this method assumes that the seqNo following the current checkpoint is processed. + */ + private void updateCheckpoint() { + assert Thread.holdsLock(this); + assert checkpoint - firstSeqNoInProcessSeqNo < bitArraysSize : "checkpoint to firstSeqNoInProcessSeqNo is larger then a bit set"; + assert getBitSetForSeqNo(checkpoint + 1).get(seqNoToBitSetOffset(checkpoint + 1)) : "updateCheckpoint is called but the bit following the checkpoint is not set"; + assert getBitSetForSeqNo(checkpoint + 1) == processedSeqNo.getFirst() : "checkpoint + 1 doesn't point to the first bit set"; + // keep it simple for now, get the checkpoint one by one. in the future we can optimize and read words + FixedBitSet current = processedSeqNo.getFirst(); + do { + checkpoint++; + // the checkpoint always falls in the first bit set or just before. If it falls + // on the last bit of the current bit set, we can clean it. + if (checkpoint == firstSeqNoInProcessSeqNo + bitArraysSize - 1) { + processedSeqNo.pop(); + firstSeqNoInProcessSeqNo += bitArraysSize; + assert checkpoint - firstSeqNoInProcessSeqNo < bitArraysSize; + current = processedSeqNo.peekFirst(); + } + } while (current != null && current.get(seqNoToBitSetOffset(checkpoint + 1))); + } + + /** + * gets the bit array for the give seqNo, allocating new ones if needed. + */ + private FixedBitSet getBitSetForSeqNo(long seqNo) { + assert Thread.holdsLock(this); + assert seqNo >= firstSeqNoInProcessSeqNo; + int bitSetOffset = ((int) (seqNo - firstSeqNoInProcessSeqNo)) / bitArraysSize; + while (bitSetOffset >= processedSeqNo.size()) { + processedSeqNo.add(new FixedBitSet(bitArraysSize)); + } + return processedSeqNo.get(bitSetOffset); + } + + + /** maps the given seqNo to a position in the bit set returned by {@link #getBitSetForSeqNo} */ + private int seqNoToBitSetOffset(long seqNo) { + assert Thread.holdsLock(this); + assert seqNo >= firstSeqNoInProcessSeqNo; + return ((int) (seqNo - firstSeqNoInProcessSeqNo)) % bitArraysSize; + } +} diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java new file mode 100644 index 0000000000000..99ffb6ad54794 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java @@ -0,0 +1,80 @@ +/* + * 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.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentBuilderString; + +import java.io.IOException; + +public class SeqNoStats implements ToXContent, Writeable { + + final long maxSeqNo; + final long localCheckpoint; + + public SeqNoStats(long maxSeqNo, long localCheckpoint) { + this.maxSeqNo = maxSeqNo; + this.localCheckpoint = localCheckpoint; + } + + public SeqNoStats(StreamInput in) throws IOException { + this(in.readZLong(), in.readZLong()); + } + + /** the maximum sequence number seen so far */ + public long getMaxSeqNo() { + return maxSeqNo; + } + + /** the maximum sequence number for which all previous operations (including) have been completed */ + public long getLocalCheckpoint() { + return localCheckpoint; + } + + @Override + public SeqNoStats readFrom(StreamInput in) throws IOException { + return new SeqNoStats(in.readLong(), in.readLong()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeZLong(maxSeqNo); + out.writeZLong(localCheckpoint); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(Fields.SEQ_NO); + builder.field(Fields.MAX_SEQ_NO, maxSeqNo); + builder.field(Fields.LOCAL_CHECKPOINT, localCheckpoint); + builder.endObject(); + return builder; + } + + + static final class Fields { + static final XContentBuilderString SEQ_NO = new XContentBuilderString("seq_no"); + static final XContentBuilderString MAX_SEQ_NO = new XContentBuilderString("max"); + static final XContentBuilderString LOCAL_CHECKPOINT = new XContentBuilderString("local_checkpoint"); + } +} diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java new file mode 100644 index 0000000000000..3ef8607c4c230 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -0,0 +1,59 @@ +/* + * 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; + +/** 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; + final LocalCheckpointService localCheckpointService; + + public SequenceNumbersService(ShardId shardId, IndexSettings indexSettings) { + super(shardId, indexSettings); + localCheckpointService = new LocalCheckpointService(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 localCheckpointService.generateSeqNo(); + } + + /** + * marks the given seqNo as completed. See {@link LocalCheckpointService#markSeqNoAsCompleted(long)} + * more details + */ + public void markSeqNoAsCompleted(long seqNo) { + localCheckpointService.markSeqNoAsCompleted(seqNo); + } + + /** + * Gets sequence number related stats + */ + public SeqNoStats stats() { + return new SeqNoStats(localCheckpointService.getMaxSeqNo(), localCheckpointService.getCheckpoint()); + } +} diff --git a/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java b/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java index 31c235e09ecd1..e632c0669f6cb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IllegalIndexShardStateException.java @@ -20,7 +20,6 @@ package org.elasticsearch.index.shard; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.rest.RestStatus; @@ -34,12 +33,12 @@ public class IllegalIndexShardStateException extends ElasticsearchException { private final IndexShardState currentState; - public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg) { - this(shardId, currentState, msg, null); + public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Object... args) { + this(shardId, currentState, msg, null, args); } - public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Throwable ex) { - super("CurrentState[" + currentState + "] " + msg, ex); + public IllegalIndexShardStateException(ShardId shardId, IndexShardState currentState, String msg, Throwable ex, Object... args) { + super("CurrentState[" + currentState + "] " + msg, ex, args); setShard(shardId); this.currentState = currentState; } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index c0bf924467399..5bc38e5616e64 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -79,6 +79,8 @@ 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.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.snapshots.IndexShardRepository; import org.elasticsearch.index.store.Store; @@ -232,6 +234,7 @@ public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, this.flushOnClose = settings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true); this.path = path; this.mergePolicyConfig = new MergePolicyConfig(logger, settings); + /* create engine config */ logger.debug("state: [CREATED]"); @@ -415,13 +418,22 @@ public IndexShardState markAsRecovering(String reason, RecoveryState recoverySta } } - public IndexShard relocated(String reason) throws IndexShardNotStartedException { + public IndexShard relocated(String reason) throws IndexShardNotStartedException, InterruptedException { synchronized (mutex) { if (state != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, state); } changeState(IndexShardState.RELOCATED, reason); + // nocommit: awful hack to work around delay replications being rejected by the primary term check. This is used to make sure all in flight operation are done + // before primary relocation is done. proper fix coming. + indexShardOperationCounter.decRef(); + } + + logger.info("waiting for op count to reach 0"); + while (indexShardOperationCounter.refCount() > 0) { + Thread.sleep(100); } + logger.info("{} waiting for op count reached 0. continuing..."); return this; } @@ -449,29 +461,33 @@ public Engine.Index prepareIndexOnPrimary(SourceToParse source, long version, Ve if (shardRouting.primary() == false) { throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary"); } - return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.PRIMARY); + return prepareIndex(docMapper(source.type()), source, SequenceNumbersService.UNASSIGNED_SEQ_NO, version, versionType, Engine.Operation.Origin.PRIMARY); } catch (Throwable t) { verifyNotClosed(t); throw t; } } - public Engine.Index prepareIndexOnReplica(SourceToParse source, long version, VersionType versionType) { + public Engine.Index prepareIndexOnReplica(SourceToParse source, long seqNo, long version, VersionType versionType) { try { - return prepareIndex(docMapper(source.type()), source, version, versionType, Engine.Operation.Origin.REPLICA); + 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 version, VersionType versionType, Engine.Operation.Origin origin) { + 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); } /** @@ -502,17 +518,22 @@ public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary"); } final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); - return prepareDelete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, Engine.Operation.Origin.PRIMARY); + 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 version, VersionType versionType) { + public Engine.Delete prepareDeleteOnReplica(String type, String id, long seqNo, long version, VersionType versionType) { + if (shardRouting.primary() && shardRouting.isRelocationTarget() == false) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a replica"); + } final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); - return prepareDelete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, Engine.Operation.Origin.REPLICA); + 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 version, VersionType versionType, Engine.Operation.Origin origin) { + 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, version, versionType, origin, startTime, false); + return new Engine.Delete(type, id, uid, seqNo, version, versionType, origin, startTime, false); } @@ -571,6 +592,15 @@ public CommitStats commitStats() { return engine == null ? null : engine.commitStats(); } + /** + * @return {@link SeqNoStats} if engine is open, otherwise null + */ + @Nullable + public SeqNoStats seqNoStats() { + Engine engine = getEngineOrNull(); + return engine == null ? null : engine.seqNoStats(); + } + public IndexingStats indexingStats(String... types) { return indexingService.stats(types); } @@ -769,8 +799,12 @@ public void close(String reason, boolean flushEngine) throws IOException { FutureUtils.cancel(mergeScheduleFuture); mergeScheduleFuture = null; } + // nocommit: done to temporary prevent operations on a relocated primary. Remove when properly fixed. + final boolean decOpCounter = state != IndexShardState.RELOCATED; changeState(IndexShardState.CLOSED, reason); - indexShardOperationCounter.decRef(); + if (decOpCounter) { + indexShardOperationCounter.decRef(); + } } finally { final Engine engine = this.currentEngineReference.getAndSet(null); try { @@ -1271,35 +1305,32 @@ public void run() { reschedule(); return; } - threadPool.executor(ThreadPool.Names.REFRESH).execute(new Runnable() { - @Override - public void run() { - try { - if (getEngine().refreshNeeded()) { - refresh("schedule"); - } - } catch (EngineClosedException e) { - // we are being closed, ignore - } catch (RefreshFailedEngineException e) { - if (e.getCause() instanceof InterruptedException) { - // ignore, we are being shutdown - } else if (e.getCause() instanceof ClosedByInterruptException) { - // ignore, we are being shutdown - } else if (e.getCause() instanceof ThreadInterruptedException) { - // ignore, we are being shutdown - } else { - if (state != IndexShardState.CLOSED) { - logger.warn("Failed to perform scheduled engine refresh", e); - } - } - } catch (Exception e) { + threadPool.executor(ThreadPool.Names.REFRESH).execute(() -> { + try { + if (getEngine().refreshNeeded()) { + refresh("schedule"); + } + } catch (EngineClosedException e) { + // we are being closed, ignore + } catch (RefreshFailedEngineException e) { + if (e.getCause() instanceof InterruptedException) { + // ignore, we are being shutdown + } else if (e.getCause() instanceof ClosedByInterruptException) { + // ignore, we are being shutdown + } else if (e.getCause() instanceof ThreadInterruptedException) { + // ignore, we are being shutdown + } else { if (state != IndexShardState.CLOSED) { logger.warn("Failed to perform scheduled engine refresh", e); } } - - reschedule(); + } catch (Exception e) { + if (state != IndexShardState.CLOSED) { + logger.warn("Failed to perform scheduled engine refresh", e); + } } + + reschedule(); }); } @@ -1515,7 +1546,29 @@ protected void alreadyClosed() { } } - public void incrementOperationCounter() { + /** + * increments the ongoing operations counter on a primary shard. Returns the primary term of this shard. + */ + public long incrementOperationCounterOnPrimary() { + if (shardRouting.primary() == false) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a primary"); + } + indexShardOperationCounter.incRef(); + return shardRouting.primaryTerm(); + } + + + /** + * 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 incrementOperationCounterOnReplica(long opPrimaryTerm) { + if (shardRouting.primaryTerm() > opPrimaryTerm) { + throw new IllegalIndexShardStateException(shardId, state, "operation term [{}] is too old (current [{}])", opPrimaryTerm, shardRouting.primaryTerm()); + } + if (shardRouting.primary() && shardRouting.isRelocationTarget() == false) { + throw new IllegalIndexShardStateException(shardId, state, "shard is not a replica"); + } indexShardOperationCounter.incRef(); } diff --git a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index 50a16fa1cee70..d99e2ccd0e803 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -18,8 +18,6 @@ */ package org.elasticsearch.index.shard; -import java.io.IOException; - import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.index.IndexSettings; @@ -31,10 +29,13 @@ import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.merge.MergeStats; +import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogStats; +import java.io.IOException; + /** * ShadowIndexShard extends {@link IndexShard} to add file synchronization * from the primary when a flush happens. It also ensures that a replica being @@ -67,6 +68,11 @@ public MergeStats mergeStats() { return new MergeStats(); } + @Override + public SeqNoStats seqNoStats() { + return null; + } + @Override public boolean canIndex() { return false; diff --git a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java index ac46f6725dec7..0c2c13c768183 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java +++ b/core/src/main/java/org/elasticsearch/index/shard/TranslogRecoveryPerformer.java @@ -128,7 +128,7 @@ public void performRecoveryOperation(Engine engine, Translog.Operation operation Translog.Index index = (Translog.Index) operation; Engine.Index engineIndex = IndexShard.prepareIndex(docMapper(index.type()), source(index.source()).type(index.type()).id(index.id()) .routing(index.routing()).parent(index.parent()).timestamp(index.timestamp()).ttl(index.ttl()), - index.version(), index.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY); + index.seqNo(), index.version(), index.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY); maybeAddMappingUpdate(engineIndex.type(), engineIndex.parsedDoc().dynamicMappingsUpdate(), engineIndex.id(), allowMappingUpdates); if (logger.isTraceEnabled()) { logger.trace("[translog] recover [index] op of [{}][{}]", index.type(), index.id()); @@ -141,8 +141,9 @@ public void performRecoveryOperation(Engine engine, Translog.Operation operation if (logger.isTraceEnabled()) { logger.trace("[translog] recover [delete] op of [{}][{}]", uid.type(), uid.id()); } - engine.delete(new Engine.Delete(uid.type(), uid.id(), delete.uid(), delete.version(), - delete.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY, System.nanoTime(), false)); + Engine.Delete engineDelete = IndexShard.prepareDelete(uid.type(), uid.id(), delete.uid(), delete.seqNo(), + delete.version(), delete.versionType().versionTypeForReplicationAndRecovery(), Engine.Operation.Origin.RECOVERY); + engine.delete(engineDelete); break; default: throw new IllegalStateException("No operation defined for [" + operation + "]"); diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index 35dd895bc2e97..f71573bf0ba0c 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -861,10 +861,11 @@ public Source(BytesReference source, String routing, String parent, long timesta } public static class Index implements Operation { - public static final int SERIALIZATION_FORMAT = 6; + public static final int SERIALIZATION_FORMAT = 7; private String id; private String type; + private long seqNo = -1; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; private BytesReference source; @@ -882,6 +883,7 @@ public Index(Engine.Index index) { this.source = index.source(); this.routing = index.routing(); this.parent = index.parent(); + this.seqNo = index.seqNo(); this.version = index.version(); this.timestamp = index.timestamp(); this.ttl = index.ttl(); @@ -892,6 +894,8 @@ public Index(String type, String id, byte[] source) { this.type = type; this.id = id; this.source = new BytesArray(source); + this.seqNo = 0; + this.version = 0; } @Override @@ -932,6 +936,10 @@ public BytesReference source() { return this.source; } + public long seqNo() { + return seqNo; + } + public long version() { return this.version; } @@ -974,6 +982,9 @@ public void readFrom(StreamInput in) throws IOException { if (version >= 6) { this.versionType = VersionType.fromValue(in.readByte()); } + if (version >= 7) { + this.seqNo = in.readVLong(); + } } catch (Exception e) { throw new ElasticsearchException("failed to read [" + type + "][" + id + "]", e); } @@ -1003,6 +1014,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(timestamp); out.writeLong(ttl); out.writeByte(versionType.getValue()); + out.writeVLong(seqNo); } @Override @@ -1017,6 +1029,7 @@ public boolean equals(Object o) { Index index = (Index) o; if (version != index.version || + seqNo != index.seqNo || timestamp != index.timestamp || ttl != index.ttl || id.equals(index.id) == false || @@ -1036,6 +1049,7 @@ public boolean equals(Object o) { public int hashCode() { int result = id.hashCode(); result = 31 * result + type.hashCode(); + result = 31 * result + Long.hashCode(seqNo); result = 31 * result + Long.hashCode(version); result = 31 * result + versionType.hashCode(); result = 31 * result + source.hashCode(); @@ -1056,9 +1070,10 @@ public String toString() { } public static class Delete implements Operation { - public static final int SERIALIZATION_FORMAT = 2; + public static final int SERIALIZATION_FORMAT = 3; private Term uid; + private long seqNo = -1L; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; @@ -1066,19 +1081,19 @@ public Delete() { } public Delete(Engine.Delete delete) { - this(delete.uid()); - this.version = delete.version(); - this.versionType = delete.versionType(); + this(delete.uid(), delete.seqNo(), delete.version(), delete.versionType()); } + /** utility for testing */ public Delete(Term uid) { - this.uid = uid; + this(uid, 0, 0, VersionType.EXTERNAL); } - public Delete(Term uid, long version, VersionType versionType) { + public Delete(Term uid, long seqNo, long version, VersionType versionType) { this.uid = uid; this.version = version; this.versionType = versionType; + this.seqNo = seqNo; } @Override @@ -1095,6 +1110,10 @@ public Term uid() { return this.uid; } + public long seqNo() { + return seqNo; + } + public long version() { return this.version; } @@ -1118,6 +1137,9 @@ public void readFrom(StreamInput in) throws IOException { if (version >= 2) { this.versionType = VersionType.fromValue(in.readByte()); } + if (version >= 3) { + this.seqNo = in.readVLong(); + } assert versionType.validateVersionForWrites(version); } @@ -1129,6 +1151,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(uid.text()); out.writeLong(version); out.writeByte(versionType.getValue()); + out.writeVLong(seqNo); } @Override @@ -1142,7 +1165,7 @@ public boolean equals(Object o) { Delete delete = (Delete) o; - return version == delete.version && + return version == delete.version && seqNo == delete.seqNo && uid.equals(delete.uid) && versionType == delete.versionType; } @@ -1150,6 +1173,7 @@ public boolean equals(Object o) { @Override public int hashCode() { int result = uid.hashCode(); + result = 31 * result + Long.hashCode(seqNo); result = 31 * result + Long.hashCode(version); result = 31 * result + versionType.hashCode(); return result; diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java index cdd7f050331bc..dd0b9b0182970 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesModule.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesModule.java @@ -167,6 +167,7 @@ private void registerBuiltInMetadataMappers() { registerMetadataMapper(TTLFieldMapper.NAME, new TTLFieldMapper.TypeParser()); registerMetadataMapper(VersionFieldMapper.NAME, new VersionFieldMapper.TypeParser()); registerMetadataMapper(ParentFieldMapper.NAME, new ParentFieldMapper.TypeParser()); + registerMetadataMapper(SeqNoFieldMapper.NAME, new SeqNoFieldMapper.TypeParser()); // _field_names is not registered here, see #getMapperRegistry: we need to register it // last so that it can see all other mappers, including those coming from plugins } diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index dead72aee8b4f..352ca12740516 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -198,7 +198,9 @@ public NodeIndicesStats stats(boolean includePrevious, CommonStatsFlags flags) { if (indexShard.routingEntry() == null) { continue; } - IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), new CommonStats(indexShard, flags), indexShard.commitStats()) }); + IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), + new ShardStats[]{new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), + new CommonStats(indexShard, flags), indexShard.commitStats(), indexShard.seqNoStats())}); if (!statsByShard.containsKey(indexService.index())) { statsByShard.put(indexService.index(), arrayAsArrayList(indexShardStats)); } else { diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 73193161d12dd..94713bc031d9c 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -40,7 +40,10 @@ import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.CancellableThreads.Interruptable; import org.elasticsearch.index.engine.RecoveryEngineException; -import org.elasticsearch.index.shard.*; +import org.elasticsearch.index.shard.IllegalIndexShardStateException; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardClosedException; +import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; @@ -218,7 +221,7 @@ public void phase1(final IndexCommit snapshot, final Translog.View translogView) totalSize += md.length(); } List phase1Files = new ArrayList<>(diff.different.size() + diff.missing.size()); - phase1Files.addAll(diff.different); + phase1Files.addAll(diff.different); phase1Files.addAll(diff.missing); for (StoreFileMetaData md : phase1Files) { if (request.metadataSnapshot().asMap().containsKey(md.name())) { @@ -319,7 +322,6 @@ public int compare(StoreFileMetaData o1, StoreFileMetaData o2) { } - protected void prepareTargetForTranslog(final Translog.View translogView) { StopWatch stopWatch = new StopWatch().start(); logger.trace("{} recovery [phase1] to {}: prepare remote engine for translog", request.shardId(), request.targetNode()); @@ -395,15 +397,17 @@ public void run() throws InterruptedException { } }); - - if (request.markAsRelocated()) { + if (request.markAsRelocated() || request.recoveryType() == RecoveryState.Type.RELOCATION) { // TODO what happens if the recovery process fails afterwards, we need to mark this back to started try { + // nocommit: awful hack to work around delay replications being rejected by the primary term check. proper fix coming. shard.relocated("to " + request.targetNode()); } catch (IllegalIndexShardStateException e) { // we can ignore this exception since, on the other node, when it moved to phase3 // it will also send shard started, which might cause the index shard we work against // to move be closed by the time we get to the the relocated method + } catch (InterruptedException e) { + throw new ElasticsearchException("interrupted while waiting for pending operation to finish on relocated primary", e); } } stopWatch.stop(); diff --git a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java index 536b73ba2b562..1bf3c46d9acd9 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java @@ -107,5 +107,4 @@ static final class Fields { static final XContentBuilderString ERRORS = new XContentBuilderString("errors"); static final XContentBuilderString TOOK = new XContentBuilderString("took"); } - } diff --git a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java index 310ce0a1248db..d55b09c37a265 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java @@ -36,7 +36,8 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.rest.RestRequest.Method.PUT; -import static org.elasticsearch.rest.RestStatus.*; +import static org.elasticsearch.rest.RestStatus.BAD_REQUEST; +import static org.elasticsearch.rest.RestStatus.CREATED; /** * @@ -84,7 +85,7 @@ public void handleRequest(final RestRequest request, final RestChannel channel, if (sOpType != null) { try { indexRequest.opType(IndexRequest.OpType.fromString(sOpType)); - } catch (IllegalArgumentException eia){ + } catch (IllegalArgumentException eia) { try { XContentBuilder builder = channel.newErrorBuilder(); channel.sendResponse(new BytesRestResponse(BAD_REQUEST, builder.startObject().field("error", eia.getMessage()).endObject())); diff --git a/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java b/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java index f59c329fbc3ac..c09998b9ef95e 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java @@ -29,13 +29,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; 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.rest.action.support.RestStatusToXContentListener; diff --git a/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java b/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java index df2362dd47bad..e842b571fa856 100644 --- a/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java +++ b/core/src/main/java/org/elasticsearch/transport/BaseTransportResponseHandler.java @@ -20,7 +20,7 @@ package org.elasticsearch.transport; /** - * A simple based class that always spawns. + * A simple base class that always spawns. */ public abstract class BaseTransportResponseHandler implements TransportResponseHandler { diff --git a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 46cdea3dadfaa..7a780642563c1 100644 --- a/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -205,7 +205,7 @@ private T serialize(T exception) throws IOException { } public void testIllegalShardRoutingStateException() throws IOException { - final ShardRouting routing = TestShardRouting.newShardRouting("test", 0, "xyz", "def", false, ShardRoutingState.STARTED, 0); + final ShardRouting routing = TestShardRouting.newShardRouting("test", 0, "xyz", "def", 1, false, ShardRoutingState.STARTED, 0); final String routingAsString = routing.toString(); IllegalShardRoutingStateException serialize = serialize(new IllegalShardRoutingStateException(routing, "foo", new NullPointerException())); assertNotNull(serialize.shard()); diff --git a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java index 6f5be64945186..c760e43ea958f 100644 --- a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java @@ -34,16 +34,12 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +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.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.ShardsIterator; -import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.cluster.routing.*; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.DummyTransportAddress; @@ -62,12 +58,7 @@ import org.junit.BeforeClass; import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -201,13 +192,16 @@ void setClusterState(TestClusterService clusterService, String index) { IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index); int shardIndex = -1; + int totalIndexShards = 0; for (int i = 0; i < numberOfNodes; i++) { final DiscoveryNode node = newNode(i); discoBuilder = discoBuilder.put(node); int numberOfShards = randomIntBetween(1, 10); + totalIndexShards += numberOfShards; for (int j = 0; j < numberOfShards; j++) { final ShardId shardId = new ShardId(index, ++shardIndex); - ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), true, ShardRoutingState.STARTED, 1); + final int primaryTerm = randomInt(200); + ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), primaryTerm, true, ShardRoutingState.STARTED, 1); IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(shardId); indexShard.addShard(shard); indexRoutingTable.addIndexShard(indexShard.build()); @@ -217,6 +211,12 @@ void setClusterState(TestClusterService clusterService, String index) { discoBuilder.masterNodeId(newNode(numberOfNodes - 1).id()); ClusterState.Builder stateBuilder = ClusterState.builder(new ClusterName(TEST_CLUSTER)); stateBuilder.nodes(discoBuilder); + final IndexMetaData.Builder indexMetaData = IndexMetaData.builder(index) + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfReplicas(0) + .numberOfShards(totalIndexShards); + + stateBuilder.metaData(MetaData.builder().put(indexMetaData)); stateBuilder.routingTable(RoutingTable.builder().add(indexRoutingTable.build()).build()); ClusterState clusterState = stateBuilder.build(); clusterService.setState(clusterState); @@ -346,7 +346,7 @@ public void testOperationExecution() throws Exception { TransportResponse response = channel.getCapturedResponse(); assertTrue(response instanceof TransportBroadcastByNodeAction.NodeResponse); - TransportBroadcastByNodeAction.NodeResponse nodeResponse = (TransportBroadcastByNodeAction.NodeResponse)response; + TransportBroadcastByNodeAction.NodeResponse nodeResponse = (TransportBroadcastByNodeAction.NodeResponse) response; // check the operation was executed on the correct node assertEquals("node id", nodeId, nodeResponse.getNodeId()); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index 4d17155f611bc..38d0163042bfc 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ReplicationResponse; 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 +59,8 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithAssignedPrimariesAndOneReplica; -import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithNoShard; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.*; +import static org.hamcrest.Matchers.*; public class BroadcastReplicationTests extends ESTestCase { @@ -131,7 +128,7 @@ public void testStartedPrimary() throws InterruptedException, ExecutionException public void testResultCombine() throws InterruptedException, ExecutionException, IOException { final String index = "test"; - int numShards = randomInt(3); + int numShards = 1 + randomInt(3); clusterService.setState(stateWithAssignedPrimariesAndOneReplica(index, numShards)); logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); Future response = (broadcastReplicationAction.execute(new BroadcastRequest().indices(index))); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java index 406e476b4e018..2a12d9c132f58 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ClusterStateCreationUtils.java @@ -36,8 +36,7 @@ import java.util.Set; import static org.elasticsearch.cluster.metadata.IndexMetaData.*; -import static org.elasticsearch.test.ESTestCase.randomFrom; -import static org.elasticsearch.test.ESTestCase.randomIntBetween; +import static org.elasticsearch.test.ESTestCase.*; /** * Helper methods for generating cluster states @@ -76,10 +75,11 @@ public static ClusterState state(String index, boolean primaryLocal, ShardRoutin } discoBuilder.localNodeId(newNode(0).id()); discoBuilder.masterNodeId(newNode(1).id()); // we need a non-local master to test shard failures + final int primaryTerm = randomInt(200); IndexMetaData indexMetaData = IndexMetaData.builder(index).settings(Settings.builder() .put(SETTING_VERSION_CREATED, Version.CURRENT) .put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, numberOfReplicas) - .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build(); + .put(SETTING_CREATION_DATE, System.currentTimeMillis())).primaryTerm(0, primaryTerm).build(); RoutingTable.Builder routing = new RoutingTable.Builder(); routing.addAsNew(indexMetaData); @@ -101,7 +101,8 @@ public static ClusterState state(String index, boolean primaryLocal, ShardRoutin } else { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null); } - indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, 0, primaryNode, relocatingNode, null, true, primaryState, 0, unassignedInfo)); + indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, 0, primaryNode, relocatingNode, null, primaryTerm, + true, primaryState, 0, unassignedInfo)); for (ShardRoutingState replicaState : replicaStates) { String replicaNode = null; @@ -117,7 +118,8 @@ public static ClusterState state(String index, boolean primaryLocal, ShardRoutin unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null); } indexShardRoutingBuilder.addShard( - TestShardRouting.newShardRouting(index, shardId.id(), replicaNode, relocatingNode, null, false, replicaState, 0, unassignedInfo)); + TestShardRouting.newShardRouting(index, shardId.id(), replicaNode, relocatingNode, null, primaryTerm, false, + replicaState, 0, unassignedInfo)); } ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); @@ -142,19 +144,22 @@ public static ClusterState stateWithAssignedPrimariesAndOneReplica(String index, discoBuilder.masterNodeId(newNode(1).id()); // we need a non-local master to test shard failures 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, 1) + .put(SETTING_NUMBER_OF_SHARDS, numberOfShards).put(SETTING_NUMBER_OF_REPLICAS, 1) .put(SETTING_CREATION_DATE, System.currentTimeMillis())).build(); ClusterState.Builder state = ClusterState.builder(new ClusterName("test")); state.nodes(discoBuilder); state.metaData(MetaData.builder().put(indexMetaData, false).generateClusterUuidIfNeeded()); IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); + final int primaryTerm = randomInt(200); for (int i = 0; i < numberOfShards; i++) { RoutingTable.Builder routing = new RoutingTable.Builder(); routing.addAsNew(indexMetaData); final ShardId shardId = new ShardId(index, i); IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); - indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(0).id(), null, null, true, ShardRoutingState.STARTED, 0, null)); - indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(1).id(), null, null, false, ShardRoutingState.STARTED, 0, null)); + indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(0).id(), null, null, primaryTerm, true, + ShardRoutingState.STARTED, 0, null)); + indexShardRoutingBuilder.addShard(TestShardRouting.newShardRouting(index, i, newNode(1).id(), null, null, primaryTerm, false, + ShardRoutingState.STARTED, 0, null)); indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build()); } state.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder.build()).build()); @@ -219,12 +224,13 @@ public static ClusterState stateWithNoShard() { /** * Creates a cluster state where local node and master node can be specified + * * @param localNode node in allNodes that is the local node * @param masterNode node in allNodes that is the master node. Can be null if no master exists * @param allNodes all nodes in the cluster * @return cluster state */ - public static ClusterState state(DiscoveryNode localNode, DiscoveryNode masterNode, DiscoveryNode... allNodes) { + public static ClusterState state(DiscoveryNode localNode, DiscoveryNode masterNode, DiscoveryNode... allNodes) { DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder(); for (DiscoveryNode node : allNodes) { discoBuilder.put(node); diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 5834b2662ad49..1f8260e39f408 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -55,7 +55,10 @@ import org.elasticsearch.test.cluster.TestClusterService; import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.*; +import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseOptions; +import org.elasticsearch.transport.TransportService; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -72,11 +75,7 @@ import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithStartedPrimary; -import static org.hamcrest.Matchers.arrayWithSize; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; public class TransportReplicationActionTests extends ESTestCase { @@ -467,10 +466,12 @@ protected void runReplicateTest(IndexShardRoutingTable shardRoutingTable, int as final ShardIterator shardIt = shardRoutingTable.shardsIt(); final ShardId shardId = shardIt.shardId(); final Request request = new Request(shardId); + final long primaryTerm = randomInt(200); + request.primaryTerm(primaryTerm); final PlainActionFuture listener = new PlainActionFuture<>(); logger.debug("expecting [{}] assigned replicas, [{}] total shards. using state: \n{}", assignedReplicas, totalShards, clusterService.state().prettyPrint()); - Releasable reference = getOrCreateIndexShardOperationsCounter(); + Releasable reference = createIndexShardReference(0); assertIndexShardCounter(2); // TODO: set a default timeout TransportReplicationAction.ReplicationPhase replicationPhase = @@ -533,6 +534,25 @@ action.new ReplicationPhase(request, } // all replicas have responded so the counter should be decreased again assertIndexShardCounter(1); + + // assert that nothing in the replica logic changes the primary term of the operation + assertThat(request.primaryTerm(), equalTo(primaryTerm)); + } + + public void testSeqNoIsSetOnPrimary() throws Exception { + final String index = "test"; + final ShardId shardId = new ShardId(index, 0); + // we use one replica to check the primary term was set on the operation and sent to the replica + clusterService.setState(state(index, true, + ShardRoutingState.STARTED, randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.STARTED))); + logger.debug("--> using initial state:\n{}", clusterService.state().prettyPrint()); + Request request = new Request(shardId); + PlainActionFuture listener = new PlainActionFuture<>(); + TransportReplicationAction.PrimaryPhase primaryPhase = action.new PrimaryPhase(request, createTransportChannel(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 { @@ -668,9 +688,14 @@ private void assertIndexShardCounter(int expected) { /* * Returns testIndexShardOperationsCounter or initializes it if it was already created in this test run. * */ - private synchronized Releasable getOrCreateIndexShardOperationsCounter() { + private synchronized TransportReplicationAction.IndexShardReference createIndexShardReference(long primaryTerm) { count.incrementAndGet(); - return new Releasable() { + return new TransportReplicationAction.IndexShardReference() { + @Override + public long opPrimaryTerm() { + return primaryTerm; + } + @Override public void close() { count.decrementAndGet(); @@ -743,9 +768,16 @@ protected boolean resolveIndex() { return false; } + + @Override + protected IndexShardReference getIndexShardOperationsCounterOnReplica(ShardId shardId, long opPrimaryTerm) { + return createIndexShardReference(opPrimaryTerm); + } + @Override - protected Releasable getIndexShardOperationsCounter(ShardId shardId) { - return getOrCreateIndexShardOperationsCounter(); + protected IndexShardReference getIndexShardOperationsCounterOnPrimary(ShardId shardId) { + final IndexMetaData indexMetaData = clusterService.state().metaData().index(shardId.getIndex()); + return createIndexShardReference(indexMetaData.primaryTerm(shardId.id())); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java index 8d4540aad3b76..f68e63118a566 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java @@ -220,13 +220,14 @@ private IndexRoutingTable randomIndexRoutingTable(String index, String[] nodeIds for (int i = 0; i < shardCount; i++) { IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(new ShardId(index, i)); int replicaCount = randomIntBetween(1, 10); + int term = randomInt(200); for (int j = 0; j < replicaCount; j++) { UnassignedInfo unassignedInfo = null; if (randomInt(5) == 1) { unassignedInfo = new UnassignedInfo(randomReason(), randomAsciiOfLength(10)); } indexShard.addShard( - TestShardRouting.newShardRouting(index, i, randomFrom(nodeIds), null, null, j == 0, + TestShardRouting.newShardRouting(index, i, randomFrom(nodeIds), null, null, term, j == 0, ShardRoutingState.fromValue((byte) randomIntBetween(2, 4)), 1, unassignedInfo)); } builder.addIndexShard(indexShard.build()); diff --git a/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java b/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java index 98eea13e673d7..9ff9ee9ee437c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/DiskUsageTests.java @@ -49,19 +49,19 @@ public void testDiskUsageCalc() { // Test that DiskUsage handles invalid numbers, as reported by some // filesystems (ZFS & NTFS) - DiskUsage du2 = new DiskUsage("node1", "n1","random", 100, 101); + DiskUsage du2 = new DiskUsage("node1", "n1", "random", 100, 101); assertThat(du2.getFreeDiskAsPercentage(), equalTo(101.0)); assertThat(du2.getFreeBytes(), equalTo(101L)); assertThat(du2.getUsedBytes(), equalTo(-1L)); assertThat(du2.getTotalBytes(), equalTo(100L)); - DiskUsage du3 = new DiskUsage("node1", "n1", "random",-1, -1); + DiskUsage du3 = new DiskUsage("node1", "n1", "random", -1, -1); assertThat(du3.getFreeDiskAsPercentage(), equalTo(100.0)); assertThat(du3.getFreeBytes(), equalTo(-1L)); assertThat(du3.getUsedBytes(), equalTo(0L)); assertThat(du3.getTotalBytes(), equalTo(-1L)); - DiskUsage du4 = new DiskUsage("node1", "n1","random", 0, 0); + DiskUsage du4 = new DiskUsage("node1", "n1", "random", 0, 0); assertThat(du4.getFreeDiskAsPercentage(), equalTo(100.0)); assertThat(du4.getFreeBytes(), equalTo(0L)); assertThat(du4.getUsedBytes(), equalTo(0L)); @@ -91,21 +91,21 @@ public void testRandomDiskUsage() { } public void testFillShardLevelInfo() { - ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, 1, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_0, "node1"); ShardRoutingHelper.moveToStarted(test_0); Path test0Path = createTempDir().resolve("indices").resolve("test").resolve("0"); CommonStats commonStats0 = new CommonStats(); commonStats0.store = new StoreStats(100, 1); - ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, 1, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_1, "node2"); ShardRoutingHelper.moveToStarted(test_1); Path test1Path = createTempDir().resolve("indices").resolve("test").resolve("1"); CommonStats commonStats1 = new CommonStats(); commonStats1.store = new StoreStats(1000, 1); - ShardStats[] stats = new ShardStats[] { - new ShardStats(test_0, new ShardPath(false, test0Path, test0Path, "0xdeadbeef", test_0.shardId()), commonStats0 , null), - new ShardStats(test_1, new ShardPath(false, test1Path, test1Path, "0xdeadbeef", test_1.shardId()), commonStats1 , null) + ShardStats[] stats = new ShardStats[]{ + new ShardStats(test_0, new ShardPath(false, test0Path, test0Path, "0xdeadbeef", test_0.shardId()), commonStats0, null, null), + new ShardStats(test_1, new ShardPath(false, test1Path, test1Path, "0xdeadbeef", test_1.shardId()), commonStats1, null, null) }; ImmutableOpenMap.Builder shardSizes = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder routingToPath = ImmutableOpenMap.builder(); @@ -131,15 +131,15 @@ public void testFillDiskUsage() { new FsInfo.Path("/least", "/dev/sdb", 200, 190, 70), new FsInfo.Path("/most", "/dev/sdc", 300, 290, 280), }; - FsInfo.Path[] node2FSInfo = new FsInfo.Path[] { + FsInfo.Path[] node2FSInfo = new FsInfo.Path[]{ new FsInfo.Path("/least_most", "/dev/sda", 100, 90, 80), }; - FsInfo.Path[] node3FSInfo = new FsInfo.Path[] { + FsInfo.Path[] node3FSInfo = new FsInfo.Path[]{ new FsInfo.Path("/least", "/dev/sda", 100, 90, 70), new FsInfo.Path("/most", "/dev/sda", 100, 90, 80), }; - NodeStats[] nodeStats = new NodeStats[] { + NodeStats[] nodeStats = new NodeStats[]{ new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT), 0, null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null,null), new NodeStats(new DiscoveryNode("node_2", DummyTransportAddress.INSTANCE, Version.CURRENT), 0, diff --git a/core/src/test/java/org/elasticsearch/cluster/health/RoutingTableGenerator.java b/core/src/test/java/org/elasticsearch/cluster/health/RoutingTableGenerator.java index 730763372ca99..ca8834313103f 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/RoutingTableGenerator.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/RoutingTableGenerator.java @@ -18,7 +18,7 @@ */ package org.elasticsearch.cluster.health; -import com.carrotsearch.randomizedtesting.RandomizedContext; +import com.carrotsearch.randomizedtesting.RandomizedTest; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.*; @@ -31,7 +31,7 @@ private ShardRouting genShardRouting(String index, int shardId, boolean primary) ShardRoutingState state; - int stateRandomizer = RandomizedContext.current().getRandom().nextInt(40); + int stateRandomizer = RandomizedTest.randomInt(40); if (stateRandomizer > 5) { state = ShardRoutingState.STARTED; } else if (stateRandomizer > 3) { @@ -40,13 +40,15 @@ private ShardRouting genShardRouting(String index, int shardId, boolean primary) state = ShardRoutingState.INITIALIZING; } + long primaryTerm = RandomizedTest.randomIntBetween(0, 200); + switch (state) { case STARTED: - return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primary, ShardRoutingState.STARTED, 1); + return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primaryTerm, primary, ShardRoutingState.STARTED, 1); case INITIALIZING: - return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primary, ShardRoutingState.INITIALIZING, 1); + return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), null, null, primaryTerm, primary, ShardRoutingState.INITIALIZING, 1); case RELOCATING: - return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), "node_" + Integer.toString(node_id++), null, primary, ShardRoutingState.RELOCATING, 1); + return TestShardRouting.newShardRouting(index, shardId, "node_" + Integer.toString(node_id++), "node_" + Integer.toString(node_id++), null, primaryTerm, primary, ShardRoutingState.RELOCATING, 1); default: throw new ElasticsearchException("Unknown state: " + state.name()); } diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/ToAndFromJsonMetaDataTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/ToAndFromJsonMetaDataTests.java index cbb5b7dfbdba6..9ac5fd6ad27a5 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/ToAndFromJsonMetaDataTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/ToAndFromJsonMetaDataTests.java @@ -41,11 +41,14 @@ public void testSimpleJsonFromAndTo() throws IOException { .put(IndexMetaData.builder("test1") .settings(settings(Version.CURRENT)) .numberOfShards(1) - .numberOfReplicas(2)) + .numberOfReplicas(2) + .primaryTerm(0, 1)) .put(IndexMetaData.builder("test2") .settings(settings(Version.CURRENT).put("setting1", "value1").put("setting2", "value2")) .numberOfShards(2) - .numberOfReplicas(3)) + .numberOfReplicas(3) + .primaryTerm(0, 2) + .primaryTerm(1, 2)) .put(IndexMetaData.builder("test3") .settings(settings(Version.CURRENT)) .numberOfShards(1) @@ -112,15 +115,15 @@ public void testSimpleJsonFromAndTo() throws IOException { .putAlias(newAliasMetaDataBuilder("alias1").filter(ALIAS_FILTER1)) .putAlias(newAliasMetaDataBuilder("alias2")) .putAlias(newAliasMetaDataBuilder("alias4").filter(ALIAS_FILTER2))) - .put(IndexTemplateMetaData.builder("foo") - .template("bar") - .order(1) - .settings(settingsBuilder() - .put("setting1", "value1") - .put("setting2", "value2")) - .putAlias(newAliasMetaDataBuilder("alias-bar1")) - .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}")) - .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar"))) + .put(IndexTemplateMetaData.builder("foo") + .template("bar") + .order(1) + .settings(settingsBuilder() + .put("setting1", "value1") + .put("setting2", "value2")) + .putAlias(newAliasMetaDataBuilder("alias-bar1")) + .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}")) + .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar"))) .put(IndexMetaData.builder("test12") .settings(settings(Version.CURRENT) .put("setting1", "value1") @@ -133,15 +136,15 @@ public void testSimpleJsonFromAndTo() throws IOException { .putAlias(newAliasMetaDataBuilder("alias1").filter(ALIAS_FILTER1)) .putAlias(newAliasMetaDataBuilder("alias2")) .putAlias(newAliasMetaDataBuilder("alias4").filter(ALIAS_FILTER2))) - .put(IndexTemplateMetaData.builder("foo") - .template("bar") - .order(1) - .settings(settingsBuilder() - .put("setting1", "value1") - .put("setting2", "value2")) - .putAlias(newAliasMetaDataBuilder("alias-bar1")) - .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}")) - .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar"))) + .put(IndexTemplateMetaData.builder("foo") + .template("bar") + .order(1) + .settings(settingsBuilder() + .put("setting1", "value1") + .put("setting2", "value2")) + .putAlias(newAliasMetaDataBuilder("alias-bar1")) + .putAlias(newAliasMetaDataBuilder("alias-bar2").filter("{\"term\":{\"user\":\"kimchy\"}}")) + .putAlias(newAliasMetaDataBuilder("alias-bar3").routing("routing-bar"))) .build(); String metaDataSource = MetaData.Builder.toXContent(metaData); @@ -150,6 +153,7 @@ public void testSimpleJsonFromAndTo() throws IOException { MetaData parsedMetaData = MetaData.Builder.fromXContent(XContentFactory.xContent(XContentType.JSON).createParser(metaDataSource)); IndexMetaData indexMetaData = parsedMetaData.index("test1"); + assertThat(indexMetaData.primaryTerm(0), equalTo(1l)); assertThat(indexMetaData.getNumberOfShards(), equalTo(1)); assertThat(indexMetaData.getNumberOfReplicas(), equalTo(2)); assertThat(indexMetaData.getCreationDate(), equalTo(-1l)); @@ -159,6 +163,8 @@ public void testSimpleJsonFromAndTo() throws IOException { indexMetaData = parsedMetaData.index("test2"); assertThat(indexMetaData.getNumberOfShards(), equalTo(2)); assertThat(indexMetaData.getNumberOfReplicas(), equalTo(3)); + assertThat(indexMetaData.primaryTerm(0), equalTo(2l)); + assertThat(indexMetaData.primaryTerm(1), equalTo(2l)); assertThat(indexMetaData.getCreationDate(), equalTo(-1l)); assertThat(indexMetaData.getSettings().getAsMap().size(), equalTo(5)); assertThat(indexMetaData.getSettings().get("setting1"), equalTo("value1")); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java index 8f7ae0c822b51..4d0980cc66d8f 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/AllocationIdTests.java @@ -37,7 +37,7 @@ public class AllocationIdTests extends ESTestCase { public void testShardToStarted() { logger.info("-- create unassigned shard"); - ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); assertThat(shard.allocationId(), nullValue()); logger.info("-- initialize the shard"); @@ -57,7 +57,7 @@ public void testShardToStarted() { public void testSuccessfulRelocation() { logger.info("-- build started shard"); - ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); shard.initialize("node1", -1); shard.moveToStarted(); @@ -80,7 +80,7 @@ public void testSuccessfulRelocation() { public void testCancelRelocation() { logger.info("-- build started shard"); - ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); shard.initialize("node1", -1); shard.moveToStarted(); @@ -100,7 +100,7 @@ public void testCancelRelocation() { public void testMoveToUnassigned() { logger.info("-- build started shard"); - ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); shard.initialize("node1", -1); shard.moveToStarted(); @@ -111,7 +111,7 @@ public void testMoveToUnassigned() { public void testReinitializing() { logger.info("-- build started shard"); - ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + ShardRouting shard = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); shard.initialize("node1", -1); shard.moveToStarted(); AllocationId allocationId = shard.allocationId(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java index d69264a1e3a2a..4a5e8ce6f68b3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/RoutingTableTests.java @@ -26,17 +26,17 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes.Builder; import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.test.ESAllocationTestCase; -import org.junit.Before; + +import java.util.*; import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.elasticsearch.common.settings.Settings.settingsBuilder; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; public class RoutingTableTests extends ESAllocationTestCase { @@ -55,8 +55,10 @@ public class RoutingTableTests extends ESAllocationTestCase { .build()); private ClusterState clusterState; + private final Map primaryTermsPerIndex = new HashMap<>(); + private final Map versionsPerIndex = new HashMap<>(); + @Override - @Before public void setUp() throws Exception { super.setUp(); this.numberOfShards = randomIntBetween(1, 5); @@ -65,6 +67,7 @@ public void setUp() throws Exception { this.totalNumberOfShards = this.shardsPerIndex * 2; logger.info("Setup test with " + this.numberOfShards + " shards and " + this.numberOfReplicas + " replicas."); this.emptyRoutingTable = new RoutingTable.Builder().build(); + this.primaryTermsPerIndex.clear(); MetaData metaData = MetaData.builder() .put(createIndexMetaData(TEST_INDEX_1)) .put(createIndexMetaData(TEST_INDEX_2)) @@ -74,6 +77,10 @@ public void setUp() throws Exception { .add(new IndexRoutingTable.Builder(TEST_INDEX_1).initializeAsNew(metaData.index(TEST_INDEX_1)).build()) .add(new IndexRoutingTable.Builder(TEST_INDEX_2).initializeAsNew(metaData.index(TEST_INDEX_2)).build()) .build(); + this.versionsPerIndex.clear(); + this.versionsPerIndex.put(TEST_INDEX_1, new long[numberOfShards]); + this.versionsPerIndex.put(TEST_INDEX_2, new long[numberOfShards]); + this.clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(testRoutingTable).build(); } @@ -90,22 +97,104 @@ private void initPrimaries() { RoutingAllocation.Result rerouteResult = ALLOCATION_SERVICE.reroute(clusterState, "reroute"); this.testRoutingTable = rerouteResult.routingTable(); assertThat(rerouteResult.changed(), is(true)); - this.clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build(); + applyRerouteResult(rerouteResult); + versionsPerIndex.keySet().forEach(this::incrementVersion); + primaryTermsPerIndex.keySet().forEach(this::incrementPrimaryTerm); + } + + private void incrementVersion(String index) { + final long[] versions = versionsPerIndex.get(index); + for (int i = 0; i < versions.length; i++) { + versions[i]++; + } + } + + private void incrementVersion(String index, int shard) { + versionsPerIndex.get(index)[shard]++; + } + + private void incrementPrimaryTerm(String index) { + final long[] primaryTerms = primaryTermsPerIndex.get(index); + for (int i = 0; i < primaryTerms.length; i++) { + primaryTerms[i]++; + } + } + + private void incrementPrimaryTerm(String index, int shard) { + primaryTermsPerIndex.get(index)[shard]++; } private void startInitializingShards(String index) { this.clusterState = ClusterState.builder(clusterState).routingTable(this.testRoutingTable).build(); logger.info("start primary shards for index " + index); RoutingAllocation.Result rerouteResult = ALLOCATION_SERVICE.applyStartedShards(this.clusterState, this.clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING)); - this.clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build(); + // TODO: this simulate the code in InternalClusterState.UpdateTask.run() we should unify this. + applyRerouteResult(rerouteResult); + incrementVersion(index); + } + + private void applyRerouteResult(RoutingAllocation.Result rerouteResult) { + ClusterState previousClusterState = this.clusterState; + ClusterState newClusterState = ClusterState.builder(previousClusterState).routingResult(rerouteResult).build(); + ClusterState.Builder builder = ClusterState.builder(newClusterState).incrementVersion(); + if (previousClusterState.routingTable() != newClusterState.routingTable()) { + builder.routingTable(RoutingTable.builder(newClusterState.routingTable()).version(newClusterState.routingTable().version() + 1).build()); + } + if (previousClusterState.metaData() != newClusterState.metaData()) { + builder.metaData(MetaData.builder(newClusterState.metaData()).version(newClusterState.metaData().version() + 1)); + } + this.clusterState = builder.build(); this.testRoutingTable = rerouteResult.routingTable(); } + private void failSomePrimaries(String index) { + this.clusterState = ClusterState.builder(clusterState).routingTable(this.testRoutingTable).build(); + final IndexRoutingTable indexShardRoutingTable = testRoutingTable.index(index); + Set shardIdsToFail = new HashSet<>(); + for (int i = 1 + randomInt(numberOfShards - 1); i > 0; i--) { + shardIdsToFail.add(randomInt(numberOfShards - 1)); + } + logger.info("failing primary shards {} for index [{}]", shardIdsToFail, index); + List failedShards = new ArrayList<>(); + for (int shard : shardIdsToFail) { + failedShards.add(new FailedRerouteAllocation.FailedShard(indexShardRoutingTable.shard(shard).primaryShard(), "test", null)); + incrementPrimaryTerm(index, shard); // the primary failure should increment the primary term; + incrementVersion(index, shard); // version is incremented once when the primary is unassigned + incrementVersion(index, shard); // and another time when the primary flag is set to false + } + RoutingAllocation.Result rerouteResult = ALLOCATION_SERVICE.applyFailedShards(this.clusterState, failedShards); + applyRerouteResult(rerouteResult); + } + private IndexMetaData.Builder createIndexMetaData(String indexName) { - return new IndexMetaData.Builder(indexName) + primaryTermsPerIndex.put(indexName, new long[numberOfShards]); + final IndexMetaData.Builder builder = new IndexMetaData.Builder(indexName) .settings(DEFAULT_SETTINGS) .numberOfReplicas(this.numberOfReplicas) .numberOfShards(this.numberOfShards); + for (int i = 0; i < numberOfShards; i++) { + builder.primaryTerm(i, randomInt(200)); + primaryTermsPerIndex.get(indexName)[i] = builder.primaryTerm(i); + } + return builder; + } + + private void assertAllVersionAndPrimaryTerm() { + versionsPerIndex.keySet().forEach(this::assertVersionAndPrimaryTerm); + } + + private void assertVersionAndPrimaryTerm(String index) { + final long[] versions = versionsPerIndex.get(index); + final long[] terms = primaryTermsPerIndex.get(index); + final IndexMetaData indexMetaData = clusterState.metaData().index(index); + for (IndexShardRoutingTable shardRoutingTable : this.testRoutingTable.index(index)) { + final int shard = shardRoutingTable.shardId().id(); + for (ShardRouting routing : shardRoutingTable) { + assertThat("wrong version in " + routing, routing.version(), equalTo(versions[shard])); + assertThat("wrong primary term in " + routing, routing.primaryTerm(), equalTo(terms[shard])); + } + assertThat("primary term mismatch between indexMetaData of [" + index + "] and shard [" + shard + "]'s routing", indexMetaData.primaryTerm(shard), equalTo(terms[shard])); + } } public void testAllShards() { @@ -162,6 +251,27 @@ public void testShardsWithState() { assertThat(this.testRoutingTable.shardsWithState(ShardRoutingState.STARTED).size(), is(this.totalNumberOfShards)); } + public void testVersionAndPrimaryTermNormalization() { + assertAllVersionAndPrimaryTerm(); + + initPrimaries(); + assertAllVersionAndPrimaryTerm(); + + startInitializingShards(TEST_INDEX_1); + assertAllVersionAndPrimaryTerm(); + + startInitializingShards(TEST_INDEX_2); + assertAllVersionAndPrimaryTerm(); + + // now start all replicas too + startInitializingShards(TEST_INDEX_1); + startInitializingShards(TEST_INDEX_2); + assertAllVersionAndPrimaryTerm(); + + failSomePrimaries(TEST_INDEX_1); + assertAllVersionAndPrimaryTerm(); + } + public void testActivePrimaryShardsGrouped() { assertThat(this.emptyRoutingTable.activePrimaryShardsGrouped(new String[0], true).size(), is(0)); assertThat(this.emptyRoutingTable.activePrimaryShardsGrouped(new String[0], false).size(), is(0)); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java index 2139cc29ae1ba..dfd7230b6a5e3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingHelper.java @@ -53,6 +53,6 @@ public static void moveToUnassigned(ShardRouting routing, UnassignedInfo info) { } public static ShardRouting newWithRestoreSource(ShardRouting routing, RestoreSource restoreSource) { - return new ShardRouting(routing.index(), routing.shardId().id(), routing.currentNodeId(), routing.relocatingNodeId(), restoreSource, routing.primary(), routing.state(), routing.version(), routing.unassignedInfo(), routing.allocationId(), true, routing.getExpectedShardSize()); + return new ShardRouting(routing.index(), routing.shardId().id(), routing.currentNodeId(), routing.relocatingNodeId(), restoreSource, routing.primaryTerm(), routing.primary(), routing.state(), routing.version(), routing.unassignedInfo(), routing.allocationId(), true, routing.getExpectedShardSize()); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java index 54e39cc227d5f..01a35d277fa66 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/ShardRoutingTests.java @@ -30,10 +30,13 @@ import java.io.IOException; +import static org.hamcrest.Matchers.equalTo; + public class ShardRoutingTests extends ESTestCase { public void testFrozenAfterRead() throws IOException { - ShardRouting routing = TestShardRouting.newShardRouting("foo", 1, "node_1", null, null, false, ShardRoutingState.INITIALIZING, 1); + long term = randomInt(200); + ShardRouting routing = TestShardRouting.newShardRouting("foo", 1, "node_1", null, null, term, false, ShardRoutingState.INITIALIZING, 1); routing.moveToPrimary(); assertTrue(routing.primary()); routing.moveFromPrimary(); @@ -49,11 +52,23 @@ public void testFrozenAfterRead() throws IOException { } } + public void testPrimaryTermIncrementOnPromotion() { + long term = randomInt(200); + ShardRouting routing = TestShardRouting.newShardRouting("foo", 1, "node_1", null, null, term, false, ShardRoutingState.STARTED, 1); + routing.moveToPrimary(); + assertTrue(routing.primary()); + assertThat(routing.primaryTerm(), equalTo(term + 1)); + routing.moveFromPrimary(); + assertFalse(routing.primary()); + assertThat(routing.primaryTerm(), equalTo(term + 1)); + } + public void testIsSameAllocation() { - ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED, 1); - ShardRouting unassignedShard1 = TestShardRouting.newShardRouting("test", 1, null, false, ShardRoutingState.UNASSIGNED, 1); - ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "1", randomBoolean(), ShardRoutingState.INITIALIZING, 1); - ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "1", randomBoolean(), ShardRoutingState.INITIALIZING, 1); + long term = randomInt(200); + ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, term, false, ShardRoutingState.UNASSIGNED, 1); + ShardRouting unassignedShard1 = TestShardRouting.newShardRouting("test", 1, null, term, false, ShardRoutingState.UNASSIGNED, 1); + ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "1", term, randomBoolean(), ShardRoutingState.INITIALIZING, 1); + ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "1", term, randomBoolean(), ShardRoutingState.INITIALIZING, 1); ShardRouting startedShard0 = new ShardRouting(initializingShard0); startedShard0.moveToStarted(); ShardRouting startedShard1 = new ShardRouting(initializingShard1); @@ -91,13 +106,14 @@ public void testIsSameShard() { private ShardRouting randomShardRouting(String index, int shard) { ShardRoutingState state = randomFrom(ShardRoutingState.values()); - return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1", state != ShardRoutingState.UNASSIGNED && randomBoolean(), state, randomInt(5)); + return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1", randomInt(200), + state != ShardRoutingState.UNASSIGNED && randomBoolean(), state, randomInt(5)); } public void testIsSourceTargetRelocation() { - ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, false, ShardRoutingState.UNASSIGNED, 1); - ShardRouting initializingShard0 = TestShardRouting.newShardRouting("test", 0, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1); - ShardRouting initializingShard1 = TestShardRouting.newShardRouting("test", 1, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1); + ShardRouting unassignedShard0 = TestShardRouting.newShardRouting("test", 0, null, randomInt(200), false, ShardRoutingState.UNASSIGNED, 1); + 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(); @@ -145,13 +161,14 @@ public void testIsSourceTargetRelocation() { assertFalse(startedShard0.isRelocationSourceOf(sourceShard0a)); } - public void testEqualsIgnoringVersion() { + public void testEqualsIgnoringMetaData() { ShardRouting routing = randomShardRouting("test", 0); ShardRouting otherRouting = new ShardRouting(routing); - assertTrue("expected equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting)); - otherRouting = new ShardRouting(routing, 1); + otherRouting = new ShardRouting(routing, + randomBoolean() ? routing.version() : routing.version() + 1, + randomBoolean() ? routing.primaryTerm() : routing.primaryTerm() + 1); assertTrue("expected equality\nthis " + routing + ",\nother " + otherRouting, routing.equalsIgnoringMetaData(otherRouting)); @@ -161,36 +178,42 @@ public void testEqualsIgnoringVersion() { switch (changeId) { case 0: // change index - otherRouting = TestShardRouting.newShardRouting(otherRouting.index() + "a", otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(), - otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); + otherRouting = TestShardRouting.newShardRouting(otherRouting.index() + "a", otherRouting.id(), otherRouting.currentNodeId(), + otherRouting.relocatingNodeId(), otherRouting.restoreSource(), otherRouting.primaryTerm(), otherRouting.primary(), + otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); break; case 1: // change shard id - otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id() + 1, otherRouting.currentNodeId(), otherRouting.relocatingNodeId(), - otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); + otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id() + 1, otherRouting.currentNodeId(), + otherRouting.relocatingNodeId(), otherRouting.restoreSource(), otherRouting.primaryTerm(), otherRouting.primary(), + otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); break; case 2: // change current node otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId() == null ? "1" : otherRouting.currentNodeId() + "_1", otherRouting.relocatingNodeId(), - otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); + otherRouting.restoreSource(), otherRouting.primaryTerm(), otherRouting.primary(), otherRouting.state(), + otherRouting.version(), otherRouting.unassignedInfo()); break; case 3: // change relocating node otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId() == null ? "1" : otherRouting.relocatingNodeId() + "_1", - otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); + otherRouting.restoreSource(), otherRouting.primaryTerm(), otherRouting.primary(), otherRouting.state(), + otherRouting.version(), otherRouting.unassignedInfo()); break; case 4: // change restore source otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(), otherRouting.restoreSource() == null ? new RestoreSource(new SnapshotId("test", "s1"), Version.CURRENT, "test") : new RestoreSource(otherRouting.restoreSource().snapshotId(), Version.CURRENT, otherRouting.index() + "_1"), - otherRouting.primary(), otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); + otherRouting.primaryTerm(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), + otherRouting.unassignedInfo()); break; case 5: // change primary flag - otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(), - otherRouting.restoreSource(), otherRouting.primary() == false, otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); + otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), + otherRouting.relocatingNodeId(), otherRouting.restoreSource(), otherRouting.primaryTerm(), + otherRouting.primary() == false, otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo()); break; case 6: // change state @@ -204,20 +227,26 @@ public void testEqualsIgnoringVersion() { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test"); } - otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(), - otherRouting.restoreSource(), otherRouting.primary(), newState, otherRouting.version(), unassignedInfo); + otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), + otherRouting.relocatingNodeId(), otherRouting.restoreSource(), otherRouting.primaryTerm(), otherRouting.primary(), + newState, otherRouting.version(), unassignedInfo); break; } if (randomBoolean()) { // change version - otherRouting = new ShardRouting(otherRouting, otherRouting.version() + 1); + otherRouting = new ShardRouting(otherRouting, otherRouting.version() + 1, otherRouting.primaryTerm()); + } + if (randomBoolean()) { + // increase term + otherRouting = new ShardRouting(otherRouting, otherRouting.version(), otherRouting.primaryTerm() + 1); } if (randomBoolean()) { // change unassigned info - otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(), - otherRouting.restoreSource(), otherRouting.primary(), otherRouting.state(), otherRouting.version(), + otherRouting = TestShardRouting.newShardRouting(otherRouting.index(), otherRouting.id(), otherRouting.currentNodeId(), + otherRouting.relocatingNodeId(), otherRouting.restoreSource(), otherRouting.primaryTerm(), otherRouting.primary(), + otherRouting.state(), otherRouting.version(), otherRouting.unassignedInfo() == null ? new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test") : new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, otherRouting.unassignedInfo().getMessage() + "_1")); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java index 3288b92cb8e5d..9615270ed3344 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/UnassignedInfoTests.java @@ -177,7 +177,7 @@ public void testReplicaAdded() { * The unassigned meta is kept when a shard goes to INITIALIZING, but cleared when it moves to STARTED. */ public void testStateTransitionMetaHandling() { - ShardRouting shard = TestShardRouting.newShardRouting("test", 1, null, null, null, true, ShardRoutingState.UNASSIGNED, 1, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + ShardRouting shard = TestShardRouting.newShardRouting("test", 1, null, null, null, 1, true, ShardRoutingState.UNASSIGNED, 1, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); ShardRouting mutable = new ShardRouting(shard); assertThat(mutable.unassignedInfo(), notNullValue()); mutable.initialize("test_node", -1); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java index 8ac6c4fcedc6c..8603d5a416e01 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/CatAllocationTestCase.java @@ -82,7 +82,7 @@ public void testRun() throws IOException { ShardRoutingState state = ShardRoutingState.valueOf(matcher.group(4)); String ip = matcher.group(5); nodes.add(ip); - ShardRouting routing = TestShardRouting.newShardRouting(index, shard, ip, null, null, primary, state, 1); + ShardRouting routing = TestShardRouting.newShardRouting(index, shard, ip, null, null, 1, primary, state, 1); idx.add(routing); logger.debug("Add routing {}", routing); } else { diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java index e994c885629c4..c001657d4bcb6 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/PrimaryElectionRoutingTests.java @@ -58,29 +58,31 @@ public void testBackupElectionToPrimaryWhenPrimaryCanBeAllocatedToAnotherNode() ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build(); logger.info("Adding two nodes and performing rerouting"); - clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build(); - RoutingTable prevRoutingTable = routingTable; - routingTable = strategy.reroute(clusterState, "reroute").routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); + clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1"))).build(); + RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute"); + clusterState = ClusterState.builder(clusterState).routingResult(result).build(); + + clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node2"))).build(); + result = strategy.reroute(clusterState, "reroute"); + clusterState = ClusterState.builder(clusterState).routingResult(result).build(); logger.info("Start the primary shard (on node1)"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); - prevRoutingTable = routingTable; - routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); + result = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)); + clusterState = ClusterState.builder(clusterState).routingResult(result).build(); logger.info("Start the backup shard (on node2)"); routingNodes = clusterState.getRoutingNodes(); - prevRoutingTable = routingTable; - routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); + result = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)); + clusterState = ClusterState.builder(clusterState).routingResult(result).build(); logger.info("Adding third node and reroute and kill first node"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node3")).remove("node1")).build(); - prevRoutingTable = routingTable; - routingTable = strategy.reroute(clusterState, "reroute").routingTable(); - clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); + RoutingTable prevRoutingTable = clusterState.routingTable(); + result = strategy.reroute(clusterState, "reroute"); + clusterState = ClusterState.builder(clusterState).routingResult(result).build(); routingNodes = clusterState.getRoutingNodes(); + routingTable = clusterState.routingTable(); assertThat(prevRoutingTable != routingTable, equalTo(true)); assertThat(routingTable.index("test").shards().size(), equalTo(1)); @@ -89,6 +91,7 @@ public void testBackupElectionToPrimaryWhenPrimaryCanBeAllocatedToAnotherNode() assertThat(routingNodes.node("node3").numberOfShardsWithState(INITIALIZING), equalTo(1)); // verify where the primary is assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node2")); + assertThat(routingTable.index("test").shard(0).primaryShard().primaryTerm(), equalTo(2l)); assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node3")); } @@ -115,7 +118,7 @@ public void testRemovingInitializingReplicasIfPrimariesFails() { logger.info("Start the primary shards"); RoutingNodes routingNodes = clusterState.getRoutingNodes(); rerouteResult = allocation.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)); - clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build(); + clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build(); routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(2)); @@ -129,12 +132,13 @@ public void testRemovingInitializingReplicasIfPrimariesFails() { .put(newNode(nodeIdRemaining)) ).build(); rerouteResult = allocation.reroute(clusterState, "reroute"); - clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build(); + clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build(); routingNodes = clusterState.getRoutingNodes(); assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(1)); assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(1)); assertThat(routingNodes.node(nodeIdRemaining).shardsWithState(INITIALIZING).get(0).primary(), equalTo(true)); + assertThat(routingNodes.node(nodeIdRemaining).shardsWithState(INITIALIZING).get(0).primaryTerm(), equalTo(2l)); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java new file mode 100644 index 0000000000000..4a6bf3d848740 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ShardStateIT.java @@ -0,0 +1,70 @@ +/* + * 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.cluster.routing.allocation; + +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.health.ClusterHealthStatus; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.test.ESIntegTestCase; + +import static org.hamcrest.Matchers.equalTo; + +public class ShardStateIT extends ESIntegTestCase { + + public void testPrimaryFailureIncreasesTerm() throws Exception { + internalCluster().ensureAtLeastNumDataNodes(2); + prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).get(); + ensureGreen(); + assertPrimaryTerms(1, 1); + + logger.info("--> disabling allocation to capture shard failure"); + disableAllocation("test"); + + ClusterState state = client().admin().cluster().prepareState().get().getState(); + final int shard = randomBoolean() ? 0 : 1; + final String nodeId = state.routingTable().index("test").shard(shard).primaryShard().currentNodeId(); + final String node = state.nodes().get(nodeId).name(); + logger.info("--> failing primary of [{}] on node [{}]", shard, node); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, node); + indicesService.indexService("test").getShard(shard).failShard("simulated test failure", null); + + logger.info("--> waiting for a yellow index"); + assertBusy(() -> assertThat(client().admin().cluster().prepareHealth().get().getStatus(), equalTo(ClusterHealthStatus.YELLOW))); + + final long term0 = shard == 0 ? 2 : 1; + final long term1 = shard == 1 ? 2 : 1; + assertPrimaryTerms(term0, term1); + + logger.info("--> enabling allocation"); + enableAllocation("test"); + ensureGreen(); + assertPrimaryTerms(term0, term1); + } + + protected void assertPrimaryTerms(long term0, long term1) { + for (String node : internalCluster().getNodeNames()) { + logger.debug("--> asserting primary terms terms on [{}]", node); + ClusterState state = client(node).admin().cluster().prepareState().setLocal(true).get().getState(); + IndexMetaData metaData = state.metaData().index("test"); + assertThat(metaData.primaryTerm(0), equalTo(term0)); + assertThat(metaData.primaryTerm(1), equalTo(term1)); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java index 28033915abe1d..42a3ffedf0fcf 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/StartedShardsRoutingTests.java @@ -24,19 +24,14 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.routing.AllocationId; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.cluster.routing.*; +import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.test.ESAllocationTestCase; import java.util.Arrays; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; + public class StartedShardsRoutingTests extends ESAllocationTestCase { public void testStartedShardsMatching() { @@ -44,59 +39,92 @@ public void testStartedShardsMatching() { logger.info("--> building initial cluster state"); final IndexMetaData indexMetaData = IndexMetaData.builder("test") - .settings(settings(Version.CURRENT)) - .numberOfShards(3).numberOfReplicas(0) + .settings(settings(Version.CURRENT).put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")) + .numberOfShards(3).numberOfReplicas(1) .build(); ClusterState.Builder stateBuilder = ClusterState.builder(ClusterName.DEFAULT) - .nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))) + .nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).put(newNode("node3"))) .metaData(MetaData.builder().put(indexMetaData, false)); - final ShardRouting initShard = TestShardRouting.newShardRouting("test", 0, "node1", randomBoolean(), ShardRoutingState.INITIALIZING, 1); - final ShardRouting startedShard = TestShardRouting.newShardRouting("test", 1, "node2", randomBoolean(), ShardRoutingState.STARTED, 1); - final ShardRouting relocatingShard = TestShardRouting.newShardRouting("test", 2, "node1", "node2", randomBoolean(), ShardRoutingState.RELOCATING, 1); - stateBuilder.routingTable(RoutingTable.builder().add(IndexRoutingTable.builder("test") - .addIndexShard(new IndexShardRoutingTable.Builder(initShard.shardId()).addShard(initShard).build()) - .addIndexShard(new IndexShardRoutingTable.Builder(startedShard.shardId()).addShard(startedShard).build()) - .addIndexShard(new IndexShardRoutingTable.Builder(relocatingShard.shardId()).addShard(relocatingShard).build())).build()); + final ShardRouting initShard; + final ShardRouting startedShard; + final ShardRouting relocatingShard; + final IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder("test"); + if (randomBoolean()) { + initShard = TestShardRouting.newShardRouting("test", 0, "node1", 1, true, ShardRoutingState.INITIALIZING, 1); + ShardRouting replica = TestShardRouting.newShardRouting("test", 0, null, 1, false, ShardRoutingState.UNASSIGNED, 1); + indexRoutingTable.addIndexShard(new IndexShardRoutingTable.Builder(initShard.shardId()).addShard(initShard).addShard(replica).build()); + } else { + ShardRouting primaryShard = TestShardRouting.newShardRouting("test", 0, "node2", 1, true, ShardRoutingState.STARTED, 1); + initShard = TestShardRouting.newShardRouting("test", 0, "node1", 1, false, ShardRoutingState.INITIALIZING, 1); + indexRoutingTable.addIndexShard(new IndexShardRoutingTable.Builder(initShard.shardId()).addShard(primaryShard).addShard(initShard).build()); + } + if (randomBoolean()) { + startedShard = TestShardRouting.newShardRouting("test", 1, "node2", 1, true, ShardRoutingState.STARTED, 1); + ShardRouting replica = TestShardRouting.newShardRouting("test", 1, null, 1, false, ShardRoutingState.UNASSIGNED, 1); + indexRoutingTable.addIndexShard(new IndexShardRoutingTable.Builder(startedShard.shardId()).addShard(startedShard).addShard(replica).build()); + } else { + ShardRouting primaryShard = TestShardRouting.newShardRouting("test", 1, "node1", 1, true, ShardRoutingState.STARTED, 1); + startedShard = TestShardRouting.newShardRouting("test", 1, "node2", 1, false, ShardRoutingState.STARTED, 1); + indexRoutingTable.addIndexShard(new IndexShardRoutingTable.Builder(startedShard.shardId()).addShard(primaryShard).addShard(startedShard).build()); + } + + if (randomBoolean()) { + relocatingShard = TestShardRouting.newShardRouting("test", 2, "node1", "node2", 1, true, ShardRoutingState.RELOCATING, 1); + ShardRouting replica = TestShardRouting.newShardRouting("test", 2, null, 1, false, ShardRoutingState.UNASSIGNED, 1); + indexRoutingTable.addIndexShard(new IndexShardRoutingTable.Builder(relocatingShard.shardId()).addShard(relocatingShard).addShard(replica).build()); + } else { + ShardRouting primaryShard = TestShardRouting.newShardRouting("test", 2, "node3", 1, true, ShardRoutingState.STARTED, 1); + relocatingShard = TestShardRouting.newShardRouting("test", 2, "node1", "node2", 1, false, ShardRoutingState.RELOCATING, 1); + indexRoutingTable.addIndexShard(new IndexShardRoutingTable.Builder(relocatingShard.shardId()) + .addShard(primaryShard).addShard(relocatingShard).build()); + } + + stateBuilder.routingTable(RoutingTable.builder().add(indexRoutingTable).build()); ClusterState state = stateBuilder.build(); logger.info("--> test starting of shard"); RoutingAllocation.Result result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(initShard.index(), initShard.id(), initShard.currentNodeId(), initShard.relocatingNodeId(), initShard.primary(), + TestShardRouting.newShardRouting(initShard.index(), initShard.id(), initShard.currentNodeId(), initShard.relocatingNodeId(), + initShard.primaryTerm(), initShard.primary(), ShardRoutingState.INITIALIZING, initShard.allocationId(), randomInt())), false); assertTrue("failed to start " + initShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); - assertTrue(initShard + "isn't started \ncurrent routing table:" + result.routingTable().prettyPrint(), - result.routingTable().index("test").shard(initShard.id()).allShardsStarted()); + final ShardRouting resultRouting = result.routingTable().index("test").shard(initShard.id()).activeShards() + .stream().filter(routing -> routing.isSameAllocation(initShard)).findFirst().get(); + assertThat(initShard + "isn't started \ncurrent routing table:" + result.routingTable().prettyPrint(), resultRouting, notNullValue()); logger.info("--> testing shard variants that shouldn't match the initializing shard"); result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(initShard.index(), initShard.id(), initShard.currentNodeId(), initShard.relocatingNodeId(), initShard.primary(), + TestShardRouting.newShardRouting(initShard.index(), initShard.id(), initShard.currentNodeId(), initShard.relocatingNodeId(), + initShard.primaryTerm(), initShard.primary(), ShardRoutingState.INITIALIZING, 1)), false); assertFalse("wrong allocation id flag shouldn't start shard " + initShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(initShard.index(), initShard.id(), "some_node", initShard.currentNodeId(), initShard.primary(), + TestShardRouting.newShardRouting(initShard.index(), initShard.id(), "some_node", initShard.currentNodeId(), + initShard.primaryTerm(), initShard.primary(), ShardRoutingState.INITIALIZING, AllocationId.newTargetRelocation(AllocationId.newRelocation(initShard.allocationId())) , 1)), false); assertFalse("relocating shard from node shouldn't start shard " + initShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); - logger.info("--> testing double starting"); result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(startedShard.index(), startedShard.id(), startedShard.currentNodeId(), startedShard.relocatingNodeId(), startedShard.primary(), + TestShardRouting.newShardRouting(startedShard.index(), startedShard.id(), startedShard.currentNodeId(), startedShard.relocatingNodeId(), + startedShard.primaryTerm(), startedShard.primary(), ShardRoutingState.INITIALIZING, startedShard.allocationId(), 1)), false); assertFalse("duplicate starting of the same shard should be ignored \ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); logger.info("--> testing starting of relocating shards"); final AllocationId targetAllocationId = AllocationId.newTargetRelocation(relocatingShard.allocationId()); result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), relocatingShard.currentNodeId(), relocatingShard.primary(), + TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), + relocatingShard.currentNodeId(), relocatingShard.primaryTerm(), relocatingShard.primary(), ShardRoutingState.INITIALIZING, targetAllocationId, randomInt())), false); assertTrue("failed to start " + relocatingShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); @@ -108,12 +136,14 @@ public void testStartedShardsMatching() { logger.info("--> testing shard variants that shouldn't match the initializing relocating shard"); result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), relocatingShard.currentNodeId(), relocatingShard.primary(), + TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), + relocatingShard.currentNodeId(), relocatingShard.primaryTerm(), relocatingShard.primary(), ShardRoutingState.INITIALIZING, relocatingShard.version()))); assertFalse("wrong allocation id shouldn't start shard" + relocatingShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); result = allocation.applyStartedShards(state, Arrays.asList( - TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), relocatingShard.currentNodeId(), relocatingShard.primary(), + TestShardRouting.newShardRouting(relocatingShard.index(), relocatingShard.id(), relocatingShard.relocatingNodeId(), + relocatingShard.currentNodeId(), relocatingShard.primaryTerm(), relocatingShard.primary(), ShardRoutingState.INITIALIZING, relocatingShard.allocationId(), randomInt())), false); assertFalse("wrong allocation id shouldn't start shard even if relocatingId==shard.id" + relocatingShard + "\ncurrent routing table:" + result.routingTable().prettyPrint(), result.changed()); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index a739f30856a61..43b74d6145730 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -112,8 +112,8 @@ public void addListener(Listener listener) { logger.info("--> adding two nodes"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder() - .put(newNode("node1")) - .put(newNode("node2")) + .put(newNode("node1")) + .put(newNode("node2")) ).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -143,7 +143,7 @@ public void addListener(Listener listener) { logger.info("--> adding node3"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .put(newNode("node3")) + .put(newNode("node3")) ).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -230,7 +230,7 @@ public void addListener(Listener listener) { logger.info("--> adding node4"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .put(newNode("node4")) + .put(newNode("node4")) ).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -378,7 +378,7 @@ public void addListener(Listener listener) { logger.info("--> adding node3"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .put(newNode("node3")) + .put(newNode("node3")) ).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -465,7 +465,7 @@ public void addListener(Listener listener) { logger.info("--> adding node4"); clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) - .put(newNode("node4")) + .put(newNode("node4")) ).build(); routingTable = strategy.reroute(clusterState, "reroute").routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); @@ -832,8 +832,8 @@ public void testCanRemainWithShardRelocatingAway() { .build(); // Two shards consuming each 80% of disk space while 70% is allowed, so shard 0 isn't allowed here - ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, true, ShardRoutingState.STARTED, 1); - ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", null, null, true, ShardRoutingState.STARTED, 1); + ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, 1, true, ShardRoutingState.STARTED, 1); + ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", null, null, 1, true, ShardRoutingState.STARTED, 1); RoutingNode firstRoutingNode = new RoutingNode("node1", discoveryNode1, Arrays.asList(firstRouting, secondRouting)); RoutingTable.Builder builder = RoutingTable.builder().add( IndexRoutingTable.builder("test") @@ -852,8 +852,8 @@ public void testCanRemainWithShardRelocatingAway() { assertThat(decision.type(), equalTo(Decision.Type.NO)); // Two shards consuming each 80% of disk space while 70% is allowed, but one is relocating, so shard 0 can stay - firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, true, ShardRoutingState.STARTED, 1); - secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", "node2", null, true, ShardRoutingState.RELOCATING, 1); + firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, null, 1, true, ShardRoutingState.STARTED, 1); + secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", "node2", null, 1, true, ShardRoutingState.RELOCATING, 1); firstRoutingNode = new RoutingNode("node1", discoveryNode1, Arrays.asList(firstRouting, secondRouting)); builder = RoutingTable.builder().add( IndexRoutingTable.builder("test") @@ -885,7 +885,7 @@ public void addListener(Listener listener) { } }; AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new HashSet<>(Arrays.asList( - new SameShardAllocationDecider(Settings.EMPTY), diskThresholdDecider + new SameShardAllocationDecider(Settings.EMPTY), diskThresholdDecider ))); AllocationService strategy = new AllocationService(settingsBuilder() .put("cluster.routing.allocation.concurrent_recoveries", 10) @@ -950,8 +950,8 @@ public void testForSingleDataNode() { .build(); // Two shards consumes 80% of disk space in data node, but we have only one data node, shards should remain. - ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED, 1); - ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", null, null, true, ShardRoutingState.STARTED, 1); + ShardRouting firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, 1, true, ShardRoutingState.STARTED, 1); + ShardRouting secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", null, null, 1, true, ShardRoutingState.STARTED, 1); RoutingNode firstRoutingNode = new RoutingNode("node2", discoveryNode2, Arrays.asList(firstRouting, secondRouting)); RoutingTable.Builder builder = RoutingTable.builder().add( @@ -1007,8 +1007,8 @@ public void addListener(Listener listener) { ClusterState updateClusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()) .put(discoveryNode3)).build(); - firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, true, ShardRoutingState.STARTED, 1); - secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", "node3", null, true, ShardRoutingState.RELOCATING, 1); + firstRouting = TestShardRouting.newShardRouting("test", 0, "node2", null, null, 1, true, ShardRoutingState.STARTED, 1); + secondRouting = TestShardRouting.newShardRouting("test", 1, "node2", "node3", null, 1, true, ShardRoutingState.RELOCATING, 1); firstRoutingNode = new RoutingNode("node2", discoveryNode2, Arrays.asList(firstRouting, secondRouting)); builder = RoutingTable.builder().add( IndexRoutingTable.builder("test") diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index a386883ad1b01..a596ec74ff268 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -90,7 +90,7 @@ public void testCanAllocateUsesMaxAvailableSpace() { ClusterInfoService cis = EmptyClusterInfoService.INSTANCE; DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null); - ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); DiscoveryNode node_0 = new DiscoveryNode("node_0", DummyTransportAddress.INSTANCE, Version.CURRENT); DiscoveryNode node_1 = new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT); @@ -135,22 +135,22 @@ public void testCanRemainUsesLeastAvailableSpace() { DiscoveryNode node_0 = new DiscoveryNode("node_0", DummyTransportAddress.INSTANCE, Version.CURRENT); DiscoveryNode node_1 = new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT); - ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_0, node_0.getId()); ShardRoutingHelper.moveToStarted(test_0); shardRoutingMap.put(test_0, "/node0/least"); - ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_1, node_1.getId()); ShardRoutingHelper.moveToStarted(test_1); shardRoutingMap.put(test_1, "/node1/least"); - ShardRouting test_2 = ShardRouting.newUnassigned("test", 2, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_2 = ShardRouting.newUnassigned("test", 2, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_2, node_1.getId()); ShardRoutingHelper.moveToStarted(test_2); shardRoutingMap.put(test_2, "/node1/most"); - ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, 1, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_3, node_1.getId()); ShardRoutingHelper.moveToStarted(test_3); // Intentionally not in the shardRoutingMap. We want to test what happens when we don't know where it is. @@ -215,17 +215,17 @@ public void testShardSizeAndRelocatingSize() { shardSizes.put("[test][2][r]", 1000L); shardSizes.put("[other][0][p]", 10000L); ClusterInfo info = new DevNullClusterInfo(ImmutableOpenMap.of(), ImmutableOpenMap.of(), shardSizes.build()); - ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, 1, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_0, "node1"); ShardRoutingHelper.moveToStarted(test_0); ShardRoutingHelper.relocate(test_0, "node2"); - ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, 1, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_1, "node2"); ShardRoutingHelper.moveToStarted(test_1); ShardRoutingHelper.relocate(test_1, "node1"); - ShardRouting test_2 = ShardRouting.newUnassigned("test", 2, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_2 = ShardRouting.newUnassigned("test", 2, null, 1, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_2, "node1"); ShardRoutingHelper.moveToStarted(test_2); @@ -239,13 +239,13 @@ public void testShardSizeAndRelocatingSize() { assertEquals(0l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/some/other/dev")); assertEquals(0l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/some/other/dev")); - ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, 1, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(test_3, "node1"); ShardRoutingHelper.moveToStarted(test_3); assertEquals(0l, DiskThresholdDecider.getShardSize(test_3, info)); - ShardRouting other_0 = ShardRouting.newUnassigned("other", 0, null, randomBoolean(), new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + ShardRouting other_0 = ShardRouting.newUnassigned("other", 0, null, 1, randomBoolean(), new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRoutingHelper.initialize(other_0, "node2"); ShardRoutingHelper.moveToStarted(other_0); ShardRoutingHelper.relocate(other_0, "node1"); diff --git a/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java b/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java index 4f28cf19d7bae..64190091a02fa 100644 --- a/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java +++ b/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java @@ -34,10 +34,7 @@ import org.elasticsearch.test.ESIntegTestCase; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.*; /** */ diff --git a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java index 73cbb51faed41..10fcd034bdb3b 100644 --- a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESAllocationTestCase; +import org.hamcrest.Matcher; import org.junit.Before; import java.io.IOException; @@ -64,12 +65,12 @@ public void buildTestAllocator() { * and processes only the applicable shard. */ public void testNoProcessReplica() { - ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, false, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); + ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, 1, false, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); assertThat(testAllocator.needToFindPrimaryCopy(shard), equalTo(false)); } - public void testNoProcessPrimayNotAllcoatedBefore() { - ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, true, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); + public void testNoProcessPrimaryNotAllocatedBefore() { + ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, 1, true, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null)); assertThat(testAllocator.needToFindPrimaryCopy(shard), equalTo(false)); } @@ -79,9 +80,7 @@ public void testNoProcessPrimayNotAllcoatedBefore() { public void testNoAsyncFetchData() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders()); boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); - assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); - assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + assertNotAllocated(allocation, changed); } /** @@ -91,9 +90,7 @@ public void testNoAllocationFound() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders()); testAllocator.addData(node1, -1); boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); - assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); - assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + assertNotAllocated(allocation, changed); } /** @@ -103,9 +100,14 @@ public void testStoreException() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders()); testAllocator.addData(node1, 3, new CorruptIndexException("test", "test")); boolean changed = testAllocator.allocateUnassigned(allocation); + assertNotAllocated(allocation, changed); + } + + protected void assertNotAllocated(RoutingAllocation allocation, boolean changed) { assertThat(changed, equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).primaryTerm(), equalTo(0l)); } /** @@ -116,9 +118,18 @@ public void testFoundAllocationAndAllocating() { testAllocator.addData(node1, 10); boolean changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(true)); + assertShardAllocated(allocation, node1); + } + + protected void assertShardAllocated(RoutingAllocation allocation, DiscoveryNode... nodes) { assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.id())); + final Matcher[] nodeMatchers = new Matcher[nodes.length]; + for (int i = 0; i < nodes.length; i++) { + nodeMatchers[i] = equalTo(nodes[i].id()); + } + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), anyOf((Matcher[]) nodeMatchers)); + assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).primaryTerm(), equalTo(1L)); } /** @@ -129,9 +140,7 @@ public void testFoundAllocationButThrottlingDecider() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders()); testAllocator.addData(node1, 10); boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); - assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); - assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + assertNotAllocated(allocation, changed); } /** @@ -143,9 +152,7 @@ public void testFoundAllocationButNoDecider() { testAllocator.addData(node1, 10); boolean changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(true)); - assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.id())); + assertShardAllocated(allocation, node1); } /** @@ -156,9 +163,7 @@ public void testAllocateToTheHighestVersion() { testAllocator.addData(node1, 10).addData(node2, 12); boolean changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(true)); - assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.id())); + assertShardAllocated(allocation, node2); } /** @@ -219,10 +224,8 @@ public void testEnoughCopiesFoundForAllocation() { allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null, System.nanoTime()); changed = testAllocator.allocateUnassigned(allocation); assertThat(changed, equalTo(true)); - assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), anyOf(equalTo(node2.id()), equalTo(node1.id()))); + assertShardAllocated(allocation, node1, node2); } /** @@ -262,12 +265,11 @@ public void testEnoughCopiesFoundForAllocationWithDifferentVersion() { assertThat(changed, equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); - assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.id())); + assertShardAllocated(allocation, node2); } public void testAllocationOnAnyNodeWithSharedFs() { - ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, false, + ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, 1, false, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); @@ -291,7 +293,7 @@ public void testAllocationOnAnyNodeWithSharedFs() { } public void testAllocationOnAnyNodeShouldPutNodesWithExceptionsLast() { - ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, false, + ShardRouting shard = TestShardRouting.newShardRouting("test", 0, null, null, null, 1, false, ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); diff --git a/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java index 88499bf96cd9e..9a114bd11d5fa 100644 --- a/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/PriorityComparatorTests.java @@ -30,8 +30,9 @@ public class PriorityComparatorTests extends ESTestCase { public void testPreferNewIndices() { RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards((RoutingNodes) null); List shardRoutings = Arrays.asList(TestShardRouting.newShardRouting("oldest", 0, null, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + 1, randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), + TestShardRouting.newShardRouting("newest", 0, null, null, null, + 1, randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); Collections.shuffle(shardRoutings, random()); for (ShardRouting routing : shardRoutings) { shards.add(routing); @@ -59,9 +60,10 @@ protected Settings getIndexSettings(String index) { public void testPreferPriorityIndices() { RoutingNodes.UnassignedShards shards = new RoutingNodes.UnassignedShards((RoutingNodes) null); - List shardRoutings = Arrays.asList(TestShardRouting.newShardRouting("oldest", 0, null, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), TestShardRouting.newShardRouting("newest", 0, null, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + List shardRoutings = Arrays.asList(TestShardRouting.newShardRouting("oldest", 0, null, null, null, 1, + randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar")), + TestShardRouting.newShardRouting("newest", 0, null, null, null, 1, + randomBoolean(), ShardRoutingState.UNASSIGNED, 0, new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); Collections.shuffle(shardRoutings, random()); for (ShardRouting routing : shardRoutings) { shards.add(routing); @@ -97,15 +99,16 @@ public void testPriorityComparatorSort() { if (frequently()) { indices[i] = new IndexMeta("idx_2015_04_" + String.format(Locale.ROOT, "%02d", i), randomIntBetween(1, 1000), randomIntBetween(1, 10000)); } else { // sometimes just use defaults - indices[i] = new IndexMeta("idx_2015_04_" + String.format(Locale.ROOT, "%02d", i)); + indices[i] = new IndexMeta("idx_2015_04_" + String.format(Locale.ROOT, "%02d", i)); } map.put(indices[i].name, indices[i]); } int numShards = randomIntBetween(10, 100); for (int i = 0; i < numShards; i++) { IndexMeta indexMeta = randomFrom(indices); - shards.add(TestShardRouting.newShardRouting(indexMeta.name, randomIntBetween(1, 5), null, null, null, - randomBoolean(), ShardRoutingState.UNASSIGNED, randomIntBetween(0, 100), new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); + shards.add(TestShardRouting.newShardRouting(indexMeta.name, randomIntBetween(1, 5), null, null, null, 1, + randomBoolean(), ShardRoutingState.UNASSIGNED, randomIntBetween(0, 100), + new UnassignedInfo(randomFrom(UnassignedInfo.Reason.values()), "foobar"))); } shards.sort(new PriorityComparator() { @Override @@ -128,7 +131,7 @@ protected Settings getIndexSettings(String index) { assertTrue("creationDate mismatch, expected:" + currentMeta.creationDate + " after " + prevMeta.creationDate, prevMeta.creationDate > currentMeta.creationDate); } } else { - assertTrue("priority mismatch, expected:" + currentMeta.priority + " after " + prevMeta.priority, prevMeta.priority > currentMeta.priority); + assertTrue("priority mismatch, expected:" + currentMeta.priority + " after " + prevMeta.priority, prevMeta.priority > currentMeta.priority); } } previous = routing; diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 01c76b465a977..95fb661d11306 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -19,11 +19,16 @@ package org.elasticsearch.gateway; +import com.carrotsearch.hppc.cursors.ObjectCursor; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.common.settings.Settings; @@ -40,6 +45,10 @@ import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSIndexStore; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.IntStream; + import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.Settings.settingsBuilder; @@ -48,10 +57,7 @@ import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.*; @ClusterScope(numDataNodes = 0, scope = Scope.TEST) public class RecoveryFromGatewayIT extends ESIntegTestCase { @@ -79,10 +85,13 @@ public void testOneNodeRecoverFromGateway() throws Exception { assertHitCount(client().prepareSearch().setSize(0).setQuery(termQuery("appAccountIds", 179)).execute().actionGet(), 2); ensureYellow("test"); // wait for primary allocations here otherwise if we have a lot of shards we might have a // shard that is still in post recovery when we restart and the ensureYellow() below will timeout + + Map primaryTerms = assertAndCapturePrimaryTerms(null); internalCluster().fullRestart(); logger.info("Running Cluster Health (wait for the shards to startup)"); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); client().admin().indices().prepareRefresh().execute().actionGet(); assertHitCount(client().prepareSearch().setSize(0).setQuery(termQuery("appAccountIds", 179)).execute().actionGet(), 2); @@ -91,11 +100,45 @@ public void testOneNodeRecoverFromGateway() throws Exception { logger.info("Running Cluster Health (wait for the shards to startup)"); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); client().admin().indices().prepareRefresh().execute().actionGet(); assertHitCount(client().prepareSearch().setSize(0).setQuery(termQuery("appAccountIds", 179)).execute().actionGet(), 2); } + private Map assertAndCapturePrimaryTerms(Map previousTerms) { + if (previousTerms == null) { + previousTerms = new HashMap<>(); + } + final Map result = new HashMap<>(); + final ClusterState state = client().admin().cluster().prepareState().get().getState(); + for (ObjectCursor cursor : state.metaData().indices().values()) { + final IndexMetaData indexMetaData = cursor.value; + final String index = indexMetaData.getIndex(); + final long[] previous = previousTerms.get(index); + final long[] current = IntStream.range(0, indexMetaData.getNumberOfShards()).mapToLong(indexMetaData::primaryTerm).toArray(); + if (previous == null) { + result.put(index, current); + } else { + assertThat("number of terms changed for index [" + index + "]", current.length, equalTo(previous.length)); + for (int shard = 0; shard < current.length; shard++) { + assertThat("primary term didn't increase for [" + index + "][" + shard + "]", current[shard], greaterThan(previous[shard])); + } + result.put(index, current); + } + } + + for (IndexRoutingTable indexRoutingTable : state.routingTable()) { + final long[] terms = result.get(indexRoutingTable.index()); + for (IndexShardRoutingTable shardRoutingTable : indexRoutingTable) { + for (ShardRouting routing : shardRoutingTable.shards()) { + assertThat("wrong primary term for " + routing, routing.primaryTerm(), equalTo(terms[routing.shardId().id()])); + } + } + } + return result; + } + public void testSingleNodeNoFlush() throws Exception { internalCluster().startNode(); @@ -154,10 +197,14 @@ SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 1) logger.info("Ensure all primaries have been started"); ensureYellow(); } + + Map primaryTerms = assertAndCapturePrimaryTerms(null); + internalCluster().fullRestart(); logger.info("Running Cluster Health (wait for the shards to startup)"); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); for (int i = 0; i <= randomInt(10); i++) { assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), value1Docs + value2Docs); @@ -171,6 +218,7 @@ SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 1) logger.info("Running Cluster Health (wait for the shards to startup)"); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); for (int i = 0; i <= randomInt(10); i++) { assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), value1Docs + value2Docs); @@ -192,10 +240,13 @@ public void testSingleNodeWithFlush() throws Exception { ensureYellow("test"); // wait for primary allocations here otherwise if we have a lot of shards we might have a // shard that is still in post recovery when we restart and the ensureYellow() below will timeout + Map primaryTerms = assertAndCapturePrimaryTerms(null); + internalCluster().fullRestart(); logger.info("Running Cluster Health (wait for the shards to startup)"); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); for (int i = 0; i < 10; i++) { assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2); @@ -205,6 +256,7 @@ public void testSingleNodeWithFlush() throws Exception { logger.info("Running Cluster Health (wait for the shards to startup)"); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); for (int i = 0; i < 10; i++) { assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2); @@ -227,6 +279,8 @@ public void testTwoNodeFirstNodeCleared() throws Exception { assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2); } + Map primaryTerms = assertAndCapturePrimaryTerms(null); + internalCluster().fullRestart(new RestartCallback() { @Override public Settings onNodeStopped(String nodeName) throws Exception { @@ -242,6 +296,7 @@ public boolean clearData(String nodeName) { logger.info("Running Cluster Health (wait for the shards to startup)"); ensureGreen(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); for (int i = 0; i < 10; i++) { assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).execute().actionGet(), 2); @@ -267,6 +322,8 @@ public void testLatestVersionLoaded() throws Exception { String metaDataUuid = client().admin().cluster().prepareState().execute().get().getState().getMetaData().clusterUUID(); assertThat(metaDataUuid, not(equalTo("_na_"))); + Map primaryTerms = assertAndCapturePrimaryTerms(null); + logger.info("--> closing first node, and indexing more data to the second node"); internalCluster().fullRestart(new RestartCallback() { @@ -306,6 +363,7 @@ public void doAfterNodes(int numNodes, Client client) throws Exception { logger.info("--> running cluster_health (wait for the shards to startup)"); ensureGreen(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); assertThat(client().admin().cluster().prepareState().execute().get().getState().getMetaData().clusterUUID(), equalTo(metaDataUuid)); @@ -379,11 +437,15 @@ public void testReusePeerRecovery() throws Exception { .setTransientSettings(settingsBuilder() .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE)) .get(); + + Map primaryTerms = assertAndCapturePrimaryTerms(null); + logger.info("--> full cluster restart"); internalCluster().fullRestart(); logger.info("--> waiting for cluster to return to green after {}shutdown", useSyncIds ? "" : "second "); ensureGreen(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); if (useSyncIds) { assertSyncIdsNotNull(); @@ -438,6 +500,8 @@ public void testRecoveryDifferentNodeOrderStartup() throws Exception { internalCluster().startNode(settingsBuilder().put("path.data", createTempDir()).build()); ensureGreen(); + Map primaryTerms = assertAndCapturePrimaryTerms(null); + internalCluster().fullRestart(new RestartCallback() { @@ -448,6 +512,7 @@ public boolean doRestart(String nodeName) { }); ensureYellow(); + primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); assertThat(client().admin().indices().prepareExists("test").execute().actionGet().isExists(), equalTo(true)); assertHitCount(client().prepareSearch("test").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet(), 1); diff --git a/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 9a053b365271d..c8cf933b92ae5 100644 --- a/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -281,8 +281,8 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide RoutingTable routingTable = RoutingTable.builder() .add(IndexRoutingTable.builder(shardId.getIndex()) .addIndexShard(new IndexShardRoutingTable.Builder(shardId) - .addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED, 10)) - .addShard(ShardRouting.newUnassigned(shardId.getIndex(), shardId.getId(), null, false, new UnassignedInfo(reason, null))) + .addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), 1, true, ShardRoutingState.STARTED, 10)) + .addShard(ShardRouting.newUnassigned(shardId.getIndex(), shardId.getId(), null, 1, false, new UnassignedInfo(reason, null))) .build()) ) .build(); @@ -300,8 +300,8 @@ private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDecid RoutingTable routingTable = RoutingTable.builder() .add(IndexRoutingTable.builder(shardId.getIndex()) .addIndexShard(new IndexShardRoutingTable.Builder(shardId) - .addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), true, ShardRoutingState.STARTED, 10)) - .addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node2.id(), null, null, false, ShardRoutingState.INITIALIZING, 10, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null))) + .addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node1.id(), 1, true, ShardRoutingState.STARTED, 10)) + .addShard(TestShardRouting.newShardRouting(shardId.getIndex(), shardId.getId(), node2.id(), null, null, 1, false, ShardRoutingState.INITIALIZING, 10, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null))) .build()) ) .build(); diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 2a122c38dde2e..d1bf85e1f2125 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -68,6 +68,7 @@ import org.elasticsearch.index.mapper.internal.SourceFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.object.RootObjectMapper; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.*; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.DirectoryService; @@ -192,9 +193,10 @@ private Document testDocument() { private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, Document document, BytesReference source, Mapping mappingUpdate) { Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); + Field seqNoField = new NumericDocValuesField("_seq_no", 0); document.add(uidField); document.add(versionField); - return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); + return new ParsedDocument(uidField, versionField, seqNoField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); } protected Store createStore() throws IOException { @@ -248,7 +250,8 @@ public EngineConfig config(IndexSettings indexSettings, Store store, Path transl public void onFailedEngine(String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test } - }, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + }, new TranslogHandler(shardId.index().getName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), + translogConfig, TimeValue.timeValueMinutes(5)); try { config.setCreate(Lucene.indexExists(store.directory()) == false); } catch (IOException e) { @@ -786,7 +789,9 @@ public void testRenewSyncFlush() throws Exception { engine.flush(); final boolean forceMergeFlushes = randomBoolean(); if (forceMergeFlushes) { - engine.index(new Engine.Index(newUid("3"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos())); + engine.index(new Engine.Index(newUid("3"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_ANY, + VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, + System.nanoTime() - engine.engineConfig.getFlushMergesAfter().nanos())); } else { engine.index(new Engine.Index(newUid("3"), doc)); } @@ -826,7 +831,7 @@ public void testRenewSyncFlush() throws Exception { } } - public void testSycnedFlushSurvivesEngineRestart() throws IOException { + public void testSyncedFlushSurvivesEngineRestart() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); engine.index(new Engine.Index(newUid("1"), doc)); @@ -845,7 +850,7 @@ public void testSycnedFlushSurvivesEngineRestart() throws IOException { assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); } - public void testSycnedFlushVanishesOnReplay() throws IOException { + public void testSyncedFlushVanishesOnReplay() throws IOException { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); engine.index(new Engine.Index(newUid("1"), doc)); @@ -874,8 +879,7 @@ public void testVersioningNewCreate() { Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED); engine.index(create); assertThat(create.version(), equalTo(1l)); - - create = new Engine.Index(newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + create = new Engine.Index(newUid("1"), doc, create.seqNo(), create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(create); assertThat(create.version(), equalTo(1l)); } @@ -886,18 +890,18 @@ public void testVersioningNewIndex() { engine.index(index); assertThat(index.version(), equalTo(1l)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(1l)); } public void testExternalVersioningNewIndex() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(12l)); - index = new Engine.Index(newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(12l)); } @@ -912,7 +916,7 @@ public void testVersioningIndexConflict() { engine.index(index); assertThat(index.version(), equalTo(2l)); - index = new Engine.Index(newUid("1"), doc, 1l, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0); try { engine.index(index); fail(); @@ -921,7 +925,7 @@ public void testVersioningIndexConflict() { } // future versions should not work as well - index = new Engine.Index(newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -932,15 +936,15 @@ public void testVersioningIndexConflict() { public void testExternalVersioningIndexConflict() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(12l)); - index = new Engine.Index(newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 14, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(14l)); - index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 13, VersionType.EXTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -961,7 +965,7 @@ public void testVersioningIndexConflictWithFlush() { engine.flush(); - index = new Engine.Index(newUid("1"), doc, 1l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -970,7 +974,7 @@ public void testVersioningIndexConflictWithFlush() { } // future versions should not work as well - index = new Engine.Index(newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -981,17 +985,17 @@ public void testVersioningIndexConflictWithFlush() { public void testExternalVersioningIndexConflictWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index index = new Engine.Index(newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + Engine.Index index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 12, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(12l)); - index = new Engine.Index(newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 14, VersionType.EXTERNAL, PRIMARY, 0); engine.index(index); assertThat(index.version(), equalTo(14l)); engine.flush(); - index = new Engine.Index(newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 13, VersionType.EXTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -1105,7 +1109,7 @@ public void testVersioningDeleteConflict() { engine.index(index); assertThat(index.version(), equalTo(2l)); - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1114,7 +1118,7 @@ public void testVersioningDeleteConflict() { } // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1123,12 +1127,12 @@ public void testVersioningDeleteConflict() { } // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0, false); engine.delete(delete); assertThat(delete.version(), equalTo(3l)); // now check if we can index to a delete doc with version - index = new Engine.Index(newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -1136,13 +1140,9 @@ public void testVersioningDeleteConflict() { // all is well } - // we shouldn't be able to create as well - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(create); - } catch (VersionConflictEngineException e) { - // all is well - } + // we should be able to create as well + Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + engine.index(create); } public void testVersioningDeleteConflictWithFlush() { @@ -1157,7 +1157,7 @@ public void testVersioningDeleteConflictWithFlush() { engine.flush(); - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 1l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1166,7 +1166,7 @@ public void testVersioningDeleteConflictWithFlush() { } // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 3l, VersionType.INTERNAL, PRIMARY, 0, false); try { engine.delete(delete); fail(); @@ -1177,14 +1177,14 @@ public void testVersioningDeleteConflictWithFlush() { engine.flush(); // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); + delete = new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0, false); engine.delete(delete); assertThat(delete.version(), equalTo(3l)); engine.flush(); // now check if we can index to a delete doc with version - index = new Engine.Index(newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + index = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2l, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(index); fail(); @@ -1192,22 +1192,18 @@ public void testVersioningDeleteConflictWithFlush() { // all is well } - // we shouldn't be able to create as well - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(create); - } catch (VersionConflictEngineException e) { - // all is well - } + // we should be able to create + Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + engine.index(create); } public void testVersioningCreateExistsException() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + Engine.Index create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); engine.index(create); assertThat(create.version(), equalTo(1l)); - create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + create = new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(create); fail(); @@ -1218,13 +1214,13 @@ public void testVersioningCreateExistsException() { public void testVersioningCreateExistsExceptionWithFlush() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, null); - Engine.Index create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + Engine.Index create = new Engine.Index(newUid("1"), doc, -1, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); engine.index(create); assertThat(create.version(), equalTo(1l)); engine.flush(); - create = new Engine.Index(newUid("1"), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); + create = new Engine.Index(newUid("1"), doc, -1, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, 0); try { engine.index(create); fail(); @@ -1244,12 +1240,12 @@ public void testVersioningReplicaConflict1() { assertThat(index.version(), equalTo(2l)); // apply the second index to the replica, should work fine - index = new Engine.Index(newUid("1"), doc, index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(2l)); // now, the old one should not work - index = new Engine.Index(newUid("1"), doc, 1l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 1l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); try { replicaEngine.index(index); fail(); @@ -1259,7 +1255,7 @@ public void testVersioningReplicaConflict1() { // second version on replica should fail as well try { - index = new Engine.Index(newUid("1"), doc, 2l + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 2l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(2l)); @@ -1275,7 +1271,7 @@ public void testVersioningReplicaConflict2() { assertThat(index.version(), equalTo(1l)); // apply the first index to the replica, should work fine - index = new Engine.Index(newUid("1"), doc, 1l + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 1l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); assertThat(index.version(), equalTo(1l)); @@ -1291,14 +1287,14 @@ public void testVersioningReplicaConflict2() { assertThat(delete.version(), equalTo(3l)); // apply the delete on the replica (skipping the second index) - delete = new Engine.Delete("test", "1", newUid("1"), 3l + delete = new Engine.Delete("test", "1", newUid("1"), delete.seqNo(), 3l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); replicaEngine.delete(delete); assertThat(delete.version(), equalTo(3l)); // second time delete with same version should fail try { - delete = new Engine.Delete("test", "1", newUid("1"), 3l + delete = new Engine.Delete("test", "1", newUid("1"), delete.seqNo(), 3l , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); replicaEngine.delete(delete); fail("excepted VersionConflictEngineException to be thrown"); @@ -1308,7 +1304,7 @@ public void testVersioningReplicaConflict2() { // now do the second index on the replica, it should fail try { - index = new Engine.Index(newUid("1"), doc, 2l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + index = new Engine.Index(newUid("1"), doc, index.seqNo(), 2l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); replicaEngine.index(index); fail("excepted VersionConflictEngineException to be thrown"); } catch (VersionConflictEngineException e) { @@ -1402,6 +1398,53 @@ public void testIndexWriterInfoStream() { } } + public void testSeqNoAndLocalCheckpoint() { + int opCount = randomIntBetween(1, 10); + long seqNoCount = -1; + for (int op = 0; op < opCount; op++) { + final String id = randomFrom("1", "2", "3"); + ParsedDocument doc = testParsedDocument(id, id, "test", null, -1, -1, testDocumentWithTextField(), B_1, null); + if (randomBoolean()) { + final Engine.Index index = new Engine.Index(newUid(id), doc, + SequenceNumbersService.UNASSIGNED_SEQ_NO, + rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, + PRIMARY, System.currentTimeMillis()); + + try { + engine.index(index); + } catch (VersionConflictEngineException e) { + // OK + } + if (index.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoCount++; + Engine.Index replica = new Engine.Index(index.uid(), index.parsedDoc(), index.seqNo(), + index.version(), VersionType.EXTERNAL, REPLICA, System.currentTimeMillis()); + replicaEngine.index(replica); + } + } else { + final Engine.Delete delete = new Engine.Delete("test", id, newUid(id), + SequenceNumbersService.UNASSIGNED_SEQ_NO, + rarely() ? 100 : Versions.MATCH_ANY, VersionType.INTERNAL, + PRIMARY, System.currentTimeMillis(), false); + try { + engine.delete(delete); + } catch (VersionConflictEngineException e) { + // OK + } + if (delete.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + seqNoCount++; + Engine.Delete replica = new Engine.Delete(delete.type(), delete.id(), delete.uid(), delete.seqNo(), + delete.version(), VersionType.EXTERNAL, REPLICA, System.currentTimeMillis(), false); + replicaEngine.delete(replica); + } + } + } + assertThat(engine.seqNoStats().getMaxSeqNo(), equalTo(seqNoCount)); + assertThat(engine.seqNoStats().getLocalCheckpoint(), equalTo(seqNoCount)); + assertThat(replicaEngine.seqNoStats().getMaxSeqNo(), equalTo(seqNoCount)); + assertThat(replicaEngine.seqNoStats().getLocalCheckpoint(), equalTo(seqNoCount)); + } + // #8603: make sure we can separately log IFD's messages public void testIndexWriterIFDInfoStream() { assumeFalse("who tests the tester?", VERBOSE); @@ -1449,10 +1492,10 @@ public void testEnableGcDeletes() throws Exception { document.add(new TextField("value", "test1", Field.Store.YES)); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_2, null); - engine.index(new Engine.Index(newUid("1"), doc, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); // Delete document we just added: - engine.delete(new Engine.Delete("test", "1", newUid("1"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", "1", newUid("1"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); // Get should not find the document Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid("1"))); @@ -1466,7 +1509,7 @@ public void testEnableGcDeletes() throws Exception { } // Delete non-existent document - engine.delete(new Engine.Delete("test", "2", newUid("2"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", "2", newUid("2"), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); // Get should not find the document (we never indexed uid=2): getResult = engine.get(new Engine.Get(true, newUid("2"))); @@ -1474,7 +1517,7 @@ public void testEnableGcDeletes() throws Exception { // Try to index uid=1 with a too-old version, should fail: try { - engine.index(new Engine.Index(newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid("1"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); fail("did not hit expected exception"); } catch (VersionConflictEngineException vcee) { // expected @@ -1486,7 +1529,7 @@ public void testEnableGcDeletes() throws Exception { // Try to index uid=2 with a too-old version, should fail: try { - engine.index(new Engine.Index(newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid("2"), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); fail("did not hit expected exception"); } catch (VersionConflictEngineException vcee) { // expected @@ -1571,7 +1614,7 @@ public void testDeletesAloneCanTriggerRefresh() throws Exception { for (int i = 0; i < 100; i++) { String id = Integer.toString(i); ParsedDocument doc = testParsedDocument(id, id, "test", null, -1, -1, testDocument(), B_1, null); - engine.index(new Engine.Index(newUid(id), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + engine.index(new Engine.Index(newUid(id), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); } // Force merge so we know all merges are done before we start deleting: @@ -1582,20 +1625,17 @@ public void testDeletesAloneCanTriggerRefresh() throws Exception { s.close(); for (int i = 0; i < 100; i++) { String id = Integer.toString(i); - engine.delete(new Engine.Delete("test", id, newUid(id), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + engine.delete(new Engine.Delete("test", id, newUid(id), SequenceNumbersService.UNASSIGNED_SEQ_NO, 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); } // We must assertBusy because refresh due to version map being full is done in background (REFRESH) thread pool: - assertBusy(new Runnable() { - @Override - public void run() { - Searcher s2 = engine.acquireSearcher("test"); - long version2 = ((DirectoryReader) s2.reader()).getVersion(); - s2.close(); - - // 100 buffered deletes will easily exceed 25% of our 1 KB indexing buffer so it should have forced a refresh: - assertThat(version2, greaterThan(version1)); - } + assertBusy(() -> { + Searcher s2 = engine.acquireSearcher("test"); + long version2 = ((DirectoryReader) s2.reader()).getVersion(); + s2.close(); + + // 100 buffered deletes will easily exceed 25% of our 1 KB indexing buffer so it should have forced a refresh: + assertThat(version2, greaterThan(version1)); }); } } @@ -1624,8 +1664,8 @@ public void testMissingTranslog() throws IOException { public void testTranslogReplayWithFailure() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, SequenceNumbersService.UNASSIGNED_SEQ_NO, -1, testDocument(), new BytesArray("{}"), null); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1677,7 +1717,8 @@ public void testSkipTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1772,7 +1813,7 @@ public void testUpgradeOldIndex() throws IOException { final int numExtraDocs = randomIntBetween(1, 10); for (int i = 0; i < numExtraDocs; i++) { ParsedDocument doc = testParsedDocument("extra" + Integer.toString(i), "extra" + Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1801,7 +1842,8 @@ public void testTranslogReplay() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } @@ -1850,7 +1892,7 @@ public void testTranslogReplay() throws IOException { int randomId = randomIntBetween(numDocs + 1, numDocs + 10); String uuidValue = "test#" + Integer.toString(randomId); ParsedDocument doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 1, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); if (flush) { @@ -1858,7 +1900,7 @@ public void testTranslogReplay() throws IOException { } doc = testParsedDocument(uuidValue, Integer.toString(randomId), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); + Engine.Index idxRequest = new Engine.Index(newUid(uuidValue), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, 2, VersionType.EXTERNAL, PRIMARY, System.nanoTime()); engine.index(idxRequest); engine.refresh("test"); assertThat(idxRequest.version(), equalTo(2l)); @@ -1925,7 +1967,8 @@ public void testRecoverFromForeignTranslog() throws IOException { final int numDocs = randomIntBetween(1, 10); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), Integer.toString(i), "test", null, -1, -1, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); + Engine.Index firstIndexRequest = new Engine.Index(newUid(Integer.toString(i)), doc, SequenceNumbersService.UNASSIGNED_SEQ_NO, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime()); engine.index(firstIndexRequest); assertThat(firstIndexRequest.version(), equalTo(1l)); } diff --git a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 3fe7a540bf8af..78fe08cd79b64 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -161,9 +161,10 @@ private ParseContext.Document testDocument() { private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, ParseContext.Document document, BytesReference source, Mapping mappingsUpdate) { Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); + Field seqNoField = new NumericDocValuesField("_seq_no", 0); document.add(uidField); document.add(versionField); - return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingsUpdate); + return new ParsedDocument(uidField, versionField, seqNoField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingsUpdate); } protected Store createStore(Path p) throws IOException { @@ -220,7 +221,9 @@ public EngineConfig config(IndexSettings indexSettings, Store store, Path transl @Override public void onFailedEngine(String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test - }}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + } + }, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5) + ); try { config.setCreate(Lucene.indexExists(store.directory()) == false); } catch (IOException e) { diff --git a/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java b/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java index ccbef6837c98a..10dd82a7ae84a 100644 --- a/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java +++ b/core/src/test/java/org/elasticsearch/index/indexing/IndexingSlowLogTests.java @@ -36,7 +36,8 @@ public class IndexingSlowLogTests extends ESTestCase { public void testSlowLogParsedDocumentPrinterSourceToLog() throws IOException { BytesReference source = JsonXContent.contentBuilder().startObject().field("foo", "bar").endObject().bytes(); - ParsedDocument pd = new ParsedDocument(new StringField("uid", "test:id", Store.YES), new IntField("version", 1, Store.YES), "id", + ParsedDocument pd = new ParsedDocument(new StringField("uid", "test:id", Store.YES), new IntField("version", 1, Store.YES), + new IntField("seqNo", 1, Store.YES), "id", "test", null, 0, -1, null, source, null); // Turning off document logging doesn't log source[] diff --git a/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java index 3f3c5702e8c7e..ef543620c9aae 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/internal/FieldNamesFieldMapperTests.java @@ -103,7 +103,7 @@ public void testInjectIntoDocDuringParsing() throws Exception { .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 { @@ -120,7 +120,7 @@ public void testExplicitEnabled() throws Exception { .endObject() .bytes()); - assertFieldNames(set("field", "_uid", "_type", "_version", "_source", "_all"), doc); + assertFieldNames(set("field", "_uid", "_type", "_version", "_seq_no", "_source", "_all"), doc); } public void testDisabled() throws Exception { diff --git a/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointServiceTests.java b/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointServiceTests.java new file mode 100644 index 0000000000000..429c60af00d76 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointServiceTests.java @@ -0,0 +1,192 @@ +/* + * 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.ElasticsearchException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.IndexSettingsModule; +import org.junit.Before; + +import java.util.*; +import java.util.concurrent.CyclicBarrier; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.isOneOf; + +public class LocalCheckpointServiceTests extends ESTestCase { + + LocalCheckpointService checkpointService; + + final int SMALL_CHUNK_SIZE = 4; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + checkpointService = getCheckpointService(); + } + + protected LocalCheckpointService getCheckpointService() { + return new LocalCheckpointService( + new ShardId("test", 0), + IndexSettingsModule.newIndexSettings("test", + Settings.builder() + .put(LocalCheckpointService.SETTINGS_BIT_ARRAYS_SIZE, SMALL_CHUNK_SIZE) + .build() + )); + } + + public void testSimplePrimary() { + long seqNo1, seqNo2; + assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO)); + seqNo1 = checkpointService.generateSeqNo(); + assertThat(seqNo1, equalTo(0L)); + checkpointService.markSeqNoAsCompleted(seqNo1); + assertThat(checkpointService.getCheckpoint(), equalTo(0L)); + seqNo1 = checkpointService.generateSeqNo(); + seqNo2 = checkpointService.generateSeqNo(); + assertThat(seqNo1, equalTo(1L)); + assertThat(seqNo2, equalTo(2L)); + checkpointService.markSeqNoAsCompleted(seqNo2); + assertThat(checkpointService.getCheckpoint(), equalTo(0L)); + checkpointService.markSeqNoAsCompleted(seqNo1); + assertThat(checkpointService.getCheckpoint(), equalTo(2L)); + } + + public void testSimpleReplica() { + assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO)); + checkpointService.markSeqNoAsCompleted(0L); + assertThat(checkpointService.getCheckpoint(), equalTo(0L)); + checkpointService.markSeqNoAsCompleted(2L); + assertThat(checkpointService.getCheckpoint(), equalTo(0L)); + checkpointService.markSeqNoAsCompleted(1L); + assertThat(checkpointService.getCheckpoint(), equalTo(2L)); + } + + public void testSimpleOverFlow() { + List seqNoList = new ArrayList<>(); + final boolean aligned = randomBoolean(); + final int maxOps = SMALL_CHUNK_SIZE * randomIntBetween(1, 5) + (aligned ? 0 : randomIntBetween(1, SMALL_CHUNK_SIZE - 1)); + + for (int i = 0; i < maxOps; i++) { + seqNoList.add(i); + } + Collections.shuffle(seqNoList, random()); + for (Integer seqNo : seqNoList) { + checkpointService.markSeqNoAsCompleted(seqNo); + } + assertThat(checkpointService.checkpoint, equalTo(maxOps - 1L)); + assertThat(checkpointService.processedSeqNo.size(), equalTo(aligned ? 0 : 1)); + } + + public void testConcurrentPrimary() throws InterruptedException { + Thread[] threads = new Thread[randomIntBetween(2, 5)]; + final int opsPerThread = randomIntBetween(10, 20); + final int maxOps = opsPerThread * threads.length; + final long unFinishedSeq = randomIntBetween(0, maxOps - 2); // make sure we always index the last seqNo to simplify maxSeq checks + logger.info("--> will run [{}] threads, maxOps [{}], unfinished seq no [{}]", threads.length, maxOps, unFinishedSeq); + final CyclicBarrier barrier = new CyclicBarrier(threads.length); + for (int t = 0; t < threads.length; t++) { + final int threadId = t; + threads[t] = new Thread(new AbstractRunnable() { + @Override + public void onFailure(Throwable t) { + throw new ElasticsearchException("failure in background thread", t); + } + + @Override + protected void doRun() throws Exception { + barrier.await(); + for (int i = 0; i < opsPerThread; i++) { + long seqNo = checkpointService.generateSeqNo(); + logger.info("[t{}] started [{}]", threadId, seqNo); + if (seqNo != unFinishedSeq) { + checkpointService.markSeqNoAsCompleted(seqNo); + logger.info("[t{}] completed [{}]", threadId, seqNo); + } + } + } + }, "testConcurrentPrimary_" + threadId); + threads[t].start(); + } + for (Thread thread : threads) { + thread.join(); + } + assertThat(checkpointService.getMaxSeqNo(), equalTo(maxOps - 1L)); + assertThat(checkpointService.getCheckpoint(), equalTo(unFinishedSeq - 1L)); + checkpointService.markSeqNoAsCompleted(unFinishedSeq); + assertThat(checkpointService.getCheckpoint(), equalTo(maxOps - 1L)); + assertThat(checkpointService.processedSeqNo.size(), isOneOf(0, 1)); + } + + public void testConcurrentReplica() throws InterruptedException { + Thread[] threads = new Thread[randomIntBetween(2, 5)]; + final int opsPerThread = randomIntBetween(10, 20); + final int maxOps = opsPerThread * threads.length; + final long unFinishedSeq = randomIntBetween(0, maxOps - 2); // make sure we always index the last seqNo to simplify maxSeq checks + Set seqNoList = new HashSet<>(); + for (int i = 0; i < maxOps; i++) { + seqNoList.add(i); + } + + final Integer[][] seqNoPerThread = new Integer[threads.length][]; + for (int t = 0; t < threads.length - 1; t++) { + int size = Math.min(seqNoList.size(), randomIntBetween(opsPerThread - 4, opsPerThread + 4)); + seqNoPerThread[t] = randomSubsetOf(size, seqNoList).toArray(new Integer[size]); + Arrays.sort(seqNoPerThread[t]); + seqNoList.removeAll(Arrays.asList(seqNoPerThread[t])); + } + seqNoPerThread[threads.length - 1] = seqNoList.toArray(new Integer[seqNoList.size()]); + logger.info("--> will run [{}] threads, maxOps [{}], unfinished seq no [{}]", threads.length, maxOps, unFinishedSeq); + final CyclicBarrier barrier = new CyclicBarrier(threads.length); + for (int t = 0; t < threads.length; t++) { + final int threadId = t; + threads[t] = new Thread(new AbstractRunnable() { + @Override + public void onFailure(Throwable t) { + throw new ElasticsearchException("failure in background thread", t); + } + + @Override + protected void doRun() throws Exception { + barrier.await(); + Integer[] ops = seqNoPerThread[threadId]; + for (int seqNo : ops) { + if (seqNo != unFinishedSeq) { + checkpointService.markSeqNoAsCompleted(seqNo); + logger.info("[t{}] completed [{}]", threadId, seqNo); + } + } + } + }, "testConcurrentPrimary_" + threadId); + threads[t].start(); + } + for (Thread thread : threads) { + thread.join(); + } + assertThat(checkpointService.getMaxSeqNo(), equalTo(maxOps - 1L)); + assertThat(checkpointService.getCheckpoint(), equalTo(unFinishedSeq - 1L)); + checkpointService.markSeqNoAsCompleted(unFinishedSeq); + assertThat(checkpointService.getCheckpoint(), equalTo(maxOps - 1L)); + } + +} diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index b5f33afa94c14..0fd9939e613a3 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -37,6 +37,8 @@ import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterInfoService; @@ -44,6 +46,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.SnapshotId; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.*; @@ -99,6 +102,7 @@ import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; /** @@ -205,8 +209,10 @@ public void testPersistenceStateMetadataPersistence() throws Exception { assertEquals(shardStateMetaData, getShardStateMetadata(shard)); assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID(), routing.allocationId())); - // test if we still write it even if the shard is not active - ShardRouting inactiveRouting = TestShardRouting.newShardRouting(shard.shardRouting.index(), shard.shardRouting.shardId().id(), shard.shardRouting.currentNodeId(), null, null, true, ShardRoutingState.INITIALIZING, shard.shardRouting.version() + 1); + // test if we don't write it if the shard is not active + ShardRouting inactiveRouting = TestShardRouting.newShardRouting(shard.shardRouting.index(), shard.shardRouting.shardId().id(), + shard.shardRouting.currentNodeId(), null, null, shard.shardRouting.primaryTerm(), true, ShardRoutingState.INITIALIZING, + shard.shardRouting.version() + 1); shard.persistMetadata(inactiveRouting, shard.shardRouting); shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, getShardStateMetadata(shard)); @@ -243,7 +249,9 @@ public void testDeleteShardState() throws IOException { ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); assertEquals(shardStateMetaData, getShardStateMetadata(shard)); - routing = TestShardRouting.newShardRouting(shard.shardId.index().getName(), shard.shardId.id(), routing.currentNodeId(), null, routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.allocationId(), shard.shardRouting.version() + 1); + routing = TestShardRouting.newShardRouting(shard.shardId.index().getName(), shard.shardId.id(), routing.currentNodeId(), null, + shard.shardRouting.primaryTerm(), routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.allocationId(), + shard.shardRouting.version() + 1); shard.updateRoutingEntry(routing, true); shard.deleteShardState(); @@ -315,7 +323,7 @@ public void testDeleteIndexDecreasesCounter() throws InterruptedException, Execu client().admin().indices().prepareDelete("test").get(); assertThat(indexShard.getOperationsCount(), equalTo(0)); try { - indexShard.incrementOperationCounter(); + indexShard.incrementOperationCounterOnPrimary(); fail("we should not be able to increment anymore"); } catch (IndexShardClosedException e) { // expected @@ -328,11 +336,39 @@ public void testIndexShardCounter() throws InterruptedException, ExecutionExcept IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService indexService = indicesService.indexServiceSafe("test"); IndexShard indexShard = indexService.getShardOrNull(0); + final long primaryTerm = indexShard.shardRouting.primaryTerm(); + // ugly hack to allow the shard to operated both as a replica and a primary + ShardRouting temp = indexShard.routingEntry(); + ShardRouting newShardRouting = TestShardRouting.newShardRouting(temp.getIndex(), temp.id(), temp.currentNodeId(), "BLA!", temp.primaryTerm(), + temp.primary(), ShardRoutingState.INITIALIZING, AllocationId.newRelocation(temp.allocationId()), temp.version() + 1); + indexShard.updateRoutingEntry(newShardRouting, false); assertEquals(0, indexShard.getOperationsCount()); - indexShard.incrementOperationCounter(); + if (randomBoolean()) { + indexShard.incrementOperationCounterOnPrimary(); + } else { + indexShard.incrementOperationCounterOnReplica(primaryTerm); + } assertEquals(1, indexShard.getOperationsCount()); - indexShard.incrementOperationCounter(); + if (randomBoolean()) { + indexShard.incrementOperationCounterOnPrimary(); + } else { + indexShard.incrementOperationCounterOnReplica(primaryTerm); + } assertEquals(2, indexShard.getOperationsCount()); + + try { + indexShard.incrementOperationCounterOnReplica(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.incrementOperationCounterOnReplica(primaryTerm + 1 + randomInt(20)); + + + indexShard.decrementOperationCounter(); indexShard.decrementOperationCounter(); indexShard.decrementOperationCounter(); assertEquals(0, indexShard.getOperationsCount()); @@ -555,7 +591,8 @@ public void testShardStats() throws IOException { IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService("test"); IndexShard shard = test.getShardOrNull(0); - ShardStats stats = new ShardStats(shard.routingEntry(), shard.shardPath(), new CommonStats(shard, new CommonStatsFlags()), shard.commitStats()); + ShardStats stats = new ShardStats(shard.routingEntry(), shard.shardPath(), new CommonStats(shard, new CommonStatsFlags()), + shard.commitStats(), shard.seqNoStats()); assertEquals(shard.shardPath().getRootDataPath().toString(), stats.getDataPath()); assertEquals(shard.shardPath().getRootStatePath().toString(), stats.getStatePath()); assertEquals(shard.shardPath().isCustomDataPath(), stats.isCustomDataPath()); @@ -583,9 +620,10 @@ public void testShardStats() throws IOException { private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, ParseContext.Document document, BytesReference source, Mapping mappingUpdate) { Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); Field versionField = new NumericDocValuesField("_version", 0); + Field seqNoField = new NumericDocValuesField("_seq_no", 0); document.add(uidField); document.add(versionField); - return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); + return new ParsedDocument(uidField, versionField, seqNoField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingUpdate); } public void testPreIndex() throws IOException { @@ -774,7 +812,7 @@ public void testRecoverFromStore() throws IOException { assertHitCount(response, 1); } - public void testFailIfIndexNotPresentInRecoverFromStore() throws IOException { + public void testFailIfIndexNotPresentInRecoverFromStore() throws Throwable { createIndex("test"); ensureGreen(); IndicesService indicesService = getInstanceFromNode(IndicesService.class); @@ -824,7 +862,12 @@ public void testFailIfIndexNotPresentInRecoverFromStore() throws IOException { newShard.updateRoutingEntry(routing, true); SearchResponse response = client().prepareSearch().get(); assertHitCount(response, 0); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(true).get(); + // we can't issue this request through a client because of the inconsistencies we created with the cluster state + // doing it directly instead + IndexRequest request = client().prepareIndex("test", "test", "0").setSource("{}").request(); + request.process(MetaData.builder().put(test.getMetaData(), false).build(), null, false, "test"); + TransportIndexAction.executeIndexRequestOnPrimary(request, newShard, null); + newShard.refresh("test"); assertHitCount(client().prepareSearch().get(), 1); } diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 26faa02a17dcc..941f797bb2ed6 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -112,9 +112,9 @@ public void tearDown() throws Exception { protected Translog create(Path path) throws IOException { Settings build = Settings.settingsBuilder() - .put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name()) - .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) - .build(); + .put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name()) + .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) + .build(); TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null); return new Translog(translogConfig); } @@ -302,7 +302,7 @@ public void testStats() throws IOException { assertThat(stats.estimatedNumberOfOperations(), equalTo(0l)); assertThat(stats.getTranslogSizeInBytes(), equalTo(firstOperationPosition)); assertEquals(6, total.estimatedNumberOfOperations()); - assertEquals(431, total.getTranslogSizeInBytes()); + assertEquals(437, total.getTranslogSizeInBytes()); BytesStreamOutput out = new BytesStreamOutput(); total.writeTo(out); @@ -310,11 +310,11 @@ public void testStats() throws IOException { copy.readFrom(StreamInput.wrap(out.bytes())); assertEquals(6, copy.estimatedNumberOfOperations()); - assertEquals(431, copy.getTranslogSizeInBytes()); + assertEquals(437, copy.getTranslogSizeInBytes()); assertEquals("\"translog\"{\n" + - " \"operations\" : 6,\n" + - " \"size_in_bytes\" : 431\n" + - "}", copy.toString().trim()); + " \"operations\" : 6,\n" + + " \"size_in_bytes\" : 437\n" + + "}", copy.toString().trim()); try { new TranslogStats(1, -1); @@ -810,7 +810,7 @@ public void testLocationComparison() throws IOException { int count = 0; for (int op = 0; op < translogOperations; op++) { locations.add(translog.add(new Translog.Index("test", "" + op, Integer.toString(++count).getBytes(Charset.forName("UTF-8"))))); - if (rarely() && translogOperations > op+1) { + if (rarely() && translogOperations > op + 1) { translog.commit(); } } @@ -889,7 +889,7 @@ public void testTranslogWriter() throws IOException { final TranslogReader reader = randomBoolean() ? writer : translog.openReader(writer.path(), Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME))); for (int i = 0; i < numOps; i++) { ByteBuffer buffer = ByteBuffer.allocate(4); - reader.readBytes(buffer, reader.getFirstOperationOffset() + 4*i); + reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * i); buffer.flip(); final int value = buffer.getInt(); assertEquals(i, value); @@ -928,9 +928,9 @@ public void testBasicRecovery() throws IOException { for (int op = 0; op < translogOperations; op++) { locations.add(translog.add(new Translog.Index("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); final boolean commit = commitOften ? frequently() : rarely(); - if (commit && op < translogOperations-1) { + if (commit && op < translogOperations - 1) { translog.commit(); - minUncommittedOp = op+1; + minUncommittedOp = op + 1; translogGeneration = translog.getGeneration(); } } @@ -964,7 +964,7 @@ public void testBasicRecovery() throws IOException { public void testRecoveryUncommitted() throws IOException { List locations = new ArrayList<>(); int translogOperations = randomIntBetween(10, 100); - final int prepareOp = randomIntBetween(0, translogOperations-1); + final int prepareOp = randomIntBetween(0, translogOperations - 1); Translog.TranslogGeneration translogGeneration = null; final boolean sync = randomBoolean(); for (int op = 0; op < translogOperations; op++) { @@ -1017,7 +1017,7 @@ public void testRecoveryUncommitted() throws IOException { public void testRecoveryUncommittedFileExists() throws IOException { List locations = new ArrayList<>(); int translogOperations = randomIntBetween(10, 100); - final int prepareOp = randomIntBetween(0, translogOperations-1); + final int prepareOp = randomIntBetween(0, translogOperations - 1); Translog.TranslogGeneration translogGeneration = null; final boolean sync = randomBoolean(); for (int op = 0; op < translogOperations; op++) { @@ -1093,12 +1093,12 @@ public void testRecoveryUncommittedCorryptedCheckpoint() throws IOException { config.setTranslogGeneration(translogGeneration); Path ckp = config.getTranslogPath().resolve(Translog.CHECKPOINT_FILE_NAME); Checkpoint read = Checkpoint.read(ckp); - Checkpoint corrupted = new Checkpoint(0,0,0); + Checkpoint corrupted = new Checkpoint(0, 0, 0); Checkpoint.write(config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation)), corrupted, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); try (Translog translog = new Translog(config)) { - fail("corrupted"); + 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)) { @@ -1134,7 +1134,7 @@ public void testLocationHashCodeEquals() throws IOException { List locations = new ArrayList<>(); List locations2 = new ArrayList<>(); int translogOperations = randomIntBetween(10, 100); - try(Translog translog2 = create(createTempDir())) { + try (Translog translog2 = create(createTempDir())) { for (int op = 0; op < translogOperations; op++) { locations.add(translog.add(new Translog.Index("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); locations2.add(translog2.add(new Translog.Index("test", "" + op, Integer.toString(op).getBytes(Charset.forName("UTF-8"))))); @@ -1173,7 +1173,7 @@ public void testOpenForeignTranslog() throws IOException { Translog.TranslogGeneration translogGeneration = translog.getGeneration(); translog.close(); - config.setTranslogGeneration(new Translog.TranslogGeneration(randomRealisticUnicodeOfCodepointLengthBetween(1, translogGeneration.translogUUID.length()),translogGeneration.translogFileGeneration)); + config.setTranslogGeneration(new Translog.TranslogGeneration(randomRealisticUnicodeOfCodepointLengthBetween(1, translogGeneration.translogUUID.length()), translogGeneration.translogFileGeneration)); try { new Translog(config); fail("translog doesn't belong to this UUID"); @@ -1260,12 +1260,12 @@ public void run() { case CREATE: case INDEX: op = new Translog.Index("test", threadId + "_" + opCount, - randomUnicodeOfLengthBetween(1, 20 * 1024).getBytes("UTF-8")); + randomUnicodeOfLengthBetween(1, 20 * 1024).getBytes("UTF-8")); break; case DELETE: op = new Translog.Delete(new Term("_uid", threadId + "_" + opCount), - 1 + randomInt(100000), - randomFrom(VersionType.values())); + opCount, 1 + randomInt(100000), + randomFrom(VersionType.values())); break; default: throw new ElasticsearchException("not supported op type"); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java index 8de3af258270e..5c08c2d60cca0 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerIT.java @@ -54,11 +54,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.Settings.builder; -import static org.elasticsearch.index.shard.IndexShardState.CLOSED; -import static org.elasticsearch.index.shard.IndexShardState.CREATED; -import static org.elasticsearch.index.shard.IndexShardState.POST_RECOVERY; -import static org.elasticsearch.index.shard.IndexShardState.RECOVERING; -import static org.elasticsearch.index.shard.IndexShardState.STARTED; +import static org.elasticsearch.index.shard.IndexShardState.*; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -181,7 +177,7 @@ public void testIndexStateShardChanged() throws Throwable { ensureGreen(); //the 3 relocated shards get closed on the first node - assertShardStatesMatch(stateChangeListenerNode1, 3, CLOSED); + assertShardStatesMatch(stateChangeListenerNode1, 3, RELOCATED, CLOSED); //the 3 relocated shards get created on the second node assertShardStatesMatch(stateChangeListenerNode2, 3, CREATED, RECOVERING, POST_RECOVERY, STARTED); diff --git a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java index 1a4bf8fd3f764..5e4411c07ea48 100644 --- a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java @@ -110,7 +110,7 @@ public void testSyncFailsIfOperationIsInFlight() throws InterruptedException { SyncedFlushService flushService = getInstanceFromNode(SyncedFlushService.class); final ShardId shardId = shard.shardId(); - shard.incrementOperationCounter(); + shard.incrementOperationCounterOnPrimary(); try { SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener<>(); flushService.attemptSyncedFlush(shardId, listener); @@ -157,7 +157,7 @@ public void testSyncFailsOnIndexClosedOrMissing() throws InterruptedException { assertNull(listener.result); assertEquals("no such index", listener.error.getMessage()); } - + public void testFailAfterIntermediateCommit() throws InterruptedException { createIndex("test"); client().prepareIndex("test", "test", "1").setSource("{}").get(); diff --git a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushUnitTests.java b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushUnitTests.java index 19cce93c6e4e6..8fad646d8b828 100644 --- a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushUnitTests.java +++ b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushUnitTests.java @@ -108,7 +108,7 @@ protected TestPlan createTestPlan() { Map shardResponses = new HashMap<>(); for (int copy = 0; copy < replicas + 1; copy++) { final ShardRouting shardRouting = TestShardRouting.newShardRouting(index, shard, "node_" + shardId + "_" + copy, null, - copy == 0, ShardRoutingState.STARTED, 0); + 1, copy == 0, ShardRoutingState.STARTED, 0); if (randomInt(5) < 2) { // shard copy failure failed++; diff --git a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java index fc4dd4f648781..cd3977e530566 100644 --- a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java @@ -404,7 +404,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { for (int i = 0; i < numShards; i++) { indexRoutingTableBuilder.addIndexShard( new IndexShardRoutingTable.Builder(new ShardId("test", i)) - .addShard(TestShardRouting.newShardRouting("test", i, masterId, true, ShardRoutingState.STARTED, shardVersions[shardIds[i]])) + .addShard(TestShardRouting.newShardRouting("test", i, masterId, 1, true, ShardRoutingState.STARTED, shardVersions[shardIds[i]])) .build() ); } diff --git a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java index ec6a3b3849151..b4b582f1e1022 100644 --- a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java +++ b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java @@ -81,10 +81,10 @@ public void testShardCanBeDeletedNoShardStarted() throws Exception { ClusterState.Builder clusterState = ClusterState.builder(new ClusterName("test")); clusterState.metaData(MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numShards).numberOfReplicas(numReplicas))); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", 1)); - + final int primaryTerm = randomInt(200); for (int i = 0; i < numShards; i++) { int unStartedShard = randomInt(numReplicas); - for (int j=0; j <= numReplicas; j++) { + for (int j = 0; j <= numReplicas; j++) { ShardRoutingState state; if (j == unStartedShard) { state = randomFrom(NOT_STARTED_STATES); @@ -95,7 +95,7 @@ public void testShardCanBeDeletedNoShardStarted() throws Exception { if (state == ShardRoutingState.UNASSIGNED) { unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null); } - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, null, j == 0, state, 0, unassignedInfo)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, null, primaryTerm, j == 0, state, 0, unassignedInfo)); } } assertFalse(indicesStore.shardCanBeDeleted(clusterState.build(), routingTable.build())); @@ -110,12 +110,13 @@ public void testShardCanBeDeletedShardExistsLocally() throws Exception { clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.id()).put(localNode).put(new DiscoveryNode("xyz", new LocalTransportAddress("xyz"), Version.CURRENT))); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", 1)); int localShardId = randomInt(numShards - 1); + final int primaryTerm = randomInt(200); for (int i = 0; i < numShards; i++) { String nodeId = i == localShardId ? localNode.getId() : randomBoolean() ? "abc" : "xyz"; String relocationNodeId = randomBoolean() ? null : randomBoolean() ? localNode.getId() : "xyz"; - routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, true, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, primaryTerm, true, ShardRoutingState.STARTED, 0)); for (int j = 0; j < numReplicas; j++) { - routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, false, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, nodeId, relocationNodeId, primaryTerm, false, ShardRoutingState.STARTED, 0)); } } @@ -131,11 +132,12 @@ public void testShardCanBeDeletedNodeNotInList() throws Exception { clusterState.metaData(MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numShards).numberOfReplicas(numReplicas))); clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.id()).put(localNode)); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", 1)); + final int primaryTerm = randomInt(200); for (int i = 0; i < numShards; i++) { String relocatingNodeId = randomBoolean() ? null : "def"; - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, true, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, primaryTerm, true, ShardRoutingState.STARTED, 0)); for (int j = 0; j < numReplicas; j++) { - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, false, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", relocatingNodeId, primaryTerm, false, ShardRoutingState.STARTED, 0)); } } @@ -153,10 +155,11 @@ public void testShardCanBeDeletedNodeVersion() throws Exception { clusterState.metaData(MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numShards).numberOfReplicas(numReplicas))); clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.id()).put(localNode).put(new DiscoveryNode("xyz", new LocalTransportAddress("xyz"), nodeVersion))); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", 1)); + final int primaryTerm = randomInt(200); for (int i = 0; i < numShards; i++) { - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, true, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, primaryTerm, true, ShardRoutingState.STARTED, 0)); for (int j = 0; j < numReplicas; j++) { - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, false, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", null, primaryTerm, false, ShardRoutingState.STARTED, 0)); } } @@ -171,7 +174,7 @@ public void testShardCanBeDeletedRelocatingNode() throws Exception { ClusterState.Builder clusterState = ClusterState.builder(new ClusterName("test")); clusterState.metaData(MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(numShards).numberOfReplicas(numReplicas))); final Version nodeVersion = randomBoolean() ? CURRENT : randomVersion(random()); - + final int primaryTerm = randomInt(200); clusterState.nodes(DiscoveryNodes.builder().localNodeId(localNode.id()) .put(localNode) .put(new DiscoveryNode("xyz", new LocalTransportAddress("xyz"), Version.CURRENT)) @@ -179,9 +182,9 @@ public void testShardCanBeDeletedRelocatingNode() throws Exception { )); IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(new ShardId("test", 1)); for (int i = 0; i < numShards; i++) { - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", true, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", primaryTerm, true, ShardRoutingState.STARTED, 0)); for (int j = 0; j < numReplicas; j++) { - routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", false, ShardRoutingState.STARTED, 0)); + routingTable.addShard(TestShardRouting.newShardRouting("test", i, "xyz", "def", primaryTerm, false, ShardRoutingState.STARTED, 0)); } } diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java b/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java index 7095639eafcbb..b8bac5090c5d9 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveryWhileUnderLoadIT.java @@ -164,6 +164,7 @@ public void testRecoverWhileUnderLoadWithReducedAllowedNodes() throws Exception try (BackgroundIndexer indexer = new BackgroundIndexer("test", "type", client(), extraDocs)) { logger.info("--> waiting for {} docs to be indexed ...", waitFor); waitForDocs(waitFor, indexer); + indexer.assertNoFailures(); logger.info("--> {} docs indexed", waitFor); @@ -176,6 +177,7 @@ public void testRecoverWhileUnderLoadWithReducedAllowedNodes() throws Exception logger.info("--> waiting for {} docs to be indexed ...", waitFor); waitForDocs(waitFor, indexer); + indexer.assertNoFailures(); logger.info("--> {} docs indexed", waitFor); @@ -294,17 +296,17 @@ private void iterateAssertCount(final int numberOfShards, final long numberOfDoc //if there was an error we try to wait and see if at some point it'll get fixed logger.info("--> trying to wait"); assertTrue(awaitBusy(() -> { - boolean errorOccurred = false; - for (int i = 0; i < iterations; i++) { - SearchResponse searchResponse = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(); - if (searchResponse.getHits().totalHits() != numberOfDocs) { - errorOccurred = true; - } - } - return !errorOccurred; - }, - 5, - TimeUnit.MINUTES + boolean errorOccurred = false; + for (int i = 0; i < iterations; i++) { + SearchResponse searchResponse = client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(); + if (searchResponse.getHits().totalHits() != numberOfDocs) { + errorOccurred = true; + } + } + return !errorOccurred; + }, + 5, + TimeUnit.MINUTES ) ); } diff --git a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 57b5e888ea92f..1765df27c11a6 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/core/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -21,7 +21,6 @@ import com.carrotsearch.hppc.IntHashSet; import com.carrotsearch.hppc.procedures.IntProcedure; - import org.apache.lucene.index.IndexFileNames; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -56,11 +55,7 @@ import org.elasticsearch.test.MockIndexEventListener; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequest; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportService; +import org.elasticsearch.transport.*; import java.io.IOException; import java.nio.file.FileVisitResult; @@ -79,9 +74,7 @@ import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.startsWith; +import static org.hamcrest.Matchers.*; /** */ @@ -144,6 +137,7 @@ public void testSimpleRelocationNoIndexing() { assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().totalHits(), equalTo(20l)); } + @TestLogging("action.index:TRACE,action.bulk:TRACE,action.search:TRACE") public void testRelocationWhileIndexingRandom() throws Exception { int numberOfRelocations = scaledRandomIntBetween(1, rarely() ? 10 : 4); int numberOfReplicas = randomBoolean() ? 0 : 1; diff --git a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java index a5b7da7796f86..fa41c9fef2d46 100644 --- a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java @@ -253,7 +253,7 @@ public void testRequiredRoutingWithPathMapping() throws Exception { assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); } } - + public void testRequiredRoutingWithPathMappingBulk() throws Exception { client().admin().indices().prepareCreate("test") .addAlias(new Alias("alias")) diff --git a/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java b/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java index c44608c4e4b75..2a57af862b088 100644 --- a/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java +++ b/plugins/delete-by-query/src/test/java/org/elasticsearch/action/deletebyquery/TransportDeleteByQueryActionTests.java @@ -226,7 +226,7 @@ public void testOnBulkResponse() { } else { deleted++; } - items[i] = new BulkItemResponse(i, "delete", new DeleteResponse(new ShardId("test", 0), "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++; @@ -282,7 +282,7 @@ public void testOnBulkResponseMultipleIndices() { deleted[0] = deleted[0] + 1; deleted[index] = deleted[index] + 1; } - items[i] = new BulkItemResponse(i, "delete", new DeleteResponse(new ShardId("test-" + index, 0), "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; @@ -409,7 +409,7 @@ private void assertNoFailures(TestActionListener listener) { private void assertSearchContextsClosed() { NodesStatsResponse nodesStats = client().admin().cluster().prepareNodesStats().setIndices(true).get(); - for (NodeStats nodeStat : nodesStats.getNodes()){ + for (NodeStats nodeStat : nodesStats.getNodes()) { assertThat(nodeStat.getIndices().getSearch().getOpenContexts(), equalTo(0L)); } } diff --git a/test-framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java b/test-framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java index df9e1f8af24ef..8a173ca4393d8 100644 --- a/test-framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java +++ b/test-framework/src/main/java/org/elasticsearch/cluster/routing/TestShardRouting.java @@ -27,26 +27,31 @@ */ public class TestShardRouting { - public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, boolean primary, ShardRoutingState state, long version) { - return new ShardRouting(index, shardId, currentNodeId, null, null, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true, -1); + public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, long primaryTerm, boolean primary, + ShardRoutingState state, long version) { + return new ShardRouting(index, shardId, currentNodeId, null, null, primaryTerm, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true, -1); } - public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, boolean primary, ShardRoutingState state, long version) { - return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true, -1); + public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, + long primaryTerm, boolean primary, ShardRoutingState state, long version) { + return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primaryTerm, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true, -1); } - public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, boolean primary, ShardRoutingState state, AllocationId allocationId, long version) { - return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primary, state, version, buildUnassignedInfo(state), allocationId, true, -1); + public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, + long primaryTerm, boolean primary, ShardRoutingState state, AllocationId allocationId, long version) { + return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, null, primaryTerm, primary, state, version, buildUnassignedInfo(state), allocationId, true, -1); } - public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, RestoreSource restoreSource, boolean primary, ShardRoutingState state, long version) { - return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, restoreSource, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true, -1); + public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, String relocatingNodeId, + RestoreSource restoreSource, long primaryTerm, boolean primary, + ShardRoutingState state, long version) { + return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, restoreSource, primaryTerm, primary, state, version, buildUnassignedInfo(state), buildAllocationId(state), true, -1); } public static ShardRouting newShardRouting(String index, int shardId, String currentNodeId, - String relocatingNodeId, RestoreSource restoreSource, boolean primary, ShardRoutingState state, long version, - UnassignedInfo unassignedInfo) { - return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, restoreSource, primary, state, version, unassignedInfo, buildAllocationId(state), true, -1); + String relocatingNodeId, RestoreSource restoreSource, long primaryTerm, boolean primary, + ShardRoutingState state, long version, UnassignedInfo unassignedInfo) { + return new ShardRouting(index, shardId, currentNodeId, relocatingNodeId, restoreSource, primaryTerm, primary, state, version, unassignedInfo, buildAllocationId(state), true, -1); } private static AllocationId buildAllocationId(ShardRoutingState state) { diff --git a/test-framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test-framework/src/main/java/org/elasticsearch/test/ESTestCase.java index c59c3ba4d4e6f..fb626d9963516 100644 --- a/test-framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test-framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -29,14 +29,12 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter; - import org.apache.lucene.uninverting.UninvertingReader; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; import org.apache.lucene.util.TestRuleMarkFailure; import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.TimeUnits; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.bootstrap.BootstrapForTesting; import org.elasticsearch.cache.recycler.MockPageCacheRecycler; @@ -50,7 +48,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -58,11 +55,7 @@ import org.elasticsearch.test.junit.listeners.LoggingListener; import org.elasticsearch.test.junit.listeners.ReproduceInfoPrinter; import org.elasticsearch.threadpool.ThreadPool; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; +import org.junit.*; import org.junit.rules.RuleChain; import java.io.IOException; @@ -562,14 +555,22 @@ private static String groupName(ThreadGroup threadGroup) { * Returns size random values */ public static List randomSubsetOf(int size, T... values) { - if (size > values.length) { - throw new IllegalArgumentException("Can\'t pick " + size + " random objects from a list of " + values.length + " objects"); + return randomSubsetOf(size, Arrays.asList(values)); + } + + /** + * Returns size random values + */ + public static List randomSubsetOf(int size, Collection values) { + if (size > values.size()) { + throw new IllegalArgumentException("Can\'t pick " + size + " random objects from a list of " + values.size() + " objects"); } - List list = arrayAsArrayList(values); - Collections.shuffle(list); + List list = new ArrayList<>(values); + Collections.shuffle(list, random()); return list.subList(0, size); } + /** * Returns true iff assertions for elasticsearch packages are enabled */ @@ -615,7 +616,7 @@ public void assertPathHasBeenCleared(Path path) throws Exception { sb.append("]"); assertThat(count + " files exist that should have been cleaned:\n" + sb.toString(), count, equalTo(0)); } - + /** Returns the suite failure marker: internal use only! */ public static TestRuleMarkFailure getSuiteFailureMarker() { return suiteFailureMarker; diff --git a/test-framework/src/main/java/org/elasticsearch/test/XContentTestUtils.java b/test-framework/src/main/java/org/elasticsearch/test/XContentTestUtils.java index 866a19e0a72f6..31380e25eb3d4 100644 --- a/test-framework/src/main/java/org/elasticsearch/test/XContentTestUtils.java +++ b/test-framework/src/main/java/org/elasticsearch/test/XContentTestUtils.java @@ -111,7 +111,7 @@ private static String differenceBetweenObjectsIgnoringArrayOrder(String path, Ob if (second instanceof Map) { return differenceBetweenMapsIgnoringArrayOrder(path, (Map) first, (Map) second); } else { - return path + ": the second element is not a map"; + return path + ": the second element is not a map (got " + second +")"; } } else { if (first.equals(second)) {